From 062c6c37f75bf98e7d5c8e2e1fb425cad22899ae Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 16 Oct 2023 13:03:16 -0700 Subject: [PATCH 01/68] wip sender scaling test --- network/p2p/p2pbuilder/libp2pNodeBuilder.go | 6 +- network/p2p/p2pbuilder/libp2pscaler.go | 4 +- network/p2p/p2pbuilder/libp2pscaler_test.go | 36 ++-- network/p2p/p2pbuilder/utils.go | 6 +- network/p2p/p2pnode/libp2pNode_test.go | 75 ++------ network/p2p/p2pnode/resourceManager_test.go | 186 ++++++++++++++++++++ network/p2p/test/fixtures.go | 8 + 7 files changed, 232 insertions(+), 89 deletions(-) create mode 100644 network/p2p/p2pnode/resourceManager_test.go diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index 5a593c2255e..5f155c029e3 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -219,11 +219,11 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { limits := rcmgr.DefaultLimits libp2p.SetDefaultServiceLimits(&limits) - mem, err := allowedMemory(builder.resourceManagerCfg.MemoryLimitRatio) + mem, err := AllowedMemory(builder.resourceManagerCfg.MemoryLimitRatio) if err != nil { return nil, fmt.Errorf("could not get allowed memory: %w", err) } - fd, err := allowedFileDescriptors(builder.resourceManagerCfg.FileDescriptorsRatio) + fd, err := AllowedFileDescriptors(builder.resourceManagerCfg.FileDescriptorsRatio) if err != nil { return nil, fmt.Errorf("could not get allowed file descriptors: %w", err) } @@ -238,7 +238,7 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { Int64("allowed_memory", mem). Int("allowed_file_descriptors", fd). Msg("allowed memory and file descriptors are fetched from the system") - newLimitConfigLogger(builder.logger).logResourceManagerLimits(l) + NewLimitConfigLogger(builder.logger).LogResourceManagerLimits(l) opts = append(opts, libp2p.ResourceManager(mgr)) builder.logger.Info().Msg("libp2p resource manager is set to default with metrics") diff --git a/network/p2p/p2pbuilder/libp2pscaler.go b/network/p2p/p2pbuilder/libp2pscaler.go index 8612a53d34d..d6bdaf2ef5a 100644 --- a/network/p2p/p2pbuilder/libp2pscaler.go +++ b/network/p2p/p2pbuilder/libp2pscaler.go @@ -16,14 +16,14 @@ func getNumFDs() (int, error) { return int(l.Cur), nil } -func allowedMemory(scaleFactor float64) (int64, error) { +func AllowedMemory(scaleFactor float64) (int64, error) { if scaleFactor <= 0 || scaleFactor > 1 { return 0, fmt.Errorf("memory scale factor must be greater than 0 and less than or equal to 1: %f", scaleFactor) } return int64(math.Floor(float64(memory.TotalMemory()) * scaleFactor)), nil } -func allowedFileDescriptors(scaleFactor float64) (int, error) { +func AllowedFileDescriptors(scaleFactor float64) (int, error) { if scaleFactor <= 0 || scaleFactor > 1 { return 0, fmt.Errorf("fd scale factor must be greater than 0 and less than or equal to 1: %f", scaleFactor) } diff --git a/network/p2p/p2pbuilder/libp2pscaler_test.go b/network/p2p/p2pbuilder/libp2pscaler_test.go index 789554866d0..094f8a4e700 100644 --- a/network/p2p/p2pbuilder/libp2pscaler_test.go +++ b/network/p2p/p2pbuilder/libp2pscaler_test.go @@ -12,44 +12,44 @@ func TestAllowedMemoryScale(t *testing.T) { require.True(t, m > 0) // scaling with factor of 1 should return the total memory. - s, err := allowedMemory(1) + s, err := AllowedMemory(1) require.NoError(t, err) require.Equal(t, int64(m), s) // scaling with factor of 0 should return an error. - _, err = allowedMemory(0) + _, err = AllowedMemory(0) require.Error(t, err) // scaling with factor of -1 should return an error. - _, err = allowedMemory(-1) + _, err = AllowedMemory(-1) require.Error(t, err) // scaling with factor of 2 should return an error. - _, err = allowedMemory(2) + _, err = AllowedMemory(2) require.Error(t, err) // scaling with factor of 0.5 should return half the total memory. - s, err = allowedMemory(0.5) + s, err = AllowedMemory(0.5) require.NoError(t, err) require.Equal(t, int64(m/2), s) // scaling with factor of 0.1 should return 10% of the total memory. - s, err = allowedMemory(0.1) + s, err = AllowedMemory(0.1) require.NoError(t, err) require.Equal(t, int64(m/10), s) // scaling with factor of 0.01 should return 1% of the total memory. - s, err = allowedMemory(0.01) + s, err = AllowedMemory(0.01) require.NoError(t, err) require.Equal(t, int64(m/100), s) // scaling with factor of 0.001 should return 0.1% of the total memory. - s, err = allowedMemory(0.001) + s, err = AllowedMemory(0.001) require.NoError(t, err) require.Equal(t, int64(m/1000), s) // scaling with factor of 0.0001 should return 0.01% of the total memory. - s, err = allowedMemory(0.0001) + s, err = AllowedMemory(0.0001) require.NoError(t, err) require.Equal(t, int64(m/10000), s) } @@ -61,44 +61,44 @@ func TestAllowedFileDescriptorsScale(t *testing.T) { require.True(t, fd > 0) // scaling with factor of 1 should return the total file descriptors. - s, err := allowedFileDescriptors(1) + s, err := AllowedFileDescriptors(1) require.NoError(t, err) require.Equal(t, fd, s) // scaling with factor of 0 should return an error. - _, err = allowedFileDescriptors(0) + _, err = AllowedFileDescriptors(0) require.Error(t, err) // scaling with factor of -1 should return an error. - _, err = allowedFileDescriptors(-1) + _, err = AllowedFileDescriptors(-1) require.Error(t, err) // scaling with factor of 2 should return an error. - _, err = allowedFileDescriptors(2) + _, err = AllowedFileDescriptors(2) require.Error(t, err) // scaling with factor of 0.5 should return half the total file descriptors. - s, err = allowedFileDescriptors(0.5) + s, err = AllowedFileDescriptors(0.5) require.NoError(t, err) require.Equal(t, fd/2, s) // scaling with factor of 0.1 should return 10% of the total file descriptors. - s, err = allowedFileDescriptors(0.1) + s, err = AllowedFileDescriptors(0.1) require.NoError(t, err) require.Equal(t, fd/10, s) // scaling with factor of 0.01 should return 1% of the total file descriptors. - s, err = allowedFileDescriptors(0.01) + s, err = AllowedFileDescriptors(0.01) require.NoError(t, err) require.Equal(t, fd/100, s) // scaling with factor of 0.001 should return 0.1% of the total file descriptors. - s, err = allowedFileDescriptors(0.001) + s, err = AllowedFileDescriptors(0.001) require.NoError(t, err) require.Equal(t, fd/1000, s) // scaling with factor of 0.0001 should return 0.01% of the total file descriptors. - s, err = allowedFileDescriptors(0.0001) + s, err = AllowedFileDescriptors(0.0001) require.NoError(t, err) require.Equal(t, fd/10000, s) } diff --git a/network/p2p/p2pbuilder/utils.go b/network/p2p/p2pbuilder/utils.go index ef2a2bc1ae9..2e05bbd5b84 100644 --- a/network/p2p/p2pbuilder/utils.go +++ b/network/p2p/p2pbuilder/utils.go @@ -32,8 +32,8 @@ type limitConfigLogger struct { logger zerolog.Logger } -// newLimitConfigLogger creates a new limitConfigLogger. -func newLimitConfigLogger(logger zerolog.Logger) *limitConfigLogger { +// NewLimitConfigLogger creates a new limitConfigLogger. +func NewLimitConfigLogger(logger zerolog.Logger) *limitConfigLogger { return &limitConfigLogger{logger: logger} } @@ -51,7 +51,7 @@ func (l *limitConfigLogger) withBaseLimit(prefix string, baseLimit rcmgr.Resourc Str(fmt.Sprintf("%s_memory", prefix), fmt.Sprintf("%v", baseLimit.Memory)).Logger() } -func (l *limitConfigLogger) logResourceManagerLimits(config rcmgr.ConcreteLimitConfig) { +func (l *limitConfigLogger) LogResourceManagerLimits(config rcmgr.ConcreteLimitConfig) { // PartialLimit config is the same as ConcreteLimit config, but with the exported fields. pCfg := config.ToPartialLimitConfig() l.logGlobalResourceLimits(pCfg) diff --git a/network/p2p/p2pnode/libp2pNode_test.go b/network/p2p/p2pnode/libp2pNode_test.go index ad42ec17108..904e5ff0aeb 100644 --- a/network/p2p/p2pnode/libp2pNode_test.go +++ b/network/p2p/p2pnode/libp2pNode_test.go @@ -28,7 +28,6 @@ import ( "github.com/onflow/flow-go/network/p2p/p2plogging" "github.com/onflow/flow-go/network/p2p/p2pnode" p2ptest "github.com/onflow/flow-go/network/p2p/test" - "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" "github.com/onflow/flow-go/utils/unittest" @@ -298,7 +297,10 @@ func TestCreateStream_SinglePairwiseConnection(t *testing.T) { go createConcurrentStreams(t, ctxWithTimeout, nodes, ids, numOfStreamsPerNode, streamChan, done) unittest.RequireCloseBefore(t, done, 5*time.Second, "could not create streamChan on time") - require.Len(t, streamChan, expectedTotalNumOfStreams, fmt.Sprintf("expected %d total number of streamChan created got %d", expectedTotalNumOfStreams, len(streamChan))) + require.Len(t, + streamChan, + expectedTotalNumOfStreams, + fmt.Sprintf("expected %d total number of streamChan created got %d", expectedTotalNumOfStreams, len(streamChan))) // ensure only a single connection exists between all nodes ensureSinglePairwiseConnection(t, nodes) @@ -401,67 +403,14 @@ func TestCreateStream_SinglePeerDial(t *testing.T) { expectedNumOfDialRetries := int64(p2pnode.MaxConnectAttempt) // we expect the second routine to retry creating a stream p2pnode.MaxConnectAttempt when dialing is in progress expectedCreateStreamRetries := int64(p2pnode.MaxConnectAttempt) - require.Equal(t, expectedNumOfDialRetries, dialPeerRetries.Load(), fmt.Sprintf("expected %d dial peer retries got %d", expectedNumOfDialRetries, dialPeerRetries.Load())) - require.Equal(t, expectedCreateStreamRetries, createStreamRetries.Load(), fmt.Sprintf("expected %d dial peer retries got %d", expectedCreateStreamRetries, createStreamRetries.Load())) -} - -// TestCreateStream_InboundConnResourceLimit ensures that the setting the resource limit config for -// PeerDefaultLimits.ConnsInbound restricts the number of inbound connections created from a peer to the configured value. -// NOTE: If this test becomes flaky, it indicates a violation of the single inbound connection guarantee. -// In such cases the test should not be quarantined but requires immediate resolution. -func TestCreateStream_InboundConnResourceLimit(t *testing.T) { - idProvider := mockmodule.NewIdentityProvider(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - - sporkID := unittest.IdentifierFixture() - - sender, id1 := p2ptest.NodeFixture( - t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithDefaultResourceManager(), - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) - - receiver, id2 := p2ptest.NodeFixture( - t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithDefaultResourceManager(), - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) - - idProvider.On("ByPeerID", sender.ID()).Return(&id1, true).Maybe() - idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() - - p2ptest.StartNodes(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}) - defer p2ptest.StopNodes(t, []p2p.LibP2PNode{sender, receiver}, cancel) - - p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}, flow.IdentityList{&id1, &id2}) - - var allStreamsCreated sync.WaitGroup - // at this point both nodes have discovered each other and we can now create an - // arbitrary number of streams from sender -> receiver. This will force libp2p - // to create multiple streams concurrently and attempt to reuse the single pairwise - // connection. If more than one connection is established while creating the conccurent - // streams this indicates a bug in the libp2p PeerBaseLimitConnsInbound limit. - defaultProtocolID := protocols.FlowProtocolID(sporkID) - expectedNumOfStreams := int64(50) - for i := int64(0); i < expectedNumOfStreams; i++ { - allStreamsCreated.Add(1) - go func() { - defer allStreamsCreated.Done() - _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) - require.NoError(t, err) - }() - } - - unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") - require.Len(t, receiver.Host().Network().ConnsToPeer(sender.ID()), 1) - actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) - require.Equal(t, expectedNumOfStreams, int64(actualNumOfStreams), fmt.Sprintf("expected to create %d number of streams got %d", expectedNumOfStreams, actualNumOfStreams)) + require.Equal(t, + expectedNumOfDialRetries, + dialPeerRetries.Load(), + fmt.Sprintf("expected %d dial peer retries got %d", expectedNumOfDialRetries, dialPeerRetries.Load())) + require.Equal(t, + expectedCreateStreamRetries, + createStreamRetries.Load(), + fmt.Sprintf("expected %d dial peer retries got %d", expectedCreateStreamRetries, createStreamRetries.Load())) } // createStreams will attempt to create n number of streams concurrently between each combination of node pairs. diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go new file mode 100644 index 00000000000..32070aca932 --- /dev/null +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -0,0 +1,186 @@ +package p2pnode_test + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/libp2p/go-libp2p" + "github.com/libp2p/go-libp2p/core/network" + rcmgr "github.com/libp2p/go-libp2p/p2p/host/resource-manager" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/irrecoverable" + mockmodule "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/network/internal/p2putils" + "github.com/onflow/flow-go/network/p2p" + p2ptest "github.com/onflow/flow-go/network/p2p/test" + "github.com/onflow/flow-go/network/p2p/unicast/protocols" + "github.com/onflow/flow-go/utils/unittest" +) + +// TestCreateStream_InboundConnResourceLimit ensures that the setting the resource limit config for +// PeerDefaultLimits.ConnsInbound restricts the number of inbound connections created from a peer to the configured value. +// NOTE: If this test becomes flaky, it indicates a violation of the single inbound connection guarantee. +// In such cases the test should not be quarantined but requires immediate resolution. +func TestCreateStream_InboundConnResourceLimit(t *testing.T) { + idProvider := mockmodule.NewIdentityProvider(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + + sporkID := unittest.IdentifierFixture() + + sender, id1 := p2ptest.NodeFixture( + t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithDefaultResourceManager(), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + receiver, id2 := p2ptest.NodeFixture( + t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithDefaultResourceManager(), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + idProvider.On("ByPeerID", sender.ID()).Return(&id1, true).Maybe() + idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() + + p2ptest.StartNodes(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}) + defer p2ptest.StopNodes(t, []p2p.LibP2PNode{sender, receiver}, cancel) + + p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}, flow.IdentityList{&id1, &id2}) + + var allStreamsCreated sync.WaitGroup + // at this point both nodes have discovered each other and we can now create an + // arbitrary number of streams from sender -> receiver. This will force libp2p + // to create multiple streams concurrently and attempt to reuse the single pairwise + // connection. If more than one connection is established while creating the conccurent + // streams this indicates a bug in the libp2p PeerBaseLimitConnsInbound limit. + defaultProtocolID := protocols.FlowProtocolID(sporkID) + expectedNumOfStreams := int64(50) + for i := int64(0); i < expectedNumOfStreams; i++ { + allStreamsCreated.Add(1) + go func() { + defer allStreamsCreated.Done() + require.NoError(t, sender.Host().Connect(ctx, receiver.Host().Peerstore().PeerInfo(receiver.ID()))) + _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) + require.NoError(t, err) + }() + } + + unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") + require.Len(t, receiver.Host().Network().ConnsToPeer(sender.ID()), 1) + actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) + require.Equal(t, + expectedNumOfStreams, + int64(actualNumOfStreams), + fmt.Sprintf("expected to create %d number of streams got %d", expectedNumOfStreams, actualNumOfStreams)) +} + +// TestCreateStream_InboundConnResourceLimit ensures that the setting the resource limit config for +// PeerDefaultLimits.ConnsInbound restricts the number of inbound connections created from a peer to the configured value. +// NOTE: If this test becomes flaky, it indicates a violation of the single inbound connection guarantee. +// In such cases the test should not be quarantined but requires immediate resolution. +func TestCreateStream_PeerResourceLimit_NoScale(t *testing.T) { + idProvider := mockmodule.NewIdentityProvider(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + + sporkID := unittest.IdentifierFixture() + + // cfg, err := flowconfig.DefaultConfig() + // require.NoError(t, err) + + // p2pbuilder.NewLimitConfigLogger(unittest.Logger()).LogResourceManagerLimits(l) + + // mem, err := p2pbuilder.AllowedMemory(cfg.NetworkConfig.MemoryLimitRatio) + // require.NoError(t, err) + // + // fd, err := p2pbuilder.AllowedFileDescriptors(cfg.NetworkConfig.FileDescriptorsRatio) + // require.NoError(t, err) + // limits.SystemBaseLimit.StreamsInbound = 1 + // limits.SystemLimitIncrease.StreamsInbound = 1 + // limits.ProtocolBaseLimit.StreamsOutbound = 10_0000 + resourceManagerSnd, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(rcmgr.InfiniteLimits)) + require.NoError(t, err) + sender, id1 := p2ptest.NodeFixture( + t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithResourceManager(resourceManagerSnd), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + limits := rcmgr.DefaultLimits + libp2p.SetDefaultServiceLimits(&limits) + // mem, err := p2pbuilder.AllowedMemory(cfg.NetworkConfig.MemoryLimitRatio) + // require.NoError(t, err) + // + // fd, err := p2pbuilder.AllowedFileDescriptors(cfg.NetworkConfig.FileDescriptorsRatio) + // require.NoError(t, err) + // limits.SystemBaseLimit.StreamsInbound = 1 + // limits.SystemLimitIncrease.StreamsInbound = 1 + // limits.ProtocolBaseLimit.StreamsOutbound = 10_0000 + l := limits.Scale(0, 0) + resourceManagerRcv, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l)) + require.NoError(t, err) + receiver, id2 := p2ptest.NodeFixture( + t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithResourceManager(resourceManagerRcv), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + idProvider.On("ByPeerID", sender.ID()).Return(&id1, true).Maybe() + idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() + + p2ptest.StartNodes(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}) + defer p2ptest.StopNodes(t, []p2p.LibP2PNode{sender, receiver}, cancel) + + p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}, flow.IdentityList{&id1, &id2}) + + var allStreamsCreated sync.WaitGroup + // at this point both nodes have discovered each other and we can now create an + // arbitrary number of streams from sender -> receiver. This will force libp2p + // to create multiple streams concurrently and attempt to reuse the single pairwise + // connection. If more than one connection is established while creating the conccurent + // streams this indicates a bug in the libp2p PeerBaseLimitConnsInbound limit. + defaultProtocolID := protocols.FlowProtocolID(sporkID) + maxAllowedStreams := l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound + t.Log("maxAllowedStreamsInbound", maxAllowedStreams) + t.Log("maxAllowedPeerStreamInbound", l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound) + t.Log("maxAllowedStreamsOutboundPeer (sender)", rcmgr.InfiniteLimits.ToPartialLimitConfig().ProtocolPeerDefault.StreamsOutbound) + t.Log("maxAllowedStreamSystemInbound", l.ToPartialLimitConfig().System.StreamsInbound) + surplus := int64(l.ToPartialLimitConfig().System.StreamsInbound) + errorCount := int64(0) + for i := int64(0); i < int64(maxAllowedStreams)+surplus; i++ { + allStreamsCreated.Add(1) + go func() { + defer allStreamsCreated.Done() + _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) + if err != nil { + atomic.AddInt64(&errorCount, 1) // count the number of errors + } + }() + } + + unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") + require.Len(t, receiver.Host().Network().ConnsToPeer(sender.ID()), 1) + actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) + require.Equal(t, + int64(maxAllowedStreams), + int64(actualNumOfStreams), + fmt.Sprintf("expected to create %d number of streams got %d", maxAllowedStreams, actualNumOfStreams)) + require.Equalf(t, int64(surplus), atomic.LoadInt64(&errorCount), "expected to get %d errors got %d", surplus, atomic.LoadInt64(&errorCount)) +} diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 656e5d6f0cb..bcdca5e3264 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -398,6 +398,14 @@ func WithDefaultResourceManager() NodeFixtureParameterOption { } } +// WithResourceManager sets the resource manager to the provided resource manager. +// Otherwise, it uses the resource manager provided by the test (the infinite resource manager). +func WithResourceManager(resourceManager network.ResourceManager) NodeFixtureParameterOption { + return func(p *NodeFixtureParameters) { + p.ResourceManager = resourceManager + } +} + func WithUnicastHandlerFunc(handler network.StreamHandler) NodeFixtureParameterOption { return func(p *NodeFixtureParameters) { p.HandlerFunc = handler From efdf38f8feb2ac388666e82a9d1e32e9face336b Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Tue, 17 Oct 2023 10:50:41 -0700 Subject: [PATCH 02/68] adds test for system-wide stream limit is not enforced --- network/internal/p2putils/utils.go | 2 +- network/p2p/p2pnode/resourceManager_test.go | 136 +++++++++++--------- 2 files changed, 74 insertions(+), 64 deletions(-) diff --git a/network/internal/p2putils/utils.go b/network/internal/p2putils/utils.go index 2325df8734a..292ce75f230 100644 --- a/network/internal/p2putils/utils.go +++ b/network/internal/p2putils/utils.go @@ -159,7 +159,7 @@ func IPPortFromMultiAddress(addrs ...multiaddr.Multiaddr) (string, string, error return "", "", err } - //there should only be one valid IPv4 address + // there should only be one valid IPv4 address return ipOrHostname, port, nil } return "", "", fmt.Errorf("ip address or hostname not found") diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 32070aca932..a6adac64219 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -4,13 +4,13 @@ import ( "context" "fmt" "sync" - "sync/atomic" "testing" "time" "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/network" rcmgr "github.com/libp2p/go-libp2p/p2p/host/resource-manager" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/flow" @@ -86,11 +86,25 @@ func TestCreateStream_InboundConnResourceLimit(t *testing.T) { fmt.Sprintf("expected to create %d number of streams got %d", expectedNumOfStreams, actualNumOfStreams)) } -// TestCreateStream_InboundConnResourceLimit ensures that the setting the resource limit config for -// PeerDefaultLimits.ConnsInbound restricts the number of inbound connections created from a peer to the configured value. -// NOTE: If this test becomes flaky, it indicates a violation of the single inbound connection guarantee. -// In such cases the test should not be quarantined but requires immediate resolution. -func TestCreateStream_PeerResourceLimit_NoScale(t *testing.T) { +// TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management +// was not being enforced. The purpose of this test is to share with the libp2p community as well as to evaluate the existence of the bug on +// future libp2p versions. +// Test scenario works as follows: +// - We have 30 senders and 1 receiver. +// - The senders are running with a resource manager that allows infinite number of streams; so that they can create as many streams as they want. +// - The receiver is running with a resource manager with base limits and no scaling. +// - The test reads the peer protocol default limits for inbound streams at receiver; say x; which is the limit for the number of inbound streams from each sender on a +// specific protocol. +// - Each sender creates x-1 streams to the receiver on a specific protocol. This is done to ensure that the receiver has x-1 streams from each sender; a total of +// 30*(x-1) streams at the receiver. +// - Test first ensures that numerically 30 * (x - 1) > max system-wide inbound stream limit; i.e., the total number of streams created by all senders is greater than +// the system-wide limit. +// - Then each sender creates x - 1 streams concurrently to the receiver. +// - At the end of the test we ensure that the total number of streams created by all senders is greater than the system-wide limit; which should not be the case if the +// system-wide limit is being enforced. +func TestCreateStream_SystemStreamLimit_NotEnforced(t *testing.T) { + nodeCount := 30 + idProvider := mockmodule.NewIdentityProvider(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -98,39 +112,16 @@ func TestCreateStream_PeerResourceLimit_NoScale(t *testing.T) { sporkID := unittest.IdentifierFixture() - // cfg, err := flowconfig.DefaultConfig() - // require.NoError(t, err) - - // p2pbuilder.NewLimitConfigLogger(unittest.Logger()).LogResourceManagerLimits(l) - - // mem, err := p2pbuilder.AllowedMemory(cfg.NetworkConfig.MemoryLimitRatio) - // require.NoError(t, err) - // - // fd, err := p2pbuilder.AllowedFileDescriptors(cfg.NetworkConfig.FileDescriptorsRatio) - // require.NoError(t, err) - // limits.SystemBaseLimit.StreamsInbound = 1 - // limits.SystemLimitIncrease.StreamsInbound = 1 - // limits.ProtocolBaseLimit.StreamsOutbound = 10_0000 resourceManagerSnd, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(rcmgr.InfiniteLimits)) require.NoError(t, err) - sender, id1 := p2ptest.NodeFixture( - t, - sporkID, - t.Name(), + senders, senderIds := p2ptest.NodesFixture(t, sporkID, t.Name(), nodeCount, idProvider, p2ptest.WithResourceManager(resourceManagerSnd), p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) limits := rcmgr.DefaultLimits libp2p.SetDefaultServiceLimits(&limits) - // mem, err := p2pbuilder.AllowedMemory(cfg.NetworkConfig.MemoryLimitRatio) - // require.NoError(t, err) - // - // fd, err := p2pbuilder.AllowedFileDescriptors(cfg.NetworkConfig.FileDescriptorsRatio) - // require.NoError(t, err) - // limits.SystemBaseLimit.StreamsInbound = 1 - // limits.SystemLimitIncrease.StreamsInbound = 1 - // limits.ProtocolBaseLimit.StreamsOutbound = 10_0000 + l := limits.Scale(0, 0) resourceManagerRcv, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l)) require.NoError(t, err) @@ -142,45 +133,64 @@ func TestCreateStream_PeerResourceLimit_NoScale(t *testing.T) { p2ptest.WithResourceManager(resourceManagerRcv), p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) - idProvider.On("ByPeerID", sender.ID()).Return(&id1, true).Maybe() + for i, sender := range senders { + idProvider.On("ByPeerID", sender.ID()).Return(senderIds[i], true).Maybe() + } idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() - p2ptest.StartNodes(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}) - defer p2ptest.StopNodes(t, []p2p.LibP2PNode{sender, receiver}, cancel) + p2ptest.StartNodes(t, signalerCtx, append(senders, receiver)) + defer p2ptest.StopNodes(t, append(senders, receiver), cancel) - p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}, flow.IdentityList{&id1, &id2}) + p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, append(senders, receiver), append(senderIds, &id2)) var allStreamsCreated sync.WaitGroup - // at this point both nodes have discovered each other and we can now create an - // arbitrary number of streams from sender -> receiver. This will force libp2p - // to create multiple streams concurrently and attempt to reuse the single pairwise - // connection. If more than one connection is established while creating the conccurent - // streams this indicates a bug in the libp2p PeerBaseLimitConnsInbound limit. defaultProtocolID := protocols.FlowProtocolID(sporkID) - maxAllowedStreams := l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound - t.Log("maxAllowedStreamsInbound", maxAllowedStreams) - t.Log("maxAllowedPeerStreamInbound", l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound) - t.Log("maxAllowedStreamsOutboundPeer (sender)", rcmgr.InfiniteLimits.ToPartialLimitConfig().ProtocolPeerDefault.StreamsOutbound) - t.Log("maxAllowedStreamSystemInbound", l.ToPartialLimitConfig().System.StreamsInbound) - surplus := int64(l.ToPartialLimitConfig().System.StreamsInbound) - errorCount := int64(0) - for i := int64(0); i < int64(maxAllowedStreams)+surplus; i++ { - allStreamsCreated.Add(1) - go func() { - defer allStreamsCreated.Done() - _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) - if err != nil { - atomic.AddInt64(&errorCount, 1) // count the number of errors - } - }() + maxInboundStreamPerPeer := l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound + maxSystemInboundStream := l.ToPartialLimitConfig().System.StreamsInbound + + t.Log("max allowed inbound stream from each sender to receiver (per protocol)", maxInboundStreamPerPeer) + t.Log("max allowed inbound stream across all peers and protocols at receiver (system-wide)", maxSystemInboundStream) + + // sanity check; if each peer creates maxInboundStreamPerPeer-1 streams, and we assume there the maxSystemInboundStream is not enforced; then to validate the hypothesis we need + // to ensure that (maxInboundStreamPerPeer - 1) * nodeCount > maxSystemInboundStream, i.e., if each peer creates maxInboundStreamPerPeer-1 streams, then the total number of streams + // end up being greater than the system-wide limit. + require.Greaterf(t, + int64(maxInboundStreamPerPeer-1)*int64(nodeCount), + int64(maxSystemInboundStream), + "(maxInboundStreamPerPeer - 1) * nodeCount should be greater than maxSystemInboundStream") + + for sIndex := range senders { + sender := senders[sIndex] + for i := int64(0); i < int64(maxInboundStreamPerPeer-1); i++ { + allStreamsCreated.Add(1) + go func() { + defer allStreamsCreated.Done() + _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) + require.NoError(t, err, "error creating stream") + }() + } } unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") - require.Len(t, receiver.Host().Network().ConnsToPeer(sender.ID()), 1) - actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) - require.Equal(t, - int64(maxAllowedStreams), - int64(actualNumOfStreams), - fmt.Sprintf("expected to create %d number of streams got %d", maxAllowedStreams, actualNumOfStreams)) - require.Equalf(t, int64(surplus), atomic.LoadInt64(&errorCount), "expected to get %d errors got %d", surplus, atomic.LoadInt64(&errorCount)) + + totalStreams := 0 + for i, sender := range senders { + actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) + t.Logf("sender %d has %d streams", i, actualNumOfStreams) + assert.Equalf(t, + int64(maxInboundStreamPerPeer-1), + int64(actualNumOfStreams), + "expected to create %d number of streams got %d", + int64(maxInboundStreamPerPeer-1), + actualNumOfStreams) + totalStreams += actualNumOfStreams + } + + // when system-wide limit is not enforced, the total number of streams created by all senders should be greater than the system-wide limit. + require.Greaterf(t, + totalStreams, + l.ToPartialLimitConfig().Stream.StreamsInbound, + "expected to create more than %d number of streams got %d", + l.ToPartialLimitConfig().Stream.StreamsInbound, + totalStreams) } From a389627ecf66ec46a86e6b57112338ebe08873ad Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 18 Oct 2023 10:34:01 -0700 Subject: [PATCH 03/68] update ledger.Set to remove empty update case --- ledger/complete/ledger.go | 5 ----- ledger/complete/ledger_test.go | 2 +- ledger/partial/ledger.go | 5 ----- 3 files changed, 1 insertion(+), 11 deletions(-) diff --git a/ledger/complete/ledger.go b/ledger/complete/ledger.go index 3d1dbed21c7..f56d63e415d 100644 --- a/ledger/complete/ledger.go +++ b/ledger/complete/ledger.go @@ -200,11 +200,6 @@ func (l *Ledger) Get(query *ledger.Query) (values []ledger.Value, err error) { func (l *Ledger) Set(update *ledger.Update) (newState ledger.State, trieUpdate *ledger.TrieUpdate, err error) { start := time.Now() - if update.Size() == 0 { - // return current state root unchanged - return update.State(), nil, nil - } - trieUpdate, err = pathfinder.UpdateToTrieUpdate(update, l.pathFinderVersion) if err != nil { return ledger.State(hash.DummyHash), nil, err diff --git a/ledger/complete/ledger_test.go b/ledger/complete/ledger_test.go index edb64365c53..b0685fb7ef4 100644 --- a/ledger/complete/ledger_test.go +++ b/ledger/complete/ledger_test.go @@ -55,7 +55,7 @@ func TestLedger_Update(t *testing.T) { newState, trieUpdate, err := l.Set(up) require.NoError(t, err) - require.Nil(t, trieUpdate) + require.True(t, trieUpdate.IsEmpty()) // state shouldn't change assert.Equal(t, currentState, newState) diff --git a/ledger/partial/ledger.go b/ledger/partial/ledger.go index 91eb9e5fbe7..b084098a50b 100644 --- a/ledger/partial/ledger.go +++ b/ledger/partial/ledger.go @@ -120,11 +120,6 @@ func (l *Ledger) Get(query *ledger.Query) (values []ledger.Value, err error) { // Set updates the ledger given an update // it returns the state after update and errors (if any) func (l *Ledger) Set(update *ledger.Update) (newState ledger.State, trieUpdate *ledger.TrieUpdate, err error) { - // TODO: add test case - if update.Size() == 0 { - // return current state root unchanged - return update.State(), nil, nil - } trieUpdate, err = pathfinder.UpdateToTrieUpdate(update, l.pathFinderVersion) if err != nil { From 543bc2c4184369a7bc0e1c70063708087ca14391 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 18 Oct 2023 16:36:52 -0700 Subject: [PATCH 04/68] wip implementing backbone test --- network/internal/p2putils/utils.go | 97 +++-- network/p2p/p2pnode/libp2pStream_test.go | 20 +- network/p2p/p2pnode/resourceManager_test.go | 402 +++++++++++++++++++- 3 files changed, 483 insertions(+), 36 deletions(-) diff --git a/network/internal/p2putils/utils.go b/network/internal/p2putils/utils.go index 292ce75f230..f5bc4f01e4c 100644 --- a/network/internal/p2putils/utils.go +++ b/network/internal/p2putils/utils.go @@ -4,11 +4,11 @@ import ( "fmt" "net" - "github.com/libp2p/go-libp2p/core" "github.com/libp2p/go-libp2p/core/crypto" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/core/protocol" "github.com/multiformats/go-multiaddr" "github.com/rs/zerolog" @@ -69,46 +69,95 @@ func ConnectednessToString(connectedness network.Connectedness) (string, bool) { } -// FindOutboundStream finds an existing outbound stream to the target id if it exists by querying libp2p -func FindOutboundStream(host host.Host, targetID peer.ID, protocol core.ProtocolID) (network.Stream, bool) { - streams := FilterStream(host, targetID, protocol, network.DirOutbound, false) - if len(streams) > 0 { - return streams[0], true +// CountStream finds total number of outbound stream to the target id +func CountStream(host host.Host, targetID peer.ID, opts ...FilterOption) int { + streams := FilterStream(host, targetID, append(opts, All())...) + return len(streams) +} + +// FilterOptions holds the filtering options used in FilterStream. +type FilterOptions struct { + // dir specifies the direction of the streams to be filtered. + // The default value is network.DirBoth, which considers both inbound and outbound streams. + dir network.Direction + + // protocol specifies the protocol ID of the streams to be filtered. + // The default value is an empty string, which considers streams of all protocol IDs. + protocol protocol.ID + + // all specifies whether to return all matching streams or just the first matching stream. + // The default value is false, which returns just the first matching stream. + all bool +} + +// FilterOption defines a function type that modifies FilterOptions. +type FilterOption func(*FilterOptions) + +// Direction is a FilterOption for setting the direction of the streams to be filtered. +func Direction(dir network.Direction) FilterOption { + return func(opts *FilterOptions) { + opts.dir = dir } - return nil, false } -// CountStream finds total number of outbound stream to the target id -func CountStream(host host.Host, targetID peer.ID, protocol core.ProtocolID, dir network.Direction) int { - streams := FilterStream(host, targetID, protocol, dir, true) - return len(streams) +// Protocol is a FilterOption for setting the protocol ID of the streams to be filtered. +func Protocol(protocol protocol.ID) FilterOption { + return func(opts *FilterOptions) { + opts.protocol = protocol + } } -// FilterStream finds one or all existing outbound streams to the target id if it exists. -// if parameter all is true - all streams are found else the first stream found is returned -func FilterStream(host host.Host, targetID peer.ID, protocol core.ProtocolID, dir network.Direction, all bool) []network.Stream { +// All is a FilterOption for setting whether to return all matching streams or just the first matching stream. +func All() FilterOption { + return func(opts *FilterOptions) { + opts.all = true + } +} +// FilterStream filters the streams to a target peer based on the provided options. +// The default behavior is to consider all directions and protocols, and return just the first matching stream. +// This behavior can be customized by providing FilterOption values. +// +// Usage: +// +// - To find all outbound streams to a target peer with a specific protocol ID: +// streams := FilterStream(host, targetID, Direction(network.DirOutbound), Protocol(myProtocolID), All(true)) +// +// - To find the first inbound stream to a target peer, regardless of protocol ID: +// stream := FilterStream(host, targetID, Direction(network.DirInbound)) +// +// host is the host from which to filter streams. +// targetID is the ID of the target peer. +// options is a variadic parameter that allows zero or more FilterOption values to be provided. +// +// It returns a slice of network.Stream values that match the filtering criteria. +func FilterStream(host host.Host, targetID peer.ID, options ...FilterOption) []network.Stream { var filteredStreams []network.Stream - // choose the connection only if it is connected + // default values + opts := FilterOptions{ + dir: network.DirUnknown, // by default, consider both inbound and outbound streams + protocol: "", // by default, consider streams of all protocol IDs + all: false, // by default, return just the first matching stream + } + + // apply provided options + for _, option := range options { + option(&opts) + } + if host.Network().Connectedness(targetID) != network.Connected { return filteredStreams } - // get all connections conns := host.Network().ConnsToPeer(targetID) - - // find a connection which is in the connected state for _, conn := range conns { - - // get all streams streams := conn.GetStreams() for _, stream := range streams { - - // choose a stream which is marked as outbound and is for the flow protocol - if stream.Stat().Direction == dir && stream.Protocol() == protocol { + if (opts.dir == network.DirUnknown || stream.Stat().Direction == opts.dir) && + (opts.protocol == "" || stream.Protocol() == opts.protocol) { filteredStreams = append(filteredStreams, stream) - if !all { + if !opts.all { return filteredStreams } } diff --git a/network/p2p/p2pnode/libp2pStream_test.go b/network/p2p/p2pnode/libp2pStream_test.go index 8d693a501cf..6a90c2c5207 100644 --- a/network/p2p/p2pnode/libp2pStream_test.go +++ b/network/p2p/p2pnode/libp2pStream_test.go @@ -164,7 +164,7 @@ func testCreateStream(t *testing.T, sporkId flow.Identifier, unicasts []protocol id2 := identities[1] // Assert that there is no outbound stream to the target yet - require.Equal(t, 0, p2putils.CountStream(nodes[0].Host(), nodes[1].ID(), protocolID, network.DirOutbound)) + require.Equal(t, 0, p2putils.CountStream(nodes[0].Host(), nodes[1].ID(), p2putils.Protocol(protocolID), p2putils.Direction(network.DirOutbound))) // Now attempt to create another 100 outbound stream to the same destination by calling CreateStream streamCount := 100 @@ -193,7 +193,7 @@ func testCreateStream(t *testing.T, sporkId flow.Identifier, unicasts []protocol } require.Eventually(t, func() bool { - return streamCount == p2putils.CountStream(nodes[0].Host(), nodes[1].ID(), protocolID, network.DirOutbound) + return streamCount == p2putils.CountStream(nodes[0].Host(), nodes[1].ID(), p2putils.Protocol(protocolID), p2putils.Direction(network.DirOutbound)) }, 5*time.Second, 100*time.Millisecond, "could not create streams on time") // checks that the number of connections is 1 despite the number of streams; i.e., all streams are created on the same connection @@ -235,8 +235,8 @@ func TestCreateStream_FallBack(t *testing.T) { // Assert that there is no outbound stream to the target yet (neither default nor preferred) defaultProtocolId := protocols.FlowProtocolID(sporkId) preferredProtocolId := protocols.FlowGzipProtocolId(sporkId) - require.Equal(t, 0, p2putils.CountStream(thisNode.Host(), otherNode.ID(), defaultProtocolId, network.DirOutbound)) - require.Equal(t, 0, p2putils.CountStream(thisNode.Host(), otherNode.ID(), preferredProtocolId, network.DirOutbound)) + require.Equal(t, 0, p2putils.CountStream(thisNode.Host(), otherNode.ID(), p2putils.Protocol(defaultProtocolId), p2putils.Direction(network.DirOutbound))) + require.Equal(t, 0, p2putils.CountStream(thisNode.Host(), otherNode.ID(), p2putils.Protocol(preferredProtocolId), p2putils.Direction(network.DirOutbound))) // Now attempt to create another 100 outbound stream to the same destination by calling CreateStream streamCount := 10 @@ -265,11 +265,11 @@ func TestCreateStream_FallBack(t *testing.T) { // wait for the stream to be created on the default protocol id. require.Eventually(t, func() bool { - return streamCount == p2putils.CountStream(nodes[0].Host(), nodes[1].ID(), defaultProtocolId, network.DirOutbound) + return streamCount == p2putils.CountStream(nodes[0].Host(), nodes[1].ID(), p2putils.Protocol(defaultProtocolId), p2putils.Direction(network.DirOutbound)) }, 5*time.Second, 100*time.Millisecond, "could not create streams on time") // no stream must be created on the preferred protocol id - require.Equal(t, 0, p2putils.CountStream(thisNode.Host(), otherNode.ID(), preferredProtocolId, network.DirOutbound)) + require.Equal(t, 0, p2putils.CountStream(thisNode.Host(), otherNode.ID(), p2putils.Protocol(preferredProtocolId), p2putils.Direction(network.DirOutbound))) // checks that the number of connections is 1 despite the number of streams; i.e., all streams are created on the same connection require.Len(t, nodes[0].Host().Network().Conns(), 1) @@ -366,16 +366,16 @@ func TestNoBackoffWhenCreatingStream(t *testing.T) { someGraceTime := 100 * time.Millisecond totalWaitTime := maxTimeToWait + someGraceTime - //each CreateStream() call may try to connect up to MaxConnectAttempt (3) times. + // each CreateStream() call may try to connect up to MaxConnectAttempt (3) times. - //there are 2 scenarios that we need to account for: + // there are 2 scenarios that we need to account for: // - //1. machines where a timeout occurs on the first connection attempt - this can be due to local firewall rules or other processes running on the machine. + // 1. machines where a timeout occurs on the first connection attempt - this can be due to local firewall rules or other processes running on the machine. // In this case, we need to create a scenario where a backoff would have normally occured. This is why we initiate a second connection attempt. // Libp2p remembers the peer we are trying to connect to between CreateStream() calls and would have initiated a backoff if backoff wasn't turned off. // The second CreateStream() call will make a second connection attempt MaxConnectAttempt times and that should never result in a backoff error. // - //2. machines where a timeout does NOT occur on the first connection attempt - this is on CI machines and some local dev machines without a firewall / too many other processes. + // 2. machines where a timeout does NOT occur on the first connection attempt - this is on CI machines and some local dev machines without a firewall / too many other processes. // In this case, there will be MaxConnectAttempt (3) connection attempts on the first CreateStream() call and MaxConnectAttempt (3) attempts on the second CreateStream() call. // make two separate stream creation attempt and assert that no connection back off happened diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index a6adac64219..a5597e660cf 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "sync" + "sync/atomic" "testing" "time" @@ -79,7 +80,7 @@ func TestCreateStream_InboundConnResourceLimit(t *testing.T) { unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") require.Len(t, receiver.Host().Network().ConnsToPeer(sender.ID()), 1) - actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) + actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), p2putils.Protocol(defaultProtocolID), p2putils.Direction(network.DirOutbound)) require.Equal(t, expectedNumOfStreams, int64(actualNumOfStreams), @@ -175,7 +176,7 @@ func TestCreateStream_SystemStreamLimit_NotEnforced(t *testing.T) { totalStreams := 0 for i, sender := range senders { - actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) + actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), p2putils.Protocol(defaultProtocolID), p2putils.Direction(network.DirOutbound)) t.Logf("sender %d has %d streams", i, actualNumOfStreams) assert.Equalf(t, int64(maxInboundStreamPerPeer-1), @@ -193,4 +194,401 @@ func TestCreateStream_SystemStreamLimit_NotEnforced(t *testing.T) { "expected to create more than %d number of streams got %d", l.ToPartialLimitConfig().Stream.StreamsInbound, totalStreams) + + totalTrackedStreams := 0 + require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { + t.Logf("transient scope; inbound stream count %d", scope.Stat().NumStreamsInbound) + totalTrackedStreams += scope.Stat().NumStreamsInbound + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { + t.Logf("protocol scope for %s; inbound stream count %d", defaultProtocolID, scope.Stat().NumStreamsInbound) + totalTrackedStreams += scope.Stat().NumStreamsInbound + return nil + })) + + for _, sender := range senders { + require.NoError(t, resourceManagerRcv.ViewPeer(sender.ID(), func(scope network.PeerScope) error { + t.Logf("peer scope for %s; inbound stream count %d", sender.ID(), scope.Stat().NumStreamsInbound) + totalTrackedStreams += scope.Stat().NumStreamsInbound + return nil + })) + } + + require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + t.Logf("system scope; inbound stream count %d", scope.Stat().NumStreamsInbound) + totalTrackedStreams += scope.Stat().NumStreamsInbound + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { + t.Logf("transient scope; inbound stream count %d", scope.Stat().NumStreamsInbound) + totalTrackedStreams += scope.Stat().NumStreamsInbound + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { + t.Logf("protocol scope for %s; inbound stream count %d", defaultProtocolID, scope.Stat().NumStreamsInbound) + totalTrackedStreams += scope.Stat().NumStreamsInbound + return nil + })) + + for _, sender := range senders { + require.NoError(t, resourceManagerRcv.ViewPeer(sender.ID(), func(scope network.PeerScope) error { + t.Logf("peer scope for %s; inbound stream count %d", sender.ID(), scope.Stat().NumStreamsInbound) + totalTrackedStreams += scope.Stat().NumStreamsInbound + return nil + })) + } + + t.Logf("total tracked streams %d", totalTrackedStreams) +} + +// TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management +// was not being enforced. The purpose of this test is to share with the libp2p community as well as to evaluate the existence of the bug on +// future libp2p versions. +// Test scenario works as follows: +// - We have 30 senders and 1 receiver. +// - The senders are running with a resource manager that allows infinite number of streams; so that they can create as many streams as they want. +// - The receiver is running with a resource manager with base limits and no scaling. +// - The test reads the peer protocol default limits for inbound streams at receiver; say x; which is the limit for the number of inbound streams from each sender on a +// specific protocol. +// - Each sender creates x-1 streams to the receiver on a specific protocol. This is done to ensure that the receiver has x-1 streams from each sender; a total of +// 30*(x-1) streams at the receiver. +// - Test first ensures that numerically 30 * (x - 1) > max system-wide inbound stream limit; i.e., the total number of streams created by all senders is greater than +// the system-wide limit. +// - Then each sender creates x - 1 streams concurrently to the receiver. +// - At the end of the test we ensure that the total number of streams created by all senders is greater than the system-wide limit; which should not be the case if the +// system-wide limit is being enforced. +func TestCreateStream_ResourceAllocation(t *testing.T) { + idProvider := mockmodule.NewIdentityProvider(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + + sporkID := unittest.IdentifierFixture() + + resourceManagerSnd, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(rcmgr.InfiniteLimits)) + require.NoError(t, err) + sender, senderId := p2ptest.NodeFixture(t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithResourceManager(resourceManagerSnd), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + limits := rcmgr.DefaultLimits + libp2p.SetDefaultServiceLimits(&limits) + + l := limits.Scale(0, 0) + resourceManagerRcv, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l)) + require.NoError(t, err) + receiver, id2 := p2ptest.NodeFixture(t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithResourceManager(resourceManagerRcv), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + idProvider.On("ByPeerID", sender.ID()).Return(&senderId, true).Maybe() + idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() + + p2ptest.StartNodes(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}) + defer p2ptest.StopNodes(t, []p2p.LibP2PNode{sender, receiver}, cancel) + + p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}, flow.IdentityList{&senderId, &id2}) + + defaultProtocolID := protocols.FlowProtocolID(sporkID) + maxInboundStreamPerPeer := l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound + maxSystemInboundStream := l.ToPartialLimitConfig().System.StreamsInbound + + t.Log("max allowed inbound stream from each sender to receiver (per protocol)", maxInboundStreamPerPeer) + t.Log("max protocol peer limit", l.ToPartialLimitConfig().ProtocolDefault.StreamsInbound) + t.Log("max allowed inbound stream across all peers and protocols at receiver (system-wide)", maxSystemInboundStream) + + for i := 0; i < 100; i++ { + streamCreated := make(chan struct{}) + go func() { + err := sender.OpenProtectedStream(ctx, receiver.ID(), t.Name(), func(stream network.Stream) error { + close(streamCreated) + <-ctx.Done() + return nil + + }) + require.NoError(t, err, "error creating stream") + }() + <-streamCreated + + t.Logf("created stream %d", i) + outStreamCnt := p2putils.CountStream(sender.Host(), receiver.ID(), p2putils.Protocol(defaultProtocolID), p2putils.Direction(network.DirOutbound)) + inStreamCnt := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Protocol(defaultProtocolID), p2putils.Direction(network.DirInbound)) + t.Logf("outbound stream count %d", outStreamCnt) + t.Logf("inbound stream count %d", inStreamCnt) + require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { + t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { + t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + } + + require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { + t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { + t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + // require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + // t.Logf("system scope") + // t.Logf("inbound stream count %d", scope.Stat().NumStreamsInbound) + // return nil + // })) + + // unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") + // + // totalStreams := 0 + // for i, sender := range senders { + // actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) + // t.Logf("sender %d has %d streams", i, actualNumOfStreams) + // assert.Equalf(t, + // int64(maxInboundStreamPerPeer-1), + // int64(actualNumOfStreams), + // "expected to create %d number of streams got %d", + // int64(maxInboundStreamPerPeer-1), + // actualNumOfStreams) + // totalStreams += actualNumOfStreams + // } + // + // // when system-wide limit is not enforced, the total number of streams created by all senders should be greater than the system-wide limit. + // require.Greaterf(t, + // totalStreams, + // l.ToPartialLimitConfig().Stream.StreamsInbound, + // "expected to create more than %d number of streams got %d", + // l.ToPartialLimitConfig().Stream.StreamsInbound, + // totalStreams) + // + // require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { + // t.Logf("protocol scope for %s", defaultProtocolID) + // t.Logf("inbound stream count %d", scope.Stat().NumStreamsInbound) + // return nil + // })) + // + // require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + // t.Logf("system scope") + // t.Logf("inbound stream count %d", scope.Stat().NumStreamsInbound) + // return nil + // })) +} + +// TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management +// was not being enforced. The purpose of this test is to share with the libp2p community as well as to evaluate the existence of the bug on +// future libp2p versions. +// Test scenario works as follows: +// - We have 30 senders and 1 receiver. +// - The senders are running with a resource manager that allows infinite number of streams; so that they can create as many streams as they want. +// - The receiver is running with a resource manager with base limits and no scaling. +// - The test reads the peer protocol default limits for inbound streams at receiver; say x; which is the limit for the number of inbound streams from each sender on a +// specific protocol. +// - Each sender creates x-1 streams to the receiver on a specific protocol. This is done to ensure that the receiver has x-1 streams from each sender; a total of +// 30*(x-1) streams at the receiver. +// - Test first ensures that numerically 30 * (x - 1) > max system-wide inbound stream limit; i.e., the total number of streams created by all senders is greater than +// the system-wide limit. +// - Then each sender creates x - 1 streams concurrently to the receiver. +// - At the end of the test we ensure that the total number of streams created by all senders is greater than the system-wide limit; which should not be the case if the +// system-wide limit is being enforced. +func TestCreateStream_PeerLimit_Enforced(t *testing.T) { + nodeCount := 10 + buff := 0 + maxStreamPerPeer := 5 + maxStreamProtocol := nodeCount * maxStreamPerPeer + maxStreamPeerProtocol := maxStreamPerPeer * maxStreamProtocol + maxTransient := nodeCount + maxSystemStream := nodeCount + + idProvider := mockmodule.NewIdentityProvider(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + + sporkID := unittest.IdentifierFixture() + + // sender nodes will have infinite stream limit to ensure that they can create as many streams as they want. + resourceManagerSnd, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(rcmgr.InfiniteLimits)) + require.NoError(t, err) + senders, senderIds := p2ptest.NodesFixture(t, + sporkID, + t.Name(), + nodeCount, + idProvider, + p2ptest.WithResourceManager(resourceManagerSnd), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + // receiver node will run with default limits and no scaling. + limits := rcmgr.DefaultLimits + libp2p.SetDefaultServiceLimits(&limits) + l := limits.Scale(0, 0) + cfg := rcmgr.PartialLimitConfig{ + System: rcmgr.ResourceLimits{ + StreamsInbound: rcmgr.LimitVal(maxSystemStream), + ConnsInbound: rcmgr.LimitVal(nodeCount), + }, + Transient: rcmgr.ResourceLimits{ + ConnsInbound: rcmgr.LimitVal(nodeCount), + StreamsInbound: rcmgr.LimitVal(maxTransient), + }, + ProtocolDefault: rcmgr.ResourceLimits{ + StreamsInbound: rcmgr.LimitVal(maxStreamProtocol + buff), + }, + ProtocolPeerDefault: rcmgr.ResourceLimits{ + StreamsInbound: rcmgr.LimitVal(maxStreamPeerProtocol + buff), + }, + PeerDefault: rcmgr.ResourceLimits{ + StreamsInbound: rcmgr.LimitVal(maxStreamPerPeer + buff), + }, + Conn: rcmgr.ResourceLimits{ + StreamsInbound: rcmgr.LimitVal(maxStreamPerPeer + buff), + }, + Stream: rcmgr.ResourceLimits{ + StreamsInbound: rcmgr.LimitVal(maxStreamPerPeer + buff), + }, + } + l = cfg.Build(l) + resourceManagerRcv, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l)) + require.NoError(t, err) + receiver, id2 := p2ptest.NodeFixture(t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithResourceManager(resourceManagerRcv), + p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) + + for i, sender := range senders { + idProvider.On("ByPeerID", sender.ID()).Return(senderIds[i], true).Maybe() + } + idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() + + nodes := append(senders, receiver) + ids := append(senderIds, &id2) + + p2ptest.StartNodes(t, signalerCtx, nodes) + defer p2ptest.StopNodes(t, nodes, cancel) + + p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, nodes, ids) + + var allStreamsCreated sync.WaitGroup + defaultProtocolID := protocols.FlowProtocolID(sporkID) + // maxTransientPerPeer := l.ToPartialLimitConfig().Transient.StreamsInbound + // + // t.Log("max allowed inbound stream from each sender to receiver (per protocol)", maxInboundStreamPerPeer, "max transient", maxTransientPerPeer) + + totalStreamsCreated := int64(0) + for sIndex := range senders { + for i := int64(0); i < int64(maxStreamPerPeer); i++ { + if i >= int64(maxSystemStream) { + // we reached the system-wide limit; no need to create more streams; as stream creation may fail; we re-examine pressure on system-wide limit later. + break + } + allStreamsCreated.Add(1) + go func(sIndex int) { + defer allStreamsCreated.Done() + sender := senders[sIndex] + _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) + require.NoError(t, err, "error creating stream") + atomic.AddInt64(&totalStreamsCreated, 1) + }(sIndex) + } + } + + unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") + + require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { + // number of in-transient streams must be less than the max transient limit + require.Less(t, int64(scope.Stat().NumStreamsInbound), int64(maxTransient)) + + // number of in-transient streams must be less than or equal the total number of streams created. + require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(totalStreamsCreated)) + // t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + for i, sender := range senders { + actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) + t.Logf("sender %d has %d streams", i, actualNumOfStreams) + // require.Equalf(t, + // int64(maxStreamPerPeer), + // int64(actualNumOfStreams), + // "expected to create %d number of streams got %d", + // int64(maxStreamPerPeer), + // actualNumOfStreams) + } + + // now the test goes beyond the protocol-peer limit and tries to create one more stream from each sender. + // this should cause receiver to close all streams from the sender and disconnect from the sender. + for sIndex := range senders { + for i := int64(0); i < int64(100); i++ { + allStreamsCreated.Add(1) + go func(sIndex int) { + defer allStreamsCreated.Done() + sender := senders[sIndex] + _, _ = sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) + }(sIndex) + } + } + + unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") + + t.Log("-----") + total := 0 + for i, sender := range senders { + actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) + t.Logf("sender %d has %d streams", i, actualNumOfStreams) + // require.Equalf(t, + // int64(0), + // int64(actualNumOfStreams), + // "expected to create %d number of streams got %d", + // int64(0), + // actualNumOfStreams) + total += actualNumOfStreams + } + + require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { + t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { + t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + return nil + })) + + t.Logf("total streams %d", total) } From a1f1d88e3b6e8961fd9d48523f662bf9ec1a3f2c Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Thu, 19 Oct 2023 05:57:07 -0500 Subject: [PATCH 05/68] alsp sync engine flag default values --- config/default-config.yml | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/config/default-config.yml b/config/default-config.yml index 42ed814d78b..05260d351f8 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -122,3 +122,18 @@ network-config: alsp-spam-report-queue-size: 10e4 alsp-disable-penalty: false alsp-heart-beat-interval: 1s + + # Base probability in [0,1] that's used in creating the final probability of creating a + # misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, + # since it's not the final probability and there are other factors that determine the final probability. + # The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. + alsp-sync-engine-batch-request-base-prob: 0.1 + + # Probability in [0,1] of creating a misbehavior report for a SyncRequest message. + alsp-sync-engine-sync-request-prob: 0.1 + + # Base probability in [0,1] that's used in creating the final probability of creating a + # misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, + # since it's not the final probability and there are other factors that determine the final probability. + # The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. + alsp-sync-engine-range-request-base-prob: 0.1 From b479b7eed9bfd76d3e0127a9bd5436c807970223 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Thu, 19 Oct 2023 06:32:29 -0500 Subject: [PATCH 06/68] alsp sync engine struct fields --- config/default-config.yml | 6 +++--- network/netconf/config.go | 15 +++++++++++++++ 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/config/default-config.yml b/config/default-config.yml index 05260d351f8..5350f3f63ce 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -129,11 +129,11 @@ network-config: # The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. alsp-sync-engine-batch-request-base-prob: 0.1 - # Probability in [0,1] of creating a misbehavior report for a SyncRequest message. - alsp-sync-engine-sync-request-prob: 0.1 - # Base probability in [0,1] that's used in creating the final probability of creating a # misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, # since it's not the final probability and there are other factors that determine the final probability. # The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. alsp-sync-engine-range-request-base-prob: 0.1 + + # Probability in [0,1] of creating a misbehavior report for a SyncRequest message. + alsp-sync-engine-sync-request-prob: 0.1 diff --git a/network/netconf/config.go b/network/netconf/config.go index f3bcfed1f93..06ca26805e2 100644 --- a/network/netconf/config.go +++ b/network/netconf/config.go @@ -67,4 +67,19 @@ type AlspConfig struct { // HeartBeatInterval is the interval between heartbeats sent by the ALSP module. The heartbeats are recurring // events that are used to perform critical ALSP tasks, such as updating the spam records cache. HearBeatInterval time.Duration `mapstructure:"alsp-heart-beat-interval"` + + // SyncEngineBatchRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a + // misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, + // since it's not the final probability and there are other factors that determine the final probability. + // The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. + SyncEngineBatchRequestBaseProb float32 `mapstructure:"alsp-sync-engine-batch-request-base-prob"` + + // SyncEngineRangeRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a + // misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, + // since it's not the final probability and there are other factors that determine the final probability. + // The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. + SyncEngineRangeRequestBaseProb float32 `mapstructure:"alsp-sync-engine-range-request-base-prob"` + + // SyncEngineSyncRequestProb is the probability in [0,1] of creating a misbehavior report for a SyncRequest message. + SyncEngineSyncRequestProb float32 `mapstructure:"alsp-sync-engine-sync-request-prob"` } From ecebde7591a9bd01ea0c9e908eeb9a4b262ecd96 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Thu, 19 Oct 2023 06:49:05 -0500 Subject: [PATCH 07/68] alsp sync engine flag definition in const --- network/netconf/flags.go | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/network/netconf/flags.go b/network/netconf/flags.go index e9e18fe4822..8c04ea9c95e 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -67,6 +67,10 @@ const ( alspSpamRecordCacheSize = "alsp-spam-record-cache-size" alspSpamRecordQueueSize = "alsp-spam-report-queue-size" alspHearBeatInterval = "alsp-heart-beat-interval" + + alspSyncEngineBatchRequestBaseProb = "alsp-sync-engine-batch-request-base-prob" + alspSyncEngineRangeRequestBaseProb = "alsp-sync-engine-range-request-base-prob" + alspSyncEngineSyncRequestProb = "alsp-sync-engine-sync-request-prob" ) func AllFlagNames() []string { @@ -76,7 +80,8 @@ func AllFlagNames() []string { fileDescriptorsRatio, peerBaseLimitConnsInbound, highWatermark, lowWatermark, gracePeriod, silencePeriod, peerScoring, localMeshLogInterval, rpcSentTrackerCacheSize, rpcSentTrackerQueueCacheSize, rpcSentTrackerNumOfWorkers, scoreTracerInterval, gossipSubRPCInspectorNotificationCacheSize, validationInspectorNumberOfWorkers, validationInspectorInspectMessageQueueCacheSize, validationInspectorClusterPrefixedTopicsReceivedCacheSize, validationInspectorClusterPrefixedTopicsReceivedCacheDecay, validationInspectorClusterPrefixHardThreshold, - ihaveMaxSampleSize, metricsInspectorNumberOfWorkers, metricsInspectorCacheSize, alspDisabled, alspSpamRecordCacheSize, alspSpamRecordQueueSize, alspHearBeatInterval, + ihaveMaxSampleSize, metricsInspectorNumberOfWorkers, metricsInspectorCacheSize, + alspDisabled, alspSpamRecordCacheSize, alspSpamRecordQueueSize, alspHearBeatInterval, alspSyncEngineBatchRequestBaseProb, alspSyncEngineRangeRequestBaseProb, alspSyncEngineSyncRequestProb, iwantMaxSampleSize, iwantMaxMessageIDSampleSize, ihaveMaxMessageIDSampleSize, iwantCacheMissThreshold, controlMessageMaxSampleSize, iwantDuplicateMsgIDThreshold, iwantCacheMissCheckSize, } } @@ -133,6 +138,9 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Uint32(alspSpamRecordCacheSize, config.AlspConfig.SpamRecordCacheSize, "size of spam record cache, recommended to be 10x the number of authorized nodes") flags.Uint32(alspSpamRecordQueueSize, config.AlspConfig.SpamReportQueueSize, "size of spam report queue, recommended to be 100x the number of authorized nodes") flags.Duration(alspHearBeatInterval, config.AlspConfig.HearBeatInterval, "interval between two consecutive heartbeat events at alsp, recommended to leave it as default unless you know what you are doing.") + flags.Float32(alspSyncEngineBatchRequestBaseProb, config.AlspConfig.SyncEngineBatchRequestBaseProb, "base probability of creating a misbehavior report for a batch request message") + flags.Float32(alspSyncEngineRangeRequestBaseProb, config.AlspConfig.SyncEngineRangeRequestBaseProb, "base probability of creating a misbehavior report for a range request message") + flags.Float32(alspSyncEngineSyncRequestProb, config.AlspConfig.SyncEngineSyncRequestProb, "probability of creating a misbehavior report for a sync request message") flags.Int(ihaveMaxSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxSampleSize, "max number of ihaves to sample when performing validation") flags.Int(ihaveMaxMessageIDSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxMessageIDSampleSize, "max number of message ids to sample when performing validation per ihave") From a99d979f8675227b19af17c827f4beedd7f78b83 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Thu, 19 Oct 2023 07:52:51 -0500 Subject: [PATCH 08/68] SpamDetectionConfig reads flags (removed hard codes), update default values --- config/default-config.yml | 7 ++++--- engine/common/synchronization/config.go | 17 +++++++++++++---- network/netconf/flags.go | 6 +++--- 3 files changed, 20 insertions(+), 10 deletions(-) diff --git a/config/default-config.yml b/config/default-config.yml index d3e9accdd47..cc7be185e2a 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -150,13 +150,14 @@ network-config: # misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, # since it's not the final probability and there are other factors that determine the final probability. # The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. - alsp-sync-engine-batch-request-base-prob: 0.1 + alsp-sync-engine-batch-request-base-prob: 0.01 # Base probability in [0,1] that's used in creating the final probability of creating a # misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, # since it's not the final probability and there are other factors that determine the final probability. # The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. - alsp-sync-engine-range-request-base-prob: 0.1 + alsp-sync-engine-range-request-base-prob: 0.01 # Probability in [0,1] of creating a misbehavior report for a SyncRequest message. - alsp-sync-engine-sync-request-prob: 0.1 + # create misbehavior report for 1% of SyncRequest messages + alsp-sync-engine-sync-request-prob: 0.01 diff --git a/engine/common/synchronization/config.go b/engine/common/synchronization/config.go index 0ab2e7dd9e0..42dc99a08f4 100644 --- a/engine/common/synchronization/config.go +++ b/engine/common/synchronization/config.go @@ -1,6 +1,10 @@ package synchronization import ( + "context" + "fmt" + "github.com/onflow/flow-go/config" + "github.com/onflow/flow-go/module/irrecoverable" "time" core "github.com/onflow/flow-go/module/chainsync" @@ -68,10 +72,16 @@ type SpamDetectionConfig struct { } func NewSpamDetectionConfig() *SpamDetectionConfig { + flowConfig, err := config.DefaultConfig() + if err != nil { + irrecoverable.Throw(context.TODO(), fmt.Errorf("failed to read default config: %w", err)) + } + return &SpamDetectionConfig{ + batchRequestBaseProb: flowConfig.NetworkConfig.SyncEngineBatchRequestBaseProb, + // create misbehavior report for 1% of SyncRequest messages - // TODO: make this configurable as a start up flag for the engine - syncRequestProb: 0.01, + syncRequestProb: flowConfig.NetworkConfig.SyncEngineSyncRequestProb, // create misbehavior report for about 0.2% of RangeRequest messages for normal range requests (i.e. not too large) // and about 15% of RangeRequest messages for very large range requests @@ -85,7 +95,6 @@ func NewSpamDetectionConfig() *SpamDetectionConfig { // creating a misbehavior report is: // rangeRequestBaseProb * (1000+1) / synccore.DefaultConfig().MaxSize // = 0.01 * 1001 / 64 = 0.15640625 = 15.640625% - // TODO: make this configurable as a start up flag for the engine - rangeRequestBaseProb: 0.01, + rangeRequestBaseProb: flowConfig.NetworkConfig.SyncEngineRangeRequestBaseProb, } } diff --git a/network/netconf/flags.go b/network/netconf/flags.go index 94a627270b1..1e021c6f589 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -128,9 +128,9 @@ func AllFlagNames() []string { alspSpamRecordCacheSize, alspSpamRecordQueueSize, alspHearBeatInterval, - alspSyncEngineBatchRequestBaseProb, - alspSyncEngineRangeRequestBaseProb, - alspSyncEngineSyncRequestProb, + alspSyncEngineBatchRequestBaseProb, + alspSyncEngineRangeRequestBaseProb, + alspSyncEngineSyncRequestProb, iwantMaxSampleSize, iwantMaxMessageIDSampleSize, ihaveMaxMessageIDSampleSize, From f80def47faceb70d7ad4a4274fb99c2ba7308172 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 09:52:01 -0700 Subject: [PATCH 09/68] wip --- network/p2p/p2pnode/resourceManager_test.go | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index a5597e660cf..e922a10e913 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -496,10 +496,8 @@ func TestCreateStream_PeerLimit_Enforced(t *testing.T) { var allStreamsCreated sync.WaitGroup defaultProtocolID := protocols.FlowProtocolID(sporkID) - // maxTransientPerPeer := l.ToPartialLimitConfig().Transient.StreamsInbound - // - // t.Log("max allowed inbound stream from each sender to receiver (per protocol)", maxInboundStreamPerPeer, "max transient", maxTransientPerPeer) + // creates max(maxStreamPerPeer * nodeCount, maxSystemStream) streams from each sender to the receiver; breaks as soon as the system-wide limit is reached. totalStreamsCreated := int64(0) for sIndex := range senders { for i := int64(0); i < int64(maxStreamPerPeer); i++ { @@ -535,16 +533,17 @@ func TestCreateStream_PeerLimit_Enforced(t *testing.T) { return nil })) - for i, sender := range senders { + totalInboundStreams := 0 + for _, sender := range senders { actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) - t.Logf("sender %d has %d streams", i, actualNumOfStreams) - // require.Equalf(t, - // int64(maxStreamPerPeer), - // int64(actualNumOfStreams), - // "expected to create %d number of streams got %d", - // int64(maxStreamPerPeer), - // actualNumOfStreams) + // t.Logf("sender %d has %d streams", i, actualNumOfStreams) + require.LessOrEqual(t, int64(actualNumOfStreams), int64(maxStreamPerPeer)) + totalInboundStreams += actualNumOfStreams } + // sanity check; the total number of inbound streams must be less than or equal to the system-wide limit. + // TODO: this must be a hard equal check; but falls short; to be shared with libp2p community. + // Failing at this line means the system-wide limit is not being enforced. + require.LessOrEqual(t, totalInboundStreams, maxSystemStream) // now the test goes beyond the protocol-peer limit and tries to create one more stream from each sender. // this should cause receiver to close all streams from the sender and disconnect from the sender. From 4853e1b4725c83862dcdba987c7a7387bd52aca4 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 11:33:10 -0700 Subject: [PATCH 10/68] cleans up tests --- network/p2p/p2pnode/resourceManager_test.go | 456 ++++---------------- 1 file changed, 92 insertions(+), 364 deletions(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index e922a10e913..beaeb23b99a 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -11,7 +11,6 @@ import ( "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/network" rcmgr "github.com/libp2p/go-libp2p/p2p/host/resource-manager" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/flow" @@ -87,316 +86,39 @@ func TestCreateStream_InboundConnResourceLimit(t *testing.T) { fmt.Sprintf("expected to create %d number of streams got %d", expectedNumOfStreams, actualNumOfStreams)) } -// TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management -// was not being enforced. The purpose of this test is to share with the libp2p community as well as to evaluate the existence of the bug on -// future libp2p versions. -// Test scenario works as follows: -// - We have 30 senders and 1 receiver. -// - The senders are running with a resource manager that allows infinite number of streams; so that they can create as many streams as they want. -// - The receiver is running with a resource manager with base limits and no scaling. -// - The test reads the peer protocol default limits for inbound streams at receiver; say x; which is the limit for the number of inbound streams from each sender on a -// specific protocol. -// - Each sender creates x-1 streams to the receiver on a specific protocol. This is done to ensure that the receiver has x-1 streams from each sender; a total of -// 30*(x-1) streams at the receiver. -// - Test first ensures that numerically 30 * (x - 1) > max system-wide inbound stream limit; i.e., the total number of streams created by all senders is greater than -// the system-wide limit. -// - Then each sender creates x - 1 streams concurrently to the receiver. -// - At the end of the test we ensure that the total number of streams created by all senders is greater than the system-wide limit; which should not be the case if the -// system-wide limit is being enforced. -func TestCreateStream_SystemStreamLimit_NotEnforced(t *testing.T) { - nodeCount := 30 +type testPeerLimitConfig struct { + // nodeCount is the number of nodes in the test. + nodeCount int - idProvider := mockmodule.NewIdentityProvider(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + // maxInboundPeerStream is the maximum number of inbound streams from a single peer to the receiver. + maxInboundPeerStream int - sporkID := unittest.IdentifierFixture() + // maxInboundStreamProtocol is the maximum number of inbound streams at the receiver using a specific protocol; it accumulates all streams from all senders. + maxInboundStreamProtocol int - resourceManagerSnd, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(rcmgr.InfiniteLimits)) - require.NoError(t, err) - senders, senderIds := p2ptest.NodesFixture(t, sporkID, t.Name(), nodeCount, - idProvider, - p2ptest.WithResourceManager(resourceManagerSnd), - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) - - limits := rcmgr.DefaultLimits - libp2p.SetDefaultServiceLimits(&limits) - - l := limits.Scale(0, 0) - resourceManagerRcv, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l)) - require.NoError(t, err) - receiver, id2 := p2ptest.NodeFixture( - t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithResourceManager(resourceManagerRcv), - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) - - for i, sender := range senders { - idProvider.On("ByPeerID", sender.ID()).Return(senderIds[i], true).Maybe() - } - idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() - - p2ptest.StartNodes(t, signalerCtx, append(senders, receiver)) - defer p2ptest.StopNodes(t, append(senders, receiver), cancel) + // maxInboundStreamPeerProtocol is the maximum number of inbound streams at the receiver from a single peer using a specific protocol. + maxInboundStreamPeerProtocol int - p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, append(senders, receiver), append(senderIds, &id2)) - - var allStreamsCreated sync.WaitGroup - defaultProtocolID := protocols.FlowProtocolID(sporkID) - maxInboundStreamPerPeer := l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound - maxSystemInboundStream := l.ToPartialLimitConfig().System.StreamsInbound + // maxInboundStreamTransient is the maximum number of inbound transient streams at the receiver; it accumulates all streams from all senders across all protocols. + // transient streams are those that are not associated fully with a peer and protocol. + maxInboundStreamTransient int - t.Log("max allowed inbound stream from each sender to receiver (per protocol)", maxInboundStreamPerPeer) - t.Log("max allowed inbound stream across all peers and protocols at receiver (system-wide)", maxSystemInboundStream) - - // sanity check; if each peer creates maxInboundStreamPerPeer-1 streams, and we assume there the maxSystemInboundStream is not enforced; then to validate the hypothesis we need - // to ensure that (maxInboundStreamPerPeer - 1) * nodeCount > maxSystemInboundStream, i.e., if each peer creates maxInboundStreamPerPeer-1 streams, then the total number of streams - // end up being greater than the system-wide limit. - require.Greaterf(t, - int64(maxInboundStreamPerPeer-1)*int64(nodeCount), - int64(maxSystemInboundStream), - "(maxInboundStreamPerPeer - 1) * nodeCount should be greater than maxSystemInboundStream") - - for sIndex := range senders { - sender := senders[sIndex] - for i := int64(0); i < int64(maxInboundStreamPerPeer-1); i++ { - allStreamsCreated.Add(1) - go func() { - defer allStreamsCreated.Done() - _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) - require.NoError(t, err, "error creating stream") - }() - } - } - - unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") - - totalStreams := 0 - for i, sender := range senders { - actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), p2putils.Protocol(defaultProtocolID), p2putils.Direction(network.DirOutbound)) - t.Logf("sender %d has %d streams", i, actualNumOfStreams) - assert.Equalf(t, - int64(maxInboundStreamPerPeer-1), - int64(actualNumOfStreams), - "expected to create %d number of streams got %d", - int64(maxInboundStreamPerPeer-1), - actualNumOfStreams) - totalStreams += actualNumOfStreams - } - - // when system-wide limit is not enforced, the total number of streams created by all senders should be greater than the system-wide limit. - require.Greaterf(t, - totalStreams, - l.ToPartialLimitConfig().Stream.StreamsInbound, - "expected to create more than %d number of streams got %d", - l.ToPartialLimitConfig().Stream.StreamsInbound, - totalStreams) - - totalTrackedStreams := 0 - require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { - t.Logf("transient scope; inbound stream count %d", scope.Stat().NumStreamsInbound) - totalTrackedStreams += scope.Stat().NumStreamsInbound - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { - t.Logf("protocol scope for %s; inbound stream count %d", defaultProtocolID, scope.Stat().NumStreamsInbound) - totalTrackedStreams += scope.Stat().NumStreamsInbound - return nil - })) - - for _, sender := range senders { - require.NoError(t, resourceManagerRcv.ViewPeer(sender.ID(), func(scope network.PeerScope) error { - t.Logf("peer scope for %s; inbound stream count %d", sender.ID(), scope.Stat().NumStreamsInbound) - totalTrackedStreams += scope.Stat().NumStreamsInbound - return nil - })) - } - - require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - t.Logf("system scope; inbound stream count %d", scope.Stat().NumStreamsInbound) - totalTrackedStreams += scope.Stat().NumStreamsInbound - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { - t.Logf("transient scope; inbound stream count %d", scope.Stat().NumStreamsInbound) - totalTrackedStreams += scope.Stat().NumStreamsInbound - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { - t.Logf("protocol scope for %s; inbound stream count %d", defaultProtocolID, scope.Stat().NumStreamsInbound) - totalTrackedStreams += scope.Stat().NumStreamsInbound - return nil - })) - - for _, sender := range senders { - require.NoError(t, resourceManagerRcv.ViewPeer(sender.ID(), func(scope network.PeerScope) error { - t.Logf("peer scope for %s; inbound stream count %d", sender.ID(), scope.Stat().NumStreamsInbound) - totalTrackedStreams += scope.Stat().NumStreamsInbound - return nil - })) - } - - t.Logf("total tracked streams %d", totalTrackedStreams) + // maxInboundStreamSystem is the maximum number of inbound streams at the receiver; it accumulates all streams from all senders across all protocols. + maxInboundStreamSystem int } -// TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management -// was not being enforced. The purpose of this test is to share with the libp2p community as well as to evaluate the existence of the bug on -// future libp2p versions. -// Test scenario works as follows: -// - We have 30 senders and 1 receiver. -// - The senders are running with a resource manager that allows infinite number of streams; so that they can create as many streams as they want. -// - The receiver is running with a resource manager with base limits and no scaling. -// - The test reads the peer protocol default limits for inbound streams at receiver; say x; which is the limit for the number of inbound streams from each sender on a -// specific protocol. -// - Each sender creates x-1 streams to the receiver on a specific protocol. This is done to ensure that the receiver has x-1 streams from each sender; a total of -// 30*(x-1) streams at the receiver. -// - Test first ensures that numerically 30 * (x - 1) > max system-wide inbound stream limit; i.e., the total number of streams created by all senders is greater than -// the system-wide limit. -// - Then each sender creates x - 1 streams concurrently to the receiver. -// - At the end of the test we ensure that the total number of streams created by all senders is greater than the system-wide limit; which should not be the case if the -// system-wide limit is being enforced. -func TestCreateStream_ResourceAllocation(t *testing.T) { - idProvider := mockmodule.NewIdentityProvider(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - - sporkID := unittest.IdentifierFixture() - - resourceManagerSnd, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(rcmgr.InfiniteLimits)) - require.NoError(t, err) - sender, senderId := p2ptest.NodeFixture(t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithResourceManager(resourceManagerSnd), - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) - - limits := rcmgr.DefaultLimits - libp2p.SetDefaultServiceLimits(&limits) - - l := limits.Scale(0, 0) - resourceManagerRcv, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l)) - require.NoError(t, err) - receiver, id2 := p2ptest.NodeFixture(t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithResourceManager(resourceManagerRcv), - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) - - idProvider.On("ByPeerID", sender.ID()).Return(&senderId, true).Maybe() - idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() - - p2ptest.StartNodes(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}) - defer p2ptest.StopNodes(t, []p2p.LibP2PNode{sender, receiver}, cancel) - - p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}, flow.IdentityList{&senderId, &id2}) - - defaultProtocolID := protocols.FlowProtocolID(sporkID) - maxInboundStreamPerPeer := l.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound - maxSystemInboundStream := l.ToPartialLimitConfig().System.StreamsInbound - - t.Log("max allowed inbound stream from each sender to receiver (per protocol)", maxInboundStreamPerPeer) - t.Log("max protocol peer limit", l.ToPartialLimitConfig().ProtocolDefault.StreamsInbound) - t.Log("max allowed inbound stream across all peers and protocols at receiver (system-wide)", maxSystemInboundStream) - - for i := 0; i < 100; i++ { - streamCreated := make(chan struct{}) - go func() { - err := sender.OpenProtectedStream(ctx, receiver.ID(), t.Name(), func(stream network.Stream) error { - close(streamCreated) - <-ctx.Done() - return nil - - }) - require.NoError(t, err, "error creating stream") - }() - <-streamCreated - - t.Logf("created stream %d", i) - outStreamCnt := p2putils.CountStream(sender.Host(), receiver.ID(), p2putils.Protocol(defaultProtocolID), p2putils.Direction(network.DirOutbound)) - inStreamCnt := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Protocol(defaultProtocolID), p2putils.Direction(network.DirInbound)) - t.Logf("outbound stream count %d", outStreamCnt) - t.Logf("inbound stream count %d", inStreamCnt) - require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { - t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { - t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - } - - require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { - t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { - t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - // require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - // t.Logf("system scope") - // t.Logf("inbound stream count %d", scope.Stat().NumStreamsInbound) - // return nil - // })) - - // unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") - // - // totalStreams := 0 - // for i, sender := range senders { - // actualNumOfStreams := p2putils.CountStream(sender.Host(), receiver.ID(), defaultProtocolID, network.DirOutbound) - // t.Logf("sender %d has %d streams", i, actualNumOfStreams) - // assert.Equalf(t, - // int64(maxInboundStreamPerPeer-1), - // int64(actualNumOfStreams), - // "expected to create %d number of streams got %d", - // int64(maxInboundStreamPerPeer-1), - // actualNumOfStreams) - // totalStreams += actualNumOfStreams - // } - // - // // when system-wide limit is not enforced, the total number of streams created by all senders should be greater than the system-wide limit. - // require.Greaterf(t, - // totalStreams, - // l.ToPartialLimitConfig().Stream.StreamsInbound, - // "expected to create more than %d number of streams got %d", - // l.ToPartialLimitConfig().Stream.StreamsInbound, - // totalStreams) - // - // require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { - // t.Logf("protocol scope for %s", defaultProtocolID) - // t.Logf("inbound stream count %d", scope.Stat().NumStreamsInbound) - // return nil - // })) - // - // require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - // t.Logf("system scope") - // t.Logf("inbound stream count %d", scope.Stat().NumStreamsInbound) - // return nil - // })) +func TestCreateStream_InboundStreamResourceLimit(t *testing.T) { + t.Run("loose-hierarchical-structure", func(t *testing.T) { + // loose hierarchical structure refers to case where maxInboundStreamSystem = maxInboundStreamTransient = maxInboundStreamProtocol > maxInboundStreamPeerProtocol = maxInboundPeerStream. + testCreateStreamInboundStreamResourceLimits(t, &testPeerLimitConfig{ + nodeCount: 10, // 10 nodes + maxInboundPeerStream: 5, // each can create 5 streams to the receiver + maxInboundStreamProtocol: 50, // which accounts for 50 streams from each sender to the receiver using the unicast protocol + maxInboundStreamPeerProtocol: 5, // each peer on using the unicast protocol can create 5 streams to the receiver + maxInboundStreamTransient: 50, // the total number of transient streams from all senders to the receiver is 50 + maxInboundStreamSystem: 50, // the total number of streams from all senders to the receiver is 50 + }) + }) } // TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management @@ -415,15 +137,7 @@ func TestCreateStream_ResourceAllocation(t *testing.T) { // - Then each sender creates x - 1 streams concurrently to the receiver. // - At the end of the test we ensure that the total number of streams created by all senders is greater than the system-wide limit; which should not be the case if the // system-wide limit is being enforced. -func TestCreateStream_PeerLimit_Enforced(t *testing.T) { - nodeCount := 10 - buff := 0 - maxStreamPerPeer := 5 - maxStreamProtocol := nodeCount * maxStreamPerPeer - maxStreamPeerProtocol := maxStreamPerPeer * maxStreamProtocol - maxTransient := nodeCount - maxSystemStream := nodeCount - +func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimitConfig) { idProvider := mockmodule.NewIdentityProvider(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -436,8 +150,7 @@ func TestCreateStream_PeerLimit_Enforced(t *testing.T) { require.NoError(t, err) senders, senderIds := p2ptest.NodesFixture(t, sporkID, - t.Name(), - nodeCount, + t.Name(), cfg.nodeCount, idProvider, p2ptest.WithResourceManager(resourceManagerSnd), p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond)) @@ -446,32 +159,32 @@ func TestCreateStream_PeerLimit_Enforced(t *testing.T) { limits := rcmgr.DefaultLimits libp2p.SetDefaultServiceLimits(&limits) l := limits.Scale(0, 0) - cfg := rcmgr.PartialLimitConfig{ + partial := rcmgr.PartialLimitConfig{ System: rcmgr.ResourceLimits{ - StreamsInbound: rcmgr.LimitVal(maxSystemStream), - ConnsInbound: rcmgr.LimitVal(nodeCount), + StreamsInbound: rcmgr.LimitVal(cfg.maxInboundStreamSystem), + ConnsInbound: rcmgr.LimitVal(cfg.nodeCount), }, Transient: rcmgr.ResourceLimits{ - ConnsInbound: rcmgr.LimitVal(nodeCount), - StreamsInbound: rcmgr.LimitVal(maxTransient), + ConnsInbound: rcmgr.LimitVal(cfg.nodeCount), + StreamsInbound: rcmgr.LimitVal(cfg.maxInboundStreamTransient), }, ProtocolDefault: rcmgr.ResourceLimits{ - StreamsInbound: rcmgr.LimitVal(maxStreamProtocol + buff), + StreamsInbound: rcmgr.LimitVal(cfg.maxInboundStreamProtocol), }, ProtocolPeerDefault: rcmgr.ResourceLimits{ - StreamsInbound: rcmgr.LimitVal(maxStreamPeerProtocol + buff), + StreamsInbound: rcmgr.LimitVal(cfg.maxInboundStreamPeerProtocol), }, PeerDefault: rcmgr.ResourceLimits{ - StreamsInbound: rcmgr.LimitVal(maxStreamPerPeer + buff), + StreamsInbound: rcmgr.LimitVal(cfg.maxInboundPeerStream), }, Conn: rcmgr.ResourceLimits{ - StreamsInbound: rcmgr.LimitVal(maxStreamPerPeer + buff), + StreamsInbound: rcmgr.LimitVal(cfg.maxInboundPeerStream), }, Stream: rcmgr.ResourceLimits{ - StreamsInbound: rcmgr.LimitVal(maxStreamPerPeer + buff), + StreamsInbound: rcmgr.LimitVal(cfg.maxInboundPeerStream), }, } - l = cfg.Build(l) + l = partial.Build(l) resourceManagerRcv, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l)) require.NoError(t, err) receiver, id2 := p2ptest.NodeFixture(t, @@ -497,11 +210,11 @@ func TestCreateStream_PeerLimit_Enforced(t *testing.T) { var allStreamsCreated sync.WaitGroup defaultProtocolID := protocols.FlowProtocolID(sporkID) - // creates max(maxStreamPerPeer * nodeCount, maxSystemStream) streams from each sender to the receiver; breaks as soon as the system-wide limit is reached. + // creates max(maxInboundPeerStream * nodeCount, maxInboundStreamSystem) streams from each sender to the receiver; breaks as soon as the system-wide limit is reached. totalStreamsCreated := int64(0) for sIndex := range senders { - for i := int64(0); i < int64(maxStreamPerPeer); i++ { - if i >= int64(maxSystemStream) { + for i := int64(0); i < int64(cfg.maxInboundPeerStream); i++ { + if i >= int64(cfg.maxInboundStreamSystem) { // we reached the system-wide limit; no need to create more streams; as stream creation may fail; we re-examine pressure on system-wide limit later. break } @@ -520,7 +233,7 @@ func TestCreateStream_PeerLimit_Enforced(t *testing.T) { require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { // number of in-transient streams must be less than the max transient limit - require.Less(t, int64(scope.Stat().NumStreamsInbound), int64(maxTransient)) + require.Less(t, int64(scope.Stat().NumStreamsInbound), int64(cfg.maxInboundStreamTransient)) // number of in-transient streams must be less than or equal the total number of streams created. require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(totalStreamsCreated)) @@ -537,57 +250,72 @@ func TestCreateStream_PeerLimit_Enforced(t *testing.T) { for _, sender := range senders { actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) // t.Logf("sender %d has %d streams", i, actualNumOfStreams) - require.LessOrEqual(t, int64(actualNumOfStreams), int64(maxStreamPerPeer)) + require.LessOrEqual(t, int64(actualNumOfStreams), int64(cfg.maxInboundPeerStream)) totalInboundStreams += actualNumOfStreams } // sanity check; the total number of inbound streams must be less than or equal to the system-wide limit. // TODO: this must be a hard equal check; but falls short; to be shared with libp2p community. // Failing at this line means the system-wide limit is not being enforced. - require.LessOrEqual(t, totalInboundStreams, maxSystemStream) + require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamSystem) - // now the test goes beyond the protocol-peer limit and tries to create one more stream from each sender. - // this should cause receiver to close all streams from the sender and disconnect from the sender. + // now the stress testing with each sender making `maxInboundStreamSystem` concurrent streams to the receiver. for sIndex := range senders { - for i := int64(0); i < int64(100); i++ { + for i := int64(0); i < int64(cfg.maxInboundStreamSystem); i++ { allStreamsCreated.Add(1) go func(sIndex int) { defer allStreamsCreated.Done() sender := senders[sIndex] + // we don't care about the error here; as we are trying to create more streams than the system-wide limit; so we expect some of the stream creations to fail. _, _ = sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) }(sIndex) } } - unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") + unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create (stress-testing) streams on time") - t.Log("-----") - total := 0 - for i, sender := range senders { + totalInboundStreams = 0 + for _, sender := range senders { actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) - t.Logf("sender %d has %d streams", i, actualNumOfStreams) - // require.Equalf(t, - // int64(0), - // int64(actualNumOfStreams), - // "expected to create %d number of streams got %d", - // int64(0), - // actualNumOfStreams) - total += actualNumOfStreams + // t.Logf("sender %d has %d streams", i, actualNumOfStreams) + require.LessOrEqual(t, actualNumOfStreams, cfg.maxInboundPeerStream) + require.LessOrEqual(t, actualNumOfStreams, cfg.maxInboundStreamPeerProtocol) + totalInboundStreams += actualNumOfStreams } - - require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { - t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { - t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - return nil - })) - - t.Logf("total streams %d", total) + // sanity check; the total number of inbound streams must be less than or equal to the system-wide limit. + // TODO: this must be a hard equal check; but falls short; to be shared with libp2p community. + // Failing at this line means the system-wide limit is not being enforced. + require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamSystem) + require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamTransient) + + // t.Log("-----") + // total := 0 + // for i, sender := range senders { + // actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) + // t.Logf("sender %d has %d streams", i, actualNumOfStreams) + // // require.Equalf(t, + // // int64(0), + // // int64(actualNumOfStreams), + // // "expected to create %d number of streams got %d", + // // int64(0), + // // actualNumOfStreams) + // total += actualNumOfStreams + // } + // + // require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { + // t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + // return nil + // })) + // + // require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { + // t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + // return nil + // })) + // + // require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { + // t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + // return nil + // })) + // + // t.Logf("total streams %d", total) + // } } From 2aff7d4c514ac4a573698fe52666f0e6c7b1b5ec Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 11:40:11 -0700 Subject: [PATCH 11/68] adds test for peer limits greater than system limit --- network/p2p/p2pnode/resourceManager_test.go | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index beaeb23b99a..37177bc3084 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -119,6 +119,17 @@ func TestCreateStream_InboundStreamResourceLimit(t *testing.T) { maxInboundStreamSystem: 50, // the total number of streams from all senders to the receiver is 50 }) }) + t.Run("peer-limit-greater-than-system", func(t *testing.T) { + // the case where peer and protocol-level limits are higher than the system-wide limit. + testCreateStreamInboundStreamResourceLimits(t, &testPeerLimitConfig{ + nodeCount: 10, + maxInboundPeerStream: 500, + maxInboundStreamProtocol: 500, + maxInboundStreamPeerProtocol: 500, + maxInboundStreamTransient: 500, + maxInboundStreamSystem: 5, + }) + }) } // TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management @@ -211,10 +222,12 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi defaultProtocolID := protocols.FlowProtocolID(sporkID) // creates max(maxInboundPeerStream * nodeCount, maxInboundStreamSystem) streams from each sender to the receiver; breaks as soon as the system-wide limit is reached. - totalStreamsCreated := int64(0) + totalStreamsCreated := int64(0) // total number of streams successfully created. + totalStreamAttempted := int64(0) // total number of stream creation attempts. for sIndex := range senders { for i := int64(0); i < int64(cfg.maxInboundPeerStream); i++ { - if i >= int64(cfg.maxInboundStreamSystem) { + totalStreamAttempted++ + if totalStreamAttempted >= int64(cfg.maxInboundStreamSystem) { // we reached the system-wide limit; no need to create more streams; as stream creation may fail; we re-examine pressure on system-wide limit later. break } From 6027558662a89e37f9c0b099cbb5e39b07c6f0a9 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Thu, 19 Oct 2023 14:14:25 -0500 Subject: [PATCH 12/68] lint fix --- engine/common/synchronization/config.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/engine/common/synchronization/config.go b/engine/common/synchronization/config.go index 42dc99a08f4..14eb630b87a 100644 --- a/engine/common/synchronization/config.go +++ b/engine/common/synchronization/config.go @@ -3,9 +3,10 @@ package synchronization import ( "context" "fmt" + "time" + "github.com/onflow/flow-go/config" "github.com/onflow/flow-go/module/irrecoverable" - "time" core "github.com/onflow/flow-go/module/chainsync" ) From 185358c68096b088267d7bf0f5ad3f1b69e91af0 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 14:12:51 -0700 Subject: [PATCH 13/68] implements all test cases --- network/internal/p2putils/utils.go | 6 +- network/p2p/p2pnode/resourceManager_test.go | 249 ++++++++++++++------ 2 files changed, 183 insertions(+), 72 deletions(-) diff --git a/network/internal/p2putils/utils.go b/network/internal/p2putils/utils.go index f5bc4f01e4c..744dd183350 100644 --- a/network/internal/p2putils/utils.go +++ b/network/internal/p2putils/utils.go @@ -133,11 +133,11 @@ func All() FilterOption { // It returns a slice of network.Stream values that match the filtering criteria. func FilterStream(host host.Host, targetID peer.ID, options ...FilterOption) []network.Stream { var filteredStreams []network.Stream - + const discardTheProtocol = "discard-the-protocol" // default values opts := FilterOptions{ dir: network.DirUnknown, // by default, consider both inbound and outbound streams - protocol: "", // by default, consider streams of all protocol IDs + protocol: discardTheProtocol, // by default, consider streams of all protocol IDs all: false, // by default, return just the first matching stream } @@ -155,7 +155,7 @@ func FilterStream(host host.Host, targetID peer.ID, options ...FilterOption) []n streams := conn.GetStreams() for _, stream := range streams { if (opts.dir == network.DirUnknown || stream.Stat().Direction == opts.dir) && - (opts.protocol == "" || stream.Protocol() == opts.protocol) { + (opts.protocol == discardTheProtocol || stream.Protocol() == opts.protocol) { filteredStreams = append(filteredStreams, stream) if !opts.all { return filteredStreams diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 37177bc3084..b729b5f6892 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -3,8 +3,8 @@ package p2pnode_test import ( "context" "fmt" + "math" "sync" - "sync/atomic" "testing" "time" @@ -105,31 +105,163 @@ type testPeerLimitConfig struct { // maxInboundStreamSystem is the maximum number of inbound streams at the receiver; it accumulates all streams from all senders across all protocols. maxInboundStreamSystem int + + // unknownProtocol when set to true will cause senders to use an unknown protocol ID when creating streams. + unknownProtocol bool } -func TestCreateStream_InboundStreamResourceLimit(t *testing.T) { - t.Run("loose-hierarchical-structure", func(t *testing.T) { - // loose hierarchical structure refers to case where maxInboundStreamSystem = maxInboundStreamTransient = maxInboundStreamProtocol > maxInboundStreamPeerProtocol = maxInboundPeerStream. - testCreateStreamInboundStreamResourceLimits(t, &testPeerLimitConfig{ - nodeCount: 10, // 10 nodes - maxInboundPeerStream: 5, // each can create 5 streams to the receiver - maxInboundStreamProtocol: 50, // which accounts for 50 streams from each sender to the receiver using the unicast protocol - maxInboundStreamPeerProtocol: 5, // each peer on using the unicast protocol can create 5 streams to the receiver - maxInboundStreamTransient: 50, // the total number of transient streams from all senders to the receiver is 50 - maxInboundStreamSystem: 50, // the total number of streams from all senders to the receiver is 50 - }) - }) - t.Run("peer-limit-greater-than-system", func(t *testing.T) { - // the case where peer and protocol-level limits are higher than the system-wide limit. - testCreateStreamInboundStreamResourceLimits(t, &testPeerLimitConfig{ - nodeCount: 10, - maxInboundPeerStream: 500, - maxInboundStreamProtocol: 500, - maxInboundStreamPeerProtocol: 500, - maxInboundStreamTransient: 500, - maxInboundStreamSystem: 5, - }) - }) +// maxLimit returns the maximum limit across all limits. +func (t testPeerLimitConfig) maxLimit() int { + max := t.maxInboundPeerStream + if t.maxInboundStreamProtocol > max { + max = t.maxInboundStreamProtocol + } + if t.maxInboundStreamPeerProtocol > max { + max = t.maxInboundStreamPeerProtocol + } + if t.maxInboundStreamTransient > max { + max = t.maxInboundStreamTransient + } + if t.maxInboundStreamSystem > max { + max = t.maxInboundStreamSystem + } + return max +} + +// baseCreateStreamInboundStreamResourceLimitConfig returns a testPeerLimitConfig with default values. +func baseCreateStreamInboundStreamResourceLimitConfig() *testPeerLimitConfig { + return &testPeerLimitConfig{ + nodeCount: 10, + maxInboundPeerStream: 100, + maxInboundStreamProtocol: 100, + maxInboundStreamPeerProtocol: 100, + maxInboundStreamTransient: 100, + maxInboundStreamSystem: 100, + } +} + +func TestCreateStream_DefaultConfig(t *testing.T) { + testCreateStreamInboundStreamResourceLimits(t, baseCreateStreamInboundStreamResourceLimitConfig()) +} + +func TestCreateStream_MinPeerLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundPeerStream = 1 + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MaxPeerLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundPeerStream = math.MaxInt + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MinProtocolLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamProtocol = 1 + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MaxProtocolLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamProtocol = math.MaxInt + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MinPeerProtocolLimit(t *testing.T) { + unittest.SkipUnless(t, + unittest.TEST_TODO, + "max inbound stream peer protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamPeerProtocol = 1 + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MaxPeerProtocolLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamPeerProtocol = math.MaxInt + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MinTransientLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamTransient = 1 + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MaxTransientLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamTransient = math.MaxInt + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MinSystemLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamSystem = 1 + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_MaxSystemLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamSystem = math.MaxInt + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_DefaultConfigWithUnknownProtocol(t *testing.T) { + unittest.SkipUnless(t, + unittest.TEST_TODO, + "limits are not enforced when using an unknown protocol ID") + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.unknownProtocol = true + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_PeerLimitLessThanPeerProtocolLimit(t *testing.T) { + // the case where peer-level limit is lower than the peer-protocol-level limit. + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundPeerStream = 5 // each peer can only create 5 streams. + base.maxInboundStreamPeerProtocol = 10 // each peer can create 10 streams on a specific protocol (but should still be limited by the peer-level limit). + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_PeerLimitGreaterThanPeerProtocolLimit(t *testing.T) { + // the case where peer-level limit is higher than the peer-protocol-level limit. + unittest.SkipUnless(t, + unittest.TEST_TODO, + "max inbound stream peer protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundPeerStream = 10 // each peer can create 10 streams. + base.maxInboundStreamPeerProtocol = 5 // each peer can create 5 streams on a specific protocol. + base.maxInboundStreamProtocol = 100 // overall limit is 100 streams on a specific protocol (across all peers). + base.maxInboundStreamTransient = 1000 // overall limit is 1000 transient streams. + base.maxInboundStreamSystem = 1000 // overall limit is 1000 system-wide streams. + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_ProtocolLimitLessThanPeerProtocolLimit(t *testing.T) { + // the case where protocol-level limit is lower than the peer-protocol-level limit. + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamProtocol = 5 // each peer can create 5 streams on a specific protocol. + base.maxInboundStreamPeerProtocol = 10 // each peer can create 10 streams on a specific protocol (but should still be limited by the protocol-level limit). + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_ProtocolLimitGreaterThanPeerProtocolLimit(t *testing.T) { + // the case where protocol-level limit is higher than the peer-protocol-level limit. + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamProtocol = 10 // overall limit is 10 streams on a specific protocol (across all peers). + base.maxInboundStreamPeerProtocol = 5 // each peer can create 5 streams on a specific protocol. + base.maxInboundStreamTransient = 1000 // overall limit is 1000 transient streams. + base.maxInboundStreamSystem = 1000 // overall limit is 1000 system-wide streams. + testCreateStreamInboundStreamResourceLimits(t, base) +} + +func TestCreateStream_TransientLimitLessThanPeerProtocolLimit(t *testing.T) { + // the case where transient-level limit is lower than the peer-protocol-level limit. + base := baseCreateStreamInboundStreamResourceLimitConfig() + base.maxInboundStreamTransient = 5 // overall limit is 5 transient streams (across all peers). + base.maxInboundStreamPeerProtocol = 10 // each peer can create 10 streams on a specific protocol (but should still be limited by the transient-level limit). + testCreateStreamInboundStreamResourceLimits(t, base) } // TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management @@ -219,13 +351,19 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi p2ptest.LetNodesDiscoverEachOther(t, signalerCtx, nodes, ids) var allStreamsCreated sync.WaitGroup - defaultProtocolID := protocols.FlowProtocolID(sporkID) - // creates max(maxInboundPeerStream * nodeCount, maxInboundStreamSystem) streams from each sender to the receiver; breaks as soon as the system-wide limit is reached. - totalStreamsCreated := int64(0) // total number of streams successfully created. + protocolID := protocols.FlowProtocolID(sporkID) + if cfg.unknownProtocol { + protocolID = protocols.FlowProtocolID(unittest.IdentifierFixture()) + } + + // creates max(maxInboundStreamPeerProtocol * nodeCount, maxInboundStreamSystem) streams from each sender to the receiver; breaks as soon as the system-wide limit is reached. totalStreamAttempted := int64(0) // total number of stream creation attempts. + + streamListMu := sync.Mutex{} // mutex to protect the streamsList. + streamsList := make([]network.Stream, 0) // list of all streams created to avoid garbage collection. for sIndex := range senders { - for i := int64(0); i < int64(cfg.maxInboundPeerStream); i++ { + for i := int64(0); i < int64(cfg.maxInboundStreamPeerProtocol); i++ { totalStreamAttempted++ if totalStreamAttempted >= int64(cfg.maxInboundStreamSystem) { // we reached the system-wide limit; no need to create more streams; as stream creation may fail; we re-examine pressure on system-wide limit later. @@ -235,9 +373,15 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi go func(sIndex int) { defer allStreamsCreated.Done() sender := senders[sIndex] - _, err := sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) - require.NoError(t, err, "error creating stream") - atomic.AddInt64(&totalStreamsCreated, 1) + s, err := sender.Host().NewStream(ctx, receiver.ID(), protocolID) + if err != nil { + return + } + + require.NotNil(t, s) + streamListMu.Lock() + streamsList = append(streamsList, s) + streamListMu.Unlock() }(sIndex) } } @@ -245,11 +389,11 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { - // number of in-transient streams must be less than the max transient limit - require.Less(t, int64(scope.Stat().NumStreamsInbound), int64(cfg.maxInboundStreamTransient)) + // number of in-transient streams must be less than or equal to the max transient limit + require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(cfg.maxInboundStreamTransient)) // number of in-transient streams must be less than or equal the total number of streams created. - require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(totalStreamsCreated)) + require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(len(streamsList))) // t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) return nil })) @@ -279,7 +423,7 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi defer allStreamsCreated.Done() sender := senders[sIndex] // we don't care about the error here; as we are trying to create more streams than the system-wide limit; so we expect some of the stream creations to fail. - _, _ = sender.Host().NewStream(ctx, receiver.ID(), defaultProtocolID) + _, _ = sender.Host().NewStream(ctx, receiver.ID(), protocolID) }(sIndex) } } @@ -288,8 +432,7 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi totalInboundStreams = 0 for _, sender := range senders { - actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) - // t.Logf("sender %d has %d streams", i, actualNumOfStreams) + actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound), p2putils.Protocol("")) require.LessOrEqual(t, actualNumOfStreams, cfg.maxInboundPeerStream) require.LessOrEqual(t, actualNumOfStreams, cfg.maxInboundStreamPeerProtocol) totalInboundStreams += actualNumOfStreams @@ -299,36 +442,4 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi // Failing at this line means the system-wide limit is not being enforced. require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamSystem) require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamTransient) - - // t.Log("-----") - // total := 0 - // for i, sender := range senders { - // actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) - // t.Logf("sender %d has %d streams", i, actualNumOfStreams) - // // require.Equalf(t, - // // int64(0), - // // int64(actualNumOfStreams), - // // "expected to create %d number of streams got %d", - // // int64(0), - // // actualNumOfStreams) - // total += actualNumOfStreams - // } - // - // require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { - // t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - // return nil - // })) - // - // require.NoError(t, resourceManagerRcv.ViewProtocol(defaultProtocolID, func(scope network.ProtocolScope) error { - // t.Logf("protocol scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - // return nil - // })) - // - // require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - // t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) - // return nil - // })) - // - // t.Logf("total streams %d", total) - // } } From f501b7287a23fb181d29d29f0a46fe1ac062eed7 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 15:32:07 -0700 Subject: [PATCH 14/68] cleans up the tests --- network/p2p/p2pnode/resourceManager_test.go | 95 ++++++++------------- 1 file changed, 35 insertions(+), 60 deletions(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index b729b5f6892..d7ca60bee40 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -112,17 +112,20 @@ type testPeerLimitConfig struct { // maxLimit returns the maximum limit across all limits. func (t testPeerLimitConfig) maxLimit() int { - max := t.maxInboundPeerStream - if t.maxInboundStreamProtocol > max { + max := 0 + if t.maxInboundPeerStream > max && t.maxInboundPeerStream != math.MaxInt { + max = t.maxInboundPeerStream + } + if t.maxInboundStreamProtocol > max && t.maxInboundStreamProtocol != math.MaxInt { max = t.maxInboundStreamProtocol } - if t.maxInboundStreamPeerProtocol > max { + if t.maxInboundStreamPeerProtocol > max && t.maxInboundStreamPeerProtocol != math.MaxInt { max = t.maxInboundStreamPeerProtocol } - if t.maxInboundStreamTransient > max { + if t.maxInboundStreamTransient > max && t.maxInboundStreamTransient != math.MaxInt { max = t.maxInboundStreamTransient } - if t.maxInboundStreamSystem > max { + if t.maxInboundStreamSystem > max && t.maxInboundStreamSystem != math.MaxInt { max = t.maxInboundStreamSystem } return max @@ -157,6 +160,9 @@ func TestCreateStream_MaxPeerLimit(t *testing.T) { } func TestCreateStream_MinProtocolLimit(t *testing.T) { + unittest.SkipUnless(t, + unittest.TEST_TODO, + "max inbound stream protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamProtocol = 1 testCreateStreamInboundStreamResourceLimits(t, base) @@ -202,6 +208,9 @@ func TestCreateStream_MinSystemLimit(t *testing.T) { } func TestCreateStream_MaxSystemLimit(t *testing.T) { + unittest.SkipUnless(t, + unittest.TEST_TODO, + "max inbound stream protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamSystem = math.MaxInt testCreateStreamInboundStreamResourceLimits(t, base) @@ -239,6 +248,9 @@ func TestCreateStream_PeerLimitGreaterThanPeerProtocolLimit(t *testing.T) { } func TestCreateStream_ProtocolLimitLessThanPeerProtocolLimit(t *testing.T) { + unittest.SkipUnless(t, + unittest.TEST_TODO, + "max inbound stream peer protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") // the case where protocol-level limit is lower than the peer-protocol-level limit. base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamProtocol = 5 // each peer can create 5 streams on a specific protocol. @@ -264,22 +276,13 @@ func TestCreateStream_TransientLimitLessThanPeerProtocolLimit(t *testing.T) { testCreateStreamInboundStreamResourceLimits(t, base) } -// TestCreateStream_SystemStreamLimit_NotEnforced is a re-production of a hypothetical bug where the system-wide inbound stream limit of libp2p resource management -// was not being enforced. The purpose of this test is to share with the libp2p community as well as to evaluate the existence of the bug on -// future libp2p versions. -// Test scenario works as follows: -// - We have 30 senders and 1 receiver. -// - The senders are running with a resource manager that allows infinite number of streams; so that they can create as many streams as they want. -// - The receiver is running with a resource manager with base limits and no scaling. -// - The test reads the peer protocol default limits for inbound streams at receiver; say x; which is the limit for the number of inbound streams from each sender on a -// specific protocol. -// - Each sender creates x-1 streams to the receiver on a specific protocol. This is done to ensure that the receiver has x-1 streams from each sender; a total of -// 30*(x-1) streams at the receiver. -// - Test first ensures that numerically 30 * (x - 1) > max system-wide inbound stream limit; i.e., the total number of streams created by all senders is greater than -// the system-wide limit. -// - Then each sender creates x - 1 streams concurrently to the receiver. -// - At the end of the test we ensure that the total number of streams created by all senders is greater than the system-wide limit; which should not be the case if the -// system-wide limit is being enforced. +// testCreateStreamInboundStreamResourceLimits tests the inbound stream limits for a given testPeerLimitConfig. It creates +// a number of senders and a single receiver. The receiver will have a resource manager with the given limits. +// The senders will have a resource manager with infinite limits to ensure that they can create as many streams as they want. +// The test will create a number of streams from each sender to the receiver. The test will then check that the limits are +// being enforced correctly. +// The number of streams is determined by the maxLimit() of the testPeerLimitConfig, which is the maximum limit across all limits (peer, protocol, transient, system), excluding +// the math.MaxInt limits. func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimitConfig) { idProvider := mockmodule.NewIdentityProvider(t) ctx, cancel := context.WithCancel(context.Background()) @@ -357,24 +360,20 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi protocolID = protocols.FlowProtocolID(unittest.IdentifierFixture()) } - // creates max(maxInboundStreamPeerProtocol * nodeCount, maxInboundStreamSystem) streams from each sender to the receiver; breaks as soon as the system-wide limit is reached. - totalStreamAttempted := int64(0) // total number of stream creation attempts. + loadLimit := cfg.maxLimit() + require.Greaterf(t, loadLimit, 0, "test limit must be greater than 0; got %d", loadLimit) streamListMu := sync.Mutex{} // mutex to protect the streamsList. streamsList := make([]network.Stream, 0) // list of all streams created to avoid garbage collection. for sIndex := range senders { - for i := int64(0); i < int64(cfg.maxInboundStreamPeerProtocol); i++ { - totalStreamAttempted++ - if totalStreamAttempted >= int64(cfg.maxInboundStreamSystem) { - // we reached the system-wide limit; no need to create more streams; as stream creation may fail; we re-examine pressure on system-wide limit later. - break - } + for i := 0; i < loadLimit; i++ { allStreamsCreated.Add(1) go func(sIndex int) { defer allStreamsCreated.Done() sender := senders[sIndex] s, err := sender.Host().NewStream(ctx, receiver.ID(), protocolID) if err != nil { + // we don't care about the error here; as we are trying to break a limit; so we expect some of the stream creations to fail. return } @@ -388,58 +387,34 @@ func testCreateStreamInboundStreamResourceLimits(t *testing.T, cfg *testPeerLimi unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create streams on time") + // transient sanity-check require.NoError(t, resourceManagerRcv.ViewTransient(func(scope network.ResourceScope) error { // number of in-transient streams must be less than or equal to the max transient limit require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(cfg.maxInboundStreamTransient)) // number of in-transient streams must be less than or equal the total number of streams created. require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(len(streamsList))) - // t.Logf("transient scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) return nil })) + // system-wide limit sanity-check require.NoError(t, resourceManagerRcv.ViewSystem(func(scope network.ResourceScope) error { - t.Logf("system scope; inbound stream count %d; inbound connections; %d", scope.Stat().NumStreamsInbound, scope.Stat().NumConnsInbound) + require.LessOrEqual(t, int64(scope.Stat().NumStreamsInbound), int64(cfg.maxInboundStreamSystem), "system-wide limit is not being enforced") return nil })) totalInboundStreams := 0 for _, sender := range senders { actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound)) - // t.Logf("sender %d has %d streams", i, actualNumOfStreams) + // number of inbound streams must be less than or equal to the peer-level limit for each sender. require.LessOrEqual(t, int64(actualNumOfStreams), int64(cfg.maxInboundPeerStream)) + require.LessOrEqual(t, int64(actualNumOfStreams), int64(cfg.maxInboundStreamPeerProtocol)) totalInboundStreams += actualNumOfStreams } // sanity check; the total number of inbound streams must be less than or equal to the system-wide limit. // TODO: this must be a hard equal check; but falls short; to be shared with libp2p community. // Failing at this line means the system-wide limit is not being enforced. require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamSystem) - - // now the stress testing with each sender making `maxInboundStreamSystem` concurrent streams to the receiver. - for sIndex := range senders { - for i := int64(0); i < int64(cfg.maxInboundStreamSystem); i++ { - allStreamsCreated.Add(1) - go func(sIndex int) { - defer allStreamsCreated.Done() - sender := senders[sIndex] - // we don't care about the error here; as we are trying to create more streams than the system-wide limit; so we expect some of the stream creations to fail. - _, _ = sender.Host().NewStream(ctx, receiver.ID(), protocolID) - }(sIndex) - } - } - - unittest.RequireReturnsBefore(t, allStreamsCreated.Wait, 2*time.Second, "could not create (stress-testing) streams on time") - - totalInboundStreams = 0 - for _, sender := range senders { - actualNumOfStreams := p2putils.CountStream(receiver.Host(), sender.ID(), p2putils.Direction(network.DirInbound), p2putils.Protocol("")) - require.LessOrEqual(t, actualNumOfStreams, cfg.maxInboundPeerStream) - require.LessOrEqual(t, actualNumOfStreams, cfg.maxInboundStreamPeerProtocol) - totalInboundStreams += actualNumOfStreams - } - // sanity check; the total number of inbound streams must be less than or equal to the system-wide limit. - // TODO: this must be a hard equal check; but falls short; to be shared with libp2p community. - // Failing at this line means the system-wide limit is not being enforced. - require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamSystem) - require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamTransient) + // sanity check; the total number of inbound streams must be less than or equal to the protocol-level limit. + require.LessOrEqual(t, totalInboundStreams, cfg.maxInboundStreamProtocol) } From 7d0a9a29acdb8fbd2390ed2373d452508989f20c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 19 Oct 2023 15:54:32 -0700 Subject: [PATCH 15/68] add optimization --- ledger/complete/ledger.go | 10 ++++++++++ ledger/partial/ledger.go | 9 +++++++++ ledger/partial/ledger_test.go | 17 +++++++++++++++++ 3 files changed, 36 insertions(+) diff --git a/ledger/complete/ledger.go b/ledger/complete/ledger.go index f56d63e415d..1af41e21ff0 100644 --- a/ledger/complete/ledger.go +++ b/ledger/complete/ledger.go @@ -198,6 +198,16 @@ func (l *Ledger) Get(query *ledger.Query) (values []ledger.Value, err error) { // Set updates the ledger given an update. // It returns the state after update and errors (if any) func (l *Ledger) Set(update *ledger.Update) (newState ledger.State, trieUpdate *ledger.TrieUpdate, err error) { + if update.Size() == 0 { + return update.State(), + &ledger.TrieUpdate{ + RootHash: ledger.RootHash(update.State()), + Paths: []ledger.Path{}, + Payloads: []*ledger.Payload{}, + }, + nil + } + start := time.Now() trieUpdate, err = pathfinder.UpdateToTrieUpdate(update, l.pathFinderVersion) diff --git a/ledger/partial/ledger.go b/ledger/partial/ledger.go index b084098a50b..33b3d141935 100644 --- a/ledger/partial/ledger.go +++ b/ledger/partial/ledger.go @@ -120,6 +120,15 @@ func (l *Ledger) Get(query *ledger.Query) (values []ledger.Value, err error) { // Set updates the ledger given an update // it returns the state after update and errors (if any) func (l *Ledger) Set(update *ledger.Update) (newState ledger.State, trieUpdate *ledger.TrieUpdate, err error) { + if update.Size() == 0 { + return update.State(), + &ledger.TrieUpdate{ + RootHash: ledger.RootHash(update.State()), + Paths: []ledger.Path{}, + Payloads: []*ledger.Payload{}, + }, + nil + } trieUpdate, err = pathfinder.UpdateToTrieUpdate(update, l.pathFinderVersion) if err != nil { diff --git a/ledger/partial/ledger_test.go b/ledger/partial/ledger_test.go index 57f12c5f1de..ac11b6930ba 100644 --- a/ledger/partial/ledger_test.go +++ b/ledger/partial/ledger_test.go @@ -16,6 +16,7 @@ import ( "github.com/onflow/flow-go/ledger/partial" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/utils/unittest" ) func TestFunctionalityWithCompleteTrie(t *testing.T) { @@ -152,3 +153,19 @@ func TestProofsForEmptyRegisters(t *testing.T) { require.Empty(t, results[0]) } + +func TestEmptyLedger(t *testing.T) { + l, err := complete.NewLedger(&fixtures.NoopWAL{}, 100, &metrics.NoopCollector{}, zerolog.Logger{}, complete.DefaultPathFinderVersion) + require.NoError(t, err) + + u, err := ledger.NewUpdate( + ledger.State(unittest.StateCommitmentFixture()), + []ledger.Key{}, + []ledger.Value{}, + ) + require.NoError(t, err) + newState, trieUpdate, err := l.Set(u) + require.NoError(t, err) + require.True(t, trieUpdate.IsEmpty()) + require.Equal(t, u.State(), newState) +} From 113e50aa79c9aeb13b2ec30de8e506ff653f068d Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 16:04:48 -0700 Subject: [PATCH 16/68] wires config --- config/default-config.yml | 13 +++ network/netconf/flags.go | 131 +++++++++++++++++++++++-------- network/p2p/p2pconf/gossipsub.go | 26 +++++- 3 files changed, 136 insertions(+), 34 deletions(-) diff --git a/config/default-config.yml b/config/default-config.yml index ba771885a1b..1809474485f 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -58,6 +58,19 @@ network-config: # Without this limit peers can end up in a state where there exists n number of connections per peer which # can lead to resource exhaustion of the libp2p node. libp2p-peer-base-limits-conns-inbound: 1 + # maximum number of inbound system-wide limit for streams, across all peers and protocols + # Note that streams are ephemeral and are created and destroyed intermittently. + libp2p-inbound-stream-limit-system: 15_000 + # maximum number of inbound transient limit for streams, across all streams that are not yet fully opened and associated with a protocol. + # Note that streams are ephemeral and are created and destroyed intermittently. + libp2p-inbound-stream-limit-transient: 15_000 + # maximum number of inbound limit for streams for each protocol across all peers; this is a per-protocol limit. We expect at least + # three protocols per node; gossipsub, unicast, and dht. Note that streams are ephemeral and are created and destroyed intermittently. + libp2p-inbound-stream-limit-protocol: 5000 + # maximum number of inbound streams from each peer across all protocols. + libp2p-inbound-stream-limit-peer: 1000 + # maximum number of inbound streams from each peer for each protocol. + libp2p-inbound-stream-limit-protocol-peer: 500 # Connection manager config # HighWatermark and LowWatermark govern the number of connections are maintained by the ConnManager. # When the peer count exceeds the HighWatermark, as many peers will be pruned (and diff --git a/network/netconf/flags.go b/network/netconf/flags.go index 5dad90c65bd..9c53ca6c348 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -33,9 +33,15 @@ const ( bandwidthRateLimit = "unicast-bandwidth-rate-limit" bandwidthBurstLimit = "unicast-bandwidth-burst-limit" // resource manager config - memoryLimitRatio = "libp2p-memory-limit-ratio" - fileDescriptorsRatio = "libp2p-file-descriptors-ratio" - peerBaseLimitConnsInbound = "libp2p-peer-base-limits-conns-inbound" + memoryLimitRatio = "libp2p-memory-limit-ratio" + fileDescriptorsRatio = "libp2p-file-descriptors-ratio" + peerBaseLimitConnsInbound = "libp2p-peer-base-limits-conns-inbound" + inboundStreamLimitSystem = "libp2p-inbound-stream-limit-system" + inboundStreamLimitPeer = "libp2p-inbound-stream-limit-peer" + inboundStreamLimitProtocol = "libp2p-inbound-stream-limit-protocol" + inboundStreamLimitProtocolPeer = "libp2p-inbound-stream-limit-protocol-peer" + inboundStreamLimitTransient = "libp2p-inbound-stream-limit-transient" + // connection manager highWatermark = "libp2p-high-watermark" lowWatermark = "libp2p-low-watermark" @@ -101,6 +107,11 @@ func AllFlagNames() []string { memoryLimitRatio, fileDescriptorsRatio, peerBaseLimitConnsInbound, + inboundStreamLimitSystem, + inboundStreamLimitPeer, + inboundStreamLimitProtocol, + inboundStreamLimitProtocolPeer, + inboundStreamLimitTransient, highWatermark, lowWatermark, gracePeriod, @@ -148,28 +159,52 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Duration(peerUpdateInterval, config.PeerUpdateInterval, "how often to refresh the peer connections for the node") flags.Duration(unicastMessageTimeout, config.UnicastMessageTimeout, "how long a unicast transmission can take to complete") // unicast manager options - flags.Duration(unicastCreateStreamRetryDelay, config.UnicastConfig.CreateStreamBackoffDelay, "initial backoff delay between failing to establish a connection with another node and retrying, "+ - "this delay increases exponentially with the number of subsequent failures to establish a connection.") - flags.Duration(unicastDialBackoffDelay, config.UnicastConfig.DialInProgressBackoffDelay, "initial backoff delay between failing to establish a connection with another node and retrying, "+ - "this delay increases exponentially with the number of subsequent failures to establish a connection.") - flags.Duration(unicastDialInProgressBackoffDelay, config.UnicastConfig.DialInProgressBackoffDelay, "initial backoff delay for concurrent stream creations to a remote peer when there is no exising connection and a dial is in progress. "+ - "this delay increases exponentially with the number of subsequent failure attempts") - flags.Uint64(unicastStreamZeroRetryResetThreshold, config.UnicastConfig.StreamZeroRetryResetThreshold, "reset stream creation retry budget from zero to the maximum after consecutive successful streams reach this threshold.") - flags.Duration(unicastDialZeroRetryResetThreshold, config.UnicastConfig.DialZeroRetryResetThreshold, "reset dial retry budget if the last successful dial is longer than this threshold.") + flags.Duration(unicastCreateStreamRetryDelay, + config.UnicastConfig.CreateStreamBackoffDelay, + "initial backoff delay between failing to establish a connection with another node and retrying, "+ + "this delay increases exponentially with the number of subsequent failures to establish a connection.") + flags.Duration(unicastDialBackoffDelay, + config.UnicastConfig.DialInProgressBackoffDelay, + "initial backoff delay between failing to establish a connection with another node and retrying, "+ + "this delay increases exponentially with the number of subsequent failures to establish a connection.") + flags.Duration(unicastDialInProgressBackoffDelay, + config.UnicastConfig.DialInProgressBackoffDelay, + "initial backoff delay for concurrent stream creations to a remote peer when there is no exising connection and a dial is in progress. "+ + "this delay increases exponentially with the number of subsequent failure attempts") + flags.Uint64(unicastStreamZeroRetryResetThreshold, + config.UnicastConfig.StreamZeroRetryResetThreshold, + "reset stream creation retry budget from zero to the maximum after consecutive successful streams reach this threshold.") + flags.Duration(unicastDialZeroRetryResetThreshold, + config.UnicastConfig.DialZeroRetryResetThreshold, + "reset dial retry budget if the last successful dial is longer than this threshold.") flags.Uint64(unicastMaxDialRetryAttemptTimes, config.UnicastConfig.MaxDialRetryAttemptTimes, "maximum attempts to establish a unicast connection.") flags.Uint64(unicastMaxStreamCreationRetryAttemptTimes, config.UnicastConfig.MaxStreamCreationRetryAttemptTimes, "max attempts to create a unicast stream.") - flags.Uint32(unicastDialConfigCacheSize, config.UnicastConfig.DialConfigCacheSize, "cache size of the dial config cache, recommended to be big enough to accommodate the entire nodes in the network.") + flags.Uint32(unicastDialConfigCacheSize, + config.UnicastConfig.DialConfigCacheSize, + "cache size of the dial config cache, recommended to be big enough to accommodate the entire nodes in the network.") // unicast stream handler rate limits flags.Int(messageRateLimit, config.UnicastConfig.UnicastRateLimitersConfig.MessageRateLimit, "maximum number of unicast messages that a peer can send per second") - flags.Int(bandwidthRateLimit, config.UnicastConfig.UnicastRateLimitersConfig.BandwidthRateLimit, "bandwidth size in bytes a peer is allowed to send via unicast streams per second") + flags.Int(bandwidthRateLimit, + config.UnicastConfig.UnicastRateLimitersConfig.BandwidthRateLimit, + "bandwidth size in bytes a peer is allowed to send via unicast streams per second") flags.Int(bandwidthBurstLimit, config.UnicastConfig.UnicastRateLimitersConfig.BandwidthBurstLimit, "bandwidth size in bytes a peer is allowed to send at one time") - flags.Duration(lockoutDuration, config.UnicastConfig.UnicastRateLimitersConfig.LockoutDuration, "the number of seconds a peer will be forced to wait before being allowed to successful reconnect to the node after being rate limited") + flags.Duration(lockoutDuration, + config.UnicastConfig.UnicastRateLimitersConfig.LockoutDuration, + "the number of seconds a peer will be forced to wait before being allowed to successful reconnect to the node after being rate limited") flags.Bool(dryRun, config.UnicastConfig.UnicastRateLimitersConfig.DryRun, "disable peer disconnects and connections gating when rate limiting peers") // resource manager cli flags flags.Float64(fileDescriptorsRatio, config.ResourceManagerConfig.FileDescriptorsRatio, "ratio of available file descriptors to be used by libp2p (in (0,1])") flags.Float64(memoryLimitRatio, config.ResourceManagerConfig.MemoryLimitRatio, "ratio of available memory to be used by libp2p (in (0,1])") flags.Int(peerBaseLimitConnsInbound, config.ResourceManagerConfig.PeerBaseLimitConnsInbound, "the maximum amount of allowed inbound connections per peer") + flags.Int(inboundStreamLimitSystem, config.ResourceManagerConfig.InboundStream.System, "the system-wide limit on the number of inbound streams") + flags.Int(inboundStreamLimitPeer, config.ResourceManagerConfig.InboundStream.Peer, "the limit on the number of inbound streams per peer (over all protocols)") + flags.Int(inboundStreamLimitProtocol, config.ResourceManagerConfig.InboundStream.Protocol, "the limit on the number of inbound streams per protocol (over all peers)") + flags.Int(inboundStreamLimitProtocolPeer, config.ResourceManagerConfig.InboundStream.ProtocolPeer, "the limit on the number of inbound streams per protocol per peer") + flags.Int(inboundStreamLimitTransient, + config.ResourceManagerConfig.InboundStream.Transient, + "the transient limit on the number of inbound streams (applied to streams that are not associated with a peer or protocol yet)") + // connection manager flags.Int(lowWatermark, config.ConnectionManagerConfig.LowWatermark, "low watermarking for libp2p connection manager") flags.Int(highWatermark, config.ConnectionManagerConfig.HighWatermark, "high watermarking for libp2p connection manager") @@ -182,30 +217,64 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Uint32(rpcSentTrackerQueueCacheSize, config.GossipSubConfig.RPCSentTrackerQueueCacheSize, "cache size of the rpc sent tracker worker queue.") flags.Int(rpcSentTrackerNumOfWorkers, config.GossipSubConfig.RpcSentTrackerNumOfWorkers, "number of workers for the rpc sent tracker worker pool.") // gossipsub RPC control message validation limits used for validation configuration and rate limiting - flags.Int(validationInspectorNumberOfWorkers, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.NumberOfWorkers, "number of gossupsub RPC control message validation inspector component workers") - flags.Uint32(validationInspectorInspectMessageQueueCacheSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.CacheSize, "cache size for gossipsub RPC validation inspector events worker pool queue.") - flags.Uint32(validationInspectorClusterPrefixedTopicsReceivedCacheSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.ClusterPrefixedControlMsgsReceivedCacheSize, "cache size for gossipsub RPC validation inspector cluster prefix received tracker.") - flags.Float64(validationInspectorClusterPrefixedTopicsReceivedCacheDecay, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.ClusterPrefixedControlMsgsReceivedCacheDecay, "the decay value used to decay cluster prefix received topics received cached counters.") - flags.Float64(validationInspectorClusterPrefixHardThreshold, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.ClusterPrefixHardThreshold, "the maximum number of cluster-prefixed control messages allowed to be processed when the active cluster id is unset or a mismatch is detected, exceeding this threshold will result in node penalization by gossipsub.") + flags.Int(validationInspectorNumberOfWorkers, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.NumberOfWorkers, + "number of gossupsub RPC control message validation inspector component workers") + flags.Uint32(validationInspectorInspectMessageQueueCacheSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.CacheSize, + "cache size for gossipsub RPC validation inspector events worker pool queue.") + flags.Uint32(validationInspectorClusterPrefixedTopicsReceivedCacheSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.ClusterPrefixedControlMsgsReceivedCacheSize, + "cache size for gossipsub RPC validation inspector cluster prefix received tracker.") + flags.Float64(validationInspectorClusterPrefixedTopicsReceivedCacheDecay, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.ClusterPrefixedControlMsgsReceivedCacheDecay, + "the decay value used to decay cluster prefix received topics received cached counters.") + flags.Float64(validationInspectorClusterPrefixHardThreshold, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.ClusterPrefixHardThreshold, + "the maximum number of cluster-prefixed control messages allowed to be processed when the active cluster id is unset or a mismatch is detected, exceeding this threshold will result in node penalization by gossipsub.") // gossipsub RPC control message metrics observer inspector configuration - flags.Int(metricsInspectorNumberOfWorkers, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCMetricsInspectorConfigs.NumberOfWorkers, "cache size for gossipsub RPC metrics inspector events worker pool queue.") - flags.Uint32(metricsInspectorCacheSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCMetricsInspectorConfigs.CacheSize, "cache size for gossipsub RPC metrics inspector events worker pool.") + flags.Int(metricsInspectorNumberOfWorkers, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCMetricsInspectorConfigs.NumberOfWorkers, + "cache size for gossipsub RPC metrics inspector events worker pool queue.") + flags.Uint32(metricsInspectorCacheSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCMetricsInspectorConfigs.CacheSize, + "cache size for gossipsub RPC metrics inspector events worker pool.") // networking event notifications - flags.Uint32(gossipSubRPCInspectorNotificationCacheSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCInspectorNotificationCacheSize, "cache size for notification events from gossipsub rpc inspector") + flags.Uint32(gossipSubRPCInspectorNotificationCacheSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCInspectorNotificationCacheSize, + "cache size for notification events from gossipsub rpc inspector") // application layer spam prevention (alsp) protocol flags.Bool(alspDisabled, config.AlspConfig.DisablePenalty, "disable the penalty mechanism of the alsp protocol. default value (recommended) is false") flags.Uint32(alspSpamRecordCacheSize, config.AlspConfig.SpamRecordCacheSize, "size of spam record cache, recommended to be 10x the number of authorized nodes") flags.Uint32(alspSpamRecordQueueSize, config.AlspConfig.SpamReportQueueSize, "size of spam report queue, recommended to be 100x the number of authorized nodes") - flags.Duration(alspHearBeatInterval, config.AlspConfig.HearBeatInterval, "interval between two consecutive heartbeat events at alsp, recommended to leave it as default unless you know what you are doing.") + flags.Duration(alspHearBeatInterval, + config.AlspConfig.HearBeatInterval, + "interval between two consecutive heartbeat events at alsp, recommended to leave it as default unless you know what you are doing.") - flags.Int(ihaveMaxSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxSampleSize, "max number of ihaves to sample when performing validation") - flags.Int(ihaveMaxMessageIDSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxMessageIDSampleSize, "max number of message ids to sample when performing validation per ihave") - flags.Int(controlMessageMaxSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.GraftPruneMessageMaxSampleSize, "max number of control messages to sample when performing validation on GRAFT and PRUNE message types") - flags.Uint(iwantMaxSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.MaxSampleSize, "max number of iwants to sample when performing validation") - flags.Int(iwantMaxMessageIDSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.MaxMessageIDSampleSize, "max number of message ids to sample when performing validation per iwant") - flags.Float64(iwantCacheMissThreshold, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.CacheMissThreshold, "max number of iwants to sample when performing validation") - flags.Int(iwantCacheMissCheckSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.CacheMissCheckSize, "the iWants size at which message id cache misses will be checked") - flags.Float64(iwantDuplicateMsgIDThreshold, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.DuplicateMsgIDThreshold, "max allowed duplicate message IDs in a single iWant control message") + flags.Int(ihaveMaxSampleSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxSampleSize, + "max number of ihaves to sample when performing validation") + flags.Int(ihaveMaxMessageIDSampleSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxMessageIDSampleSize, + "max number of message ids to sample when performing validation per ihave") + flags.Int(controlMessageMaxSampleSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.GraftPruneMessageMaxSampleSize, + "max number of control messages to sample when performing validation on GRAFT and PRUNE message types") + flags.Uint(iwantMaxSampleSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.MaxSampleSize, + "max number of iwants to sample when performing validation") + flags.Int(iwantMaxMessageIDSampleSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.MaxMessageIDSampleSize, + "max number of message ids to sample when performing validation per iwant") + flags.Float64(iwantCacheMissThreshold, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.CacheMissThreshold, + "max number of iwants to sample when performing validation") + flags.Int(iwantCacheMissCheckSize, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.CacheMissCheckSize, + "the iWants size at which message id cache misses will be checked") + flags.Float64(iwantDuplicateMsgIDThreshold, + config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IWantRPCInspectionConfig.DuplicateMsgIDThreshold, + "max allowed duplicate message IDs in a single iWant control message") } // SetAliases this func sets an aliases for each CLI flag defined for network config overrides to it's corresponding diff --git a/network/p2p/p2pconf/gossipsub.go b/network/p2p/p2pconf/gossipsub.go index 683dff67fdc..62cde38ff73 100644 --- a/network/p2p/p2pconf/gossipsub.go +++ b/network/p2p/p2pconf/gossipsub.go @@ -8,9 +8,29 @@ import ( // The resource manager is used to limit the number of open connections and streams (as well as any other resources // used by libp2p) for each peer. type ResourceManagerConfig struct { - MemoryLimitRatio float64 `mapstructure:"libp2p-memory-limit-ratio"` // maximum allowed fraction of memory to be allocated by the libp2p resources in (0,1] - FileDescriptorsRatio float64 `mapstructure:"libp2p-file-descriptors-ratio"` // maximum allowed fraction of file descriptors to be allocated by the libp2p resources in (0,1] - PeerBaseLimitConnsInbound int `mapstructure:"libp2p-peer-base-limits-conns-inbound"` // the maximum amount of allowed inbound connections per peer + InboundStream InboundStreamLimit `mapstructure:",squash"` + MemoryLimitRatio float64 `mapstructure:"libp2p-memory-limit-ratio"` // maximum allowed fraction of memory to be allocated by the libp2p resources in (0,1] + FileDescriptorsRatio float64 `mapstructure:"libp2p-file-descriptors-ratio"` // maximum allowed fraction of file descriptors to be allocated by the libp2p resources in (0,1] + PeerBaseLimitConnsInbound int `mapstructure:"libp2p-peer-base-limits-conns-inbound"` // the maximum amount of allowed inbound connections per peer +} + +// InboundStreamLimit is the configuration for the inbound stream limit. The inbound stream limit is used to limit the +// number of inbound streams that can be opened by the node. +type InboundStreamLimit struct { + // the system-wide limit on the number of inbound streams + System int `validate:"gt=0" mapstructure:"libp2p-inbound-stream-limit-system"` + + // Transient is the transient limit on the number of inbound streams (applied to streams that are not associated with a peer or protocol yet) + Transient int `validate:"gt=0" mapstructure:"libp2p-inbound-stream-limit-transient"` + + // Protocol is the limit on the number of inbound streams per protocol (over all peers). + Protocol int `validate:"gt=0" mapstructure:"libp2p-inbound-stream-limit-protocol"` + + // Peer is the limit on the number of inbound streams per peer (over all protocols). + Peer int `validate:"gt=0" mapstructure:"libp2p-inbound-stream-limit-peer"` + + // ProtocolPeer is the limit on the number of inbound streams per protocol per peer. + ProtocolPeer int `validate:"gt=0" mapstructure:"libp2p-inbound-stream-limit-protocol-peer"` } // GossipSubConfig is the configuration for the GossipSub pubsub implementation. From 4745ab05793e4f04aefa60b405bd204eb40be9d6 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 16:19:25 -0700 Subject: [PATCH 17/68] adds utility function for applying inbound limits --- network/p2p/p2pbuilder/utils.go | 61 +++++++++++++++++++++++++++++++++ 1 file changed, 61 insertions(+) diff --git a/network/p2p/p2pbuilder/utils.go b/network/p2p/p2pbuilder/utils.go index 2e05bbd5b84..232ea571d7d 100644 --- a/network/p2p/p2pbuilder/utils.go +++ b/network/p2p/p2pbuilder/utils.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network/p2p/p2pconf" "github.com/onflow/flow-go/network/p2p/p2plogging" ) @@ -123,3 +124,63 @@ func (l *limitConfigLogger) logPeerProtocolLimits(p map[protocol.ID]rcmgr.Resour lg.Info().Msg("protocol peer limits set") } } + +// ApplyInboundStreamLimits applies the inbound stream limits to the concrete limit config. The concrete limit config is assumed coming from scaling the +// base limit config by the scaling factor. The inbound stream limits are applied to the concrete limit config if the concrete limit config is greater than +// the inbound stream limits. +// The inbound limits are assumed coming from the config file. +// Args: +// +// logger: the logger to log the applied limits. +// concrete: the concrete limit config. +// limit: the inbound stream limits. +// +// Returns: +// +// a copy of the concrete limit config with the inbound stream limits applied and overridden. +func ApplyInboundStreamLimits(logger zerolog.Logger, concrete rcmgr.ConcreteLimitConfig, limit p2pconf.InboundStreamLimit) rcmgr.ConcreteLimitConfig { + c := concrete.ToPartialLimitConfig() + + partial := rcmgr.PartialLimitConfig{} + lg := logger.With().Logger() + + if int(c.System.StreamsInbound) > limit.System { + lg = lg.With().Int("concrete_system_inbound_streams", int(c.System.StreamsInbound)).Int("partial_system_inbound_streams", limit.System).Logger() + partial.System.StreamsInbound = rcmgr.LimitVal(limit.System) + } + + if int(c.Transient.StreamsInbound) > limit.Transient { + lg = lg.With().Int("concrete_transient_inbound_streams", int(c.Transient.StreamsInbound)).Int("partial_transient_inbound_streams", limit.Transient).Logger() + partial.Transient.StreamsInbound = rcmgr.LimitVal(limit.Transient) + } + + if int(c.ProtocolDefault.StreamsInbound) > limit.Protocol { + lg = lg.With().Int("concrete_protocol_default_inbound_streams", int(c.ProtocolDefault.StreamsInbound)).Int("partial_protocol_default_inbound_streams", + limit.Protocol).Logger() + partial.ProtocolDefault.StreamsInbound = rcmgr.LimitVal(limit.Protocol) + } + + if int(c.PeerDefault.StreamsInbound) > limit.Peer { + lg = lg.With().Int("concrete_peer_default_inbound_streams", int(c.PeerDefault.StreamsInbound)).Int("partial_peer_default_inbound_streams", limit.Peer).Logger() + partial.PeerDefault.StreamsInbound = rcmgr.LimitVal(limit.Peer) + } + + if int(c.ProtocolPeerDefault.StreamsInbound) > limit.ProtocolPeer { + lg = lg.With().Int("concrete_protocol_peer_default_inbound_streams", + int(c.ProtocolPeerDefault.StreamsInbound)).Int("partial_protocol_peer_default_inbound_streams", limit.ProtocolPeer).Logger() + partial.ProtocolPeerDefault.StreamsInbound = rcmgr.LimitVal(limit.ProtocolPeer) + } + + if int(c.Stream.StreamsInbound) > limit.Peer { + lg = lg.With().Int("concrete_stream_inbound_streams", int(c.Stream.StreamsInbound)).Int("partial_stream_inbound_streams", limit.Peer).Logger() + partial.ServiceDefault.StreamsInbound = rcmgr.LimitVal(limit.Peer) + } + + if int(c.Conn.StreamsInbound) > limit.Peer { + lg = lg.With().Int("concrete_conn_inbound_streams", int(c.Conn.StreamsInbound)).Int("partial_conn_inbound_streams", limit.Peer).Logger() + partial.ServicePeerDefault.StreamsInbound = rcmgr.LimitVal(limit.Peer) + } + + lg.Info().Msg("inbound stream limits applied") + return partial.Build(concrete) +} From dae85dabd16d273d2e7ad2e818042e1ccf6b4509 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 16:48:08 -0700 Subject: [PATCH 18/68] adds tests --- network/p2p/p2pbuilder/libp2pNodeBuilder.go | 7 ++- network/p2p/p2pbuilder/libp2pscaler_test.go | 67 +++++++++++++++++++++ 2 files changed, 71 insertions(+), 3 deletions(-) diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index 5bbcf4f6ed3..ddac941a2b5 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -223,8 +223,9 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { return nil, fmt.Errorf("could not get allowed file descriptors: %w", err) } limits.PeerBaseLimit.ConnsInbound = builder.resourceManagerCfg.PeerBaseLimitConnsInbound - l := limits.Scale(mem, fd) - mgr, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(l), rcmgr.WithMetrics(builder.metricsConfig.Metrics)) + scaledLimits := limits.Scale(mem, fd) + appliedLimits := ApplyInboundStreamLimits(builder.logger, scaledLimits, builder.resourceManagerCfg.InboundStream) + mgr, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(appliedLimits), rcmgr.WithMetrics(builder.metricsConfig.Metrics)) if err != nil { return nil, fmt.Errorf("could not create libp2p resource manager: %w", err) } @@ -233,7 +234,7 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { Int64("allowed_memory", mem). Int("allowed_file_descriptors", fd). Msg("allowed memory and file descriptors are fetched from the system") - NewLimitConfigLogger(builder.logger).LogResourceManagerLimits(l) + NewLimitConfigLogger(builder.logger).LogResourceManagerLimits(appliedLimits) opts = append(opts, libp2p.ResourceManager(mgr)) builder.logger.Info().Msg("libp2p resource manager is set to default with metrics") diff --git a/network/p2p/p2pbuilder/libp2pscaler_test.go b/network/p2p/p2pbuilder/libp2pscaler_test.go index 094f8a4e700..a952193a715 100644 --- a/network/p2p/p2pbuilder/libp2pscaler_test.go +++ b/network/p2p/p2pbuilder/libp2pscaler_test.go @@ -3,8 +3,13 @@ package p2pbuilder import ( "testing" + "github.com/libp2p/go-libp2p" + rcmgr "github.com/libp2p/go-libp2p/p2p/host/resource-manager" "github.com/pbnjay/memory" "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/config" + "github.com/onflow/flow-go/utils/unittest" ) func TestAllowedMemoryScale(t *testing.T) { @@ -102,3 +107,65 @@ func TestAllowedFileDescriptorsScale(t *testing.T) { require.NoError(t, err) require.Equal(t, fd/10000, s) } + +// TestApplyInboundStreamLimits tests that the inbound stream limits are applied correctly, i.e., the limits from the config file +// are applied to the concrete limit config when the concrete limit config is greater than the limits from the config file. +func TestApplyInboundStreamLimits(t *testing.T) { + cfg, err := config.DefaultConfig() + require.NoError(t, err) + + mem, err := AllowedMemory(cfg.NetworkConfig.ResourceManagerConfig.MemoryLimitRatio) + require.NoError(t, err) + + fd, err := AllowedFileDescriptors(cfg.NetworkConfig.FileDescriptorsRatio) + require.NoError(t, err) + limits := rcmgr.DefaultLimits + libp2p.SetDefaultServiceLimits(&limits) + scaled := limits.Scale(mem, fd) + + concrete := rcmgr.PartialLimitConfig{ + System: rcmgr.ResourceLimits{ + // intentionally sets to 1 to test that it is not overridden. + StreamsInbound: 1, + }, + Transient: rcmgr.ResourceLimits{ + // sets it higher than the default to test that it is overridden. + StreamsInbound: rcmgr.LimitVal(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Transient + 1), + }, + ProtocolDefault: rcmgr.ResourceLimits{ + // sets it higher than the default to test that it is overridden. + StreamsInbound: rcmgr.LimitVal(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Protocol + 1), + }, + ProtocolPeerDefault: rcmgr.ResourceLimits{ + StreamsInbound: 1, // intentionally sets to 1 to test that it is not overridden. + }, + PeerDefault: rcmgr.ResourceLimits{ + StreamsInbound: rcmgr.LimitVal(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Peer + 1), + }, + Conn: rcmgr.ResourceLimits{ + StreamsInbound: 1, // intentionally sets to 1 to test that it is not overridden. + }, + Stream: rcmgr.ResourceLimits{ + StreamsInbound: 1, // intentionally sets to 1 to test that it is not overridden. + }, + }.Build(scaled) + + // apply inbound stream limits from the config file. + applied := ApplyInboundStreamLimits(unittest.Logger(), concrete, cfg.NetworkConfig.ResourceManagerConfig.InboundStream) + + // check that the applied limits are overridden. + // transient limit should be overridden. + require.Equal(t, int(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Transient), int(applied.ToPartialLimitConfig().Transient.StreamsInbound)) + // protocol default limit should be overridden. + require.Equal(t, int(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Protocol), int(applied.ToPartialLimitConfig().ProtocolDefault.StreamsInbound)) + // peer default limit should be overridden. + require.Equal(t, int(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Peer), int(applied.ToPartialLimitConfig().PeerDefault.StreamsInbound)) + // protocol peer default limit should not be overridden. + require.Equal(t, int(1), int(applied.ToPartialLimitConfig().ProtocolPeerDefault.StreamsInbound)) + // conn limit should not be overridden. + require.Equal(t, int(1), int(applied.ToPartialLimitConfig().Conn.StreamsInbound)) + // stream limit should not be overridden. + require.Equal(t, int(1), int(applied.ToPartialLimitConfig().Stream.StreamsInbound)) + // system limit should not be overridden. + require.Equal(t, int(1), int(applied.ToPartialLimitConfig().System.StreamsInbound)) +} From a99bcb9ac79f5fc32822d14ba24d425b7fef1e8f Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 19 Oct 2023 17:41:27 -0700 Subject: [PATCH 19/68] fixes tests --- network/p2p/p2pbuilder/libp2pNodeBuilder.go | 7 +++--- network/p2p/p2pbuilder/libp2pscaler_test.go | 10 +++++++- network/p2p/p2pbuilder/utils.go | 28 +++++++++++++++++++++ 3 files changed, 41 insertions(+), 4 deletions(-) diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index ddac941a2b5..3c768b44ad6 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -222,9 +222,10 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { if err != nil { return nil, fmt.Errorf("could not get allowed file descriptors: %w", err) } - limits.PeerBaseLimit.ConnsInbound = builder.resourceManagerCfg.PeerBaseLimitConnsInbound - scaledLimits := limits.Scale(mem, fd) - appliedLimits := ApplyInboundStreamLimits(builder.logger, scaledLimits, builder.resourceManagerCfg.InboundStream) + + // scales the default limits by the allowed memory and file descriptors and applies the inbound connection and stream limits. + appliedLimits := ApplyInboundConnectionLimits(builder.logger, limits.Scale(mem, fd), builder.resourceManagerCfg.PeerBaseLimitConnsInbound) + appliedLimits = ApplyInboundStreamLimits(builder.logger, appliedLimits, builder.resourceManagerCfg.InboundStream) mgr, err := rcmgr.NewResourceManager(rcmgr.NewFixedLimiter(appliedLimits), rcmgr.WithMetrics(builder.metricsConfig.Metrics)) if err != nil { return nil, fmt.Errorf("could not create libp2p resource manager: %w", err) diff --git a/network/p2p/p2pbuilder/libp2pscaler_test.go b/network/p2p/p2pbuilder/libp2pscaler_test.go index a952193a715..176b3a6bb3c 100644 --- a/network/p2p/p2pbuilder/libp2pscaler_test.go +++ b/network/p2p/p2pbuilder/libp2pscaler_test.go @@ -110,7 +110,7 @@ func TestAllowedFileDescriptorsScale(t *testing.T) { // TestApplyInboundStreamLimits tests that the inbound stream limits are applied correctly, i.e., the limits from the config file // are applied to the concrete limit config when the concrete limit config is greater than the limits from the config file. -func TestApplyInboundStreamLimits(t *testing.T) { +func TestApplyInboundStreamAndConnectionLimits(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) @@ -135,6 +135,8 @@ func TestApplyInboundStreamLimits(t *testing.T) { ProtocolDefault: rcmgr.ResourceLimits{ // sets it higher than the default to test that it is overridden. StreamsInbound: rcmgr.LimitVal(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Protocol + 1), + // intentionally sets it lower than the default to test that it is not overridden. + ConnsInbound: rcmgr.LimitVal(cfg.NetworkConfig.ResourceManagerConfig.PeerBaseLimitConnsInbound - 1), }, ProtocolPeerDefault: rcmgr.ResourceLimits{ StreamsInbound: 1, // intentionally sets to 1 to test that it is not overridden. @@ -153,6 +155,9 @@ func TestApplyInboundStreamLimits(t *testing.T) { // apply inbound stream limits from the config file. applied := ApplyInboundStreamLimits(unittest.Logger(), concrete, cfg.NetworkConfig.ResourceManagerConfig.InboundStream) + // then applies the peer base limit connections from the config file. + applied = ApplyInboundConnectionLimits(unittest.Logger(), applied, cfg.NetworkConfig.ResourceManagerConfig.PeerBaseLimitConnsInbound) + // check that the applied limits are overridden. // transient limit should be overridden. require.Equal(t, int(cfg.NetworkConfig.ResourceManagerConfig.InboundStream.Transient), int(applied.ToPartialLimitConfig().Transient.StreamsInbound)) @@ -168,4 +173,7 @@ func TestApplyInboundStreamLimits(t *testing.T) { require.Equal(t, int(1), int(applied.ToPartialLimitConfig().Stream.StreamsInbound)) // system limit should not be overridden. require.Equal(t, int(1), int(applied.ToPartialLimitConfig().System.StreamsInbound)) + + // check that the applied peer base limit connections are overridden. + require.Equal(t, int(cfg.NetworkConfig.ResourceManagerConfig.PeerBaseLimitConnsInbound), int(applied.ToPartialLimitConfig().PeerDefault.ConnsInbound)) } diff --git a/network/p2p/p2pbuilder/utils.go b/network/p2p/p2pbuilder/utils.go index 232ea571d7d..5af43ba9e81 100644 --- a/network/p2p/p2pbuilder/utils.go +++ b/network/p2p/p2pbuilder/utils.go @@ -184,3 +184,31 @@ func ApplyInboundStreamLimits(logger zerolog.Logger, concrete rcmgr.ConcreteLimi lg.Info().Msg("inbound stream limits applied") return partial.Build(concrete) } + +// ApplyInboundConnectionLimits applies the inbound connection limits to the concrete limit config. The concrete limit config is assumed coming from scaling the +// base limit config by the scaling factor. The inbound connection limits are applied to the concrete limit config if the concrete limit config is greater than +// the inbound connection limits. +// The inbound limits are assumed coming from the config file. +// Args: +// +// logger: the logger to log the applied limits. +// concrete: the concrete limit config. +// peerLimit: the inbound connection limit from each remote peer. +// +// Returns: +// +// a copy of the concrete limit config with the inbound connection limits applied and overridden. +func ApplyInboundConnectionLimits(logger zerolog.Logger, concrete rcmgr.ConcreteLimitConfig, peerLimit int) rcmgr.ConcreteLimitConfig { + c := concrete.ToPartialLimitConfig() + + partial := rcmgr.PartialLimitConfig{} + lg := logger.With().Logger() + + if int(c.PeerDefault.ConnsInbound) > peerLimit { + lg = lg.With().Int("concrete_peer_inbound_conns", int(c.PeerDefault.ConnsInbound)).Int("partial_peer_inbound_conns", peerLimit).Logger() + partial.PeerDefault.ConnsInbound = rcmgr.LimitVal(peerLimit) + } + + lg.Info().Msg("inbound connection limits applied") + return partial.Build(concrete) +} From 26e817fdf608eb4bfe4151638c78c44e8c07dd95 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 07:22:17 -0500 Subject: [PATCH 20/68] validation tags to make sure probability values in range [0,1] --- network/netconf/config.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/network/netconf/config.go b/network/netconf/config.go index 100eb1e0d00..3ae0de87780 100644 --- a/network/netconf/config.go +++ b/network/netconf/config.go @@ -119,14 +119,14 @@ type AlspConfig struct { // misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, // since it's not the final probability and there are other factors that determine the final probability. // The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. - SyncEngineBatchRequestBaseProb float32 `mapstructure:"alsp-sync-engine-batch-request-base-prob"` + SyncEngineBatchRequestBaseProb float32 `validate:"range=0,1" mapstructure:"alsp-sync-engine-batch-request-base-prob"` // SyncEngineRangeRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a // misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, // since it's not the final probability and there are other factors that determine the final probability. // The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. - SyncEngineRangeRequestBaseProb float32 `mapstructure:"alsp-sync-engine-range-request-base-prob"` + SyncEngineRangeRequestBaseProb float32 `validate:"range=0,1" mapstructure:"alsp-sync-engine-range-request-base-prob"` // SyncEngineSyncRequestProb is the probability in [0,1] of creating a misbehavior report for a SyncRequest message. - SyncEngineSyncRequestProb float32 `mapstructure:"alsp-sync-engine-sync-request-prob"` + SyncEngineSyncRequestProb float32 `validate:"range=0,1" mapstructure:"alsp-sync-engine-sync-request-prob"` } From a04084960ac6baaf353f766f86a97f50c333ab7f Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 07:39:22 -0500 Subject: [PATCH 21/68] more docs for ALSP flags --- config/default-config.yml | 23 +++++++++++++++++++++++ engine/common/synchronization/config.go | 18 ++---------------- 2 files changed, 25 insertions(+), 16 deletions(-) diff --git a/config/default-config.yml b/config/default-config.yml index cc7be185e2a..a215519cfc2 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -150,12 +150,35 @@ network-config: # misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, # since it's not the final probability and there are other factors that determine the final probability. # The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. + # Create misbehavior report for about 0.2% of BatchRequest messages for normal batch requests (i.e. not too large) + # The final batch request probability is calculated as follows: + # batchRequestBaseProb * (len(batchRequest.BlockIDs) + 1) / synccore.DefaultConfig().MaxSize + # Example 1 (small batch of block IDs) if the batch request is for 10 blocks IDs and batchRequestBaseProb is 0.01, then the probability of + # creating a misbehavior report is: + # batchRequestBaseProb * (10+1) / synccore.DefaultConfig().MaxSize + # = 0.01 * 11 / 64 = 0.00171875 = 0.171875% + # Example 2 (large batch of block IDs) if the batch request is for 1000 block IDs and batchRequestBaseProb is 0.01, then the probability of + # creating a misbehavior report is: + # batchRequestBaseProb * (1000+1) / synccore.DefaultConfig().MaxSize + # = 0.01 * 1001 / 64 = 0.15640625 = 15.640625% alsp-sync-engine-batch-request-base-prob: 0.01 # Base probability in [0,1] that's used in creating the final probability of creating a # misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, # since it's not the final probability and there are other factors that determine the final probability. # The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. + # Create misbehavior report for about 0.2% of RangeRequest messages for normal range requests (i.e. not too large) + # and about 15% of RangeRequest messages for very large range requests. + # The final probability is calculated as follows: + # rangeRequestBaseProb * ((rangeRequest.ToHeight-rangeRequest.FromHeight) + 1) / synccore.DefaultConfig().MaxSize + # Example 1 (small range) if the range request is for 10 blocks and rangeRequestBaseProb is 0.01, then the probability of + # creating a misbehavior report is: + # rangeRequestBaseProb * (10+1) / synccore.DefaultConfig().MaxSize + # = 0.01 * 11 / 64 = 0.00171875 = 0.171875% + # Example 2 (large range) if the range request is for 1000 blocks and rangeRequestBaseProb is 0.01, then the probability of + # creating a misbehavior report is: + # rangeRequestBaseProb * (1000+1) / synccore.DefaultConfig().MaxSize + # = 0.01 * 1001 / 64 = 0.15640625 = 15.640625% alsp-sync-engine-range-request-base-prob: 0.01 # Probability in [0,1] of creating a misbehavior report for a SyncRequest message. diff --git a/engine/common/synchronization/config.go b/engine/common/synchronization/config.go index 14eb630b87a..b7393414897 100644 --- a/engine/common/synchronization/config.go +++ b/engine/common/synchronization/config.go @@ -79,23 +79,9 @@ func NewSpamDetectionConfig() *SpamDetectionConfig { } return &SpamDetectionConfig{ + // see config/default-config.yml for more information on the following fields batchRequestBaseProb: flowConfig.NetworkConfig.SyncEngineBatchRequestBaseProb, - - // create misbehavior report for 1% of SyncRequest messages - syncRequestProb: flowConfig.NetworkConfig.SyncEngineSyncRequestProb, - - // create misbehavior report for about 0.2% of RangeRequest messages for normal range requests (i.e. not too large) - // and about 15% of RangeRequest messages for very large range requests - // The final probability is calculated as follows: - // rangeRequestBaseProb * ((rangeRequest.ToHeight-rangeRequest.FromHeight) + 1) / synccore.DefaultConfig().MaxSize - // Example 1 (small range) if the range request is for 10 blocks and rangeRequestBaseProb is 0.01, then the probability of - // creating a misbehavior report is: - // rangeRequestBaseProb * (10+1) / synccore.DefaultConfig().MaxSize - // = 0.01 * 11 / 64 = 0.00171875 = 0.171875% - // Example 2 (large range) if the range request is for 1000 blocks and rangeRequestBaseProb is 0.01, then the probability of - // creating a misbehavior report is: - // rangeRequestBaseProb * (1000+1) / synccore.DefaultConfig().MaxSize - // = 0.01 * 1001 / 64 = 0.15640625 = 15.640625% + syncRequestProb: flowConfig.NetworkConfig.SyncEngineSyncRequestProb, rangeRequestBaseProb: flowConfig.NetworkConfig.SyncEngineRangeRequestBaseProb, } } From 91607b899ecb1dbc4414582de6c27817a388c8f6 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 07:51:33 -0500 Subject: [PATCH 22/68] NewSpamDetectionConfig() passes up error instead of throwing error --- engine/common/synchronization/config.go | 9 +++------ engine/common/synchronization/engine_suite_test.go | 5 +++-- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/engine/common/synchronization/config.go b/engine/common/synchronization/config.go index b7393414897..36c52dbf5fd 100644 --- a/engine/common/synchronization/config.go +++ b/engine/common/synchronization/config.go @@ -1,13 +1,10 @@ package synchronization import ( - "context" "fmt" "time" "github.com/onflow/flow-go/config" - "github.com/onflow/flow-go/module/irrecoverable" - core "github.com/onflow/flow-go/module/chainsync" ) @@ -72,10 +69,10 @@ type SpamDetectionConfig struct { rangeRequestBaseProb float32 } -func NewSpamDetectionConfig() *SpamDetectionConfig { +func NewSpamDetectionConfig() (*SpamDetectionConfig, error) { flowConfig, err := config.DefaultConfig() if err != nil { - irrecoverable.Throw(context.TODO(), fmt.Errorf("failed to read default config: %w", err)) + return nil, fmt.Errorf("failed to read default config: %w", err) } return &SpamDetectionConfig{ @@ -83,5 +80,5 @@ func NewSpamDetectionConfig() *SpamDetectionConfig { batchRequestBaseProb: flowConfig.NetworkConfig.SyncEngineBatchRequestBaseProb, syncRequestProb: flowConfig.NetworkConfig.SyncEngineSyncRequestProb, rangeRequestBaseProb: flowConfig.NetworkConfig.SyncEngineRangeRequestBaseProb, - } + }, nil } diff --git a/engine/common/synchronization/engine_suite_test.go b/engine/common/synchronization/engine_suite_test.go index 1b9b4f7681e..ddd9497268e 100644 --- a/engine/common/synchronization/engine_suite_test.go +++ b/engine/common/synchronization/engine_suite_test.go @@ -159,6 +159,8 @@ func (ss *SyncSuite) SetupTest() { idCache, err := cache.NewProtocolStateIDCache(log, ss.state, protocolEvents.NewDistributor()) require.NoError(ss.T(), err, "could not create protocol state identity cache") + spamConfig, err := NewSpamDetectionConfig() + require.NoError(ss.T(), err, "could not create spam detection config") e, err := New(log, metrics, ss.net, ss.me, ss.state, ss.blocks, ss.comp, ss.core, id.NewIdentityFilterIdentifierProvider( filter.And( @@ -167,8 +169,7 @@ func (ss *SyncSuite) SetupTest() { ), idCache, ), - NewSpamDetectionConfig()) + spamConfig) require.NoError(ss.T(), err, "should pass engine initialization") - ss.e = e } From 58c93f6262a66b3f8d1c1cb3492c1cab952b94f3 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 08:45:19 -0500 Subject: [PATCH 23/68] fix broken link in config README --- config/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/README.md b/config/README.md index f8a31bda478..93c0334a5be 100644 --- a/config/README.md +++ b/config/README.md @@ -36,7 +36,7 @@ so that configuration can be updated in one place these sub-packages can live an const workersCLIFlag = "app-workers" flags.String(workersCLIFlag, 1, "number of app workers") ``` - The network package can be used as a good example of how to structure CLI flag initialization. All flags are initialized in a single function [InitializeNetworkFlags](https://github.com/onflow/flow-go/blob/master/config/network/flags.go#L80), this function is then used during flag initialization + The network package can be used as a good example of how to structure CLI flag initialization. All flags are initialized in a single function [InitializeNetworkFlags](https://github.com/onflow/flow-go/blob/master/network/netconf/flags.go#L80), this function is then used during flag initialization of the [config package](https://github.com/onflow/flow-go/blob/master/config/base_flags.go#L22). 3. Add the config as a new field to an existing configuration struct or create a new one. Each configuration struct must be a field on the FlowConfig struct so that it is unmarshalled during configuration initialization. Each field on a configuration struct must contain the following field tags. From 5abfe85be693f0b40aeaf40c2fa8d401f54393e7 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 09:44:18 -0500 Subject: [PATCH 24/68] config README update --- config/README.md | 36 ++++++++++++++++++++++++++++-------- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/config/README.md b/config/README.md index 93c0334a5be..01e999ef3a6 100644 --- a/config/README.md +++ b/config/README.md @@ -3,11 +3,11 @@ config is a package to hold all configuration values for each Flow component. Th to the entire FlowConfig and utilities to add a new config value, corresponding CLI flag, and validation. ### Package structure -The root config package contains the FlowConfig struct and the default config file [default-config.yml](https://github.com/onflow/flow-go/blob/master/config/default-config.yml). The `default-config.yml` file is the default configuration that is loaded when the config package is initialize. +The root config package contains the FlowConfig struct and the default config file [default-config.yml](https://github.com/onflow/flow-go/blob/master/config/default-config.yml). The `default-config.yml` file is the default configuration that is loaded when the config package is initialized. The `default-config.yml` is a snapshot of all the configuration values defined for Flow. -Each subpackage contains configuration structs and utilities for components and their related subcomponents. These packages also contain the CLI flags for each configuration value. The [network](https://github.com/onflow/flow-go/tree/master/config/network) package +Each subpackage contains configuration structs and utilities for components and their related subcomponents. These packages also contain the CLI flags for each configuration value. The [netconf](https://github.com/onflow/flow-go/tree/master/network/netconf) package is a good example of this pattern. The network component is a large component made of many other large components and subcomponents. Each configuration -struct is defined for all of these network related components in the network subpackage and CLI flags. +struct is defined for all of these network related components in the netconf subpackage and CLI flags. ### Overriding default values The entire default config can be overridden using the `--config-file` CLI flag. When set the config package will attempt to parse the specified config file and override all the values @@ -26,19 +26,30 @@ go build -tags relic -o flow-access-node ./cmd/access ### Adding a new config value Adding a new config to the FlowConfig can be done in a few easy steps. +The network package can be used as a good example of how to add CLI flags and will be used in the steps below. + 1. Create a new subpackage in the config package for the new configuration structs to live. Although it is encouraged to put all configuration sub-packages in the config package so that configuration can be updated in one place these sub-packages can live anywhere. This package will define the configuration structs and CLI flags for overriding. ```shell mkdir example_config ``` + For the network package we have a subpackage created in [network/netconf](https://github.com/onflow/flow-go/tree/master/network/netconf). + 2. Add a new CLI flag for the config value. ```go const workersCLIFlag = "app-workers" flags.String(workersCLIFlag, 1, "number of app workers") ``` - The network package can be used as a good example of how to structure CLI flag initialization. All flags are initialized in a single function [InitializeNetworkFlags](https://github.com/onflow/flow-go/blob/master/network/netconf/flags.go#L80), this function is then used during flag initialization - of the [config package](https://github.com/onflow/flow-go/blob/master/config/base_flags.go#L22). -3. Add the config as a new field to an existing configuration struct or create a new one. Each configuration struct must be a field on the FlowConfig struct so that it is unmarshalled during configuration initialization. + + All network package CLI flags are defined in [network/netconf/flags.go](https://github.com/onflow/flow-go/blob/master/network/netconf/flags.go) in: + - `const` block + - `AllFlagNames` function + - `InitializeNetworkFlags` function + + `InitializeNetworkFlags` is used during initialization of all flags + in the `InitializePFlagSet` function in the [config/base_flags.go](https://github.com/onflow/flow-go/blob/master/config/base_flags.go). + +3. Add the config as a new field to an existing configuration struct or create a new struct. Each configuration struct must be a field on the FlowConfig struct so that it is unmarshalled during configuration initialization. Each field on a configuration struct must contain the following field tags. 1. `validate` - validate tag is used to perform validation on field structs using the [validator](https://github.com/go-playground/validator) package. In the example below you will notice the `validate:"gt=0"` tag, this will ensure that the value of `AppWorkers` is greater than 0. The top level `FlowConfig` struct has a Validate method that performs struct validation. This @@ -50,6 +61,9 @@ so that configuration can be updated in one place these sub-packages can live an } ``` It's important to make sure that the CLI flag name matches the mapstructure field tag to avoid parsing errors. + + All network package configuration structs are defined in [network/netconf/config.go](https://github.com/onflow/flow-go/blob/master/network/netconf/config.go) + 4. Add the new config and a default value to the `default-config.yml` file. Ensure that the new property added matches the configuration struct structure for the subpackage the config belongs to. ```yaml config-file: "./default-config.yml" @@ -58,8 +72,11 @@ so that configuration can be updated in one place these sub-packages can live an my-component: app-workers: 1 ``` -5. Finally, if a new struct was created add it as a new field to the FlowConfig. In the previous steps we added a new config struct and added a new property to the default-config.yml for this struct `my-component`. This property name - must match the mapstructure field tag for the struct when added to the FlowConfig. + + All network package configuration values are defined under `network-config` in `default-config.yml` + +5. If a new struct was created in step 3, add it as a new field to `FlowConfig` struct in [config/config.go](https://github.com/onflow/flow-go/blob/master/config/config.go). In the previous steps we added a new config struct and added a new property to the `default-config.yml` for this struct `my-component`. This property name + must match the mapstructure field tag for the struct when added to `FlowConfig`. ```go // FlowConfig Flow configuration. type FlowConfig struct { @@ -68,6 +85,9 @@ so that configuration can be updated in one place these sub-packages can live an MyComponentConfig *mypackage.MyComponentConfig `mapstructure:"my-component"` } ``` + + The network package configuration struct, `NetworkConfig`, is already embedded as a field in `FlowConfig` struct. + This means that new flags can be added to the network package configuration struct without having to update the `FlowConfig` struct. ### Nested structs In an effort to keep the configuration yaml structure readable some configuration will be in nested properties. When this is the case the mapstructure `squash` tag can be used so that the corresponding nested struct will be From d5f8bf361f70d2175e08a1067d35b9e970b8ef6e Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 10:15:53 -0500 Subject: [PATCH 25/68] lint fix --- cmd/access/node_builder/access_node_builder.go | 6 +++++- cmd/collection/main.go | 6 +++++- cmd/consensus/main.go | 7 ++++++- cmd/execution_builder.go | 8 ++++++-- cmd/observer/node_builder/observer_builder.go | 7 ++++++- cmd/verification_builder.go | 7 ++++++- consensus/integration/nodes_test.go | 5 ++++- engine/testutil/nodes.go | 4 +++- 8 files changed, 41 insertions(+), 9 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b1a97d930ce..3c615d744ca 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -430,6 +430,10 @@ func (builder *FlowAccessNodeBuilder) buildFollowerEngine() *FlowAccessNodeBuild func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.Component("sync engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + spamConfig, err := synceng.NewSpamDetectionConfig() + if err != nil { + return nil, fmt.Errorf("could not initialize spam detection config: %w", err) + } sync, err := synceng.New( node.Logger, node.Metrics.Engine, @@ -440,7 +444,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.SyncEngineParticipantsProviderFactory(), - synceng.NewSpamDetectionConfig(), + spamConfig, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/collection/main.go b/cmd/collection/main.go index f285911bfdd..3b86932b84b 100644 --- a/cmd/collection/main.go +++ b/cmd/collection/main.go @@ -354,6 +354,10 @@ func main() { return followerEng, nil }). Component("main chain sync engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + spamConfig, err := consync.NewSpamDetectionConfig() + if err != nil { + return nil, fmt.Errorf("could not initialize spam detection config: %w", err) + } // create a block synchronization engine to handle follower getting out of sync sync, err := consync.New( @@ -366,7 +370,7 @@ func main() { followerEng, mainChainSyncCore, node.SyncEngineIdentifierProvider, - consync.NewSpamDetectionConfig(), + spamConfig, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 2090311b1ae..fc685afc247 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -839,6 +839,11 @@ func main() { return messageHub, nil }). Component("sync engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + spamConfig, err := synceng.NewSpamDetectionConfig() + if err != nil { + return nil, fmt.Errorf("could not initialize spam detection config: %w", err) + } + sync, err := synceng.New( node.Logger, node.Metrics.Engine, @@ -849,7 +854,7 @@ func main() { comp, syncCore, node.SyncEngineIdentifierProvider, - synceng.NewSpamDetectionConfig(), + spamConfig, ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index 60903942d4a..9f0fe5114f3 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -1071,7 +1071,11 @@ func (exeNode *ExecutionNode) LoadSynchronizationEngine( error, ) { // initialize the synchronization engine - var err error + //var err error + spamConfig, err := synchronization.NewSpamDetectionConfig() + if err != nil { + return nil, fmt.Errorf("could not initialize spam detection config: %w", err) + } exeNode.syncEngine, err = synchronization.New( node.Logger, node.Metrics.Engine, @@ -1082,7 +1086,7 @@ func (exeNode *ExecutionNode) LoadSynchronizationEngine( exeNode.followerEng, exeNode.syncCore, node.SyncEngineIdentifierProvider, - synchronization.NewSpamDetectionConfig(), + spamConfig, ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 0321069070c..88ddb1bf36c 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -394,6 +394,11 @@ func (builder *ObserverServiceBuilder) buildFollowerEngine() *ObserverServiceBui func (builder *ObserverServiceBuilder) buildSyncEngine() *ObserverServiceBuilder { builder.Component("sync engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + spamConfig, err := synceng.NewSpamDetectionConfig() + if err != nil { + return nil, fmt.Errorf("could not initialize spam detection config: %w", err) + } + sync, err := synceng.New( node.Logger, node.Metrics.Engine, @@ -404,7 +409,7 @@ func (builder *ObserverServiceBuilder) buildSyncEngine() *ObserverServiceBuilder builder.FollowerEng, builder.SyncCore, builder.SyncEngineParticipantsProviderFactory(), - synceng.NewSpamDetectionConfig(), + spamConfig, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/verification_builder.go b/cmd/verification_builder.go index ea5ddf65a8e..15be77689ec 100644 --- a/cmd/verification_builder.go +++ b/cmd/verification_builder.go @@ -395,6 +395,11 @@ func (v *VerificationNodeBuilder) LoadComponentsAndModules() { return followerEng, nil }). Component("sync engine", func(node *NodeConfig) (module.ReadyDoneAware, error) { + spamConfig, err := commonsync.NewSpamDetectionConfig() + if err != nil { + return nil, fmt.Errorf("could not initialize spam detection config: %w", err) + } + sync, err := commonsync.New( node.Logger, node.Metrics.Engine, @@ -405,7 +410,7 @@ func (v *VerificationNodeBuilder) LoadComponentsAndModules() { followerEng, syncCore, node.SyncEngineIdentifierProvider, - commonsync.NewSpamDetectionConfig(), + spamConfig, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index ddfad8ee52a..8b8963cdc32 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -627,6 +627,9 @@ func createNode( require.NoError(t, err) idProvider := id.NewFixedIdentifierProvider(identities.NodeIDs()) + spamConfig, err := synceng.NewSpamDetectionConfig() + require.NoError(t, err, "could not initialize spam detection config") + // initialize the synchronization engine sync, err := synceng.New( log, @@ -638,7 +641,7 @@ func createNode( comp, syncCore, idProvider, - synceng.NewSpamDetectionConfig(), + spamConfig, func(cfg *synceng.Config) { // use a small pool and scan interval for sync engine cfg.ScanInterval = 500 * time.Millisecond diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index f653ff6c72f..ddee974f573 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -773,6 +773,8 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit idCache, err := cache.NewProtocolStateIDCache(node.Log, node.State, events.NewDistributor()) require.NoError(t, err, "could not create finalized snapshot cache") + spamConfig, err := synchronization.NewSpamDetectionConfig() + require.NoError(t, err, "could not initialize spam detection config") syncEngine, err := synchronization.New( node.Log, node.Metrics, @@ -789,7 +791,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit ), idCache, ), - synchronization.NewSpamDetectionConfig(), + spamConfig, synchronization.WithPollInterval(time.Duration(0)), ) require.NoError(t, err) From 175adafcdb7272d3227807aae4af7907655c0ce0 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 10:57:08 -0500 Subject: [PATCH 26/68] lint fix --- follower/follower_builder.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/follower/follower_builder.go b/follower/follower_builder.go index 067994f4f31..cebee84d0ef 100644 --- a/follower/follower_builder.go +++ b/follower/follower_builder.go @@ -288,6 +288,11 @@ func (builder *FollowerServiceBuilder) buildFollowerEngine() *FollowerServiceBui func (builder *FollowerServiceBuilder) buildSyncEngine() *FollowerServiceBuilder { builder.Component("sync engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + spamConfig, err := synceng.NewSpamDetectionConfig() + if err != nil { + return nil, fmt.Errorf("could not initialize spam detection config: %w", err) + } + sync, err := synceng.New( node.Logger, node.Metrics.Engine, @@ -298,7 +303,7 @@ func (builder *FollowerServiceBuilder) buildSyncEngine() *FollowerServiceBuilder builder.FollowerEng, builder.SyncCore, builder.SyncEngineParticipantsProviderFactory(), - synceng.NewSpamDetectionConfig(), + spamConfig, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) From ac06fd009466ddf2fb2d7d9db4f458588ac96b32 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 20 Oct 2023 09:47:57 -0700 Subject: [PATCH 27/68] fixes lint --- network/p2p/p2pnode/libp2pNode_test.go | 106 ------------------------- 1 file changed, 106 deletions(-) diff --git a/network/p2p/p2pnode/libp2pNode_test.go b/network/p2p/p2pnode/libp2pNode_test.go index b4eaf4f6496..9ec66082a0e 100644 --- a/network/p2p/p2pnode/libp2pNode_test.go +++ b/network/p2p/p2pnode/libp2pNode_test.go @@ -285,112 +285,6 @@ func TestCreateStream_SinglePairwiseConnection(t *testing.T) { close(streamChan) } -// TestCreateStream_SinglePeerDial ensures that the unicast manager only attempts to dial a peer once, retries dialing a peer the expected max amount of times when an -// error is encountered and retries creating the stream the expected max amount of times when unicast.ErrDialInProgress is encountered. -func TestCreateStream_SinglePeerDial(t *testing.T) { - createStreamRetries := atomic.NewInt64(0) - dialPeerRetries := atomic.NewInt64(0) - hook := zerolog.HookFunc(func(e *zerolog.Event, level zerolog.Level, message string) { - if level == zerolog.WarnLevel { - switch { - case strings.Contains(message, "retrying create stream, dial to peer in progress"): - createStreamRetries.Inc() - case strings.Contains(message, "retrying peer dialing"): - dialPeerRetries.Inc() - } - } - }) - logger := zerolog.New(os.Stdout).Level(zerolog.InfoLevel).Hook(hook) - idProvider := mockmodule.NewIdentityProvider(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - - sporkID := unittest.IdentifierFixture() - - // mock metrics we expected only a single call to CreateStream to initiate the dialing to the peer, which will result in 3 failed attempts - // the next call to CreateStream will encounter a DialInProgress error which will result in 3 failed attempts - m := mockmodule.NewNetworkMetrics(t) - m.On("OnPeerDialFailure", mock.Anything, 3).Once() - m.On("OnStreamCreationFailure", mock.Anything, mock.Anything).Twice().Run(func(args mock.Arguments) { - attempts := args.Get(1).(int) - // We expect OnCreateStream to be called twice: once in each separate call to CreateStream. The first call that initializes - // the peer dialing should not attempt to retry CreateStream because all peer dialing attempts will be made which will not - // return the DialInProgress err that kicks off the CreateStream retries so we expect attempts to be 1 in this case. In the - // second call to CreateStream we expect all 3 attempts to be made as we wait for the DialInProgress to complete, in this case - // we expect attempts to be 3. Thus we only expect this method to be called twice with either 1 or 3 attempts. - require.False(t, attempts != 1 && attempts != 3, fmt.Sprintf("expected either 1 or 3 attempts got %d", attempts)) - }) - - sender, id1 := p2ptest.NodeFixture( - t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithConnectionGater(p2ptest.NewConnectionGater(idProvider, func(pid peer.ID) error { - // avoid connection gating outbound messages on sender - return nil - })), - // add very small delay so that when the sender attempts to create multiple streams - // the func fails fast before the first routine can finish the peer dialing retries - // this prevents us from making another call to dial peer - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond), - p2ptest.WithLogger(logger), - p2ptest.WithMetricsCollector(m)) - - receiver, id2 := p2ptest.NodeFixture( - t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithConnectionGater(p2ptest.NewConnectionGater(idProvider, func(pid peer.ID) error { - // connection gate all incoming connections forcing the senders unicast manager to perform retries - return fmt.Errorf("gate keep") - })), - p2ptest.WithCreateStreamRetryDelay(10*time.Millisecond), - p2ptest.WithLogger(logger)) - - idProvider.On("ByPeerID", sender.ID()).Return(&id1, true).Maybe() - idProvider.On("ByPeerID", receiver.ID()).Return(&id2, true).Maybe() - - p2ptest.StartNodes(t, signalerCtx, []p2p.LibP2PNode{sender, receiver}) - defer p2ptest.StopNodes(t, []p2p.LibP2PNode{sender, receiver}, cancel) - - var wg sync.WaitGroup - wg.Add(2) - // attempt to create two concurrent streams - go func() { - defer wg.Done() - err := sender.OpenProtectedStream(ctx, receiver.ID(), t.Name(), func(stream network.Stream) error { - return nil - }) - require.Error(t, err) - }() - go func() { - defer wg.Done() - err := sender.OpenProtectedStream(ctx, receiver.ID(), t.Name(), func(stream network.Stream) error { - return nil - }) - require.Error(t, err) - }() - - unittest.RequireReturnsBefore(t, wg.Wait, 3*time.Second, "cannot create streams on time") - - // we expect a single routine to start attempting to dial thus the number of retries - // before failure should be at most p2pnode.MaxConnectAttempt - expectedNumOfDialRetries := int64(p2pnode.MaxConnectAttempt) - // we expect the second routine to retry creating a stream p2pnode.MaxConnectAttempt when dialing is in progress - expectedCreateStreamRetries := int64(p2pnode.MaxConnectAttempt) - require.Equal(t, - expectedNumOfDialRetries, - dialPeerRetries.Load(), - fmt.Sprintf("expected %d dial peer retries got %d", expectedNumOfDialRetries, dialPeerRetries.Load())) - require.Equal(t, - expectedCreateStreamRetries, - createStreamRetries.Load(), - fmt.Sprintf("expected %d dial peer retries got %d", expectedCreateStreamRetries, createStreamRetries.Load())) -} - // createStreams will attempt to create n number of streams concurrently between each combination of node pairs. func createConcurrentStreams(t *testing.T, ctx context.Context, nodes []p2p.LibP2PNode, ids flow.IdentityList, n int, streams chan network.Stream, done chan struct{}) { defer close(done) From 41f175080e73e5a8af31bdad3b05d20f3785bc47 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 19 Oct 2023 08:27:07 -0700 Subject: [PATCH 28/68] jobqueue to accept default index from constructor instead of Start method --- cmd/verification_builder.go | 6 +- engine/testutil/nodes.go | 3 +- .../assigner/blockconsumer/consumer.go | 20 ++++--- .../fetcher/chunkconsumer/consumer.go | 11 ++-- .../fetcher/chunkconsumer/consumer_test.go | 3 +- module/jobqueue.go | 2 +- module/jobqueue/component_consumer.go | 25 ++++---- module/jobqueue/consumer.go | 58 +++++++++++-------- 8 files changed, 77 insertions(+), 51 deletions(-) diff --git a/cmd/verification_builder.go b/cmd/verification_builder.go index ea5ddf65a8e..79f5de44595 100644 --- a/cmd/verification_builder.go +++ b/cmd/verification_builder.go @@ -245,7 +245,7 @@ func (v *VerificationNodeBuilder) LoadComponentsAndModules() { v.verConf.stopAtHeight) // requester and fetcher engines are started by chunk consumer - chunkConsumer = chunkconsumer.NewChunkConsumer( + chunkConsumer, err = chunkconsumer.NewChunkConsumer( node.Logger, collector, processedChunkIndex, @@ -253,6 +253,10 @@ func (v *VerificationNodeBuilder) LoadComponentsAndModules() { fetcherEngine, v.verConf.chunkWorkers) + if err != nil { + return nil, fmt.Errorf("could not create chunk consumer: %w", err) + } + err = node.Metrics.Mempool.Register(metrics.ResourceChunkConsumer, chunkConsumer.Size) if err != nil { return nil, fmt.Errorf("could not register backend metric: %w", err) diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index f653ff6c72f..0671a4ed567 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -1055,12 +1055,13 @@ func VerificationNode(t testing.TB, } if node.ChunkConsumer == nil { - node.ChunkConsumer = chunkconsumer.NewChunkConsumer(node.Log, + node.ChunkConsumer, err = chunkconsumer.NewChunkConsumer(node.Log, collector, node.ProcessedChunkIndex, node.ChunksQueue, node.FetcherEngine, chunkconsumer.DefaultChunkWorkers) // defaults number of workers to 3. + require.NoError(t, err) err = mempoolCollector.Register(metrics.ResourceChunkConsumer, node.ChunkConsumer.Size) require.NoError(t, err) } diff --git a/engine/verification/assigner/blockconsumer/consumer.go b/engine/verification/assigner/blockconsumer/consumer.go index 982fe418688..10cf18a461f 100644 --- a/engine/verification/assigner/blockconsumer/consumer.go +++ b/engine/verification/assigner/blockconsumer/consumer.go @@ -21,10 +21,9 @@ const DefaultBlockWorkers = uint64(2) // and notifies the consumer to check in the job queue // (i.e., its block reader) for new block jobs. type BlockConsumer struct { - consumer module.JobConsumer - defaultIndex uint64 - unit *engine.Unit - metrics module.VerificationMetrics + consumer module.JobConsumer + unit *engine.Unit + metrics module.VerificationMetrics } // defaultProcessedIndex returns the last sealed block height from the protocol state. @@ -59,17 +58,20 @@ func NewBlockConsumer(log zerolog.Logger, // the block reader is where the consumer reads new finalized blocks from (i.e., jobs). jobs := jobqueue.NewFinalizedBlockReader(state, blocks) - consumer := jobqueue.NewConsumer(lg, jobs, processedHeight, worker, maxProcessing, 0) defaultIndex, err := defaultProcessedIndex(state) if err != nil { return nil, 0, fmt.Errorf("could not read default processed index: %w", err) } + consumer, err := jobqueue.NewConsumer(lg, jobs, processedHeight, worker, maxProcessing, 0, defaultIndex) + if err != nil { + return nil, 0, fmt.Errorf("could not create block consumer: %w", err) + } + blockConsumer := &BlockConsumer{ - consumer: consumer, - defaultIndex: defaultIndex, - unit: engine.NewUnit(), - metrics: metrics, + consumer: consumer, + unit: engine.NewUnit(), + metrics: metrics, } worker.withBlockConsumer(blockConsumer) diff --git a/engine/verification/fetcher/chunkconsumer/consumer.go b/engine/verification/fetcher/chunkconsumer/consumer.go index 97cccdb4ab2..703b51d5d7e 100644 --- a/engine/verification/fetcher/chunkconsumer/consumer.go +++ b/engine/verification/fetcher/chunkconsumer/consumer.go @@ -33,14 +33,17 @@ func NewChunkConsumer( chunksQueue storage.ChunksQueue, // to read jobs (chunks) from chunkProcessor fetcher.AssignedChunkProcessor, // to process jobs (chunks) maxProcessing uint64, // max number of jobs to be processed in parallel -) *ChunkConsumer { +) (*ChunkConsumer, error) { worker := NewWorker(chunkProcessor) chunkProcessor.WithChunkConsumerNotifier(worker) jobs := &ChunkJobs{locators: chunksQueue} lg := log.With().Str("module", "chunk_consumer").Logger() - consumer := jobqueue.NewConsumer(lg, jobs, processedIndex, worker, maxProcessing, 0) + consumer, err := jobqueue.NewConsumer(lg, jobs, processedIndex, worker, maxProcessing, 0, DefaultJobIndex) + if err != nil { + return nil, err + } chunkConsumer := &ChunkConsumer{ consumer: consumer, @@ -50,7 +53,7 @@ func NewChunkConsumer( worker.consumer = chunkConsumer - return chunkConsumer + return chunkConsumer, nil } func (c *ChunkConsumer) NotifyJobIsDone(jobID module.JobID) { @@ -68,7 +71,7 @@ func (c ChunkConsumer) Check() { } func (c *ChunkConsumer) Ready() <-chan struct{} { - err := c.consumer.Start(DefaultJobIndex) + err := c.consumer.Start() if err != nil { panic(fmt.Errorf("could not start the chunk consumer for match engine: %w", err)) } diff --git a/engine/verification/fetcher/chunkconsumer/consumer_test.go b/engine/verification/fetcher/chunkconsumer/consumer_test.go index 6ca73d10aba..1aabce2bd14 100644 --- a/engine/verification/fetcher/chunkconsumer/consumer_test.go +++ b/engine/verification/fetcher/chunkconsumer/consumer_test.go @@ -155,7 +155,7 @@ func WithConsumer( } collector := &metrics.NoopCollector{} - consumer := chunkconsumer.NewChunkConsumer( + consumer, err := chunkconsumer.NewChunkConsumer( unittest.Logger(), collector, processedIndex, @@ -163,6 +163,7 @@ func WithConsumer( engine, maxProcessing, ) + require.NoError(t, err) withConsumer(consumer, chunksQueue) }) diff --git a/module/jobqueue.go b/module/jobqueue.go index c0275cc9241..e78cc8e670b 100644 --- a/module/jobqueue.go +++ b/module/jobqueue.go @@ -30,7 +30,7 @@ type JobConsumer interface { // Start starts processing jobs from a job queue. If this is the first time, a processed index // will be initialized in the storage. If it fails to initialize, an error will be returned - Start(defaultIndex uint64) error + Start() error // Stop gracefully stops the consumer from reading new jobs from the job queue. It does not stop // the existing worker finishing their jobs diff --git a/module/jobqueue/component_consumer.go b/module/jobqueue/component_consumer.go index 605dd81e7ae..86c16acba7d 100644 --- a/module/jobqueue/component_consumer.go +++ b/module/jobqueue/component_consumer.go @@ -33,13 +33,7 @@ func NewComponentConsumer( processor JobProcessor, // method used to process jobs maxProcessing uint64, maxSearchAhead uint64, -) *ComponentConsumer { - - c := &ComponentConsumer{ - workSignal: workSignal, - jobs: jobs, - log: log, - } +) (*ComponentConsumer, error) { // create a worker pool with maxProcessing workers to process jobs worker := NewWorkerPool( @@ -47,12 +41,23 @@ func NewComponentConsumer( func(id module.JobID) { c.NotifyJobIsDone(id) }, maxProcessing, ) - c.consumer = NewConsumer(c.log, c.jobs, progress, worker, maxProcessing, maxSearchAhead) + + consumer, err := NewConsumer(log, jobs, progress, worker, maxProcessing, maxSearchAhead, defaultIndex) + if err != nil { + return nil, err + } + + c := &ComponentConsumer{ + workSignal: workSignal, + jobs: jobs, + log: log, + consumer: consumer, + } builder := component.NewComponentManagerBuilder(). AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { c.log.Info().Msg("job consumer starting") - err := c.consumer.Start(defaultIndex) + err := c.consumer.Start() if err != nil { ctx.Throw(fmt.Errorf("could not start consumer: %w", err)) } @@ -95,7 +100,7 @@ func NewComponentConsumer( c.cm = cm c.Component = cm - return c + return c, nil } // SetPreNotifier sets a notification function that is invoked before marking a job as done in the diff --git a/module/jobqueue/consumer.go b/module/jobqueue/consumer.go index b13fc890d4e..ba9a7cf4d94 100644 --- a/module/jobqueue/consumer.go +++ b/module/jobqueue/consumer.go @@ -54,7 +54,14 @@ func NewConsumer( worker Worker, maxProcessing uint64, maxSearchAhead uint64, -) *Consumer { + defaultIndex uint64, +) (*Consumer, error) { + + processedIndex, err := readProcessedIndex(log, progress, defaultIndex) + if err != nil { + return nil, fmt.Errorf("could not read processed index: %w", err) + } + return &Consumer{ log: log.With().Str("sub_module", "job_queue").Logger(), @@ -71,48 +78,51 @@ func NewConsumer( running: false, isChecking: atomic.NewBool(false), started: atomic.NewBool(false), - processedIndex: 0, + processedIndex: processedIndex, processings: make(map[uint64]*jobStatus), processingsIndex: make(map[module.JobID]uint64), - } + }, nil } -// Start starts consuming the jobs from the job queue. -func (c *Consumer) Start(defaultIndex uint64) error { - c.mu.Lock() - defer c.mu.Unlock() - - if !c.started.CompareAndSwap(false, true) { - return fmt.Errorf("consumer has already been started") - } - c.running = true - +func readProcessedIndex(log zerolog.Logger, progress storage.ConsumerProgress, defaultIndex uint64) (uint64, error) { // on startup, sync with storage for the processed index // to ensure the consistency - processedIndex, err := c.progress.ProcessedIndex() + processedIndex, err := progress.ProcessedIndex() if errors.Is(err, storage.ErrNotFound) { - err := c.progress.InitProcessedIndex(defaultIndex) + err := progress.InitProcessedIndex(defaultIndex) if errors.Is(err, storage.ErrAlreadyExists) { - return fmt.Errorf("processed index has already been inited, no effect for the second time. default index: %v", + return 0, fmt.Errorf("processed index has already been inited, no effect for the second time. default index: %v", defaultIndex) } if err != nil { - return fmt.Errorf("could not init processed index: %w", err) + return 0, fmt.Errorf("could not init processed index: %w", err) } - processedIndex = defaultIndex - - c.log.Warn().Uint64("processed index", processedIndex). + log.Warn().Uint64("processed index", processedIndex). Msg("processed index not found, initialized.") - } else if err != nil { - return fmt.Errorf("could not read processed index: %w", err) + return defaultIndex, nil + } + + if err != nil { + return 0, fmt.Errorf("could not read processed index: %w", err) } - c.processedIndex = processedIndex + return processedIndex, nil +} + +// Start starts consuming the jobs from the job queue. +func (c *Consumer) Start() error { + c.mu.Lock() + defer c.mu.Unlock() + + if !c.started.CompareAndSwap(false, true) { + return fmt.Errorf("consumer has already been started") + } + c.running = true c.log.Info(). - Uint64("processed", processedIndex). + Uint64("processed", c.processedIndex). Msg("consumer started") c.checkProcessable() From 1acdb1fa91a5c8debd9f4dbb83b0085fe61e343f Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 19 Oct 2023 08:37:03 -0700 Subject: [PATCH 29/68] fix consumer_behavior_test --- module/jobqueue/consumer_behavior_test.go | 89 ++++++++++++----------- 1 file changed, 47 insertions(+), 42 deletions(-) diff --git a/module/jobqueue/consumer_behavior_test.go b/module/jobqueue/consumer_behavior_test.go index 1fac55faa96..98fc7395377 100644 --- a/module/jobqueue/consumer_behavior_test.go +++ b/module/jobqueue/consumer_behavior_test.go @@ -89,15 +89,15 @@ func TestConsumer(t *testing.T) { } func testOnStartup(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) assertProcessed(t, cp, 0) }) } func TestProcessedOrder(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(5)) + runWith(t, 5, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) assertProcessed(t, cp, 5) }) } @@ -105,8 +105,8 @@ func TestProcessedOrder(t *testing.T) { // [+1] => [0#, 1!] // when received job 1, it will be processed func testOnReceiveOneJob(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -121,8 +121,8 @@ func testOnReceiveOneJob(t *testing.T) { // [+1, 1*] => [0#, 1#] // when job 1 is finished, it will be marked as processed func testOnJobFinished(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -138,8 +138,8 @@ func testOnJobFinished(t *testing.T) { // [+1, +2, 1*, 2*] => [0#, 1#, 2#] // when job 2 and 1 are finished, they will be marked as processed func testOnJobsFinished(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -159,8 +159,8 @@ func testOnJobsFinished(t *testing.T) { // [+1, +2, +3, +4] => [0#, 1!, 2!, 3!, 4] // when more jobs are arrived than the max number of workers, only the first 3 jobs will be processed func testMaxWorker(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -183,8 +183,8 @@ func testMaxWorker(t *testing.T) { // [+1, +2, +3, +4, +5, +6] => [0#, !1, *2, *3, *4, *5, 6, +7] => [0#, *1, *2, *3, *4, *5, !6, !7] // when processing lags behind, the consumer is paused until processing catches up func testPauseResume(t *testing.T) { - runWithSeatchAhead(t, 5, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWithSeatchAhead(t, 5, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -229,8 +229,8 @@ func testPauseResume(t *testing.T) { // [+1, +2, +3, +4, 3*] => [0#, 1!, 2!, 3*, 4!] // when job 3 is finished, which is not the next processing job 1, the processed index won't change func testNonNextFinished(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -259,8 +259,8 @@ func testNonNextFinished(t *testing.T) { // // [+1, +2, +3, +3, +4] => [1, 2, 3*, 4] => [1, 2, 3*, 4*] => => [1#, 2, 3*, 4*] => [1#, 2#, 3#, 4#] func testMovingProcessedIndex(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -312,8 +312,8 @@ func testMovingProcessedIndex(t *testing.T) { // [+1, +2, +3, +4, 3*, 2*] => [0#, 1!, 2*, 3*, 4!] // when job 3 and 2 are finished, the processed index won't change, because 1 is still not finished func testTwoNonNextFinished(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -339,8 +339,8 @@ func testTwoNonNextFinished(t *testing.T) { // [+1, +2, +3, +4, 3*, 2*, +5] => [0#, 1!, 2*, 3*, 4!, 5!] // when job 5 is received, it will be processed, because the worker has capacity func testProcessingWithNonNextFinished(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -369,8 +369,8 @@ func testProcessingWithNonNextFinished(t *testing.T) { // [+1, +2, +3, +4, 3*, 2*, +5, +6] => [0#, 1!, 2*, 3*, 4!, 5!, 6] // when job 6 is received, no more worker can process it, it will be buffered func testMaxWorkerWithFinishedNonNexts(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -402,8 +402,8 @@ func testMaxWorkerWithFinishedNonNexts(t *testing.T) { // [+1, +2, +3, +4, 3*, 2*, +5, 1*] => [0#, 1#, 2#, 3#, 4!, 5!] // when job 1 is finally finished, it will fast forward the processed index to 3 func testFastforward(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -434,8 +434,8 @@ func testFastforward(t *testing.T) { // [+1, +2, +3, +4, 3*, 2*, +5, 1*, +6, +7, 6*], restart => [0#, 1#, 2#, 3#, 4!, 5!, 6*, 7!] // when job queue crashed and restarted, the queue can be resumed func testWorkOnNextAfterFastforward(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) require.NoError(t, j.PushOne()) // +1 c.Check() @@ -470,9 +470,9 @@ func testWorkOnNextAfterFastforward(t *testing.T) { // jobs need to be reused, since it stores all the jobs reWorker := newMockWorker() reProgress := badger.NewConsumerProgress(db, ConsumerTag) - reConsumer := newTestConsumer(reProgress, j, reWorker, 0) + reConsumer := newTestConsumer(t, reProgress, j, reWorker, 0, DefaultIndex) - err := reConsumer.Start(DefaultIndex) + err := reConsumer.Start() require.NoError(t, err) time.Sleep(1 * time.Millisecond) @@ -485,8 +485,8 @@ func testWorkOnNextAfterFastforward(t *testing.T) { // [+1, +2, +3, +4, Stop, 2*] => [0#, 1!, 2*, 3!, 4] // when Stop is called, it won't work on any job any more func testStopRunning(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) for i := 0; i < 4; i++ { require.NoError(t, j.PushOne()) c.Check() @@ -507,8 +507,8 @@ func testStopRunning(t *testing.T) { } func testConcurrency(t *testing.T) { - runWith(t, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { - require.NoError(t, c.Start(DefaultIndex)) + runWith(t, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + require.NoError(t, c.Start()) var finishAll sync.WaitGroup finishAll.Add(100) // Finish job concurrently @@ -549,16 +549,19 @@ func testConcurrency(t *testing.T) { type JobID = module.JobID type Job = module.Job -func runWith(t testing.TB, runTestWith func(module.JobConsumer, storage.ConsumerProgress, *mockWorker, *jobqueue.MockJobs, *badgerdb.DB)) { - runWithSeatchAhead(t, 0, runTestWith) +func runWith(t testing.TB, + defaultIndex uint64, + runTestWith func(module.JobConsumer, storage.ConsumerProgress, *mockWorker, *jobqueue.MockJobs, *badgerdb.DB)) { + runWithSeatchAhead(t, 0, defaultIndex, runTestWith) } -func runWithSeatchAhead(t testing.TB, maxSearchAhead uint64, runTestWith func(module.JobConsumer, storage.ConsumerProgress, *mockWorker, *jobqueue.MockJobs, *badgerdb.DB)) { +func runWithSeatchAhead(t testing.TB, maxSearchAhead uint64, defaultIndex uint64, + runTestWith func(module.JobConsumer, storage.ConsumerProgress, *mockWorker, *jobqueue.MockJobs, *badgerdb.DB)) { unittest.RunWithBadgerDB(t, func(db *badgerdb.DB) { jobs := jobqueue.NewMockJobs() worker := newMockWorker() progress := badger.NewConsumerProgress(db, ConsumerTag) - consumer := newTestConsumer(progress, jobs, worker, maxSearchAhead) + consumer := newTestConsumer(t, progress, jobs, worker, maxSearchAhead, defaultIndex) runTestWith(consumer, progress, worker, jobs, db) }) } @@ -569,10 +572,12 @@ func assertProcessed(t testing.TB, cp storage.ConsumerProgress, expectProcessed require.Equal(t, expectProcessed, processed) } -func newTestConsumer(cp storage.ConsumerProgress, jobs module.Jobs, worker jobqueue.Worker, maxSearchAhead uint64) module.JobConsumer { +func newTestConsumer(t testing.TB, cp storage.ConsumerProgress, jobs module.Jobs, worker jobqueue.Worker, maxSearchAhead uint64, defaultIndex uint64) module.JobConsumer { log := unittest.Logger().With().Str("module", "consumer").Logger() maxProcessing := uint64(3) - return jobqueue.NewConsumer(log, jobs, cp, worker, maxProcessing, maxSearchAhead) + c, err := jobqueue.NewConsumer(log, jobs, cp, worker, maxProcessing, maxSearchAhead, defaultIndex) + require.NoError(t, err) + return c } // a Mock worker that stores all the jobs that it was asked to work on @@ -629,7 +634,7 @@ func (w *mockWorker) AssertCalled(t *testing.T, expectCalled []int64) { // 0.22 ms to finish job func BenchmarkPushAndConsume(b *testing.B) { b.StopTimer() - runWith(b, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { + runWith(b, DefaultIndex, func(c module.JobConsumer, cp storage.ConsumerProgress, w *mockWorker, j *jobqueue.MockJobs, db *badgerdb.DB) { var wg sync.WaitGroup wg.Add(b.N) @@ -641,7 +646,7 @@ func BenchmarkPushAndConsume(b *testing.B) { }() } - require.NoError(b, c.Start(DefaultIndex)) + require.NoError(b, c.Start()) b.StartTimer() for i := 0; i < b.N; i++ { From 402e6a596594736fcdf281803cf35948d9bb2bb2 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 19 Oct 2023 08:42:25 -0700 Subject: [PATCH 30/68] fix lint --- .../assigner/blockconsumer/consumer.go | 2 +- module/jobqueue/component_consumer.go | 14 +++++++------- module/jobqueue/component_consumer_test.go | 4 +++- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/engine/verification/assigner/blockconsumer/consumer.go b/engine/verification/assigner/blockconsumer/consumer.go index 10cf18a461f..7b6341be000 100644 --- a/engine/verification/assigner/blockconsumer/consumer.go +++ b/engine/verification/assigner/blockconsumer/consumer.go @@ -101,7 +101,7 @@ func (c *BlockConsumer) OnFinalizedBlock(*model.Block) { } func (c *BlockConsumer) Ready() <-chan struct{} { - err := c.consumer.Start(c.defaultIndex) + err := c.consumer.Start() if err != nil { panic(fmt.Errorf("could not start block consumer for finder engine: %w", err)) } diff --git a/module/jobqueue/component_consumer.go b/module/jobqueue/component_consumer.go index 86c16acba7d..1b174e712ad 100644 --- a/module/jobqueue/component_consumer.go +++ b/module/jobqueue/component_consumer.go @@ -35,6 +35,12 @@ func NewComponentConsumer( maxSearchAhead uint64, ) (*ComponentConsumer, error) { + c := &ComponentConsumer{ + workSignal: workSignal, + jobs: jobs, + log: log, + } + // create a worker pool with maxProcessing workers to process jobs worker := NewWorkerPool( processor, @@ -46,13 +52,7 @@ func NewComponentConsumer( if err != nil { return nil, err } - - c := &ComponentConsumer{ - workSignal: workSignal, - jobs: jobs, - log: log, - consumer: consumer, - } + c.consumer = consumer builder := component.NewComponentManagerBuilder(). AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { diff --git a/module/jobqueue/component_consumer_test.go b/module/jobqueue/component_consumer_test.go index c02f43a6c24..574423b1bc9 100644 --- a/module/jobqueue/component_consumer_test.go +++ b/module/jobqueue/component_consumer_test.go @@ -11,6 +11,7 @@ import ( "github.com/rs/zerolog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.uber.org/atomic" @@ -88,7 +89,7 @@ func (suite *ComponentConsumerSuite) prepareTest( progress.On("ProcessedIndex").Return(suite.defaultIndex, nil) progress.On("SetProcessedIndex", mock.AnythingOfType("uint64")).Return(nil) - consumer := NewComponentConsumer( + consumer, err := NewComponentConsumer( zerolog.New(os.Stdout).With().Timestamp().Logger(), workSignal, progress, @@ -98,6 +99,7 @@ func (suite *ComponentConsumerSuite) prepareTest( suite.maxProcessing, suite.maxSearchAhead, ) + require.NoError(suite.T(), err) consumer.SetPreNotifier(preNotifier) consumer.SetPostNotifier(postNotifier) From 4cb3062ab9209f4537b768f02b2ca3076a3084ba Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 19 Oct 2023 08:47:19 -0700 Subject: [PATCH 31/68] fix lint --- cmd/access/node_builder/access_node_builder.go | 11 +++++++++-- module/jobqueue/consumer_test.go | 5 +++-- module/state_synchronization/indexer/indexer.go | 12 +++++++++--- .../state_synchronization/indexer/indexer_test.go | 3 ++- .../requester/execution_data_requester.go | 15 +++++++++++---- .../requester/execution_data_requester_test.go | 3 ++- 6 files changed, 36 insertions(+), 13 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b1a97d930ce..4019f8d7b52 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -622,7 +622,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess execDataCacheBackend, ) - builder.ExecutionDataRequester = edrequester.New( + r, err := edrequester.New( builder.Logger, metrics.NewExecutionDataRequesterCollector(), builder.ExecutionDataDownloader, @@ -634,6 +634,10 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.executionDataConfig, execDataDistributor, ) + if err != nil { + return nil, fmt.Errorf("failed to create execution data requester: %w", err) + } + builder.ExecutionDataRequester = r builder.FollowerDistributor.AddOnBlockFinalizedConsumer(builder.ExecutionDataRequester.OnBlockFinalized) @@ -737,7 +741,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.ExecutionIndexerCore = indexerCore // execution state worker uses a jobqueue to process new execution data and indexes it by using the indexer. - builder.ExecutionIndexer = indexer.NewIndexer( + builder.ExecutionIndexer, err = indexer.NewIndexer( builder.Logger, registers.FirstHeight(), registers, @@ -746,6 +750,9 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.ExecutionDataRequester.HighestConsecutiveHeight, indexedBlockHeight, ) + if err != nil { + return nil, err + } // setup requester to notify indexer when new execution data is received execDataDistributor.AddOnExecutionDataReceivedConsumer(builder.ExecutionIndexer.OnExecutionData) diff --git a/module/jobqueue/consumer_test.go b/module/jobqueue/consumer_test.go index 47b35a70edc..3af02c057cd 100644 --- a/module/jobqueue/consumer_test.go +++ b/module/jobqueue/consumer_test.go @@ -164,7 +164,8 @@ func TestProcessedIndexDeletion(t *testing.T) { progress := badger.NewConsumerProgress(db, "consumer") worker := newMockWorker() maxProcessing := uint64(3) - c := NewConsumer(log, jobs, progress, worker, maxProcessing, 0) + c, err := NewConsumer(log, jobs, progress, worker, maxProcessing, 0, 0) + require.NoError(t, err) worker.WithConsumer(c) f(c, jobs) @@ -173,7 +174,7 @@ func TestProcessedIndexDeletion(t *testing.T) { setup(t, func(c *Consumer, jobs *MockJobs) { require.NoError(t, jobs.PushN(10)) - require.NoError(t, c.Start(0)) + require.NoError(t, c.Start()) require.Eventually(t, func() bool { c.mu.Lock() diff --git a/module/state_synchronization/indexer/indexer.go b/module/state_synchronization/indexer/indexer.go index e19ad58f027..e3fcaa2551b 100644 --- a/module/state_synchronization/indexer/indexer.go +++ b/module/state_synchronization/indexer/indexer.go @@ -1,6 +1,7 @@ package indexer import ( + "fmt" "time" "github.com/rs/zerolog" @@ -55,7 +56,7 @@ func NewIndexer( executionCache *cache.ExecutionDataCache, executionDataLatestHeight func() (uint64, error), processedHeight storage.ConsumerProgress, -) *Indexer { +) (*Indexer, error) { r := &Indexer{ log: log.With().Str("module", "execution_indexer").Logger(), exeDataNotifier: engine.NewNotifier(), @@ -67,7 +68,7 @@ func NewIndexer( // create a jobqueue that will process new available block execution data. The `exeDataNotifier` is used to // signal new work, which is being triggered on the `OnExecutionData` handler. - r.jobConsumer = jobqueue.NewComponentConsumer( + jobConsumer, err := jobqueue.NewComponentConsumer( r.log, r.exeDataNotifier.Channel(), processedHeight, @@ -77,10 +78,15 @@ func NewIndexer( workersCount, searchAhead, ) + if err != nil { + return nil, fmt.Errorf("error creating execution data jobqueue: %w", err) + } + + r.jobConsumer = jobConsumer r.Component = r.jobConsumer - return r + return r, nil } // Start the worker jobqueue to consume the available data. diff --git a/module/state_synchronization/indexer/indexer_test.go b/module/state_synchronization/indexer/indexer_test.go index f7b1104d1ea..126afb75274 100644 --- a/module/state_synchronization/indexer/indexer_test.go +++ b/module/state_synchronization/indexer/indexer_test.go @@ -75,7 +75,7 @@ func newIndexerTest(t *testing.T, availableBlocks int, lastIndexedIndex int) *in executionData: executionData, } - test.worker = NewIndexer( + test.worker, err = NewIndexer( unittest.Logger(), test.first().Header.Height, registers, @@ -84,6 +84,7 @@ func newIndexerTest(t *testing.T, availableBlocks int, lastIndexedIndex int) *in test.latestHeight, progress, ) + require.NoError(t, err) return test } diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 624d7c74ed9..4ed489371dd 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -151,7 +151,7 @@ func New( headers storage.Headers, cfg ExecutionDataConfig, distributor *ExecutionDataDistributor, -) state_synchronization.ExecutionDataRequester { +) (state_synchronization.ExecutionDataRequester, error) { e := &executionDataRequester{ log: log.With().Str("component", "execution_data_requester").Logger(), downloader: downloader, @@ -179,7 +179,7 @@ func New( // from `processedHeight + 1`. If the database is empty, rootHeight will be used to init the // last processed height. Once the execution data is fetched and stored, it notifies // `executionDataNotifier`. - e.blockConsumer = jobqueue.NewComponentConsumer( + blockConsumer, err := jobqueue.NewComponentConsumer( e.log.With().Str("module", "block_consumer").Logger(), e.finalizationNotifier.Channel(), // to listen to finalization events to find newly sealed blocks processedHeight, // read and persist the downloaded height @@ -189,6 +189,10 @@ func New( fetchWorkers, // the number of concurrent workers e.config.MaxSearchAhead, // max number of unsent notifications to allow before pausing new fetches ) + if err != nil { + return nil, fmt.Errorf("failed to create block consumer: %w", err) + } + e.blockConsumer = blockConsumer // notifies notificationConsumer when new ExecutionData blobs are available // SetPostNotifier will notify executionDataNotifier AFTER e.blockConsumer.LastProcessedIndex is updated. @@ -222,7 +226,7 @@ func New( // `e.consumers`. // Note: the `e.consumers` will be guaranteed to receive at least one `OnExecutionDataFetched` event // for each sealed block in consecutive block height order. - e.notificationConsumer = jobqueue.NewComponentConsumer( + e.notificationConsumer, err = jobqueue.NewComponentConsumer( e.log.With().Str("module", "notification_consumer").Logger(), executionDataNotifier.Channel(), // listen for notifications from the block consumer processedNotifications, // read and persist the notified height @@ -232,13 +236,16 @@ func New( 1, // use a single worker to ensure notification is delivered in consecutive order 0, // search ahead limit controlled by worker count ) + if err != nil { + return nil, fmt.Errorf("failed to create notification consumer: %w", err) + } e.Component = component.NewComponentManagerBuilder(). AddWorker(e.runBlockConsumer). AddWorker(e.runNotificationConsumer). Build() - return e + return e, nil } // OnBlockFinalized accepts block finalization notifications from the FollowerDistributor diff --git a/module/state_synchronization/requester/execution_data_requester_test.go b/module/state_synchronization/requester/execution_data_requester_test.go index b66c7fc9408..b85ce646fa2 100644 --- a/module/state_synchronization/requester/execution_data_requester_test.go +++ b/module/state_synchronization/requester/execution_data_requester_test.go @@ -415,7 +415,7 @@ func (suite *ExecutionDataRequesterSuite) prepareRequesterTest(cfg *fetchTestRun processedHeight := bstorage.NewConsumerProgress(suite.db, module.ConsumeProgressExecutionDataRequesterBlockHeight) processedNotification := bstorage.NewConsumerProgress(suite.db, module.ConsumeProgressExecutionDataRequesterNotification) - edr := requester.New( + edr, err := requester.New( logger, metrics, suite.downloader, @@ -433,6 +433,7 @@ func (suite *ExecutionDataRequesterSuite) prepareRequesterTest(cfg *fetchTestRun }, suite.distributor, ) + require.NoError(suite.T(), err) followerDistributor.AddOnBlockFinalizedConsumer(edr.OnBlockFinalized) From 5a5439d0eef5cc091b5536935348232740273a4c Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 14:33:27 -0500 Subject: [PATCH 32/68] encapsulate SyncEngine ALSP config in separate struct --- engine/common/synchronization/config.go | 6 +++--- network/netconf/config.go | 5 +++++ network/netconf/flags.go | 6 +++--- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/engine/common/synchronization/config.go b/engine/common/synchronization/config.go index 36c52dbf5fd..bf67c2da572 100644 --- a/engine/common/synchronization/config.go +++ b/engine/common/synchronization/config.go @@ -77,8 +77,8 @@ func NewSpamDetectionConfig() (*SpamDetectionConfig, error) { return &SpamDetectionConfig{ // see config/default-config.yml for more information on the following fields - batchRequestBaseProb: flowConfig.NetworkConfig.SyncEngineBatchRequestBaseProb, - syncRequestProb: flowConfig.NetworkConfig.SyncEngineSyncRequestProb, - rangeRequestBaseProb: flowConfig.NetworkConfig.SyncEngineRangeRequestBaseProb, + batchRequestBaseProb: flowConfig.NetworkConfig.SyncEngine.SyncEngineBatchRequestBaseProb, + syncRequestProb: flowConfig.NetworkConfig.SyncEngine.SyncEngineSyncRequestProb, + rangeRequestBaseProb: flowConfig.NetworkConfig.SyncEngine.SyncEngineRangeRequestBaseProb, }, nil } diff --git a/network/netconf/config.go b/network/netconf/config.go index 3ae0de87780..cfe424214d2 100644 --- a/network/netconf/config.go +++ b/network/netconf/config.go @@ -115,6 +115,11 @@ type AlspConfig struct { // events that are used to perform critical ALSP tasks, such as updating the spam records cache. HearBeatInterval time.Duration `mapstructure:"alsp-heart-beat-interval"` + SyncEngine SyncEngineAlspConfig +} + +// SyncEngineAlspConfig is the ALSP config for the SyncEngine. +type SyncEngineAlspConfig struct { // SyncEngineBatchRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a // misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, // since it's not the final probability and there are other factors that determine the final probability. diff --git a/network/netconf/flags.go b/network/netconf/flags.go index 1e021c6f589..da6ec0c4ed5 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -204,9 +204,9 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Uint32(alspSpamRecordCacheSize, config.AlspConfig.SpamRecordCacheSize, "size of spam record cache, recommended to be 10x the number of authorized nodes") flags.Uint32(alspSpamRecordQueueSize, config.AlspConfig.SpamReportQueueSize, "size of spam report queue, recommended to be 100x the number of authorized nodes") flags.Duration(alspHearBeatInterval, config.AlspConfig.HearBeatInterval, "interval between two consecutive heartbeat events at alsp, recommended to leave it as default unless you know what you are doing.") - flags.Float32(alspSyncEngineBatchRequestBaseProb, config.AlspConfig.SyncEngineBatchRequestBaseProb, "base probability of creating a misbehavior report for a batch request message") - flags.Float32(alspSyncEngineRangeRequestBaseProb, config.AlspConfig.SyncEngineRangeRequestBaseProb, "base probability of creating a misbehavior report for a range request message") - flags.Float32(alspSyncEngineSyncRequestProb, config.AlspConfig.SyncEngineSyncRequestProb, "probability of creating a misbehavior report for a sync request message") + flags.Float32(alspSyncEngineBatchRequestBaseProb, config.AlspConfig.SyncEngine.SyncEngineBatchRequestBaseProb, "base probability of creating a misbehavior report for a batch request message") + flags.Float32(alspSyncEngineRangeRequestBaseProb, config.AlspConfig.SyncEngine.SyncEngineRangeRequestBaseProb, "base probability of creating a misbehavior report for a range request message") + flags.Float32(alspSyncEngineSyncRequestProb, config.AlspConfig.SyncEngine.SyncEngineSyncRequestProb, "probability of creating a misbehavior report for a sync request message") flags.Int(ihaveMaxSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxSampleSize, "max number of ihaves to sample when performing validation") flags.Int(ihaveMaxMessageIDSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxMessageIDSampleSize, "max number of message ids to sample when performing validation per ihave") From 32e847d768e484bf03994f59d38da06683abc091 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 20 Oct 2023 13:23:35 -0700 Subject: [PATCH 33/68] update mock --- module/mock/job_consumer.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/module/mock/job_consumer.go b/module/mock/job_consumer.go index 346231f09fc..9c4ebb17d04 100644 --- a/module/mock/job_consumer.go +++ b/module/mock/job_consumer.go @@ -59,13 +59,13 @@ func (_m *JobConsumer) Size() uint { return r0 } -// Start provides a mock function with given fields: defaultIndex -func (_m *JobConsumer) Start(defaultIndex uint64) error { - ret := _m.Called(defaultIndex) +// Start provides a mock function with given fields: +func (_m *JobConsumer) Start() error { + ret := _m.Called() var r0 error - if rf, ok := ret.Get(0).(func(uint64) error); ok { - r0 = rf(defaultIndex) + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() } else { r0 = ret.Error(0) } From 541a7182778071145f4bd16b82063ef2740f8908 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 20 Oct 2023 13:24:47 -0700 Subject: [PATCH 34/68] fix lint --- module/jobqueue/consumer_test.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/module/jobqueue/consumer_test.go b/module/jobqueue/consumer_test.go index 3af02c057cd..90db5332f79 100644 --- a/module/jobqueue/consumer_test.go +++ b/module/jobqueue/consumer_test.go @@ -201,21 +201,23 @@ func TestCheckBeforeStartIsNoop(t *testing.T) { err := progress.InitProcessedIndex(storedProcessedIndex) require.NoError(t, err) - c := NewConsumer( + c, err := NewConsumer( unittest.Logger(), NewMockJobs(), progress, worker, uint64(3), 0, + 10, ) + require.NoError(t, err) worker.WithConsumer(c) // check will store the processedIndex. Before start, it will be uninitialized (0) c.Check() // start will load the processedIndex from storage - err = c.Start(10) + err = c.Start() require.NoError(t, err) // make sure that the processedIndex at the end is from storage From bf7b982735791f976ec86ec319cab48455f46c2b Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Fri, 20 Oct 2023 16:02:42 -0500 Subject: [PATCH 35/68] SyncEngineAlspConfig - squashed, fixed valid range values --- network/netconf/config.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/network/netconf/config.go b/network/netconf/config.go index cfe424214d2..ae41cde6a0d 100644 --- a/network/netconf/config.go +++ b/network/netconf/config.go @@ -115,7 +115,7 @@ type AlspConfig struct { // events that are used to perform critical ALSP tasks, such as updating the spam records cache. HearBeatInterval time.Duration `mapstructure:"alsp-heart-beat-interval"` - SyncEngine SyncEngineAlspConfig + SyncEngine SyncEngineAlspConfig `mapstructure:",squash"` } // SyncEngineAlspConfig is the ALSP config for the SyncEngine. @@ -124,14 +124,14 @@ type SyncEngineAlspConfig struct { // misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, // since it's not the final probability and there are other factors that determine the final probability. // The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. - SyncEngineBatchRequestBaseProb float32 `validate:"range=0,1" mapstructure:"alsp-sync-engine-batch-request-base-prob"` + SyncEngineBatchRequestBaseProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-batch-request-base-prob"` // SyncEngineRangeRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a // misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, // since it's not the final probability and there are other factors that determine the final probability. // The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. - SyncEngineRangeRequestBaseProb float32 `validate:"range=0,1" mapstructure:"alsp-sync-engine-range-request-base-prob"` + SyncEngineRangeRequestBaseProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-range-request-base-prob"` // SyncEngineSyncRequestProb is the probability in [0,1] of creating a misbehavior report for a SyncRequest message. - SyncEngineSyncRequestProb float32 `validate:"range=0,1" mapstructure:"alsp-sync-engine-sync-request-prob"` + SyncEngineSyncRequestProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-sync-request-prob"` } From 590e81b796562340d80f56738a0ea9cd690107c0 Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Sat, 21 Oct 2023 08:42:53 -0500 Subject: [PATCH 36/68] shorten field names in encapsulated SyncEngine ALSP config separate struct --- engine/common/synchronization/config.go | 6 +++--- network/netconf/config.go | 12 ++++++------ network/netconf/flags.go | 6 +++--- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/engine/common/synchronization/config.go b/engine/common/synchronization/config.go index bf67c2da572..48ecf8b94a5 100644 --- a/engine/common/synchronization/config.go +++ b/engine/common/synchronization/config.go @@ -77,8 +77,8 @@ func NewSpamDetectionConfig() (*SpamDetectionConfig, error) { return &SpamDetectionConfig{ // see config/default-config.yml for more information on the following fields - batchRequestBaseProb: flowConfig.NetworkConfig.SyncEngine.SyncEngineBatchRequestBaseProb, - syncRequestProb: flowConfig.NetworkConfig.SyncEngine.SyncEngineSyncRequestProb, - rangeRequestBaseProb: flowConfig.NetworkConfig.SyncEngine.SyncEngineRangeRequestBaseProb, + batchRequestBaseProb: flowConfig.NetworkConfig.SyncEngine.BatchRequestBaseProb, + syncRequestProb: flowConfig.NetworkConfig.SyncEngine.SyncRequestProb, + rangeRequestBaseProb: flowConfig.NetworkConfig.SyncEngine.RangeRequestBaseProb, }, nil } diff --git a/network/netconf/config.go b/network/netconf/config.go index ae41cde6a0d..8ae6099c376 100644 --- a/network/netconf/config.go +++ b/network/netconf/config.go @@ -120,18 +120,18 @@ type AlspConfig struct { // SyncEngineAlspConfig is the ALSP config for the SyncEngine. type SyncEngineAlspConfig struct { - // SyncEngineBatchRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a + // BatchRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a // misbehavior report for a BatchRequest message. This is why the word "base" is used in the name of this field, // since it's not the final probability and there are other factors that determine the final probability. // The reason for this is that we want to increase the probability of creating a misbehavior report for a large batch. - SyncEngineBatchRequestBaseProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-batch-request-base-prob"` + BatchRequestBaseProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-batch-request-base-prob"` - // SyncEngineRangeRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a + // RangeRequestBaseProb is the base probability in [0,1] that's used in creating the final probability of creating a // misbehavior report for a RangeRequest message. This is why the word "base" is used in the name of this field, // since it's not the final probability and there are other factors that determine the final probability. // The reason for this is that we want to increase the probability of creating a misbehavior report for a large range. - SyncEngineRangeRequestBaseProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-range-request-base-prob"` + RangeRequestBaseProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-range-request-base-prob"` - // SyncEngineSyncRequestProb is the probability in [0,1] of creating a misbehavior report for a SyncRequest message. - SyncEngineSyncRequestProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-sync-request-prob"` + // SyncRequestProb is the probability in [0,1] of creating a misbehavior report for a SyncRequest message. + SyncRequestProb float32 `validate:"gte=0,lte=1" mapstructure:"alsp-sync-engine-sync-request-prob"` } diff --git a/network/netconf/flags.go b/network/netconf/flags.go index da6ec0c4ed5..8069a5e607d 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -204,9 +204,9 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Uint32(alspSpamRecordCacheSize, config.AlspConfig.SpamRecordCacheSize, "size of spam record cache, recommended to be 10x the number of authorized nodes") flags.Uint32(alspSpamRecordQueueSize, config.AlspConfig.SpamReportQueueSize, "size of spam report queue, recommended to be 100x the number of authorized nodes") flags.Duration(alspHearBeatInterval, config.AlspConfig.HearBeatInterval, "interval between two consecutive heartbeat events at alsp, recommended to leave it as default unless you know what you are doing.") - flags.Float32(alspSyncEngineBatchRequestBaseProb, config.AlspConfig.SyncEngine.SyncEngineBatchRequestBaseProb, "base probability of creating a misbehavior report for a batch request message") - flags.Float32(alspSyncEngineRangeRequestBaseProb, config.AlspConfig.SyncEngine.SyncEngineRangeRequestBaseProb, "base probability of creating a misbehavior report for a range request message") - flags.Float32(alspSyncEngineSyncRequestProb, config.AlspConfig.SyncEngine.SyncEngineSyncRequestProb, "probability of creating a misbehavior report for a sync request message") + flags.Float32(alspSyncEngineBatchRequestBaseProb, config.AlspConfig.SyncEngine.BatchRequestBaseProb, "base probability of creating a misbehavior report for a batch request message") + flags.Float32(alspSyncEngineRangeRequestBaseProb, config.AlspConfig.SyncEngine.RangeRequestBaseProb, "base probability of creating a misbehavior report for a range request message") + flags.Float32(alspSyncEngineSyncRequestProb, config.AlspConfig.SyncEngine.SyncRequestProb, "probability of creating a misbehavior report for a sync request message") flags.Int(ihaveMaxSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxSampleSize, "max number of ihaves to sample when performing validation") flags.Int(ihaveMaxMessageIDSampleSize, config.GossipSubConfig.GossipSubRPCInspectorsConfig.GossipSubRPCValidationInspectorConfigs.IHaveRPCInspectionConfig.MaxMessageIDSampleSize, "max number of message ids to sample when performing validation per ihave") From 9bf7ec8c1eaf365ae8990f327046ba4cb5d08b8e Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Mon, 23 Oct 2023 08:06:17 -0500 Subject: [PATCH 37/68] config/README.md updated, clarified, fixed broken links --- config/README.md | 53 ++++++++++++++++++++++++--------------- network/netconf/config.go | 2 +- 2 files changed, 34 insertions(+), 21 deletions(-) diff --git a/config/README.md b/config/README.md index 01e999ef3a6..eceb591dc2c 100644 --- a/config/README.md +++ b/config/README.md @@ -11,7 +11,7 @@ struct is defined for all of these network related components in the netconf sub ### Overriding default values The entire default config can be overridden using the `--config-file` CLI flag. When set the config package will attempt to parse the specified config file and override all the values -defined. A single default value can be overridden by setting the CLI flag for that specific config. For example `--network-connection-pruning=false` will override the default network connection pruning +defined. A single default value can be overridden by setting the CLI flag for that specific config. For example `--networking-connection-pruning=false` will override the default network connection pruning config to false. Override entire config file. ```shell @@ -21,19 +21,19 @@ go build -tags relic -o flow-access-node ./cmd/access Override a single configuration value. ```shell go build -tags relic -o flow-access-node ./cmd/access -./flow-access-node --network-connection-pruning=false +./flow-access-node --networking-connection-pruning=false ``` ### Adding a new config value Adding a new config to the FlowConfig can be done in a few easy steps. The network package can be used as a good example of how to add CLI flags and will be used in the steps below. -1. Create a new subpackage in the config package for the new configuration structs to live. Although it is encouraged to put all configuration sub-packages in the config package -so that configuration can be updated in one place these sub-packages can live anywhere. This package will define the configuration structs and CLI flags for overriding. +1. Create a new configuration package for the new configuration structs and CLI flags. Although it is encouraged to define all package configuration structs and CLI flags as a subpackage of the component they are related to, +the configuration package can live anywhere. This configuration package will define the configuration structs and CLI flags for overriding. ```shell mkdir example_config ``` - For the network package we have a subpackage created in [network/netconf](https://github.com/onflow/flow-go/tree/master/network/netconf). + For the network package we have a configuration subpackage created in [network/netconf](https://github.com/onflow/flow-go/tree/master/network/netconf). 2. Add a new CLI flag for the config value. ```go @@ -49,22 +49,22 @@ so that configuration can be updated in one place these sub-packages can live an `InitializeNetworkFlags` is used during initialization of all flags in the `InitializePFlagSet` function in the [config/base_flags.go](https://github.com/onflow/flow-go/blob/master/config/base_flags.go). -3. Add the config as a new field to an existing configuration struct or create a new struct. Each configuration struct must be a field on the FlowConfig struct so that it is unmarshalled during configuration initialization. +3. Add the configuration as a new field to an existing configuration struct or create a new configuration struct. Each configuration struct must be a field on the FlowConfig struct so that it is unmarshalled during configuration initialization. Each field on a configuration struct must contain the following field tags. 1. `validate` - validate tag is used to perform validation on field structs using the [validator](https://github.com/go-playground/validator) package. In the example below you will notice the `validate:"gt=0"` tag, this will ensure that the value of `AppWorkers` is greater than 0. The top level `FlowConfig` struct has a Validate method that performs struct validation. This - validation is done with the validator package, each validate tag on ever struct field and sub struct field will be validated and validation errors are returned. - 2. `mapstructure` - mapstructure tag is used for unmarshalling and must match the CLI flag name defined in step or else the field will not be set when the config is unmarshalled. + validation is done with the validator package, each validate tag on every struct field and sub struct field will be validated and validation errors are returned. + 2. `mapstructure` - [mapstructure](https://github.com/mitchellh/mapstructure) tag is used for unmarshalling and must match the CLI flag name defined in step or else the field will not be set when the config is unmarshalled. ```go type MyComponentConfig struct { AppWorkers int `validate:"gt=0" mapstructure:"app-workers"` } ``` - It's important to make sure that the CLI flag name matches the mapstructure field tag to avoid parsing errors. + It's important to make sure that each mapstructure field tag matches the CLI flag name in [config/default-config.yml](https://github.com/onflow/flow-go/blob/master/config/default-config.yml) to avoid parsing errors. All network package configuration structs are defined in [network/netconf/config.go](https://github.com/onflow/flow-go/blob/master/network/netconf/config.go) -4. Add the new config and a default value to the `default-config.yml` file. Ensure that the new property added matches the configuration struct structure for the subpackage the config belongs to. +4. Add the new configuration properties and default values to [config/default-config.yml](https://github.com/onflow/flow-go/blob/master/config/default-config.yml). Ensure that each new property added matches the mapstructure value of the configuration struct field. ```yaml config-file: "./default-config.yml" network-config: @@ -75,7 +75,7 @@ so that configuration can be updated in one place these sub-packages can live an All network package configuration values are defined under `network-config` in `default-config.yml` -5. If a new struct was created in step 3, add it as a new field to `FlowConfig` struct in [config/config.go](https://github.com/onflow/flow-go/blob/master/config/config.go). In the previous steps we added a new config struct and added a new property to the `default-config.yml` for this struct `my-component`. This property name +5. If a new struct was created in step 3, add it as a new field to `FlowConfig` struct in [config/config.go](https://github.com/onflow/flow-go/blob/master/config/config.go). In the previous steps we added a new config struct and added a new property to [config/default-config.yml](https://github.com/onflow/flow-go/blob/master/config/default-config.yml) for the `my-component` struct. This property name must match the mapstructure field tag for the struct when added to `FlowConfig`. ```go // FlowConfig Flow configuration. @@ -87,26 +87,39 @@ so that configuration can be updated in one place these sub-packages can live an ``` The network package configuration struct, `NetworkConfig`, is already embedded as a field in `FlowConfig` struct. - This means that new flags can be added to the network package configuration struct without having to update the `FlowConfig` struct. + This means that new fields can be added to the network package configuration struct without having to update the `FlowConfig` struct. ### Nested structs -In an effort to keep the configuration yaml structure readable some configuration will be in nested properties. When this is the case the mapstructure `squash` tag can be used so that the corresponding nested struct will be -flattened before the configuration is unmarshalled. This is used in the network package which is a collection of configuration structs nested on the network.Config struct. +In an effort to keep the configuration yaml structure readable some configuration properties will be defined in nested structs. When this is the case the mapstructure [squash](https://pkg.go.dev/github.com/mitchellh/mapstructure#example-Decode-EmbeddedStruct) tag must be used in the parent struct so that the corresponding nested struct will be +flattened before the configuration is unmarshalled. This is used in the network package which is a collection of configuration structs nested in the `network.Config` struct: ```go type Config struct { + UnicastConfig `mapstructure:",squash"` + p2pconf.ResourceManagerConfig `mapstructure:",squash"` + ConnectionManagerConfig `mapstructure:",squash"` + + p2pconf.GossipSubConfig `mapstructure:",squash"` + AlspConfig `mapstructure:",squash"` + ... +} +``` +Each nested struct must have the mapstructure squash tag so that the nested struct will be flattened before unmarshalling. +The nesting can be as deep as needed. For example, the `UnicastConfig` struct in the `Config` struct has a nested `UnicastRateLimitersConfig` struct that also uses the `squash` tag: + +```go +type UnicastConfig struct { // UnicastRateLimitersConfig configuration for all unicast rate limiters. UnicastRateLimitersConfig `mapstructure:",squash"` ... } ``` -`UnicastRateLimitersConfig` is a nested struct that defines configuration for unicast rate limiter component. In our configuration yaml structure you will see that all network configs are defined under the `network-config` property. ### Setting Aliases Most configs will not be defined on the top layer FlowConfig but instead be defined on nested structs and in nested properties of the configuration yaml. When the default config is initially loaded the underlying config [viper](https://github.com/spf13/viper) store will store -each configuration with a key that is prefixed with each parent property. For example, because `network-connection-pruning` is found on the `network-config` property of the configuration yaml, the key used by the config store to -store this config value will be prefixed with `network` e.g. -```network.network-connection-pruning``` +each configuration with a key that is prefixed with each parent property. For example, because `networking-connection-pruning` is found on the `network-config` property of the configuration yaml, the key used by the config store to +store this config value will be prefixed with `network-config` e.g. +```network-config.networking-connection-pruning``` Later in the config process we bind the underlying config store with our pflag set, this allows us to override default values using CLI flags. -At this time the underlying config store would have 2 separate keys `network-connection-pruning` and `network.network-connection-pruning` for the same configuration value. This is because we don't use the network prefix for the CLI flags -used to override network configs. As a result, an alias must be set from `network.network-connection-pruning` -> `network-connection-pruning` so that they both point to the value loaded from the CLI flag. See [SetAliases](https://github.com/onflow/flow-go/blob/master/config/network/config.go#L84) in the network package for a reference. +At this time the underlying config store would have 2 separate keys `networking-connection-pruning` and `network-config.networking-connection-pruning` for the same configuration value. This is because we don't use the network prefix for the CLI flags +used to override network configs. As a result, an alias must be set from `network-config.networking-connection-pruning` -> `networking-connection-pruning` so that they both point to the value loaded from the CLI flag. See `SetAliases` in [network/netconf/flags.go](https://github.com/onflow/flow-go/blob/master/config/network/netconf/flags.go) in the network package for a reference. diff --git a/network/netconf/config.go b/network/netconf/config.go index 8ae6099c376..561ec62a764 100644 --- a/network/netconf/config.go +++ b/network/netconf/config.go @@ -8,7 +8,7 @@ import ( // Config encapsulation of configuration structs for all components related to the Flow network. type Config struct { - UnicastConfig UnicastConfig `mapstructure:",squash"` + UnicastConfig `mapstructure:",squash"` p2pconf.ResourceManagerConfig `mapstructure:",squash"` ConnectionManagerConfig `mapstructure:",squash"` // GossipSubConfig core gossipsub configuration. From d05c1c4ec3ef64c5c080dd552aa9cb1a2e4091f3 Mon Sep 17 00:00:00 2001 From: Leo Zhang Date: Mon, 23 Oct 2023 11:11:50 -0700 Subject: [PATCH 38/68] Update module/jobqueue.go Co-authored-by: Peter Argue <89119817+peterargue@users.noreply.github.com> --- module/jobqueue.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/module/jobqueue.go b/module/jobqueue.go index e78cc8e670b..b320b0375f4 100644 --- a/module/jobqueue.go +++ b/module/jobqueue.go @@ -28,8 +28,7 @@ type NewJobListener interface { type JobConsumer interface { NewJobListener - // Start starts processing jobs from a job queue. If this is the first time, a processed index - // will be initialized in the storage. If it fails to initialize, an error will be returned + // Start starts processing jobs from a job queue. Start() error // Stop gracefully stops the consumer from reading new jobs from the job queue. It does not stop From a96dec6b3a202e98d6af59c4223048a4f0635908 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 23 Oct 2023 12:07:13 -0700 Subject: [PATCH 39/68] fix test --- module/jobqueue/component_consumer_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/module/jobqueue/component_consumer_test.go b/module/jobqueue/component_consumer_test.go index 574423b1bc9..de9d13b5981 100644 --- a/module/jobqueue/component_consumer_test.go +++ b/module/jobqueue/component_consumer_test.go @@ -232,7 +232,7 @@ func (suite *ComponentConsumerSuite) TestSignalsBeforeReadyDoNotCheck() { started := atomic.NewBool(false) jobConsumer := modulemock.NewJobConsumer(suite.T()) - jobConsumer.On("Start", suite.defaultIndex).Return(func(_ uint64) error { + jobConsumer.On("Start").Return(func() error { // force Start to take a while so the processingLoop is ready first // the processingLoop should wait to start, otherwise Check would be called time.Sleep(500 * time.Millisecond) From f0efa0b0c81a84af7ef5594f274568e924a25bd9 Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Mon, 23 Oct 2023 12:32:21 -0700 Subject: [PATCH 40/68] Update network/p2p/p2pnode/resourceManager_test.go Co-authored-by: Khalil Claybon --- network/p2p/p2pnode/resourceManager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index d7ca60bee40..89aeb73bba2 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -210,7 +210,7 @@ func TestCreateStream_MinSystemLimit(t *testing.T) { func TestCreateStream_MaxSystemLimit(t *testing.T) { unittest.SkipUnless(t, unittest.TEST_TODO, - "max inbound stream protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") + "max inbound stream protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamSystem = math.MaxInt testCreateStreamInboundStreamResourceLimits(t, base) From 10f394d64625420d6645b6d0f21dc0045924d614 Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Mon, 23 Oct 2023 12:33:04 -0700 Subject: [PATCH 41/68] Update network/p2p/p2pnode/resourceManager_test.go Co-authored-by: Khalil Claybon --- network/p2p/p2pnode/resourceManager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 89aeb73bba2..2a04e0aef8d 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -177,7 +177,7 @@ func TestCreateStream_MaxProtocolLimit(t *testing.T) { func TestCreateStream_MinPeerProtocolLimit(t *testing.T) { unittest.SkipUnless(t, unittest.TEST_TODO, - "max inbound stream peer protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") + "max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamPeerProtocol = 1 testCreateStreamInboundStreamResourceLimits(t, base) From ce9fb463b90bacac76667619b61f8d39a4e08482 Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Mon, 23 Oct 2023 12:33:12 -0700 Subject: [PATCH 42/68] Update network/p2p/p2pnode/resourceManager_test.go Co-authored-by: Khalil Claybon --- network/p2p/p2pnode/resourceManager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 2a04e0aef8d..1d89998de12 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -162,7 +162,7 @@ func TestCreateStream_MaxPeerLimit(t *testing.T) { func TestCreateStream_MinProtocolLimit(t *testing.T) { unittest.SkipUnless(t, unittest.TEST_TODO, - "max inbound stream protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") + "max inbound stream protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamProtocol = 1 testCreateStreamInboundStreamResourceLimits(t, base) From 5c0ebd4af50c874151262e534954bee627762cdf Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Mon, 23 Oct 2023 12:33:20 -0700 Subject: [PATCH 43/68] Update network/p2p/p2pnode/resourceManager_test.go Co-authored-by: Khalil Claybon --- network/p2p/p2pnode/resourceManager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 1d89998de12..1c5263b448d 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -250,7 +250,7 @@ func TestCreateStream_PeerLimitGreaterThanPeerProtocolLimit(t *testing.T) { func TestCreateStream_ProtocolLimitLessThanPeerProtocolLimit(t *testing.T) { unittest.SkipUnless(t, unittest.TEST_TODO, - "max inbound stream peer protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") + "max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") // the case where protocol-level limit is lower than the peer-protocol-level limit. base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamProtocol = 5 // each peer can create 5 streams on a specific protocol. From 2d046d8666ac61e998a4aa592bc7367e7e7183ff Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Mon, 23 Oct 2023 12:33:34 -0700 Subject: [PATCH 44/68] Update network/p2p/p2pnode/resourceManager_test.go Co-authored-by: Khalil Claybon --- network/p2p/p2pnode/resourceManager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 1c5263b448d..c521f23e367 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -237,7 +237,7 @@ func TestCreateStream_PeerLimitGreaterThanPeerProtocolLimit(t *testing.T) { // the case where peer-level limit is higher than the peer-protocol-level limit. unittest.SkipUnless(t, unittest.TEST_TODO, - "max inbound stream peer protocol is not preserved; can be partially due to count steam not counting inbound streams on a protocol") + "max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundPeerStream = 10 // each peer can create 10 streams. base.maxInboundStreamPeerProtocol = 5 // each peer can create 5 streams on a specific protocol. From 7936f72b37b85956d8cce02bb061ad7e5585b06c Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Mon, 23 Oct 2023 12:35:38 -0700 Subject: [PATCH 45/68] Update network/internal/p2putils/utils.go Co-authored-by: Khalil Claybon --- network/internal/p2putils/utils.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/internal/p2putils/utils.go b/network/internal/p2putils/utils.go index 744dd183350..6ca15b7b5f0 100644 --- a/network/internal/p2putils/utils.go +++ b/network/internal/p2putils/utils.go @@ -133,7 +133,7 @@ func All() FilterOption { // It returns a slice of network.Stream values that match the filtering criteria. func FilterStream(host host.Host, targetID peer.ID, options ...FilterOption) []network.Stream { var filteredStreams []network.Stream - const discardTheProtocol = "discard-the-protocol" + const allProtocols = "*" // default values opts := FilterOptions{ dir: network.DirUnknown, // by default, consider both inbound and outbound streams From 0314cde84b79dd082c984f705259fdf925707f8e Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 12:36:31 -0700 Subject: [PATCH 46/68] fixes build issue --- network/internal/p2putils/utils.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/network/internal/p2putils/utils.go b/network/internal/p2putils/utils.go index 6ca15b7b5f0..0ec8b8aba11 100644 --- a/network/internal/p2putils/utils.go +++ b/network/internal/p2putils/utils.go @@ -137,7 +137,7 @@ func FilterStream(host host.Host, targetID peer.ID, options ...FilterOption) []n // default values opts := FilterOptions{ dir: network.DirUnknown, // by default, consider both inbound and outbound streams - protocol: discardTheProtocol, // by default, consider streams of all protocol IDs + protocol: allProtocols, // by default, consider streams of all protocol IDs all: false, // by default, return just the first matching stream } @@ -155,7 +155,7 @@ func FilterStream(host host.Host, targetID peer.ID, options ...FilterOption) []n streams := conn.GetStreams() for _, stream := range streams { if (opts.dir == network.DirUnknown || stream.Stat().Direction == opts.dir) && - (opts.protocol == discardTheProtocol || stream.Protocol() == opts.protocol) { + (opts.protocol == allProtocols || stream.Protocol() == opts.protocol) { filteredStreams = append(filteredStreams, stream) if !opts.all { return filteredStreams From a328d8a5ffaf3f7039a72f3660cccd8f607c7c8f Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Mon, 23 Oct 2023 12:58:17 -0700 Subject: [PATCH 47/68] Update config/default-config.yml Co-authored-by: Peter Argue <89119817+peterargue@users.noreply.github.com> --- config/default-config.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/config/default-config.yml b/config/default-config.yml index 1809474485f..2b901b0b3db 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -58,13 +58,13 @@ network-config: # Without this limit peers can end up in a state where there exists n number of connections per peer which # can lead to resource exhaustion of the libp2p node. libp2p-peer-base-limits-conns-inbound: 1 - # maximum number of inbound system-wide limit for streams, across all peers and protocols + # maximum number of inbound system-wide streams, across all peers and protocols # Note that streams are ephemeral and are created and destroyed intermittently. libp2p-inbound-stream-limit-system: 15_000 - # maximum number of inbound transient limit for streams, across all streams that are not yet fully opened and associated with a protocol. + # maximum number of inbound transient streams, across all streams that are not yet fully opened and associated with a protocol. # Note that streams are ephemeral and are created and destroyed intermittently. libp2p-inbound-stream-limit-transient: 15_000 - # maximum number of inbound limit for streams for each protocol across all peers; this is a per-protocol limit. We expect at least + # maximum number of inbound streams for each protocol across all peers; this is a per-protocol limit. We expect at least # three protocols per node; gossipsub, unicast, and dht. Note that streams are ephemeral and are created and destroyed intermittently. libp2p-inbound-stream-limit-protocol: 5000 # maximum number of inbound streams from each peer across all protocols. From 3bbaf2470a3825ea4e1ee72f7bc468c4870b3867 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 14:08:34 -0700 Subject: [PATCH 48/68] un-exports allowed momory --- network/p2p/p2pbuilder/libp2pNodeBuilder.go | 4 +-- network/p2p/p2pbuilder/libp2pscaler.go | 4 +-- network/p2p/p2pbuilder/libp2pscaler_test.go | 40 ++++++++++----------- 3 files changed, 24 insertions(+), 24 deletions(-) diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index 3c768b44ad6..07dd4110567 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -214,11 +214,11 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { limits := rcmgr.DefaultLimits libp2p.SetDefaultServiceLimits(&limits) - mem, err := AllowedMemory(builder.resourceManagerCfg.MemoryLimitRatio) + mem, err := allowedMemory(builder.resourceManagerCfg.MemoryLimitRatio) if err != nil { return nil, fmt.Errorf("could not get allowed memory: %w", err) } - fd, err := AllowedFileDescriptors(builder.resourceManagerCfg.FileDescriptorsRatio) + fd, err := allowedFileDescriptors(builder.resourceManagerCfg.FileDescriptorsRatio) if err != nil { return nil, fmt.Errorf("could not get allowed file descriptors: %w", err) } diff --git a/network/p2p/p2pbuilder/libp2pscaler.go b/network/p2p/p2pbuilder/libp2pscaler.go index d6bdaf2ef5a..8612a53d34d 100644 --- a/network/p2p/p2pbuilder/libp2pscaler.go +++ b/network/p2p/p2pbuilder/libp2pscaler.go @@ -16,14 +16,14 @@ func getNumFDs() (int, error) { return int(l.Cur), nil } -func AllowedMemory(scaleFactor float64) (int64, error) { +func allowedMemory(scaleFactor float64) (int64, error) { if scaleFactor <= 0 || scaleFactor > 1 { return 0, fmt.Errorf("memory scale factor must be greater than 0 and less than or equal to 1: %f", scaleFactor) } return int64(math.Floor(float64(memory.TotalMemory()) * scaleFactor)), nil } -func AllowedFileDescriptors(scaleFactor float64) (int, error) { +func allowedFileDescriptors(scaleFactor float64) (int, error) { if scaleFactor <= 0 || scaleFactor > 1 { return 0, fmt.Errorf("fd scale factor must be greater than 0 and less than or equal to 1: %f", scaleFactor) } diff --git a/network/p2p/p2pbuilder/libp2pscaler_test.go b/network/p2p/p2pbuilder/libp2pscaler_test.go index 176b3a6bb3c..dd5cfe48088 100644 --- a/network/p2p/p2pbuilder/libp2pscaler_test.go +++ b/network/p2p/p2pbuilder/libp2pscaler_test.go @@ -17,44 +17,44 @@ func TestAllowedMemoryScale(t *testing.T) { require.True(t, m > 0) // scaling with factor of 1 should return the total memory. - s, err := AllowedMemory(1) + s, err := allowedMemory(1) require.NoError(t, err) require.Equal(t, int64(m), s) // scaling with factor of 0 should return an error. - _, err = AllowedMemory(0) + _, err = allowedMemory(0) require.Error(t, err) // scaling with factor of -1 should return an error. - _, err = AllowedMemory(-1) + _, err = allowedMemory(-1) require.Error(t, err) // scaling with factor of 2 should return an error. - _, err = AllowedMemory(2) + _, err = allowedMemory(2) require.Error(t, err) // scaling with factor of 0.5 should return half the total memory. - s, err = AllowedMemory(0.5) + s, err = allowedMemory(0.5) require.NoError(t, err) require.Equal(t, int64(m/2), s) // scaling with factor of 0.1 should return 10% of the total memory. - s, err = AllowedMemory(0.1) + s, err = allowedMemory(0.1) require.NoError(t, err) require.Equal(t, int64(m/10), s) // scaling with factor of 0.01 should return 1% of the total memory. - s, err = AllowedMemory(0.01) + s, err = allowedMemory(0.01) require.NoError(t, err) require.Equal(t, int64(m/100), s) // scaling with factor of 0.001 should return 0.1% of the total memory. - s, err = AllowedMemory(0.001) + s, err = allowedMemory(0.001) require.NoError(t, err) require.Equal(t, int64(m/1000), s) // scaling with factor of 0.0001 should return 0.01% of the total memory. - s, err = AllowedMemory(0.0001) + s, err = allowedMemory(0.0001) require.NoError(t, err) require.Equal(t, int64(m/10000), s) } @@ -66,44 +66,44 @@ func TestAllowedFileDescriptorsScale(t *testing.T) { require.True(t, fd > 0) // scaling with factor of 1 should return the total file descriptors. - s, err := AllowedFileDescriptors(1) + s, err := allowedFileDescriptors(1) require.NoError(t, err) require.Equal(t, fd, s) // scaling with factor of 0 should return an error. - _, err = AllowedFileDescriptors(0) + _, err = allowedFileDescriptors(0) require.Error(t, err) // scaling with factor of -1 should return an error. - _, err = AllowedFileDescriptors(-1) + _, err = allowedFileDescriptors(-1) require.Error(t, err) // scaling with factor of 2 should return an error. - _, err = AllowedFileDescriptors(2) + _, err = allowedFileDescriptors(2) require.Error(t, err) // scaling with factor of 0.5 should return half the total file descriptors. - s, err = AllowedFileDescriptors(0.5) + s, err = allowedFileDescriptors(0.5) require.NoError(t, err) require.Equal(t, fd/2, s) // scaling with factor of 0.1 should return 10% of the total file descriptors. - s, err = AllowedFileDescriptors(0.1) + s, err = allowedFileDescriptors(0.1) require.NoError(t, err) require.Equal(t, fd/10, s) // scaling with factor of 0.01 should return 1% of the total file descriptors. - s, err = AllowedFileDescriptors(0.01) + s, err = allowedFileDescriptors(0.01) require.NoError(t, err) require.Equal(t, fd/100, s) // scaling with factor of 0.001 should return 0.1% of the total file descriptors. - s, err = AllowedFileDescriptors(0.001) + s, err = allowedFileDescriptors(0.001) require.NoError(t, err) require.Equal(t, fd/1000, s) // scaling with factor of 0.0001 should return 0.01% of the total file descriptors. - s, err = AllowedFileDescriptors(0.0001) + s, err = allowedFileDescriptors(0.0001) require.NoError(t, err) require.Equal(t, fd/10000, s) } @@ -114,10 +114,10 @@ func TestApplyInboundStreamAndConnectionLimits(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) - mem, err := AllowedMemory(cfg.NetworkConfig.ResourceManagerConfig.MemoryLimitRatio) + mem, err := allowedMemory(cfg.NetworkConfig.ResourceManagerConfig.MemoryLimitRatio) require.NoError(t, err) - fd, err := AllowedFileDescriptors(cfg.NetworkConfig.FileDescriptorsRatio) + fd, err := allowedFileDescriptors(cfg.NetworkConfig.FileDescriptorsRatio) require.NoError(t, err) limits := rcmgr.DefaultLimits libp2p.SetDefaultServiceLimits(&limits) From df61bacbd012f3f9efa375e726fb0836cf02b954 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 14:10:19 -0700 Subject: [PATCH 49/68] un-exports newLimitConfigLogger --- network/p2p/p2pbuilder/libp2pNodeBuilder.go | 2 +- network/p2p/p2pbuilder/utils.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index 07dd4110567..0340069a5de 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -235,7 +235,7 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { Int64("allowed_memory", mem). Int("allowed_file_descriptors", fd). Msg("allowed memory and file descriptors are fetched from the system") - NewLimitConfigLogger(builder.logger).LogResourceManagerLimits(appliedLimits) + newLimitConfigLogger(builder.logger).LogResourceManagerLimits(appliedLimits) opts = append(opts, libp2p.ResourceManager(mgr)) builder.logger.Info().Msg("libp2p resource manager is set to default with metrics") diff --git a/network/p2p/p2pbuilder/utils.go b/network/p2p/p2pbuilder/utils.go index 5af43ba9e81..3f54c886fd5 100644 --- a/network/p2p/p2pbuilder/utils.go +++ b/network/p2p/p2pbuilder/utils.go @@ -33,8 +33,8 @@ type limitConfigLogger struct { logger zerolog.Logger } -// NewLimitConfigLogger creates a new limitConfigLogger. -func NewLimitConfigLogger(logger zerolog.Logger) *limitConfigLogger { +// newLimitConfigLogger creates a new limitConfigLogger. +func newLimitConfigLogger(logger zerolog.Logger) *limitConfigLogger { return &limitConfigLogger{logger: logger} } From 9a4769c6dcbf0d5b0078149d54f06005b01e3d1e Mon Sep 17 00:00:00 2001 From: Misha <15269764+gomisha@users.noreply.github.com> Date: Mon, 23 Oct 2023 16:11:24 -0500 Subject: [PATCH 50/68] README.md update re:encourage config subpackage --- config/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/README.md b/config/README.md index eceb591dc2c..3a4fe42c918 100644 --- a/config/README.md +++ b/config/README.md @@ -28,7 +28,7 @@ Adding a new config to the FlowConfig can be done in a few easy steps. The network package can be used as a good example of how to add CLI flags and will be used in the steps below. -1. Create a new configuration package for the new configuration structs and CLI flags. Although it is encouraged to define all package configuration structs and CLI flags as a subpackage of the component they are related to, +1. Create a new configuration package for the new configuration structs and CLI flags. Although it is encouraged to define all package configuration structs and CLI flags as a subpackage of the [config package](https://github.com/onflow/flow-go/tree/master/config), the configuration package can live anywhere. This configuration package will define the configuration structs and CLI flags for overriding. ```shell mkdir example_config From dd83fe6a55a89ee813a23484e8f7d528c3e56354 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 14:12:59 -0700 Subject: [PATCH 51/68] formats default values greater than 9999 --- config/default-config.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/config/default-config.yml b/config/default-config.yml index 2b901b0b3db..a5a9ba34f84 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -6,7 +6,7 @@ network-config: networking-connection-pruning: true # Preferred unicasts protocols list of unicast protocols in preferred order preferred-unicast-protocols: [ ] - received-message-cache-size: 10e4 + received-message-cache-size: 10_000 peerupdate-interval: 10m unicast-message-timeout: 5s # Unicast create stream retry delay is initial delay used in the exponential backoff for create stream retries @@ -47,7 +47,7 @@ network-config: unicast-dial-in-progress-backoff-delay: 1s # The size of the dial config cache used to keep track of the dial config for each remote peer. The dial config is used to keep track of the dial retry budget for each remote peer. # Recommended to set it to the maximum number of remote peers in the network. - unicast-dial-config-cache-size: 10000 + unicast-dial-config-cache-size: 10_000 # Resource manager config # Maximum allowed fraction of file descriptors to be allocated by the libp2p resources in (0,1] libp2p-memory-limit-ratio: 0.5 # flow default @@ -111,7 +111,7 @@ network-config: # Gossipsub rpc inspectors configs # The size of the queue for notifications about invalid RPC messages - gossipsub-rpc-inspector-notification-cache-size: 10000 + gossipsub-rpc-inspector-notification-cache-size: 10_000 # RPC control message validation inspector configs # Rpc validation inspector number of pool workers gossipsub-rpc-validation-inspector-workers: 5 @@ -154,8 +154,8 @@ network-config: # The size of the queue used by worker pool for the control message metrics inspector gossipsub-rpc-metrics-inspector-cache-size: 100 # Application layer spam prevention - alsp-spam-record-cache-size: 10e3 - alsp-spam-report-queue-size: 10e4 + alsp-spam-record-cache-size: 1000 + alsp-spam-report-queue-size: 10_000 alsp-disable-penalty: false alsp-heart-beat-interval: 1s From 00e77c858e2eef60dec221c5924fdac6287670b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bastian=20M=C3=BCller?= Date: Mon, 23 Oct 2023 16:04:08 -0700 Subject: [PATCH 52/68] auto update to onflow/cadence v0.42.1 --- go.mod | 4 ++-- go.sum | 8 ++++---- insecure/go.mod | 4 ++-- insecure/go.sum | 8 ++++---- integration/go.mod | 4 ++-- integration/go.sum | 8 ++++---- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/go.mod b/go.mod index 7efb815d44b..2fee1c30772 100644 --- a/go.mod +++ b/go.mod @@ -51,11 +51,11 @@ require ( github.com/multiformats/go-multiaddr-dns v0.3.1 github.com/multiformats/go-multihash v0.2.3 github.com/onflow/atree v0.6.0 - github.com/onflow/cadence v0.42.0 + github.com/onflow/cadence v0.42.1 github.com/onflow/flow v0.3.4 github.com/onflow/flow-core-contracts/lib/go/contracts v0.14.0 github.com/onflow/flow-core-contracts/lib/go/templates v0.14.0 - github.com/onflow/flow-go-sdk v0.41.9 + github.com/onflow/flow-go-sdk v0.41.12 github.com/onflow/flow-go/crypto v0.24.9 github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231017162044-5d0f9b6dfdb2 github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d diff --git a/go.sum b/go.sum index d3df5493af6..6204ab003d9 100644 --- a/go.sum +++ b/go.sum @@ -1309,8 +1309,8 @@ github.com/onflow/atree v0.1.0-beta1.0.20211027184039-559ee654ece9/go.mod h1:+6x github.com/onflow/atree v0.6.0 h1:j7nQ2r8npznx4NX39zPpBYHmdy45f4xwoi+dm37Jk7c= github.com/onflow/atree v0.6.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= github.com/onflow/cadence v0.20.1/go.mod h1:7mzUvPZUIJztIbr9eTvs+fQjWWHTF8veC+yk4ihcNIA= -github.com/onflow/cadence v0.42.0 h1:XatyCy1pZu10x+JouRU6cZ9A50dF3uM+ubqYUER1/Vk= -github.com/onflow/cadence v0.42.0/go.mod h1:raU8va8QRyTa/eUbhej4mbyW2ETePfSaywoo36MddgE= +github.com/onflow/cadence v0.42.1 h1:Til0aa+TX2o9CM/0OFUPEssP0BoZBfpFRa4qefRgr0E= +github.com/onflow/cadence v0.42.1/go.mod h1:raU8va8QRyTa/eUbhej4mbyW2ETePfSaywoo36MddgE= github.com/onflow/flow v0.3.4 h1:FXUWVdYB90f/rjNcY0Owo30gL790tiYff9Pb/sycXYE= github.com/onflow/flow v0.3.4/go.mod h1:lzyAYmbu1HfkZ9cfnL5/sjrrsnJiUU8fRL26CqLP7+c= github.com/onflow/flow-core-contracts/lib/go/contracts v0.14.0 h1:DpkgyNAP3SAe7dMQX/Tb7BWFstodqtREo4hxWFHwdS0= @@ -1320,8 +1320,8 @@ github.com/onflow/flow-core-contracts/lib/go/templates v0.14.0/go.mod h1:ZeLxwaB github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20230711213910-baad011d2b13 h1:B4ll7e3j+MqTJv2122Enq3RtDNzmIGRu9xjV7fo7un0= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20230711213910-baad011d2b13/go.mod h1:kTMFIySzEJJeupk+7EmXs0EJ6CBWY/MV9fv9iYQk+RU= github.com/onflow/flow-go-sdk v0.24.0/go.mod h1:IoptMLPyFXWvyd9yYA6/4EmSeeozl6nJoIv4FaEMg74= -github.com/onflow/flow-go-sdk v0.41.9 h1:cyplhhhc0RnfOAan2t7I/7C9g1hVGDDLUhWj6ZHAkk4= -github.com/onflow/flow-go-sdk v0.41.9/go.mod h1:e9Q5TITCy7g08lkdQJxP8fAKBnBoC5FjALvUKr36j4I= +github.com/onflow/flow-go-sdk v0.41.12 h1:SrdHxpjqUHSjwToCTtcuvquSmjW9AA/QuwY9aex9f8c= +github.com/onflow/flow-go-sdk v0.41.12/go.mod h1:WJh4pkajUdLdcwqXqwhd0eLm4wz0cvw1vTgHZo4//NE= github.com/onflow/flow-go/crypto v0.21.3/go.mod h1:vI6V4CY3R6c4JKBxdcRiR/AnjBfL8OSD97bJc60cLuQ= github.com/onflow/flow-go/crypto v0.24.9 h1:0EQp+kSZYJepMIiSypfJVe7tzsPcb6UXOdOtsTCDhBs= github.com/onflow/flow-go/crypto v0.24.9/go.mod h1:fqCzkIBBMRRkciVrvW21rECKq1oD7Q6u+bCI78lfNX0= diff --git a/insecure/go.mod b/insecure/go.mod index 177aa4fad1c..4977282e2d3 100644 --- a/insecure/go.mod +++ b/insecure/go.mod @@ -190,11 +190,11 @@ require ( github.com/multiformats/go-multistream v0.4.1 // indirect github.com/multiformats/go-varint v0.0.7 // indirect github.com/onflow/atree v0.6.0 // indirect - github.com/onflow/cadence v0.42.0 // indirect + github.com/onflow/cadence v0.42.1 // indirect github.com/onflow/flow-core-contracts/lib/go/contracts v1.2.4-0.20231016154253-a00dbf7c061f // indirect github.com/onflow/flow-core-contracts/lib/go/templates v1.2.4-0.20231016154253-a00dbf7c061f // indirect github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20230711213910-baad011d2b13 // indirect - github.com/onflow/flow-go-sdk v0.41.10 // indirect + github.com/onflow/flow-go-sdk v0.41.12 // indirect github.com/onflow/flow-nft/lib/go/contracts v1.1.0 // indirect github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231017162044-5d0f9b6dfdb2 // indirect github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d // indirect diff --git a/insecure/go.sum b/insecure/go.sum index 2c266d0e39c..0c4bd6a72df 100644 --- a/insecure/go.sum +++ b/insecure/go.sum @@ -1285,8 +1285,8 @@ github.com/onflow/atree v0.1.0-beta1.0.20211027184039-559ee654ece9/go.mod h1:+6x github.com/onflow/atree v0.6.0 h1:j7nQ2r8npznx4NX39zPpBYHmdy45f4xwoi+dm37Jk7c= github.com/onflow/atree v0.6.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= github.com/onflow/cadence v0.20.1/go.mod h1:7mzUvPZUIJztIbr9eTvs+fQjWWHTF8veC+yk4ihcNIA= -github.com/onflow/cadence v0.42.0 h1:XatyCy1pZu10x+JouRU6cZ9A50dF3uM+ubqYUER1/Vk= -github.com/onflow/cadence v0.42.0/go.mod h1:raU8va8QRyTa/eUbhej4mbyW2ETePfSaywoo36MddgE= +github.com/onflow/cadence v0.42.1 h1:Til0aa+TX2o9CM/0OFUPEssP0BoZBfpFRa4qefRgr0E= +github.com/onflow/cadence v0.42.1/go.mod h1:raU8va8QRyTa/eUbhej4mbyW2ETePfSaywoo36MddgE= github.com/onflow/flow-core-contracts/lib/go/contracts v1.2.4-0.20231016154253-a00dbf7c061f h1:S8yIZw9LFXfYD1V5H9BiixihHw3GrXVPrmfplSzYaww= github.com/onflow/flow-core-contracts/lib/go/contracts v1.2.4-0.20231016154253-a00dbf7c061f/go.mod h1:jM6GMAL+m0hjusUgiYDNrixPQ6b9s8xjoJQoEu5bHQI= github.com/onflow/flow-core-contracts/lib/go/templates v1.2.4-0.20231016154253-a00dbf7c061f h1:Ep+Mpo2miWMe4pjPGIaEvEzshRep30dvNgxqk+//FrQ= @@ -1294,8 +1294,8 @@ github.com/onflow/flow-core-contracts/lib/go/templates v1.2.4-0.20231016154253-a github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20230711213910-baad011d2b13 h1:B4ll7e3j+MqTJv2122Enq3RtDNzmIGRu9xjV7fo7un0= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20230711213910-baad011d2b13/go.mod h1:kTMFIySzEJJeupk+7EmXs0EJ6CBWY/MV9fv9iYQk+RU= github.com/onflow/flow-go-sdk v0.24.0/go.mod h1:IoptMLPyFXWvyd9yYA6/4EmSeeozl6nJoIv4FaEMg74= -github.com/onflow/flow-go-sdk v0.41.10 h1:Cio6GJhtx532TUY+cqrqWglD5sZCXkWeM5QvaRha3p4= -github.com/onflow/flow-go-sdk v0.41.10/go.mod h1:0a0LiQFbFt8RW/ptoMUU7YkvW9ArVcbjLE0XS78uz1E= +github.com/onflow/flow-go-sdk v0.41.12 h1:SrdHxpjqUHSjwToCTtcuvquSmjW9AA/QuwY9aex9f8c= +github.com/onflow/flow-go-sdk v0.41.12/go.mod h1:WJh4pkajUdLdcwqXqwhd0eLm4wz0cvw1vTgHZo4//NE= github.com/onflow/flow-go/crypto v0.21.3/go.mod h1:vI6V4CY3R6c4JKBxdcRiR/AnjBfL8OSD97bJc60cLuQ= github.com/onflow/flow-go/crypto v0.24.9 h1:0EQp+kSZYJepMIiSypfJVe7tzsPcb6UXOdOtsTCDhBs= github.com/onflow/flow-go/crypto v0.24.9/go.mod h1:fqCzkIBBMRRkciVrvW21rECKq1oD7Q6u+bCI78lfNX0= diff --git a/integration/go.mod b/integration/go.mod index 213dd632568..f6c126c9377 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -18,12 +18,12 @@ require ( github.com/ipfs/go-datastore v0.6.0 github.com/ipfs/go-ds-badger2 v0.1.3 github.com/ipfs/go-ipfs-blockstore v1.3.0 - github.com/onflow/cadence v0.42.0 + github.com/onflow/cadence v0.42.1 github.com/onflow/flow-core-contracts/lib/go/contracts v1.2.4-0.20231016154253-a00dbf7c061f github.com/onflow/flow-core-contracts/lib/go/templates v1.2.4-0.20231016154253-a00dbf7c061f github.com/onflow/flow-emulator v0.54.1 github.com/onflow/flow-go v0.31.1-0.20230808172820-f074502a67e3 - github.com/onflow/flow-go-sdk v0.41.10 + github.com/onflow/flow-go-sdk v0.41.12 github.com/onflow/flow-go/crypto v0.24.9 github.com/onflow/flow-go/insecure v0.0.0-00010101000000-000000000000 github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231017162044-5d0f9b6dfdb2 diff --git a/integration/go.sum b/integration/go.sum index 8b8743d7dc3..54413759e04 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -1427,8 +1427,8 @@ github.com/onflow/atree v0.1.0-beta1.0.20211027184039-559ee654ece9/go.mod h1:+6x github.com/onflow/atree v0.6.0 h1:j7nQ2r8npznx4NX39zPpBYHmdy45f4xwoi+dm37Jk7c= github.com/onflow/atree v0.6.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= github.com/onflow/cadence v0.20.1/go.mod h1:7mzUvPZUIJztIbr9eTvs+fQjWWHTF8veC+yk4ihcNIA= -github.com/onflow/cadence v0.42.0 h1:XatyCy1pZu10x+JouRU6cZ9A50dF3uM+ubqYUER1/Vk= -github.com/onflow/cadence v0.42.0/go.mod h1:raU8va8QRyTa/eUbhej4mbyW2ETePfSaywoo36MddgE= +github.com/onflow/cadence v0.42.1 h1:Til0aa+TX2o9CM/0OFUPEssP0BoZBfpFRa4qefRgr0E= +github.com/onflow/cadence v0.42.1/go.mod h1:raU8va8QRyTa/eUbhej4mbyW2ETePfSaywoo36MddgE= github.com/onflow/flow-core-contracts/lib/go/contracts v1.2.4-0.20231016154253-a00dbf7c061f h1:S8yIZw9LFXfYD1V5H9BiixihHw3GrXVPrmfplSzYaww= github.com/onflow/flow-core-contracts/lib/go/contracts v1.2.4-0.20231016154253-a00dbf7c061f/go.mod h1:jM6GMAL+m0hjusUgiYDNrixPQ6b9s8xjoJQoEu5bHQI= github.com/onflow/flow-core-contracts/lib/go/templates v1.2.4-0.20231016154253-a00dbf7c061f h1:Ep+Mpo2miWMe4pjPGIaEvEzshRep30dvNgxqk+//FrQ= @@ -1438,8 +1438,8 @@ github.com/onflow/flow-emulator v0.54.1/go.mod h1:cPKNx2eaxUDtXNHN9nnrt/qydWUHNQ github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20230711213910-baad011d2b13 h1:B4ll7e3j+MqTJv2122Enq3RtDNzmIGRu9xjV7fo7un0= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20230711213910-baad011d2b13/go.mod h1:kTMFIySzEJJeupk+7EmXs0EJ6CBWY/MV9fv9iYQk+RU= github.com/onflow/flow-go-sdk v0.24.0/go.mod h1:IoptMLPyFXWvyd9yYA6/4EmSeeozl6nJoIv4FaEMg74= -github.com/onflow/flow-go-sdk v0.41.10 h1:Cio6GJhtx532TUY+cqrqWglD5sZCXkWeM5QvaRha3p4= -github.com/onflow/flow-go-sdk v0.41.10/go.mod h1:0a0LiQFbFt8RW/ptoMUU7YkvW9ArVcbjLE0XS78uz1E= +github.com/onflow/flow-go-sdk v0.41.12 h1:SrdHxpjqUHSjwToCTtcuvquSmjW9AA/QuwY9aex9f8c= +github.com/onflow/flow-go-sdk v0.41.12/go.mod h1:WJh4pkajUdLdcwqXqwhd0eLm4wz0cvw1vTgHZo4//NE= github.com/onflow/flow-go/crypto v0.21.3/go.mod h1:vI6V4CY3R6c4JKBxdcRiR/AnjBfL8OSD97bJc60cLuQ= github.com/onflow/flow-go/crypto v0.24.9 h1:0EQp+kSZYJepMIiSypfJVe7tzsPcb6UXOdOtsTCDhBs= github.com/onflow/flow-go/crypto v0.24.9/go.mod h1:fqCzkIBBMRRkciVrvW21rECKq1oD7Q6u+bCI78lfNX0= From c6fe7146031dc52b664ef2664a2c016b9183bc30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bastian=20M=C3=BCller?= Date: Mon, 23 Oct 2023 16:31:37 -0700 Subject: [PATCH 53/68] add argument for new parameter --- fvm/runtime/reusable_cadence_runtime.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fvm/runtime/reusable_cadence_runtime.go b/fvm/runtime/reusable_cadence_runtime.go index 4b249b73b47..307d6959bb2 100644 --- a/fvm/runtime/reusable_cadence_runtime.go +++ b/fvm/runtime/reusable_cadence_runtime.go @@ -78,7 +78,7 @@ func NewReusableCadenceRuntime(rt runtime.Runtime, config runtime.Config) *Reusa ), } - reusable.DeclareValue(blockRandomSource) + reusable.DeclareValue(blockRandomSource, nil) return reusable } From 9ce45a9b67e0a256ff2d812646a3314fb81ca302 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 16:32:21 -0700 Subject: [PATCH 54/68] bumps up the startup timeout --- network/p2p/test/fixtures.go | 38 +++++++++++++++++++++++++++++------- 1 file changed, 31 insertions(+), 7 deletions(-) diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 65fe7558b8c..9e9b8e4c8e8 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -50,13 +50,13 @@ const ( // the test is run in parallel with other tests. Hence, no further increase of the timeout is // expected to be necessary. Any failure to start a node within this timeout is likely to be // caused by a bug in the code. - libp2pNodeStartupTimeout = 5 * time.Second + libp2pNodeStartupTimeout = 10 * time.Second // libp2pNodeStartupTimeout is the timeout for starting a libp2p node in tests. Note that the // timeout has been selected to be large enough to allow for the node to start up on a CI even when // the test is run in parallel with other tests. Hence, no further increase of the timeout is // expected to be necessary. Any failure to start a node within this timeout is likely to be // caused by a bug in the code. - libp2pNodeShutdownTimeout = 5 * time.Second + libp2pNodeShutdownTimeout = 10 * time.Second ) // NetworkingKeyFixtures is a test helper that generates a ECDSA flow key pair. @@ -122,7 +122,9 @@ func NodeFixture( opt(parameters) } - identity := unittest.IdentityFixture(unittest.WithNetworkingKey(parameters.Key.PublicKey()), unittest.WithAddress(parameters.Address), unittest.WithRole(parameters.Role)) + identity := unittest.IdentityFixture(unittest.WithNetworkingKey(parameters.Key.PublicKey()), + unittest.WithAddress(parameters.Address), + unittest.WithRole(parameters.Role)) logger = parameters.Logger.With().Hex("node_id", logging.ID(identity.NodeID)).Logger() @@ -663,7 +665,11 @@ func EnsurePubsubMessageExchange( for i := 0; i < count; i++ { // creates a unique message to be published by the node payload := messageFactory() - outgoingMessageScope, err := message.NewOutgoingScope(flow.IdentifierList{unittest.IdentifierFixture()}, topic, payload, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) + outgoingMessageScope, err := message.NewOutgoingScope(flow.IdentifierList{unittest.IdentifierFixture()}, + topic, + payload, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) require.NoError(t, err) require.NoError(t, node.Publish(ctx, outgoingMessageScope)) @@ -689,7 +695,14 @@ func EnsurePubsubMessageExchange( // - count: the number of messages to exchange from `sender` to `receiver`. // - messageFactory: a function that creates a unique message to be published by the node. func EnsurePubsubMessageExchangeFromNode( - t *testing.T, ctx context.Context, sender p2p.LibP2PNode, receiverNode p2p.LibP2PNode, receiverIdentifier flow.Identifier, topic channels.Topic, count int, messageFactory func() interface{}, + t *testing.T, + ctx context.Context, + sender p2p.LibP2PNode, + receiverNode p2p.LibP2PNode, + receiverIdentifier flow.Identifier, + topic channels.Topic, + count int, + messageFactory func() interface{}, ) { _, err := sender.Subscribe(topic, validator.TopicValidator(unittest.Logger(), unittest.AllowAllPeerFilter())) require.NoError(t, err) @@ -703,7 +716,11 @@ func EnsurePubsubMessageExchangeFromNode( for i := 0; i < count; i++ { // creates a unique message to be published by the node payload := messageFactory() - outgoingMessageScope, err := message.NewOutgoingScope(flow.IdentifierList{receiverIdentifier}, topic, payload, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) + outgoingMessageScope, err := message.NewOutgoingScope(flow.IdentifierList{receiverIdentifier}, + topic, + payload, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) require.NoError(t, err) require.NoError(t, sender.Publish(ctx, outgoingMessageScope)) @@ -746,7 +763,14 @@ func EnsureNotConnectedBetweenGroups(t *testing.T, ctx context.Context, groupA [ // - count: the number of messages to exchange from each node. // - messageFactory: a function that creates a unique message to be published by the node. func EnsureNoPubsubMessageExchange( - t *testing.T, ctx context.Context, from []p2p.LibP2PNode, to []p2p.LibP2PNode, toIdentifiers flow.IdentifierList, topic channels.Topic, count int, messageFactory func() interface{}, + t *testing.T, + ctx context.Context, + from []p2p.LibP2PNode, + to []p2p.LibP2PNode, + toIdentifiers flow.IdentifierList, + topic channels.Topic, + count int, + messageFactory func() interface{}, ) { subs := make([]p2p.Subscription, len(to)) tv := validator.TopicValidator(unittest.Logger(), unittest.AllowAllPeerFilter()) From b067caa0bd0ade077e185764398711f3eb06a04b Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 18:24:20 -0700 Subject: [PATCH 55/68] shortens the skip message --- network/p2p/p2pnode/resourceManager_test.go | 29 +++++++++------------ 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index c521f23e367..00d8c44cee5 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -160,9 +160,8 @@ func TestCreateStream_MaxPeerLimit(t *testing.T) { } func TestCreateStream_MinProtocolLimit(t *testing.T) { - unittest.SkipUnless(t, - unittest.TEST_TODO, - "max inbound stream protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") + // max inbound protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol + unittest.SkipUnless(t, unittest.TEST_TODO, "broken test") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamProtocol = 1 testCreateStreamInboundStreamResourceLimits(t, base) @@ -175,9 +174,8 @@ func TestCreateStream_MaxProtocolLimit(t *testing.T) { } func TestCreateStream_MinPeerProtocolLimit(t *testing.T) { - unittest.SkipUnless(t, - unittest.TEST_TODO, - "max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") + // max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol + unittest.SkipUnless(t, unittest.TEST_TODO, "broken test") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamPeerProtocol = 1 testCreateStreamInboundStreamResourceLimits(t, base) @@ -208,18 +206,16 @@ func TestCreateStream_MinSystemLimit(t *testing.T) { } func TestCreateStream_MaxSystemLimit(t *testing.T) { - unittest.SkipUnless(t, - unittest.TEST_TODO, - "max inbound stream protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") + // max inbound stream protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol + unittest.SkipUnless(t, unittest.TEST_TODO, "broken test") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamSystem = math.MaxInt testCreateStreamInboundStreamResourceLimits(t, base) } func TestCreateStream_DefaultConfigWithUnknownProtocol(t *testing.T) { - unittest.SkipUnless(t, - unittest.TEST_TODO, - "limits are not enforced when using an unknown protocol ID") + // limits are not enforced when using an unknown protocol ID + unittest.SkipUnless(t, unittest.TEST_TODO, "broken test") base := baseCreateStreamInboundStreamResourceLimitConfig() base.unknownProtocol = true testCreateStreamInboundStreamResourceLimits(t, base) @@ -235,9 +231,8 @@ func TestCreateStream_PeerLimitLessThanPeerProtocolLimit(t *testing.T) { func TestCreateStream_PeerLimitGreaterThanPeerProtocolLimit(t *testing.T) { // the case where peer-level limit is higher than the peer-protocol-level limit. - unittest.SkipUnless(t, - unittest.TEST_TODO, - "max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") + // max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol + unittest.SkipUnless(t, unittest.TEST_TODO, "broken test") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundPeerStream = 10 // each peer can create 10 streams. base.maxInboundStreamPeerProtocol = 5 // each peer can create 5 streams on a specific protocol. @@ -248,9 +243,9 @@ func TestCreateStream_PeerLimitGreaterThanPeerProtocolLimit(t *testing.T) { } func TestCreateStream_ProtocolLimitLessThanPeerProtocolLimit(t *testing.T) { + // max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol unittest.SkipUnless(t, - unittest.TEST_TODO, - "max inbound stream peer protocol is not preserved; can be partially due to count stream not counting inbound streams on a protocol") + unittest.TEST_TODO, "broken test") // the case where protocol-level limit is lower than the peer-protocol-level limit. base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamProtocol = 5 // each peer can create 5 streams on a specific protocol. From 951672fb092410f0c8975df6b1bfed69cae2e30d Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 19:02:19 -0700 Subject: [PATCH 56/68] quaranttines TestCreateStream_MaxTransientLimit --- network/p2p/p2pnode/resourceManager_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 00d8c44cee5..743ce015c94 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -154,6 +154,7 @@ func TestCreateStream_MinPeerLimit(t *testing.T) { } func TestCreateStream_MaxPeerLimit(t *testing.T) { + base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundPeerStream = math.MaxInt testCreateStreamInboundStreamResourceLimits(t, base) @@ -194,6 +195,7 @@ func TestCreateStream_MinTransientLimit(t *testing.T) { } func TestCreateStream_MaxTransientLimit(t *testing.T) { + unittest.SkipUnless(t, unittest.TEST_TODO, "fails on CI constantly") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamTransient = math.MaxInt testCreateStreamInboundStreamResourceLimits(t, base) From 488e9e0a629cc50f0bd62a19103fb6cf5875bd21 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 23 Oct 2023 21:22:24 -0700 Subject: [PATCH 57/68] reduces test loads --- network/p2p/p2pnode/resourceManager_test.go | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 743ce015c94..863d6bb2146 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -135,11 +135,11 @@ func (t testPeerLimitConfig) maxLimit() int { func baseCreateStreamInboundStreamResourceLimitConfig() *testPeerLimitConfig { return &testPeerLimitConfig{ nodeCount: 10, - maxInboundPeerStream: 100, - maxInboundStreamProtocol: 100, - maxInboundStreamPeerProtocol: 100, - maxInboundStreamTransient: 100, - maxInboundStreamSystem: 100, + maxInboundPeerStream: 20, + maxInboundStreamProtocol: 20, + maxInboundStreamPeerProtocol: 20, + maxInboundStreamTransient: 20, + maxInboundStreamSystem: 20, } } @@ -195,7 +195,6 @@ func TestCreateStream_MinTransientLimit(t *testing.T) { } func TestCreateStream_MaxTransientLimit(t *testing.T) { - unittest.SkipUnless(t, unittest.TEST_TODO, "fails on CI constantly") base := baseCreateStreamInboundStreamResourceLimitConfig() base.maxInboundStreamTransient = math.MaxInt testCreateStreamInboundStreamResourceLimits(t, base) From 4f4edbc09be5743c84d9e404255b68bc5556a887 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Tue, 24 Oct 2023 06:24:39 -0700 Subject: [PATCH 58/68] reduces load of test --- network/p2p/p2pnode/resourceManager_test.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/network/p2p/p2pnode/resourceManager_test.go b/network/p2p/p2pnode/resourceManager_test.go index 863d6bb2146..bcef3cb8f7f 100644 --- a/network/p2p/p2pnode/resourceManager_test.go +++ b/network/p2p/p2pnode/resourceManager_test.go @@ -134,12 +134,12 @@ func (t testPeerLimitConfig) maxLimit() int { // baseCreateStreamInboundStreamResourceLimitConfig returns a testPeerLimitConfig with default values. func baseCreateStreamInboundStreamResourceLimitConfig() *testPeerLimitConfig { return &testPeerLimitConfig{ - nodeCount: 10, - maxInboundPeerStream: 20, - maxInboundStreamProtocol: 20, - maxInboundStreamPeerProtocol: 20, - maxInboundStreamTransient: 20, - maxInboundStreamSystem: 20, + nodeCount: 5, + maxInboundPeerStream: 10, + maxInboundStreamProtocol: 10, + maxInboundStreamPeerProtocol: 10, + maxInboundStreamTransient: 10, + maxInboundStreamSystem: 10, } } From 7e742fa945c3d21d545318227778f40383758726 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 10:54:50 -0700 Subject: [PATCH 59/68] apply PR comments --- fvm/evm/types/account.go | 16 ++++++++-------- fvm/evm/types/result.go | 4 +--- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/fvm/evm/types/account.go b/fvm/evm/types/account.go index 7aa05f5e95d..9e247fe1992 100644 --- a/fvm/evm/types/account.go +++ b/fvm/evm/types/account.go @@ -2,13 +2,13 @@ package types // Account is an EVM account, currently // three types of accounts are supported on Flow EVM, -// externally owned accounts (EOAs), smart contract accounts and flow owned accounts (FOAs) -// FlowOwnedAccount is a new type of account in the environment, +// externally owned accounts (EOAs), smart contract accounts and bridged accounts +// BridgedAccount is a new type of account in the environment, // that instead of being managed by public key, // it is managed by a resource owned by a Flow account. // // In other words, the FVM account who owns the FOA resource -// can bridge native tokens to and from the account associated with the FOA, +// can bridge native tokens to and from the account associated with the bridged account, // deploy contracts to the environment, // or call methods on contracts without the need to sign a transaction. type Account interface { @@ -23,7 +23,7 @@ type Account interface { // Withdraw withdraws the balance from account and // return it as a FlowTokenVault - // works only for FOA accounts + // works only for bridged accounts Withdraw(Balance) *FLOWTokenVault // Transfer is a utility method on top of call for transfering tokens to another account @@ -31,17 +31,17 @@ type Account interface { // Deploy deploys a contract to the environment // the new deployed contract would be at the returned address and - // the contract data is not controlled by the FOA accounts - // works only for FOA accounts + // the contract data is not controlled by the bridge account + // works only for bridged accounts Deploy(Code, GasLimit, Balance) Address // Call calls a smart contract function with the given data. // The gas usage is limited by the given gas limit, // and the Flow transaction's computation limit. - // The fees are deducted from the FOA + // The fees are deducted from the bridged account // and are transferred to the target address. // if no data is provided it would behave as transfering tokens to the // target address - // works only for FOA accounts + // works only for bridged accounts Call(Address, Data, GasLimit, Balance) Data } diff --git a/fvm/evm/types/result.go b/fvm/evm/types/result.go index 5203249a91a..cf1b69e1e1b 100644 --- a/fvm/evm/types/result.go +++ b/fvm/evm/types/result.go @@ -8,9 +8,7 @@ import ( // tx type 255 is used for direct calls from bridged accounts var DirectCallTxType = uint8(255) -// Result captures the result of an interaction to the emulator -// it could be the out put of a direct call or output of running an -// evm transaction. +// Result captures the result of an interaction with the emulator (direct call or evm tx) // Its more comprehensive than typical evm receipt, usually // the receipt generation requires some extra calculation (e.g. Deployed contract address) // but we take a different apporach here and include more data so that From d2aeb6d5b481825cbfb37ba1f52e41d02c11b158 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 11:05:25 -0700 Subject: [PATCH 60/68] apply PR comments --- fvm/evm/types/address.go | 3 +-- fvm/evm/types/balance.go | 20 +++++++++++++------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/fvm/evm/types/address.go b/fvm/evm/types/address.go index db4f999d93d..877e77c2571 100644 --- a/fvm/evm/types/address.go +++ b/fvm/evm/types/address.go @@ -14,8 +14,7 @@ const AddressLength = gethCommon.AddressLength // NewAddress constructs a new Address func NewAddress(addr gethCommon.Address) Address { - fa := Address(addr) - return fa + return Address(addr) } // Bytes returns a byte slice for the address diff --git a/fvm/evm/types/balance.go b/fvm/evm/types/balance.go index 0d8c74765de..e7108b7ea89 100644 --- a/fvm/evm/types/balance.go +++ b/fvm/evm/types/balance.go @@ -7,17 +7,24 @@ import ( "github.com/onflow/cadence" ) +var SmallestAcceptableBalanceValueInAttoFlow = new(big.Int).SetInt64(1e10) + // Balance represents the balance of an address -// in the evm environment, balances are kept in attoflow, +// in the evm environment, balances are kept in attoflow (1e10^-18 flow), // the smallest denomination of FLOW token (similar to how Wei is used to store Eth) // But on the FLOW Vaults, we use Cadence.UFix64 to store values in Flow. -// this type is defined to minimize the chance of mistake when dealing with the converision +// this could result in accidental conversion mistakes, the balance object here would +// do the conversions and does appropriate checks. +// +// For example the smallest unit of Flow token that a FlowVault could store is 1e10^-8, +// so transfering smaller values (or values with smalls fractions) could result in loss in +// conversion. The balance object checks it to prevent invalid balance. +// This means that values smaller than 1e10^-8 flow could not be bridged between FVM and Flow EVM. type Balance cadence.UFix64 // ToAttoFlow converts the balance into AttoFlow func (b Balance) ToAttoFlow() *big.Int { - conv := new(big.Int).SetInt64(1e10) - return new(big.Int).Mul(new(big.Int).SetUint64(uint64(b)), conv) + return new(big.Int).Mul(new(big.Int).SetUint64(uint64(b)), SmallestAcceptableBalanceValueInAttoFlow) } // Sub subtract the other balance from this balance @@ -47,12 +54,11 @@ func DecodeBalance(encoded []byte) (Balance, error) { // NewBalanceFromAttoFlow constructs a new balance from atto flow value func NewBalanceFromAttoFlow(inp *big.Int) (Balance, error) { - conv := new(big.Int).SetInt64(1e10) - if new(big.Int).Mod(inp, conv).Cmp(big.NewInt(0)) != 0 { + if new(big.Int).Mod(inp, SmallestAcceptableBalanceValueInAttoFlow).Cmp(big.NewInt(0)) != 0 { return 0, ErrBalanceConversion } // we only need to divide by 10 given we already have 8 as factor - converted := new(big.Int).Div(inp, conv) + converted := new(big.Int).Div(inp, SmallestAcceptableBalanceValueInAttoFlow) return Balance(cadence.UFix64(converted.Uint64())), nil } From a91af8e26abc8ad47c1b37794018252843adc549 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 11:08:10 -0700 Subject: [PATCH 61/68] remove unused const --- fvm/evm/emulator/database/database.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/fvm/evm/emulator/database/database.go b/fvm/evm/emulator/database/database.go index abb3cb78ad2..f79532ad4f9 100644 --- a/fvm/evm/emulator/database/database.go +++ b/fvm/evm/emulator/database/database.go @@ -15,9 +15,8 @@ import ( ) const ( - FlowEVMLatextBlockKey = "LatestBlock" - FlowEVMRootSlabKey = "RootSlabKey" - FlowEVMRootHashKey = "RootHash" + FlowEVMRootSlabKey = "RootSlabKey" + FlowEVMRootHashKey = "RootHash" ) var ( From 26691704c9a670898b3cc268831a25bd3ca633b0 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 15:05:23 -0700 Subject: [PATCH 62/68] apply comment feedback --- fvm/environment/event_emitter.go | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/fvm/environment/event_emitter.go b/fvm/environment/event_emitter.go index 046272d13b0..992d32f6f3d 100644 --- a/fvm/environment/event_emitter.go +++ b/fvm/environment/event_emitter.go @@ -241,7 +241,7 @@ func (emitter *eventEmitter) EmitFlowEvent(etype flow.EventType, payload []byte) err := emitter.meter.MeterComputation(ComputationKindEmitEvent, 1) if err != nil { - return fmt.Errorf("emit event failed: %w", err) + return fmt.Errorf("emit flow event failed: %w", err) } eventSize := uint64(len(etype) + len(payload)) @@ -254,16 +254,7 @@ func (emitter *eventEmitter) EmitFlowEvent(etype flow.EventType, payload []byte) Payload: payload, } - // TODO: to set limit to maximum when it is service account and get rid of this flag - isServiceAccount := emitter.payer == emitter.chain.ServiceAddress() - - eventEmitError := emitter.eventCollection.AppendEvent(flowEvent, eventSize) - // skip limit if payer is service account - if !isServiceAccount { - return eventEmitError - } - - return nil + return emitter.eventCollection.AppendEvent(flowEvent, eventSize) } func (emitter *eventEmitter) Events() flow.EventsList { From c6e19155b117453182f31324fcc971d135e0ca78 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 15:06:45 -0700 Subject: [PATCH 63/68] minor changes --- fvm/environment/event_emitter.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fvm/environment/event_emitter.go b/fvm/environment/event_emitter.go index 992d32f6f3d..921ccd2ca75 100644 --- a/fvm/environment/event_emitter.go +++ b/fvm/environment/event_emitter.go @@ -44,8 +44,8 @@ type EventEmitter interface { // EmitFlowEvent is used to emit events that are not generated by // Cadence runtime. - // Warning: EmitFlowEvent does not support handling service events - // that functionality might need to be added if needed in the future + // Warning: current implementation of EmitFlowEvent does not support handling service events + // that functionality should be added if needed in the future EmitFlowEvent(etype flow.EventType, payload []byte) error Events() flow.EventsList From 48c4b164620c4363f6c92ceea32b054fb0048aa8 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 15:09:05 -0700 Subject: [PATCH 64/68] add as a todo --- fvm/environment/event_emitter.go | 1 + 1 file changed, 1 insertion(+) diff --git a/fvm/environment/event_emitter.go b/fvm/environment/event_emitter.go index 921ccd2ca75..6a05fefe1f3 100644 --- a/fvm/environment/event_emitter.go +++ b/fvm/environment/event_emitter.go @@ -46,6 +46,7 @@ type EventEmitter interface { // Cadence runtime. // Warning: current implementation of EmitFlowEvent does not support handling service events // that functionality should be added if needed in the future + // TODO: we could merge this one with the EmitEvent endpoint EmitFlowEvent(etype flow.EventType, payload []byte) error Events() flow.EventsList From 4403cf631b9b6f126c1cf9b58030889e6d5394ed Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 15:31:39 -0700 Subject: [PATCH 65/68] update mocks --- fvm/environment/mock/environment.go | 28 ++++++++++++++-------------- fvm/environment/mock/meter.go | 28 ++++++++++++++-------------- 2 files changed, 28 insertions(+), 28 deletions(-) diff --git a/fvm/environment/mock/environment.go b/fvm/environment/mock/environment.go index 56cc8382840..11b9cda285c 100644 --- a/fvm/environment/mock/environment.go +++ b/fvm/environment/mock/environment.go @@ -278,6 +278,20 @@ func (_m *Environment) CheckPayerBalanceAndGetMaxTxFees(payer flow.Address, incl return r0, r1 } +// ComputationAvailable provides a mock function with given fields: _a0, _a1 +func (_m *Environment) ComputationAvailable(_a0 common.ComputationKind, _a1 uint) bool { + ret := _m.Called(_a0, _a1) + + var r0 bool + if rf, ok := ret.Get(0).(func(common.ComputationKind, uint) bool); ok { + r0 = rf(_a0, _a1) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + // ComputationIntensities provides a mock function with given fields: func (_m *Environment) ComputationIntensities() meter.MeteredComputationIntensities { ret := _m.Called() @@ -903,20 +917,6 @@ func (_m *Environment) GetValue(owner []byte, key []byte) ([]byte, error) { return r0, r1 } -// ComputationAvailable provides a mock function with given fields: _a0, _a1 -func (_m *Environment) ComputationAvailable(_a0 common.ComputationKind, _a1 uint) bool { - ret := _m.Called(_a0, _a1) - - var r0 bool - if rf, ok := ret.Get(0).(func(common.ComputationKind, uint) bool); ok { - r0 = rf(_a0, _a1) - } else { - r0 = ret.Get(0).(bool) - } - - return r0 -} - // Hash provides a mock function with given fields: data, tag, hashAlgorithm func (_m *Environment) Hash(data []byte, tag string, hashAlgorithm sema.HashAlgorithm) ([]byte, error) { ret := _m.Called(data, tag, hashAlgorithm) diff --git a/fvm/environment/mock/meter.go b/fvm/environment/mock/meter.go index b7a8d923f4f..d8b2cdd7f99 100644 --- a/fvm/environment/mock/meter.go +++ b/fvm/environment/mock/meter.go @@ -15,6 +15,20 @@ type Meter struct { mock.Mock } +// ComputationAvailable provides a mock function with given fields: _a0, _a1 +func (_m *Meter) ComputationAvailable(_a0 common.ComputationKind, _a1 uint) bool { + ret := _m.Called(_a0, _a1) + + var r0 bool + if rf, ok := ret.Get(0).(func(common.ComputationKind, uint) bool); ok { + r0 = rf(_a0, _a1) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + // ComputationIntensities provides a mock function with given fields: func (_m *Meter) ComputationIntensities() meter.MeteredComputationIntensities { ret := _m.Called() @@ -55,20 +69,6 @@ func (_m *Meter) ComputationUsed() (uint64, error) { return r0, r1 } -// ComputationAvailable provides a mock function with given fields: _a0, _a1 -func (_m *Meter) ComputationAvailable(_a0 common.ComputationKind, _a1 uint) bool { - ret := _m.Called(_a0, _a1) - - var r0 bool - if rf, ok := ret.Get(0).(func(common.ComputationKind, uint) bool); ok { - r0 = rf(_a0, _a1) - } else { - r0 = ret.Get(0).(bool) - } - - return r0 -} - // InteractionUsed provides a mock function with given fields: func (_m *Meter) InteractionUsed() (uint64, error) { ret := _m.Called() From 88d69a25bb319ff3acded7737e8d7bc909658c71 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Tue, 24 Oct 2023 15:52:01 -0700 Subject: [PATCH 66/68] update docs --- fvm/environment/meter.go | 2 ++ fvm/evm/handler/handler.go | 20 +++++++++++--------- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/fvm/environment/meter.go b/fvm/environment/meter.go index 9ebbdf6e28e..757ec0ea8be 100644 --- a/fvm/environment/meter.go +++ b/fvm/environment/meter.go @@ -49,6 +49,8 @@ const ( ComputationKindGenerateAccountLocalID = 2035 ComputationKindGetRandomSourceHistory = 2036 ComputationKindEVMGasUsage = 2037 + ComputationKindRLPEncoding = 2038 + ComputationKindRLPDecoding = 2039 ) type Meter interface { diff --git a/fvm/evm/handler/handler.go b/fvm/evm/handler/handler.go index 628224357b1..a7569919f98 100644 --- a/fvm/evm/handler/handler.go +++ b/fvm/evm/handler/handler.go @@ -55,7 +55,7 @@ func NewContractHandler( } } -// AllocateAddress allocates an address to be used by FOA resources +// AllocateAddress allocates an address to be used by the bridged accounts func (h *ContractHandler) AllocateAddress() types.Address { target := types.Address{} // first 12 bytes would be zero @@ -72,7 +72,7 @@ func (h *ContractHandler) AllocateAddress() types.Address { } // AccountByAddress returns the account for the given address, -// if isAuthorized is set, account is controlled by the FVM and FOA resources +// if isAuthorized is set, account is controlled by the FVM (bridged accounts) func (h *ContractHandler) AccountByAddress(addr types.Address, isAuthorized bool) types.Account { return newAccount(h, addr, isAuthorized) } @@ -101,8 +101,11 @@ func (h *ContractHandler) updateLastExecutedBlock(stateRoot, eventRoot gethCommo func (h ContractHandler) Run(rlpEncodedTx []byte, coinbase types.Address) bool { // Decode transaction encoding tx := gethTypes.Transaction{} - // TODO: update the max limit on the encoded size to a meaningful value - err := tx.DecodeRLP( + + err := h.backend.MeterComputation(environment.ComputationKindRLPDecoding, uint(len(rlpEncodedTx))) + handleError(err) + + err = tx.DecodeRLP( rlp.NewStream( bytes.NewReader(rlpEncodedTx), uint64(len(rlpEncodedTx)))) @@ -214,7 +217,7 @@ func (a *Account) Balance() types.Balance { } // Deposit deposits the token from the given vault into the flow evm main vault -// and update the FOA balance with the new amount +// and update the account balance with the new amount func (a *Account) Deposit(v *types.FLOWTokenVault) { cfg := a.fch.getBlockContext() a.fch.checkGasLimit(types.GasLimit(cfg.DirectCallBaseGasUsage)) @@ -232,7 +235,7 @@ func (a *Account) Deposit(v *types.FLOWTokenVault) { a.fch.updateLastExecutedBlock(res.StateRootHash, gethTypes.EmptyRootHash) } -// Withdraw deducts the balance from the FOA account and +// Withdraw deducts the balance from the account and // withdraw and return flow token from the Flex main vault. func (a *Account) Withdraw(b types.Balance) *types.FLOWTokenVault { a.checkAuthorized() @@ -280,7 +283,7 @@ func (a *Account) Transfer(to types.Address, balance types.Balance) { // Deploy deploys a contract to the EVM environment // the new deployed contract would be at the returned address and -// the contract data is not controlled by the FOA accounts +// the contract data is not controlled by the caller accounts func (a *Account) Deploy(code types.Code, gaslimit types.GasLimit, balance types.Balance) types.Address { a.checkAuthorized() a.fch.checkGasLimit(gaslimit) @@ -300,7 +303,6 @@ func (a *Account) Deploy(code types.Code, gaslimit types.GasLimit, balance types // it would limit the gas used according to the limit provided // given it doesn't goes beyond what Flow transaction allows. // the balance would be deducted from the OFA account and would be transferred to the target address -// contract data is not controlled by the FOA accounts func (a *Account) Call(to types.Address, data types.Data, gaslimit types.GasLimit, balance types.Balance) types.Data { a.checkAuthorized() a.fch.checkGasLimit(gaslimit) @@ -318,7 +320,7 @@ func (a *Account) Call(to types.Address, data types.Data, gaslimit types.GasLimi } func (a *Account) checkAuthorized() { - // check if account is authorized to to FOA related opeartions + // check if account is authorized (i.e. is a bridged account) if !a.isAuthorized { handleError(types.ErrUnAuthroizedMethodCall) } From c45b4055ab85144f6f4fc2e4c5430abb56d25386 Mon Sep 17 00:00:00 2001 From: ramtinms Date: Wed, 25 Oct 2023 14:17:24 -0700 Subject: [PATCH 67/68] add test for transaction integration --- fvm/evm/evm_test.go | 119 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 119 insertions(+) diff --git a/fvm/evm/evm_test.go b/fvm/evm/evm_test.go index 23c66973bcb..2c56880a1d7 100644 --- a/fvm/evm/evm_test.go +++ b/fvm/evm/evm_test.go @@ -21,11 +21,14 @@ import ( "github.com/stretchr/testify/require" "go.opentelemetry.io/otel/attribute" + "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/stdlib/emulator" "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/fvm/evm/types" + "github.com/onflow/flow-go/fvm/storage/snapshot" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" ) func encodeArgs(argValues []cadence.Value) [][]byte { @@ -722,3 +725,119 @@ func (i *testRuntimeInterface) invalidateUpdatedPrograms() { i.updatedContractCode = false } } + +func TestTransactionIntegration(t *testing.T) { + chain := flow.Emulator.Chain() + testutils.RunWithTestBackend(t, func(backend types.Backend) { + testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { + RunWithNewTestVM(t, chain, func(ctx fvm.Context, vm fvm.VM, snapshot snapshot.SnapshotTree) { + + code := []byte(` + transaction(bytes: [UInt8; 20]) { + execute { + Flex.FlexAddress(bytes: bytes) + } + } + `) + + addressBytesArray := cadence.NewArray([]cadence.Value{ + cadence.UInt8(1), cadence.UInt8(1), + cadence.UInt8(2), cadence.UInt8(2), + cadence.UInt8(3), cadence.UInt8(3), + cadence.UInt8(4), cadence.UInt8(4), + cadence.UInt8(5), cadence.UInt8(5), + cadence.UInt8(6), cadence.UInt8(6), + cadence.UInt8(7), cadence.UInt8(7), + cadence.UInt8(8), cadence.UInt8(8), + cadence.UInt8(9), cadence.UInt8(9), + cadence.UInt8(10), cadence.UInt8(10), + }).WithType(evmAddressBytesCadenceType) + + tx := flow.NewTransactionBody(). + SetScript(code). + SetPayer(unittest.AddressFixture()). + AddArgument(json.MustEncode(addressBytesArray)) + + executionSnapshot, output, err := vm.Run( + ctx, + fvm.Transaction(tx, 0), + snapshot) + require.NoError(t, err) + require.NoError(t, output.Err) + + _ = executionSnapshot + // snapshot = snapshot.Append(executionSnapshot) + }) + }) + }) +} + +func TestScriptIntegration(t *testing.T) { + chain := flow.Emulator.Chain() + testutils.RunWithTestBackend(t, func(backend types.Backend) { + testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { + RunWithNewTestVM(t, chain, func(ctx fvm.Context, vm fvm.VM, snapshot snapshot.SnapshotTree) { + + code := []byte(` + pub fun main(_ bytes: [UInt8; 20]): Flex.FlexAddress { + return Flex.FlexAddress(bytes: bytes) + } + `) + + addressBytesArray := cadence.NewArray([]cadence.Value{ + cadence.UInt8(1), cadence.UInt8(1), + cadence.UInt8(2), cadence.UInt8(2), + cadence.UInt8(3), cadence.UInt8(3), + cadence.UInt8(4), cadence.UInt8(4), + cadence.UInt8(5), cadence.UInt8(5), + cadence.UInt8(6), cadence.UInt8(6), + cadence.UInt8(7), cadence.UInt8(7), + cadence.UInt8(8), cadence.UInt8(8), + cadence.UInt8(9), cadence.UInt8(9), + cadence.UInt8(10), cadence.UInt8(10), + }).WithType(evmAddressBytesCadenceType) + + script := fvm.Script(code).WithArguments( + json.MustEncode(addressBytesArray), + ) + + executionSnapshot, output, err := vm.Run( + ctx, + script, + snapshot) + require.NoError(t, err) + require.NoError(t, output.Err) + + _ = executionSnapshot + // snapshot = snapshot.Append(executionSnapshot) + }) + }) + }) +} + +func RunWithNewTestVM(t *testing.T, chain flow.Chain, f func(fvm.Context, fvm.VM, snapshot.SnapshotTree)) { + opts := []fvm.Option{ + fvm.WithChain(chain), + fvm.WithAuthorizationChecksEnabled(false), + fvm.WithSequenceNumberCheckAndIncrementEnabled(false), + fvm.WithEVMEnabled(true), + } + ctx := fvm.NewContext(opts...) + + vm := fvm.NewVirtualMachine() + snapshotTree := snapshot.NewSnapshotTree(nil) + + baseBootstrapOpts := []fvm.BootstrapProcedureOption{ + fvm.WithInitialTokenSupply(unittest.GenesisTokenSupply), + } + + executionSnapshot, _, err := vm.Run( + ctx, + fvm.Bootstrap(unittest.ServiceAccountPublicKey, baseBootstrapOpts...), + snapshotTree) + require.NoError(t, err) + + snapshotTree = snapshotTree.Append(executionSnapshot) + + f(ctx, vm, snapshotTree) +} From 97315e9a49484c759016c579c05ece2495fc4bdb Mon Sep 17 00:00:00 2001 From: ramtinms Date: Wed, 25 Oct 2023 14:37:47 -0700 Subject: [PATCH 68/68] add reusable Environment to scripts --- fvm/runtime/reusable_cadence_runtime.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/fvm/runtime/reusable_cadence_runtime.go b/fvm/runtime/reusable_cadence_runtime.go index 4b249b73b47..07ec643c3f4 100644 --- a/fvm/runtime/reusable_cadence_runtime.go +++ b/fvm/runtime/reusable_cadence_runtime.go @@ -149,8 +149,9 @@ func (reusable *ReusableCadenceRuntime) ExecuteScript( return reusable.Runtime.ExecuteScript( script, runtime.Context{ - Interface: reusable.fvmEnv, - Location: location, + Interface: reusable.fvmEnv, + Location: location, + Environment: reusable.Environment, }, ) }