From a198ebd27aed70fff3fba0122abd1c3e117a6908 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 6 Aug 2021 22:18:19 -0700 Subject: [PATCH 001/291] init --- .../node_builder/access_node_builder.go | 18 +- .../staked_access_node_builder.go | 23 ++ cmd/node_builder.go | 33 +-- cmd/scaffold.go | 1 + engine/access/ping/engine.go | 6 +- engine/access/relay/engine.go | 3 +- integration/Makefile | 6 +- integration/testnet/container.go | 13 +- integration/testnet/network.go | 64 +++++- integration/testnet/network_test.go | 2 + .../tests/access/unstaked_node_test.go | 204 ++++++++++++++++++ 11 files changed, 324 insertions(+), 49 deletions(-) create mode 100644 integration/tests/access/unstaked_node_test.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b2fc8f6b6cb..73eb762ed0b 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -70,8 +70,8 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // ParticipatesInUnstakedNetwork returns True if this is a staked Access node which also participates - // in the unstaked network acting as an upstream for other unstaked access nodes, False otherwise. + // ParticipatesInUnstakedNetwork returns True if this an Access Node which participates in the unstaked network, + // False otherwise ParticipatesInUnstakedNetwork() bool // Build defines all of the Access node's components and modules. @@ -143,8 +143,8 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware + UnstakedNetwork p2p.ReadyDoneAwareNetwork + unstakedMiddleware network.Middleware FollowerState protocol.MutableState SyncCore *synchronization.Core RpcEng *rpc.Engine @@ -517,12 +517,6 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { } func (builder *FlowAccessNodeBuilder) ParticipatesInUnstakedNetwork() bool { - // unstaked access nodes can't be upstream of other unstaked access nodes for now - if !builder.IsStaked() { - return false - } - // if an unstaked network bind address is provided, then this staked access node will act as the upstream for - // unstaked access nodes return builder.unstakedNetworkBindAddr != cmd.NotSet } @@ -611,7 +605,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, unicastMessageTimeout time.Duration, connectionGating bool, managerPeerConnections bool, - validators ...network.MessageValidator) *p2p.Middleware { + validators ...network.MessageValidator) network.Middleware { builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, factoryFunc, nodeID, @@ -630,7 +624,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, // updated by calling network.SetIDs. func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, - middleware *p2p.Middleware, + middleware network.Middleware, participants flow.IdentityList, topology network.Topology) (*p2p.Network, error) { diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 419901b1bf5..ba79a866866 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -6,6 +6,8 @@ import ( "github.com/onflow/flow-go/cmd" pingeng "github.com/onflow/flow-go/engine/access/ping" + "github.com/onflow/flow-go/engine/access/relay" + splitternetwork "github.com/onflow/flow-go/engine/common/splitter/network" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/metrics" @@ -46,6 +48,10 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { + anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) + return node.Network, nil + }) anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -63,6 +69,23 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { } return ping, nil }) + if anb.ParticipatesInUnstakedNetwork() { + // create relay engine + anb.Component("relay engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + channels := node.SubscriptionManager.Channels() + if len(channels) == 0 { + return nil, fmt.Errorf("no subscribed channels to relay") + } + + relayEngine, err := relay.New(node.Logger, channels, node.Network, anb.UnstakedNetwork) + + if err != nil { + return nil, fmt.Errorf("could not create relay engine: %w", err) + } + + return relayEngine, nil + }) + } return anb } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index fbc2ce99f60..830eb06300b 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -112,22 +112,23 @@ type BaseConfig struct { // NodeBuilder functions as a node is bootstrapped. type NodeConfig struct { BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware *p2p.Middleware - Network *p2p.Network - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware network.Middleware + Network p2p.ReadyDoneAwareNetwork + SubscriptionManager network.SubscriptionManager + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // root state information RootBlock *flow.Block diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 88ceb2eda1e..07ef57814d4 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -179,6 +179,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit() { } subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) + node.SubscriptionManager = subscriptionManager top, err := topology.NewTopicBasedTopology(fnb.NodeID, fnb.Logger, fnb.State) if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) diff --git a/engine/access/ping/engine.go b/engine/access/ping/engine.go index be93a5d3283..7ac71ec89b5 100644 --- a/engine/access/ping/engine.go +++ b/engine/access/ping/engine.go @@ -9,7 +9,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -22,7 +22,7 @@ type Engine struct { pingEnabled bool pingInterval time.Duration - middleware *p2p.Middleware + middleware network.Middleware nodeInfo map[flow.Identifier]string // additional details about a node such as operator name } @@ -32,7 +32,7 @@ func New( me module.Local, metrics module.PingMetrics, pingEnabled bool, - mw *p2p.Middleware, + mw network.Middleware, nodeInfoFile string, ) (*Engine, error) { diff --git a/engine/access/relay/engine.go b/engine/access/relay/engine.go index b850f3d48f0..6c20ab72544 100644 --- a/engine/access/relay/engine.go +++ b/engine/access/relay/engine.go @@ -99,10 +99,11 @@ func (e *Engine) process(channel network.Channel, originID flow.Identifier, even conduit, ok := e.conduits[channel] if !ok { - e.log.Trace().Interface("event", event).Str("channel", channel.String()).Str("originID", originID.String()).Msg("unknown channel") return fmt.Errorf("received message on unknown channel %s", channel) } + e.log.Trace().Interface("event", event).Str("channel", channel.String()).Str("originID", originID.String()).Msg("relaying message") + // We use a dummy target ID here so that events are broadcast to the entire network if err := conduit.Publish(event, flow.ZeroID); err != nil { return fmt.Errorf("could not relay message: %w", err) diff --git a/integration/Makefile b/integration/Makefile index 231543fd848..81276cd00c6 100644 --- a/integration/Makefile +++ b/integration/Makefile @@ -12,7 +12,11 @@ integration-test: common-tests execution-tests verification-tests collection-tes # NOTE: Currently skipping collection and consensus tests due to them not passing properly on CI .PHONY: ci-integration-test -ci-integration-test: common-tests execution-tests epoch-tests verification-tests # collection-tests # consensus-tests +ci-integration-test: common-tests execution-tests epoch-tests verification-tests access-tests # collection-tests # consensus-tests + +.PHONY: access-tests +access-tests: + GO111MODULE=on go test -tags relic -count=1 ./tests/access .PHONY: collection-tests collection-tests: diff --git a/integration/testnet/container.go b/integration/testnet/container.go index 38183028ae9..8d700d18a2e 100644 --- a/integration/testnet/container.go +++ b/integration/testnet/container.go @@ -38,12 +38,13 @@ func init() { // ContainerConfig represents configuration for a node container in the network. type ContainerConfig struct { bootstrap.NodeInfo - ContainerName string - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - Unstaked bool + ContainerName string + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + Unstaked bool + ParticipatesInUnstakedNetwork bool } // ImageName returns the Docker image name for the given config. diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 04d5c4e88a4..def109f29ff 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -4,7 +4,6 @@ import ( "context" "encoding/hex" "fmt" - "github.com/onflow/flow-go/cmd/bootstrap/utils" "io/ioutil" "math/rand" "os" @@ -15,6 +14,8 @@ import ( "testing" "time" + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/docker/docker/api/types/container" dockerclient "github.com/docker/docker/client" "github.com/rs/zerolog" @@ -65,12 +66,20 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" + // UnstakedNetworkPort is the name used for the access node unstaked libp2p network port. + UnstakedNetworkPort = "access-unstaked-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" // ExeNodeMetricsPort ExeNodeMetricsPort = "exe-metrics-port" + // default staked network port + DefaultStakedFlowPort = 2137 + + // default unstaked network port + DefaultUnstakedFlowPort = 7312 + DefaultViewsInStakingAuction uint64 = 5 DefaultViewsInDKGPhase uint64 = 50 DefaultViewsInEpoch uint64 = 180 @@ -288,7 +297,8 @@ type NodeConfig struct { Debug bool // Unstaked - only applicable to Access Node. Access nodes can be staked or unstaked. // Unstaked nodes are not part of the identity table - Unstaked bool // only applicable to Access node + Unstaked bool // only applicable to Access node + ParticipatesInUnstakedNetwork bool } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -364,6 +374,12 @@ func AsGhost() func(config *NodeConfig) { } } +func AsUnstakedNetworkParticipant() func(config *NodeConfig) { + return func(config *NodeConfig) { + config.ParticipatesInUnstakedNetwork = true + } +} + // WithAdditionalFlag adds additional flags to the command func WithAdditionalFlag(flag string) func(config *NodeConfig) { return func(config *NodeConfig) { @@ -562,6 +578,15 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort + if nodeConf.ParticipatesInUnstakedNetwork { + hostUnstakedPort := testingdock.RandomPort(t) + containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) + nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) + nodeContainer.addFlag("unstaked-bind-addr", fmt.Sprintf("%s:%d", nodeContainer.Name(), DefaultUnstakedFlowPort)) + nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + net.AccessPorts[UnstakedNetworkPort] = hostUnstakedPort + } + case flow.RoleConsensus: // use 1 here instead of the default 5, because the integration // tests only start 1 verification node @@ -581,6 +606,20 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.addFlag("rpc-addr", fmt.Sprintf("%s:9000", nodeContainer.Name())) nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort + + if nodeConf.ParticipatesInUnstakedNetwork { + hostUnstakedPort := testingdock.RandomPort(t) + containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) + + // TODO: Currently, it is not possible to create a staked ghost AN which + // participates on the unstaked network, because the ghost node only joins + // a single network during startup. The ghost node needs to support the + // "unstaked-bind-addr" flag which can be used to specify a bind address + // for the unstaked network. + + nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) + nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + } } if nodeConf.Debug { @@ -817,7 +856,11 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { // define the node's name _ and address : name := fmt.Sprintf("%s_%d", conf.Role.String(), roleCounter[conf.Role]+1) - addr := fmt.Sprintf("%s:%d", name, 2137) + flowPort := DefaultStakedFlowPort + if conf.Unstaked { + flowPort = DefaultUnstakedFlowPort + } + addr := fmt.Sprintf("%s:%d", name, flowPort) roleCounter[conf.Role]++ info := bootstrap.NewPrivateNodeInfo( @@ -830,13 +873,14 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - Unstaked: conf.Unstaked, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + Unstaked: conf.Unstaked, + ParticipatesInUnstakedNetwork: conf.ParticipatesInUnstakedNetwork, } confs = append(confs, containerConf) diff --git a/integration/testnet/network_test.go b/integration/testnet/network_test.go index 1a3be04cbdb..fa979f50c00 100644 --- a/integration/testnet/network_test.go +++ b/integration/testnet/network_test.go @@ -33,6 +33,7 @@ func TestNetworkSetupBasic(t *testing.T) { testnet.NewNodeConfig(flow.RoleConsensus), testnet.NewNodeConfig(flow.RoleExecution), testnet.NewNodeConfig(flow.RoleVerification), + testnet.NewNodeConfig(flow.RoleAccess), } conf := testnet.NewNetworkConfig("meta_test_basic", nodes) @@ -50,6 +51,7 @@ func TestNetworkSetupBasic(t *testing.T) { {image: defaultRegistry + "/consensus:latest", name: "consensus_3", address: "consensus_3:2137"}, {image: defaultRegistry + "/execution:latest", name: "execution_1", address: "execution_1:2137"}, {image: defaultRegistry + "/verification:latest", name: "verification_1", address: "verification_1:2137"}, + {image: defaultRegistry + "/access:latest", name: "access_1", address: "access_1:2137"}, } assert.Subset(t, realData, expectedData) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go new file mode 100644 index 00000000000..194d151b1a8 --- /dev/null +++ b/integration/tests/access/unstaked_node_test.go @@ -0,0 +1,204 @@ +package access + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/engine" + ghostclient "github.com/onflow/flow-go/engine/ghost/client" + "github.com/onflow/flow-go/integration/testnet" + "github.com/onflow/flow-go/integration/tests/common" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/messages" + "github.com/onflow/flow-go/utils/unittest" +) + +type UnstakedAccessSuite struct { + suite.Suite + + // root context for the current test + ctx context.Context + cancel context.CancelFunc + + net *testnet.FlowNetwork + unstakedGhost *ghostclient.GhostClient + conGhost *ghostclient.GhostClient + stakedID flow.Identifier + unstakedID flow.Identifier + conID flow.Identifier +} + +func TestUnstakedAccessSuite(t *testing.T) { + suite.Run(t, new(UnstakedAccessSuite)) +} + +func (suite *UnstakedAccessSuite) TearDownTest() { + // avoid nil pointer errors for skipped tests + if suite.cancel != nil { + defer suite.cancel() + } + if suite.net != nil { + suite.net.Remove() + } +} + +func (suite *UnstakedAccessSuite) SetupTest() { + nodeConfigs := []testnet.NodeConfig{} + + // staked access node + suite.stakedID = unittest.IdentifierFixture() + stakedConfig := testnet.NewNodeConfig( + flow.RoleAccess, + testnet.WithID(suite.stakedID), + testnet.AsUnstakedNetworkParticipant(), + testnet.WithLogLevel(zerolog.InfoLevel) + ) + nodeConfigs = append(nodeConfigs, stakedConfig) + + // consensus node (ghost) + suite.conID = unittest.IdentifierFixture() + conConfig := testnet.NewNodeConfig( + flow.RoleConsensus, + testnet.WithID(suite.conID), + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, conConfig) + + // execution node (unused) + exeConfig := testnet.NewNodeConfig( + flow.RoleExecution, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, exeConfig) + + // verification node (unused) + verConfig := testnet.NewNodeConfig( + flow.RoleVerification, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, verConfig) + + // collection node (unused) + collConfig := testnet.NewNodeConfig( + flow.RoleCollection, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, collConfig) + + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs) + suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + + suite.setupConsensusFollower() + + // start the network + suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.net.Start(suite.ctx) + + conGhost := suite.net.ContainerByID(suite.conID) + client, err = common.GetGhostClient(conGhost) + require.NoError(suite.T(), err, "could not get ghost client") + suite.conGhost = client +} + +func (suite *UnstakedAccessSuite) setupConsensusFollower() { + // create a temporary directory to store all bootstrapping files, these + // will be shared between all nodes + bootstrapDir, err := ioutil.TempDir(TmpRoot, "flow-integration-bootstrap") + require.Nil(t, err) + + // get a temporary directory in the host. On macOS the default tmp + // directory is NOT accessible to Docker by default, so we use /tmp + // instead. + tmpdir, err := ioutil.TempDir(TmpRoot, "flow-integration-node") + if err != nil { + return fmt.Errorf("could not get tmp dir: %w", err) + } + + // create a directory for the node database + dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) + err = os.Mkdir(flowDBDir, 0700) + require.NoError(t, err) + + // create a directory for the bootstrap files + // we create a node-specific bootstrap directory to enable testing nodes + // bootstrapping from different root state snapshots and epochs + followerBootstrapDir := filepath.Join(tmpdir, DefaultBootstrapDir) + err = os.Mkdir(nodeBootstrapDir, 0700) + require.NoError(t, err) + + // copy bootstrap files to node-specific bootstrap directory + err = io.CopyDirectory(bootstrapDir, followerBootstrapDir) + require.NoError(t, err) + + // consensus follower + suite.unstakedID = unittest.IdentifierFixture() + bindPort := testingdock.RandomPort(suite.T()) + bindAddr := fmt.Sprintf(":%v", bindPort) // TODO: verify this + opts := []consensus_follower.Option{ + consensus_follower.WithDataDir(dataDir), + consensus_follower.WithBootstrapDir(bootstrapDir), + } // TODO + upstreamANPort := suite.net.ContainerByID(suite.stakedID).Ports[testnet.UnstakedNetworkPort] // TODO + consensus_follower.NewConsensusFollower( + suite.unstakedID, + suite.stakedID, + bindAddr, + opts..., + ) +} + +func (suite *UnstakedAccessSuite) TestReceiveBlocks() { + // 1. Send new block from consensus node to staked AN + // 2. Check that unstaked AN (ghost) receives it + // 3. Check that staked AN also processed the block. This can be done by calling the + // Access API on the staked AN. + + block := unittest.BlockFixture() + + proposal := &messages.BlockProposal{ + Header: block.Header, + Payload: block.Payload, + } + + // Send block proposal fron consensus node to staked AN + suite.conGhost.Send(suite.ctx, engine.PushBlocks, proposal, suite.stakedID) + + m := make(chan interface{}) + go func() { + _, msg, err := suite.unstakedReader.Next() + suite.Require().Nil(err, "could not read next message") + suite.T().Logf("unstaked ghost recv: %T", msg) + + m <- msg + }() + + // Check that the unstaked AN receives the message + select { + case msg := <-m: + suite.Assert().Equal(msg, proposal) + case <-time.After(5 * time.Second): + suite.T().Fatal("timed out waiting for next message") + } + + // TODO: Since the staked AN follower engine will perform validation on received blocks, + // the following check may not work unless we send a "valid" block. In particular we will + // probably at least need to generate a block with ParentID equal to the root block ID + // (suite.net.Root().ID()) + + // chain := suite.net.Root().Header.ChainID.Chain() + + // stakedContainer := suite.net.ContainerByID(suite.stakedID) + // stakedClient, err := testnet.NewClient(stakedContainer.Addr(testnet.AccessNodeAPIPort), chain) + // require.NoError(suite.T(), err) + + // suite.Assert().Equal(stakedClient.GetLatestBlockID(suite.ctx), block.ID()) + +} From 7ea4e72a01c72b5ae4c5803878c65880d7f81d44 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 15:14:03 -0700 Subject: [PATCH 002/291] s --- follower/consensus_follower.go | 2 +- integration/testnet/network.go | 100 +++++++++-- .../tests/access/unstaked_node_test.go | 158 ++++-------------- 3 files changed, 121 insertions(+), 139 deletions(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 162faf4b3c0..e1051091e35 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -1,4 +1,4 @@ -package consensus_follower +package follower import ( "context" diff --git a/integration/testnet/network.go b/integration/testnet/network.go index def109f29ff..cfbfeed0a52 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -32,6 +32,7 @@ import ( "github.com/onflow/cadence" "github.com/onflow/flow-go/cmd/bootstrap/run" + consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" dkgmod "github.com/onflow/flow-go/model/dkg" @@ -91,17 +92,18 @@ func init() { // FlowNetwork represents a test network of Flow nodes running in Docker containers. type FlowNetwork struct { - t *testing.T - suite *testingdock.Suite - config NetworkConfig - cli *dockerclient.Client - network *testingdock.Network - Containers map[string]*Container - AccessPorts map[string]string - root *flow.Block - result *flow.ExecutionResult - seal *flow.Seal - bootstrapDir string + t *testing.T + suite *testingdock.Suite + config NetworkConfig + cli *dockerclient.Client + network *testingdock.Network + Containers map[string]*Container + ConsensusFollowers map[flow.Identifier]consensus_follower.ConsensusFollower + AccessPorts map[string]string + root *flow.Block + result *flow.ExecutionResult + seal *flow.Seal + bootstrapDir string } // Identities returns a list of identities, one for each node in the network. @@ -133,6 +135,9 @@ func (net *FlowNetwork) Start(ctx context.Context) { // makes it easier to see logs for a specific test case fmt.Println(">>>> starting network: ", net.config.Name) net.suite.Start(ctx) + for _, cf := range net.ConsensusFollowers { + go cf.Run(ctx) + } } // Remove stops the network, removes all the containers and cleans up all resources. @@ -204,6 +209,14 @@ func (net *FlowNetwork) ContainerByID(id flow.Identifier) *Container { return nil } +// ConsensusFollowerByID returns the ConsensusFollower with the given node ID, if it exists. +// Otherwise fails the test. +func (net *FlowNetwork) ConsensusFollowerByID(id flow.Identifier) consensus_follower.ConsensusFollower { + follower, ok := net.ConsensusFollowers[id] + require.True(net.t, ok) + return follower +} + // ContainerByName returns the container with the given name, if it exists. // Otherwise fails the test. func (net *FlowNetwork) ContainerByName(name string) *Container { @@ -212,9 +225,22 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { return container } +type ConsensusFollowerConfig struct { + nodeID flow.Identifier + upstreamNodeID flow.Identifier +} + +func NewConsensusFollowerConfig(nodeID flow.Identifier, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { + return ConsensusFollowerConfig{ + nodeID: nodeID, + upstreamNodeID: upstreamNodeID, + } +} + // NetworkConfig is the config for the network. type NetworkConfig struct { Nodes []NodeConfig + ConsensusFollowers []ConsensusFollowerConfig Name string NClusters uint ViewsInDKGPhase uint64 @@ -224,9 +250,10 @@ type NetworkConfig struct { type NetworkConfigOpt func(*NetworkConfig) -func NewNetworkConfig(name string, nodes []NodeConfig, opts ...NetworkConfigOpt) NetworkConfig { +func NewNetworkConfig(name string, nodes []NodeConfig, followers []ConsensusFollowerConfig, opts ...NetworkConfigOpt) NetworkConfig { c := NetworkConfig{ Nodes: nodes, + ConsensusFollowers: followers, Name: name, NClusters: 1, // default to 1 cluster ViewsInStakingAuction: DefaultViewsInStakingAuction, @@ -440,9 +467,58 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { require.NoError(t, err) } + // add each follower to the network + for _, followerConf := range networkConf.ConsensusFollowers { + err = flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) + require.NoError(t, err) + } + return flowNetwork } +func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) error { + tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") + if err != nil { + return fmt.Errorf("could not get tmp dir: %w", err) + } + + // create a directory for the follower database + dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) + err = os.Mkdir(dataDir, 0700) + require.NoError(t, err) + + // create a follower-specific directory for the bootstrap files + followerBootstrapDir := filepath.Join(tmpdir, DefaultBootstrapDir) + err = os.Mkdir(followerBootstrapDir, 0700) + require.NoError(t, err) + + // copy bootstrap files to follower-specific bootstrap directory + err = io.CopyDirectory(bootstrapDir, followerBootstrapDir) + require.NoError(t, err) + + // consensus follower + bindPort := testingdock.RandomPort(t) + bindAddr := fmt.Sprintf("0.0.0.0:%d", bindPort) + opts := []consensus_follower.Option{ + consensus_follower.WithDataDir(dataDir), + consensus_follower.WithBootstrapDir(followerBootstrapDir), + } + + // TODO: eventually we will need upstream node's address + // + // upstreamANPort := net.ContainerByID(followerConf.upstreamNodeID).Ports[testnet.UnstakedNetworkPort] + // upstreamANAddress := fmt.Sprintf("127.0.0.1:%d", upstreamANPort) + + follower := consensus_follower.NewConsensusFollower( + followerConf.nodeID, + followerConf.upstreamNodeID, + bindAddr, + opts..., + ) + + net.ConsensusFollowers[followerConf.nodeID] = follower +} + // AddNode creates a node container with the given config and adds it to the // network. func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf ContainerConfig) error { diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 194d151b1a8..bb87daae6de 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -3,17 +3,12 @@ package access import ( "context" "testing" - "time" - "github.com/stretchr/testify/require" + "github.com/rs/zerolog" "github.com/stretchr/testify/suite" - "github.com/onflow/flow-go/engine" - ghostclient "github.com/onflow/flow-go/engine/ghost/client" "github.com/onflow/flow-go/integration/testnet" - "github.com/onflow/flow-go/integration/tests/common" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/utils/unittest" ) @@ -24,12 +19,11 @@ type UnstakedAccessSuite struct { ctx context.Context cancel context.CancelFunc - net *testnet.FlowNetwork - unstakedGhost *ghostclient.GhostClient - conGhost *ghostclient.GhostClient - stakedID flow.Identifier - unstakedID flow.Identifier - conID flow.Identifier + net *testnet.FlowNetwork + stakedID flow.Identifier + unstakedID flow.Identifier + conID flow.Identifier + follower consensus_follower.ConsensusFollower } func TestUnstakedAccessSuite(t *testing.T) { @@ -55,150 +49,62 @@ func (suite *UnstakedAccessSuite) SetupTest() { flow.RoleAccess, testnet.WithID(suite.stakedID), testnet.AsUnstakedNetworkParticipant(), - testnet.WithLogLevel(zerolog.InfoLevel) + testnet.WithLogLevel(zerolog.InfoLevel), ) nodeConfigs = append(nodeConfigs, stakedConfig) // consensus node (ghost) suite.conID = unittest.IdentifierFixture() conConfig := testnet.NewNodeConfig( - flow.RoleConsensus, - testnet.WithID(suite.conID), - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleConsensus, + testnet.WithID(suite.conID), + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, conConfig) // execution node (unused) exeConfig := testnet.NewNodeConfig( - flow.RoleExecution, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleExecution, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, exeConfig) // verification node (unused) verConfig := testnet.NewNodeConfig( - flow.RoleVerification, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleVerification, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, verConfig) // collection node (unused) collConfig := testnet.NewNodeConfig( - flow.RoleCollection, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleCollection, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, collConfig) - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs) - suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + // consensus follower + suite.unstakedID = unittest.IdentifierFixture() + followerConfigs := []testnet.ConsensusFollowerConfig{ + testnet.ConsensusFollowerConfig{ + nodeID: suite.unstakedID, + upstreamNodeID: suite.stakedID, + }, + } - suite.setupConsensusFollower() + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) + suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) // start the network suite.ctx, suite.cancel = context.WithCancel(context.Background()) suite.net.Start(suite.ctx) - - conGhost := suite.net.ContainerByID(suite.conID) - client, err = common.GetGhostClient(conGhost) - require.NoError(suite.T(), err, "could not get ghost client") - suite.conGhost = client -} - -func (suite *UnstakedAccessSuite) setupConsensusFollower() { - // create a temporary directory to store all bootstrapping files, these - // will be shared between all nodes - bootstrapDir, err := ioutil.TempDir(TmpRoot, "flow-integration-bootstrap") - require.Nil(t, err) - - // get a temporary directory in the host. On macOS the default tmp - // directory is NOT accessible to Docker by default, so we use /tmp - // instead. - tmpdir, err := ioutil.TempDir(TmpRoot, "flow-integration-node") - if err != nil { - return fmt.Errorf("could not get tmp dir: %w", err) - } - - // create a directory for the node database - dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) - err = os.Mkdir(flowDBDir, 0700) - require.NoError(t, err) - - // create a directory for the bootstrap files - // we create a node-specific bootstrap directory to enable testing nodes - // bootstrapping from different root state snapshots and epochs - followerBootstrapDir := filepath.Join(tmpdir, DefaultBootstrapDir) - err = os.Mkdir(nodeBootstrapDir, 0700) - require.NoError(t, err) - - // copy bootstrap files to node-specific bootstrap directory - err = io.CopyDirectory(bootstrapDir, followerBootstrapDir) - require.NoError(t, err) - - // consensus follower - suite.unstakedID = unittest.IdentifierFixture() - bindPort := testingdock.RandomPort(suite.T()) - bindAddr := fmt.Sprintf(":%v", bindPort) // TODO: verify this - opts := []consensus_follower.Option{ - consensus_follower.WithDataDir(dataDir), - consensus_follower.WithBootstrapDir(bootstrapDir), - } // TODO - upstreamANPort := suite.net.ContainerByID(suite.stakedID).Ports[testnet.UnstakedNetworkPort] // TODO - consensus_follower.NewConsensusFollower( - suite.unstakedID, - suite.stakedID, - bindAddr, - opts..., - ) } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - // 1. Send new block from consensus node to staked AN - // 2. Check that unstaked AN (ghost) receives it - // 3. Check that staked AN also processed the block. This can be done by calling the - // Access API on the staked AN. - - block := unittest.BlockFixture() - - proposal := &messages.BlockProposal{ - Header: block.Header, - Payload: block.Payload, - } - - // Send block proposal fron consensus node to staked AN - suite.conGhost.Send(suite.ctx, engine.PushBlocks, proposal, suite.stakedID) - - m := make(chan interface{}) - go func() { - _, msg, err := suite.unstakedReader.Next() - suite.Require().Nil(err, "could not read next message") - suite.T().Logf("unstaked ghost recv: %T", msg) - - m <- msg - }() - - // Check that the unstaked AN receives the message - select { - case msg := <-m: - suite.Assert().Equal(msg, proposal) - case <-time.After(5 * time.Second): - suite.T().Fatal("timed out waiting for next message") - } - - // TODO: Since the staked AN follower engine will perform validation on received blocks, - // the following check may not work unless we send a "valid" block. In particular we will - // probably at least need to generate a block with ParentID equal to the root block ID - // (suite.net.Root().ID()) - - // chain := suite.net.Root().Header.ChainID.Chain() - - // stakedContainer := suite.net.ContainerByID(suite.stakedID) - // stakedClient, err := testnet.NewClient(stakedContainer.Addr(testnet.AccessNodeAPIPort), chain) - // require.NoError(suite.T(), err) - - // suite.Assert().Equal(stakedClient.GetLatestBlockID(suite.ctx), block.ID()) - + suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) + // TODO } From e67c8604c87fac6692e411541fea3aa4cdc1d11c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 15:15:26 -0700 Subject: [PATCH 003/291] Update unstaked_node_test.go --- integration/tests/access/unstaked_node_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index bb87daae6de..1ee76939237 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -7,6 +7,7 @@ import ( "github.com/rs/zerolog" "github.com/stretchr/testify/suite" + consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/utils/unittest" From 4c102b83d928f83f2eb5ba0914c9450fbe070665 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 16:51:26 -0700 Subject: [PATCH 004/291] Add consensus follower to integration tests --- integration/go.mod | 1 + integration/go.sum | 8 ++ integration/testnet/network.go | 77 +++++++++++++------ integration/tests/access/access_api_test.go | 2 +- .../tests/access/unstaked_node_test.go | 11 ++- 5 files changed, 69 insertions(+), 30 deletions(-) diff --git a/integration/go.mod b/integration/go.mod index 8d34c4cb857..4eaf5587bf4 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -6,6 +6,7 @@ require ( github.com/DataDog/zstd v1.4.8 // indirect github.com/HdrHistogram/hdrhistogram-go v1.0.1 // indirect github.com/dapperlabs/testingdock v0.4.3-0.20200626075145-ea23fc16bb90 + github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f // indirect github.com/dgraph-io/badger/v2 v2.2007.2 github.com/dgraph-io/ristretto v0.0.3 // indirect github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect diff --git a/integration/go.sum b/integration/go.sum index 223642df5f9..17fae948dce 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -213,6 +213,8 @@ github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c h1:pFUpOrbxDR github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c/go.mod h1:6UhI8N9EjYm1c2odKpFpAYeR8dsBeM7PtzQhRgxRr9U= github.com/deckarep/golang-set v0.0.0-20180603214616-504e848d77ea/go.mod h1:93vsz/8Wt4joVM7c2AVqh+YRMiUSc14yDtF28KmMOgQ= github.com/decred/dcrd/lru v1.0.0/go.mod h1:mxKOwFd7lFjN2GZYsiz/ecgqR6kkYAl+0pz0tEMk218= +github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f h1:U5y3Y5UE0w7amNe7Z5G/twsBW0KEalRQXZzf8ufSh9I= +github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f/go.mod h1:xH/i4TFMt8koVQZ6WFms69WAsDWr2XsYL3Hkl7jkoLE= github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= github.com/dgraph-io/badger v1.6.0-rc1/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= @@ -472,9 +474,13 @@ github.com/graph-gophers/graphql-go v0.0.0-20191115155744-f33e81362277/go.mod h1 github.com/graph-gophers/graphql-go v0.0.0-20201113091052-beb923fada29/go.mod h1:9CQHMSxwO4MprSdzoIEobiHpoLtHm77vfxsvsIN5Vuc= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2 h1:uxUHSMwWDJ/9jVPHNumRC8WZOi3hrBL22ObVOoLg4ww= github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2/go.mod h1:BL7w7qd2l/j9jgY6WMhYutfOFQc0I8RTVwtjpnAMoTM= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea h1:1Tk1IbruXbunEnaIZEFb+Hpv9BIZti3OxKwKn5wWyKk= github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea/go.mod h1:GugMBs30ZSAkckqXEAIEGyYdDH6EgqowG8ppA3Zt+AY= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= @@ -523,6 +529,7 @@ github.com/huin/goupnp v1.0.1-0.20200620063722-49508fba0031 h1:HarGZ5h9HD9LgEg1y github.com/huin/goupnp v1.0.1-0.20200620063722-49508fba0031/go.mod h1:nNs7wvRfN1eKaMknBydLNQU6146XQim8t4h+q90biWo= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/improbable-eng/grpc-web v0.12.0 h1:GlCS+lMZzIkfouf7CNqY+qqpowdKuJLSLLcKVfM1oLc= github.com/improbable-eng/grpc-web v0.12.0/go.mod h1:6hRR09jOEG81ADP5wCQju1z71g6OL4eEvELdran/3cs= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= @@ -1168,6 +1175,7 @@ github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.2.2/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/cors v0.0.0-20160617231935-a62a804a8a00/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/rs/xhandler v0.0.0-20160618193221-ed27b6fd6521/go.mod h1:RvLn4FgxWubrpZHtQLnOf6EwhN2hEMusxZOhcW9H3UQ= github.com/rs/xid v1.2.1/go.mod h1:+uKXf+4Djp6Md1KODXJxgGQPKngRmWyn10oCKFzNHOQ= diff --git a/integration/testnet/network.go b/integration/testnet/network.go index cfbfeed0a52..b1cc9941529 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -135,9 +135,6 @@ func (net *FlowNetwork) Start(ctx context.Context) { // makes it easier to see logs for a specific test case fmt.Println(">>>> starting network: ", net.config.Name) net.suite.Start(ctx) - for _, cf := range net.ConsensusFollowers { - go cf.Run(ctx) - } } // Remove stops the network, removes all the containers and cleans up all resources. @@ -448,17 +445,18 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { require.Nil(t, err) flowNetwork := &FlowNetwork{ - t: t, - cli: dockerClient, - config: networkConf, - suite: suite, - network: network, - Containers: make(map[string]*Container, nNodes), - AccessPorts: make(map[string]string), - root: root, - seal: seal, - result: result, - bootstrapDir: bootstrapDir, + t: t, + cli: dockerClient, + config: networkConf, + suite: suite, + network: network, + Containers: make(map[string]*Container, nNodes), + ConsensusFollowers: make(map[flow.Identifier]consensus_follower.ConsensusFollower, len(networkConf.ConsensusFollowers)), + AccessPorts: make(map[string]string), + root: root, + seal: seal, + result: result, + bootstrapDir: bootstrapDir, } // add each node to the network @@ -469,18 +467,15 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { // add each follower to the network for _, followerConf := range networkConf.ConsensusFollowers { - err = flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) - require.NoError(t, err) + flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) } return flowNetwork } -func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) error { +func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) { tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") - if err != nil { - return fmt.Errorf("could not get tmp dir: %w", err) - } + require.NoError(t, err) // create a directory for the follower database dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) @@ -498,7 +493,7 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // consensus follower bindPort := testingdock.RandomPort(t) - bindAddr := fmt.Sprintf("0.0.0.0:%d", bindPort) + bindAddr := fmt.Sprintf("0.0.0.0:%s", bindPort) opts := []consensus_follower.Option{ consensus_follower.WithDataDir(dataDir), consensus_follower.WithBootstrapDir(followerBootstrapDir), @@ -721,6 +716,37 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { require.NoError(net.t, err) } +func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { + var nodeInfos []bootstrap.NodeInfo + + // get networking keys for all followers + networkKeys, err := unittest.NetworkingKeys(len(confs)) + if err != nil { + return nil, err + } + + // get staking keys for all followers + stakingKeys, err := unittest.StakingKeys(len(confs)) + if err != nil { + return nil, err + } + + for i, conf := range confs { + info := bootstrap.NewPrivateNodeInfo( + conf.nodeID, + flow.RoleAccess, // use Access role + "", // no address + 0, // no stake + networkKeys[i], + stakingKeys[i], + ) + + nodeInfos = append(nodeInfos, info) + } + + return nodeInfos, nil +} + func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Block, *flow.ExecutionResult, *flow.Seal, []ContainerConfig, error) { chainID := flow.Localnet chain := chainID.Chain() @@ -746,9 +772,14 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo stakedConfs := filterContainerConfigs(allConfs, func(config ContainerConfig) bool { return !config.Unstaked }) - fmt.Println(len(stakedConfs)) - fmt.Println(len(allConfs)) + + followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) + if err != nil { + return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) + } + allNodeInfos := toNodeInfos(allConfs) + allNodeInfos = append(allNodeInfos, followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices // IMPORTANT: these nodes infos must include exactly the identity table diff --git a/integration/tests/access/access_api_test.go b/integration/tests/access/access_api_test.go index 56e437afc26..24fd024f720 100644 --- a/integration/tests/access/access_api_test.go +++ b/integration/tests/access/access_api_test.go @@ -66,7 +66,7 @@ func (suite *AccessSuite) SetupTest() { collConfig := testnet.NewNodeConfig(flow.RoleCollection, testnet.WithID(collID)) nodeConfigs = append(nodeConfigs, collConfig) - conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs) + conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs, nil) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) // start the network diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 1ee76939237..f5259c853aa 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -91,21 +91,20 @@ func (suite *UnstakedAccessSuite) SetupTest() { // consensus follower suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.ConsensusFollowerConfig{ - nodeID: suite.unstakedID, - upstreamNodeID: suite.stakedID, - }, + testnet.NewConsensusFollowerConfig(suite.unstakedID, suite.stakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) + // start the network suite.ctx, suite.cancel = context.WithCancel(context.Background()) suite.net.Start(suite.ctx) } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) - // TODO + go suite.follower.Run(suite.ctx) + // TODO: to be implemented later } From e9bfcb18d36ee56b067e7b1d5b9750acf2771034 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 16:58:07 -0700 Subject: [PATCH 005/291] Update staked_access_node_builder.go --- cmd/access/node_builder/staked_access_node_builder.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index ba79a866866..c697a3e1c31 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -48,10 +48,12 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { - anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) - return node.Network, nil - }) + if anb.ParticipatesInUnstakedNetwork() { + anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) + return node.Network, nil + }) + } anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { From eb24e516373366ca51ebe8abc9c2cb72417536d6 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 00:07:25 -0700 Subject: [PATCH 006/291] initial --- go.mod | 27 +++--- go.sum | 118 +++++++++++++++++++++++++ integration/go.mod | 8 +- integration/go.sum | 167 +++++++++++++++++++++++++++++------- module/id/id_provider.go | 148 ++++++++++++++++++++++++++++++++ network/p2p/idTranslator.go | 33 +++++++ network/p2p/libp2pNode.go | 10 ++- 7 files changed, 463 insertions(+), 48 deletions(-) create mode 100644 module/id/id_provider.go create mode 100644 network/p2p/idTranslator.go diff --git a/go.mod b/go.mod index afe3b5b96da..5383fbc401f 100644 --- a/go.mod +++ b/go.mod @@ -13,31 +13,34 @@ require ( github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8 github.com/gammazero/workerpool v1.1.2 github.com/gogo/protobuf v1.3.2 - github.com/golang/mock v1.5.0 + github.com/golang/mock v1.6.0 github.com/golang/protobuf v1.5.2 github.com/google/go-cmp v0.5.5 - github.com/google/uuid v1.1.2 + github.com/google/uuid v1.3.0 github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2 github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 - github.com/hashicorp/go-multierror v1.0.0 + github.com/hashicorp/go-multierror v1.1.1 github.com/hashicorp/golang-lru v0.5.4 github.com/improbable-eng/grpc-web v0.12.0 - github.com/ipfs/go-log v1.0.4 + github.com/ipfs/go-log v1.0.5 + github.com/ipfs/go-todocounter v0.0.2 // indirect github.com/jrick/bitset v1.0.0 github.com/kr/text v0.2.0 // indirect - github.com/libp2p/go-addr-util v0.0.2 - github.com/libp2p/go-libp2p v0.14.1 - github.com/libp2p/go-libp2p-core v0.8.5 + github.com/libp2p/go-addr-util v0.1.0 + github.com/libp2p/go-libp2p v0.14.4 + github.com/libp2p/go-libp2p-core v0.8.6 github.com/libp2p/go-libp2p-discovery v0.5.0 + github.com/libp2p/go-libp2p-kad-dht v0.13.0 // indirect github.com/libp2p/go-libp2p-pubsub v0.4.1 - github.com/libp2p/go-libp2p-swarm v0.5.0 + github.com/libp2p/go-libp2p-routing v0.1.0 // indirect + github.com/libp2p/go-libp2p-swarm v0.5.3 github.com/libp2p/go-libp2p-tls v0.1.3 - github.com/libp2p/go-libp2p-transport-upgrader v0.4.2 - github.com/libp2p/go-tcp-transport v0.2.1 + github.com/libp2p/go-libp2p-transport-upgrader v0.4.6 + github.com/libp2p/go-tcp-transport v0.2.7 github.com/m4ksio/wal v1.0.0 github.com/mitchellh/mapstructure v1.3.3 // indirect - github.com/multiformats/go-multiaddr v0.3.1 + github.com/multiformats/go-multiaddr v0.3.3 github.com/onflow/cadence v0.18.1-0.20210729032058-d9eb6683d6ed github.com/onflow/flow-core-contracts/lib/go/contracts v0.7.5 github.com/onflow/flow-core-contracts/lib/go/templates v0.7.5 @@ -48,7 +51,7 @@ require ( github.com/opentracing/opentracing-go v1.2.0 github.com/pelletier/go-toml v1.7.0 // indirect github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.7.1 + github.com/prometheus/client_golang v1.10.0 github.com/rs/zerolog v1.19.0 github.com/spf13/cobra v1.1.3 github.com/spf13/pflag v1.0.5 diff --git a/go.sum b/go.sum index f1ee7a2c6f0..fa2415cdf7c 100644 --- a/go.sum +++ b/go.sum @@ -43,6 +43,7 @@ dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= +github.com/AndreasBriese/bbloom v0.0.0-20190825152654-46b345b51c96/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/Azure/azure-pipeline-go v0.2.1/go.mod h1:UGSo8XybXnIGZ3epmeBw7Jdz+HiUVpqIlpz/HKHylF4= github.com/Azure/azure-pipeline-go v0.2.2/go.mod h1:4rQ/NZncSvGqNkkOsNpOU1tgoNuIlp9AfUH5G1tvCHc= github.com/Azure/azure-storage-blob-go v0.7.0/go.mod h1:f9YQKtsG1nMisotuTPpO0tjNuEjKRYAcJU8/ydDI++4= @@ -99,6 +100,7 @@ github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZw github.com/benbjohnson/clock v1.0.2/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= github.com/benbjohnson/clock v1.0.3 h1:vkLuvpK4fmtSCuo60+yC63p7y0BmQ8gm5ZXGuBCJyXg= github.com/benbjohnson/clock v1.0.3/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= +github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -185,6 +187,7 @@ github.com/dgraph-io/badger v1.6.0-rc1/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhY github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger v1.6.1 h1:w9pSFNSdq/JPM1N12Fz/F/bzo993Is1W+Q7HjPzi7yg= github.com/dgraph-io/badger v1.6.1/go.mod h1:FRmFw3uxvcpa8zG3Rxs0th+hCLIuaQg8HlNV5bjgnuU= +github.com/dgraph-io/badger v1.6.2/go.mod h1:JW2yswe3V058sS0kZ2h/AXeDSqFjxnZcRrVH//y2UQE= github.com/dgraph-io/badger/v2 v2.0.3 h1:inzdf6VF/NZ+tJ8RwwYMjJMvsOALTHYdozn0qSl6XJI= github.com/dgraph-io/badger/v2 v2.0.3/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= @@ -230,6 +233,7 @@ github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJn github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= +github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= @@ -261,6 +265,7 @@ github.com/go-sourcemap/sourcemap v2.1.2+incompatible/go.mod h1:F8jJfvm2KbVjc5Nq github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/go-test/deep v1.0.5 h1:AKODKU3pDH1RzZzm6YZu77YWtEAq6uh1rLIAQlay2qc= github.com/go-test/deep v1.0.5/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= @@ -291,6 +296,7 @@ github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0 h1:jlYHihg//f7RRwuPfptm04yp4s7O6Kw8EZiVYIGcH0g= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= +github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -325,6 +331,7 @@ github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= @@ -349,6 +356,8 @@ github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= +github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go v2.0.0+incompatible h1:j0GKcs05QVmm7yesiZq2+9cxHkNK9YM6zKx4D2qucQU= github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= @@ -393,6 +402,9 @@ github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjh github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-multierror v1.1.0/go.mod h1:spPvp8C1qA32ftKqdAHm4hHTbPw+vmowP0z+KUhOZdA= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= @@ -433,34 +445,47 @@ github.com/ipfs/go-cid v0.0.6/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqg github.com/ipfs/go-cid v0.0.7 h1:ysQJVJA3fNDF1qigJbsSQOdjhVLsOEoPdh0+R97k3jY= github.com/ipfs/go-cid v0.0.7/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= github.com/ipfs/go-datastore v0.0.1/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= +github.com/ipfs/go-datastore v0.1.0/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= +github.com/ipfs/go-datastore v0.1.1/go.mod h1:w38XXW9kVFNp57Zj5knbKWM2T+KOZCGDRVNdgPHtbHw= github.com/ipfs/go-datastore v0.4.0/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.1/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.4/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= +github.com/ipfs/go-datastore v0.4.5 h1:cwOUcGMLdLPWgu3SlrCckCMznaGADbPqE0r8h768/Dg= github.com/ipfs/go-datastore v0.4.5/go.mod h1:eXTcaaiN6uOlVCLS9GjJUJtlvJfM3xk23w3fyfrmmJs= github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= github.com/ipfs/go-ds-badger v0.0.2/go.mod h1:Y3QpeSFWQf6MopLTiZD+VT6IC1yZqaGmjvRcKeSGij8= github.com/ipfs/go-ds-badger v0.0.5/go.mod h1:g5AuuCGmr7efyzQhLL8MzwqcauPojGPUaHzfGTzuE3s= +github.com/ipfs/go-ds-badger v0.0.7/go.mod h1:qt0/fWzZDoPW6jpQeqUjR5kBfhDNB65jd9YlmAvpQBk= github.com/ipfs/go-ds-badger v0.2.1/go.mod h1:Tx7l3aTph3FMFrRS838dcSJh+jjA7cX9DrGVwx/NOwE= github.com/ipfs/go-ds-badger v0.2.3/go.mod h1:pEYw0rgg3FIrywKKnL+Snr+w/LjJZVMTBRn4FS6UHUk= +github.com/ipfs/go-ds-badger v0.2.7/go.mod h1:02rnztVKA4aZwDuaRPTf8mpqcKmXP7mLl6JPxd14JHA= github.com/ipfs/go-ds-leveldb v0.0.1/go.mod h1:feO8V3kubwsEF22n0YRQCffeb79OOYIykR4L04tMOYc= +github.com/ipfs/go-ds-leveldb v0.1.0/go.mod h1:hqAW8y4bwX5LWcCtku2rFNX3vjDZCy5LZCg+cSZvYb8= github.com/ipfs/go-ds-leveldb v0.4.1/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ds-leveldb v0.4.2/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-util v0.0.1/go.mod h1:spsl5z8KUnrve+73pOhSVZND1SIxPW5RyBCNzQxlJBc= github.com/ipfs/go-ipfs-util v0.0.2 h1:59Sswnk1MFaiq+VcaknX7aYEyGyGDAA73ilhEK2POp8= github.com/ipfs/go-ipfs-util v0.0.2/go.mod h1:CbPtkWJzjLdEcezDns2XYaehFVNXG9zrdrtMecczcsQ= +github.com/ipfs/go-ipns v0.1.2 h1:O/s/0ht+4Jl9+VoxoUo0zaHjnZUS+aBQIKTuzdZ/ucI= +github.com/ipfs/go-ipns v0.1.2/go.mod h1:ioQ0j02o6jdIVW+bmi18f4k2gRf0AV3kZ9KeHYHICnQ= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-log v1.0.2/go.mod h1:1MNjMxe0u6xvJZgeqbJ8vdo2TKaGwZ1a0Bpza+sr2Sk= github.com/ipfs/go-log v1.0.3/go.mod h1:OsLySYkwIbiSUR/yBTdv1qPtcE4FW3WPWk/ewz9Ru+A= github.com/ipfs/go-log v1.0.4 h1:6nLQdX4W8P9yZZFH7mO+X/PzjN8Laozm/lMJ6esdgzY= github.com/ipfs/go-log v1.0.4/go.mod h1:oDCg2FkjogeFOhqqb+N39l2RpTNPL6F/StPkB3kPgcs= +github.com/ipfs/go-log v1.0.5 h1:2dOuUCB1Z7uoczMWgAyDck5JLb72zHzrMnGnCNNbvY8= +github.com/ipfs/go-log v1.0.5/go.mod h1:j0b8ZoR+7+R99LD9jZ6+AJsrzkPbSXbZfGakb5JPtIo= github.com/ipfs/go-log/v2 v2.0.2/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= github.com/ipfs/go-log/v2 v2.0.3/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= github.com/ipfs/go-log/v2 v2.0.5/go.mod h1:eZs4Xt4ZUJQFM3DlanGhy7TkwwawCZcSByscwkWG+dw= github.com/ipfs/go-log/v2 v2.1.1/go.mod h1:2v2nsGfZsvvAJz13SyFzf9ObaqwHiHxsPLEHntrv9KM= github.com/ipfs/go-log/v2 v2.1.3 h1:1iS3IU7aXRlbgUpN8yTTpJ53NXYjAe37vcI5+5nYrzk= github.com/ipfs/go-log/v2 v2.1.3/go.mod h1:/8d0SH3Su5Ooc31QlL1WysJhvyOTDCjcCZ9Axpmri6g= +github.com/ipfs/go-todocounter v0.0.2/go.mod h1:l5aErvQc8qKE2r7NDMjmq5UNAvuZy0rC8BHOplkWvZ4= +github.com/ipld/go-ipld-prime v0.9.0 h1:N2OjJMb+fhyFPwPnVvJcWU/NsumP8etal+d2v3G4eww= +github.com/ipld/go-ipld-prime v0.9.0/go.mod h1:KvBLMr4PX1gWptgkzRjVZCrLmSGcZCb/jioOQwCqZN8= github.com/jackpal/gateway v1.0.5/go.mod h1:lTpwd4ACLXmpyiCTRtfiNyVnUmqT9RivzCDQetPfnjA= github.com/jackpal/go-nat-pmp v1.0.1/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= @@ -509,6 +534,9 @@ github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQL github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= +github.com/klauspost/compress v1.11.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/cpuid/v2 v2.0.4 h1:g0I61F2K2DjRHz1cnxlkNSBIaePVoJIjjnHui8QHbiw= +github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -528,9 +556,12 @@ github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+ github.com/libp2p/go-addr-util v0.0.1/go.mod h1:4ac6O7n9rIAKB1dnd+s8IbbMXkt+oBpzX4/+RACcnlQ= github.com/libp2p/go-addr-util v0.0.2 h1:7cWK5cdA5x72jX0g8iLrQWm5TRJZ6CzGdPEhWj7plWU= github.com/libp2p/go-addr-util v0.0.2/go.mod h1:Ecd6Fb3yIuLzq4bD7VcywcVSBtefcAwnUISBM3WG15E= +github.com/libp2p/go-addr-util v0.1.0/go.mod h1:6I3ZYuFr2O/9D+SoyM0zEw0EF3YkldtTX406BpdQMqw= github.com/libp2p/go-buffer-pool v0.0.1/go.mod h1:xtyIz9PMobb13WaxR6Zo1Pd1zXJKYg0a8KiIvDp3TzQ= github.com/libp2p/go-buffer-pool v0.0.2 h1:QNK2iAFa8gjAe1SPz6mHSMuCcjs+X1wlHzeOSqcmlfs= github.com/libp2p/go-buffer-pool v0.0.2/go.mod h1:MvaB6xw5vOrDl8rYZGLFdKAuk/hRoRZd1Vi32+RXyFM= +github.com/libp2p/go-cidranger v1.1.0 h1:ewPN8EZ0dd1LSnrtuwd4709PXVcITVeuwbag38yPW7c= +github.com/libp2p/go-cidranger v1.1.0/go.mod h1:KWZTfSr+r9qEo9OkI9/SIEeAtw+NNoU0dXIXt15Okic= github.com/libp2p/go-conn-security-multistream v0.1.0/go.mod h1:aw6eD7LOsHEX7+2hJkDxw1MteijaVcI+/eP2/x3J1xc= github.com/libp2p/go-conn-security-multistream v0.2.0/go.mod h1:hZN4MjlNetKD3Rq5Jb/P5ohUnFLNzEAR4DLSzpn2QLU= github.com/libp2p/go-conn-security-multistream v0.2.1 h1:ft6/POSK7F+vl/2qzegnHDaXFU0iWB4yVTYrioC6Zy0= @@ -539,6 +570,7 @@ github.com/libp2p/go-eventbus v0.1.0/go.mod h1:vROgu5cs5T7cv7POWlWxBaVLxfSegC5UG github.com/libp2p/go-eventbus v0.2.1 h1:VanAdErQnpTioN2TowqNcOijf6YwhuODe4pPKSDpxGc= github.com/libp2p/go-eventbus v0.2.1/go.mod h1:jc2S4SoEVPP48H9Wpzm5aiGwUCBMfGhVhhBjyhhCJs8= github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZxBdp967ls1g+k8= +github.com/libp2p/go-flow-metrics v0.0.2/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-flow-metrics v0.0.3 h1:8tAs/hSdNvUiLgtlSy3mxwxWP4I9y/jlkPFT7epKdeM= github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-libp2p v0.6.1/go.mod h1:CTFnWXogryAHjXAKEbOf1OWY+VeAP3lDMZkfEI5sT54= @@ -547,6 +579,9 @@ github.com/libp2p/go-libp2p v0.7.4/go.mod h1:oXsBlTLF1q7pxr+9w6lqzS1ILpyHsaBPniV github.com/libp2p/go-libp2p v0.8.1/go.mod h1:QRNH9pwdbEBpx5DTJYg+qxcVaDMAz3Ee/qDKwXujH5o= github.com/libp2p/go-libp2p v0.14.1 h1:R0vNY7nkU8IISlDuHd2yk4eNAZsVQ0rCr2bPfWU3sXo= github.com/libp2p/go-libp2p v0.14.1/go.mod h1:0PQMADQEjCM2l8cSMYDpTgsb8gr6Zq7i4LUgq1mlW2E= +github.com/libp2p/go-libp2p v0.14.4/go.mod h1:EIRU0Of4J5S8rkockZM7eJp2S0UrCyi55m2kJVru3rM= +github.com/libp2p/go-libp2p-asn-util v0.0.0-20200825225859-85005c6cf052 h1:BM7aaOF7RpmNn9+9g6uTjGJ0cTzWr5j9i9IKeun2M8U= +github.com/libp2p/go-libp2p-asn-util v0.0.0-20200825225859-85005c6cf052/go.mod h1:nRMRTab+kZuk0LnKZpxhOVH/ndsdr2Nr//Zltc/vwgo= github.com/libp2p/go-libp2p-autonat v0.1.1/go.mod h1:OXqkeGOY2xJVWKAGV2inNF5aKN/djNA3fdpCWloIudE= github.com/libp2p/go-libp2p-autonat v0.2.0/go.mod h1:DX+9teU4pEEoZUqR1PiMlqliONQdNbfzE1C718tcViI= github.com/libp2p/go-libp2p-autonat v0.2.1/go.mod h1:MWtAhV5Ko1l6QBsHQNSuM6b1sRkXrpk0/LqCr+vCVxI= @@ -568,27 +603,37 @@ github.com/libp2p/go-libp2p-core v0.0.4/go.mod h1:jyuCQP356gzfCFtRKyvAbNkyeuxb7O github.com/libp2p/go-libp2p-core v0.2.0/go.mod h1:X0eyB0Gy93v0DZtSYbEM7RnMChm9Uv3j7yRXjO77xSI= github.com/libp2p/go-libp2p-core v0.2.2/go.mod h1:8fcwTbsG2B+lTgRJ1ICZtiM5GWCWZVoVrLaDRvIRng0= github.com/libp2p/go-libp2p-core v0.2.4/go.mod h1:STh4fdfa5vDYr0/SzYYeqnt+E6KfEV5VxfIrm0bcI0g= +github.com/libp2p/go-libp2p-core v0.2.5/go.mod h1:6+5zJmKhsf7yHn1RbmYDu08qDUpIUxGdqHuEZckmZOA= github.com/libp2p/go-libp2p-core v0.3.0/go.mod h1:ACp3DmS3/N64c2jDzcV429ukDpicbL6+TrrxANBjPGw= github.com/libp2p/go-libp2p-core v0.3.1/go.mod h1:thvWy0hvaSBhnVBaW37BvzgVV68OUhgJJLAa6almrII= github.com/libp2p/go-libp2p-core v0.4.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= github.com/libp2p/go-libp2p-core v0.5.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= github.com/libp2p/go-libp2p-core v0.5.1/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= +github.com/libp2p/go-libp2p-core v0.5.3/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= github.com/libp2p/go-libp2p-core v0.5.4/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= github.com/libp2p/go-libp2p-core v0.5.5/go.mod h1:vj3awlOr9+GMZJFH9s4mpt9RHHgGqeHCopzbYKZdRjM= github.com/libp2p/go-libp2p-core v0.5.6/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.5.7/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.6.0/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= +github.com/libp2p/go-libp2p-core v0.6.1/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.7.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.1/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.2/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.5 h1:aEgbIcPGsKy6zYcC+5AJivYFedhYa4sW7mIpWpUaLKw= github.com/libp2p/go-libp2p-core v0.8.5/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= +github.com/libp2p/go-libp2p-core v0.8.6 h1:3S8g006qG6Tjpj1JdRK2S+TWc2DJQKX/RG9fdLeiLSU= +github.com/libp2p/go-libp2p-core v0.8.6/go.mod h1:dgHr0l0hIKfWpGpqAMbpo19pen9wJfdCGv51mTmdpmM= github.com/libp2p/go-libp2p-crypto v0.1.0/go.mod h1:sPUokVISZiy+nNuTTH/TY+leRSxnFj/2GLjtOTW90hI= github.com/libp2p/go-libp2p-discovery v0.2.0/go.mod h1:s4VGaxYMbw4+4+tsoQTqh7wfxg97AEdo4GYBt6BadWg= github.com/libp2p/go-libp2p-discovery v0.3.0/go.mod h1:o03drFnz9BVAZdzC/QUQ+NeQOu38Fu7LJGEOK2gQltw= github.com/libp2p/go-libp2p-discovery v0.5.0 h1:Qfl+e5+lfDgwdrXdu4YNCWyEo3fWuP+WgN9mN0iWviQ= github.com/libp2p/go-libp2p-discovery v0.5.0/go.mod h1:+srtPIU9gDaBNu//UHvcdliKBIcr4SfDcm0/PfPJLug= +github.com/libp2p/go-libp2p-kad-dht v0.13.0 h1:qBNYzee8BVS6RkD8ukIAGRG6LmVz8+kkeponyI7W+yA= +github.com/libp2p/go-libp2p-kad-dht v0.13.0/go.mod h1:NkGf28RNhPrcsGYWJHm6EH8ULkiJ2qxsWmpE7VTL3LI= +github.com/libp2p/go-libp2p-kbucket v0.3.1/go.mod h1:oyjT5O7tS9CQurok++ERgc46YLwEpuGoFq9ubvoUOio= +github.com/libp2p/go-libp2p-kbucket v0.4.7 h1:spZAcgxifvFZHBD8tErvppbnNiKA5uokDu3CV7axu70= +github.com/libp2p/go-libp2p-kbucket v0.4.7/go.mod h1:XyVo99AfQH0foSf176k4jY1xUJ2+jUJIZCSDm7r2YKk= github.com/libp2p/go-libp2p-loggables v0.1.0/go.mod h1:EyumB2Y6PrYjr55Q3/tiJ/o3xoDasoRYM7nOzEpoa90= github.com/libp2p/go-libp2p-mplex v0.2.0/go.mod h1:Ejl9IyjvXJ0T9iqUTE1jpYATQ9NM3g+OtR+EMMODbKo= github.com/libp2p/go-libp2p-mplex v0.2.1/go.mod h1:SC99Rxs8Vuzrf/6WhmH41kNn13TiYdAWNYHrwImKLnE= @@ -607,18 +652,27 @@ github.com/libp2p/go-libp2p-noise v0.2.0/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhU github.com/libp2p/go-libp2p-peer v0.2.0/go.mod h1:RCffaCvUyW2CJmG2gAWVqwePwW7JMgxjsHm7+J5kjWY= github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= github.com/libp2p/go-libp2p-peerstore v0.1.3/go.mod h1:BJ9sHlm59/80oSkpWgr1MyY1ciXAXV397W6h1GH/uKI= +github.com/libp2p/go-libp2p-peerstore v0.1.4/go.mod h1:+4BDbDiiKf4PzpANZDAT+knVdLxvqh7hXOujessqdzs= github.com/libp2p/go-libp2p-peerstore v0.2.0/go.mod h1:N2l3eVIeAitSg3Pi2ipSrJYnqhVnMNQZo9nkSCuAbnQ= github.com/libp2p/go-libp2p-peerstore v0.2.1/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.7 h1:83JoLxyR9OYTnNfB5vvFqvMUv/xDNa6NoPHnENhBsGw= github.com/libp2p/go-libp2p-peerstore v0.2.7/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= +github.com/libp2p/go-libp2p-peerstore v0.2.8 h1:nJghUlUkFVvyk7ccsM67oFA6kqUkwyCM1G4WPVMCWYA= +github.com/libp2p/go-libp2p-peerstore v0.2.8/go.mod h1:gGiPlXdz7mIHd2vfAsHzBNAMqSDkt2UBFwgcITgw1lA= github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6na5f0/k= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= github.com/libp2p/go-libp2p-pubsub v0.4.1 h1:j4umIg5nyus+sqNfU+FWvb9aeYFQH/A+nDFhWj+8yy8= github.com/libp2p/go-libp2p-pubsub v0.4.1/go.mod h1:izkeMLvz6Ht8yAISXjx60XUQZMq9ZMe5h2ih4dLIBIQ= github.com/libp2p/go-libp2p-quic-transport v0.10.0 h1:koDCbWD9CCHwcHZL3/WEvP2A+e/o5/W5L3QS/2SPMA0= github.com/libp2p/go-libp2p-quic-transport v0.10.0/go.mod h1:RfJbZ8IqXIhxBRm5hqUEJqjiiY8xmEuq3HUDS993MkA= +github.com/libp2p/go-libp2p-quic-transport v0.11.2/go.mod h1:wlanzKtIh6pHrq+0U3p3DY9PJfGqxMgPaGKaK5LifwQ= +github.com/libp2p/go-libp2p-record v0.1.2/go.mod h1:pal0eNcT5nqZaTV7UGhqeGqxFgGdsU/9W//C8dqjQDk= +github.com/libp2p/go-libp2p-record v0.1.3 h1:R27hoScIhQf/A8XJZ8lYpnqh9LatJ5YbHs28kCIfql0= +github.com/libp2p/go-libp2p-record v0.1.3/go.mod h1:yNUff/adKIfPnYQXgp6FQmNu3gLJ6EMg7+/vv2+9pY4= +github.com/libp2p/go-libp2p-routing v0.1.0/go.mod h1:zfLhI1RI8RLEzmEaaPwzonRvXeeSHddONWkcTcB54nE= +github.com/libp2p/go-libp2p-routing-helpers v0.2.3/go.mod h1:795bh+9YeoFl99rMASoiVgHdi5bjack0N1+AFAdbvBw= github.com/libp2p/go-libp2p-secio v0.1.0/go.mod h1:tMJo2w7h3+wN4pgU2LSYeiKPrfqBgkOsdiKK77hE7c8= github.com/libp2p/go-libp2p-secio v0.2.0/go.mod h1:2JdZepB8J5V9mBp79BmwsaPQhRPNN2NrnB2lKQcdy6g= github.com/libp2p/go-libp2p-secio v0.2.1/go.mod h1:cWtZpILJqkqrSkiYcDBh5lA3wbT2Q+hz3rJQq3iftD8= @@ -631,6 +685,7 @@ github.com/libp2p/go-libp2p-swarm v0.3.0/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJeg github.com/libp2p/go-libp2p-swarm v0.3.1/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= github.com/libp2p/go-libp2p-swarm v0.5.0 h1:HIK0z3Eqoo8ugmN8YqWAhD2RORgR+3iNXYG4U2PFd1E= github.com/libp2p/go-libp2p-swarm v0.5.0/go.mod h1:sU9i6BoHE0Ve5SKz3y9WfKrh8dUat6JknzUehFx8xW4= +github.com/libp2p/go-libp2p-swarm v0.5.3/go.mod h1:NBn7eNW2lu568L7Ns9wdFrOhgRlkRnIDg0FLKbuu3i8= github.com/libp2p/go-libp2p-testing v0.0.2/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.3/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.4/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= @@ -640,6 +695,7 @@ github.com/libp2p/go-libp2p-testing v0.1.2-0.20200422005655-8775583591d8/go.mod github.com/libp2p/go-libp2p-testing v0.3.0/go.mod h1:efZkql4UZ7OVsEfaxNHZPzIehtsBXMrXnCfJIgDti5g= github.com/libp2p/go-libp2p-testing v0.4.0 h1:PrwHRi0IGqOwVQWR3xzgigSlhlLfxgfXgkHxr77EghQ= github.com/libp2p/go-libp2p-testing v0.4.0/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= +github.com/libp2p/go-libp2p-testing v0.4.2/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= github.com/libp2p/go-libp2p-tls v0.1.3 h1:twKMhMu44jQO+HgQK9X8NHO5HkeJu2QbhLzLJpa8oNM= github.com/libp2p/go-libp2p-tls v0.1.3/go.mod h1:wZfuewxOndz5RTnCAxFliGjvYSDA40sKitV4c50uI1M= github.com/libp2p/go-libp2p-transport-upgrader v0.1.1/go.mod h1:IEtA6or8JUbsV07qPW4r01GnTenLW4oi3lOPbUMGJJA= @@ -647,6 +703,8 @@ github.com/libp2p/go-libp2p-transport-upgrader v0.2.0/go.mod h1:mQcrHj4asu6ArfSo github.com/libp2p/go-libp2p-transport-upgrader v0.3.0/go.mod h1:i+SKzbRnvXdVbU3D1dwydnTmKRPXiAR/fyvi1dXuL4o= github.com/libp2p/go-libp2p-transport-upgrader v0.4.2 h1:4JsnbfJzgZeRS9AWN7B9dPqn/LY/HoQTlO9gtdJTIYM= github.com/libp2p/go-libp2p-transport-upgrader v0.4.2/go.mod h1:NR8ne1VwfreD5VIWIU62Agt/J18ekORFU/j1i2y8zvk= +github.com/libp2p/go-libp2p-transport-upgrader v0.4.6/go.mod h1:JE0WQuQdy+uLZ5zOaI3Nw9dWGYJIA7mywEtP2lMvnyk= +github.com/libp2p/go-libp2p-xor v0.0.0-20210714161855-5c005aca55db/go.mod h1:LSTM5yRnjGZbWNTA/hRwq2gGFrvRIbQJscoIL/u6InY= github.com/libp2p/go-libp2p-yamux v0.2.0/go.mod h1:Db2gU+XfLpm6E4rG5uGCFX6uXA8MEXOxFcRoXUODaK8= github.com/libp2p/go-libp2p-yamux v0.2.2/go.mod h1:lIohaR0pT6mOt0AZ0L2dFze9hds9Req3OfS+B+dv4qw= github.com/libp2p/go-libp2p-yamux v0.2.5/go.mod h1:Zpgj6arbyQrmZ3wxSZxfBmbdnWtbZ48OpsfmQVTErwA= @@ -676,6 +734,7 @@ github.com/libp2p/go-nat v0.0.5 h1:qxnwkco8RLKqVh1NmjQ+tJ8p8khNLFxuElYG/TwqW4Q= github.com/libp2p/go-nat v0.0.5/go.mod h1:B7NxsVNPZmRLvMOwiEO1scOSyjA56zxYAGv1yQgRkEU= github.com/libp2p/go-netroute v0.1.2/go.mod h1:jZLDV+1PE8y5XxBySEBgbuVAXbhtuHSdmLPL2n9MKbk= github.com/libp2p/go-netroute v0.1.3/go.mod h1:jZLDV+1PE8y5XxBySEBgbuVAXbhtuHSdmLPL2n9MKbk= +github.com/libp2p/go-netroute v0.1.5/go.mod h1:V1SR3AaECRkEQCoFFzYwVYWvYIEtlxx89+O3qcpCl4A= github.com/libp2p/go-netroute v0.1.6 h1:ruPJStbYyXVYGQ81uzEDzuvbYRLKRrLvTYd33yomC38= github.com/libp2p/go-netroute v0.1.6/go.mod h1:AqhkMh0VuWmfgtxKPp3Oc1LdU5QSWS7wl0QLhSZqXxQ= github.com/libp2p/go-openssl v0.0.2/go.mod h1:v8Zw2ijCSWBQi8Pq5GAixw6DbFfa9u6VIYDXnvOXkc0= @@ -691,7 +750,9 @@ github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA2 github.com/libp2p/go-reuseport-transport v0.0.3/go.mod h1:Spv+MPft1exxARzP2Sruj2Wb5JSyHNncjf1Oi2dEbzM= github.com/libp2p/go-reuseport-transport v0.0.4 h1:OZGz0RB620QDGpv300n1zaOcKGGAoGVf8h9txtt/1uM= github.com/libp2p/go-reuseport-transport v0.0.4/go.mod h1:trPa7r/7TJK/d+0hdBLOCGvpQQVOU74OXbNCIMkufGw= +github.com/libp2p/go-reuseport-transport v0.0.5/go.mod h1:TC62hhPc8qs5c/RoXDZG6YmjK+/YWUPC0yYmeUecbjc= github.com/libp2p/go-sockaddr v0.0.2/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= +github.com/libp2p/go-sockaddr v0.1.0/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-sockaddr v0.1.1 h1:yD80l2ZOdGksnOyHrhxDdTDFrf7Oy+v3FMVArIRgZxQ= github.com/libp2p/go-sockaddr v0.1.1/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-stream-muxer v0.0.1/go.mod h1:bAo8x7YkSpadMTbtTaxGVHWUQsR/l5MEaHbKaliuT14= @@ -703,6 +764,8 @@ github.com/libp2p/go-tcp-transport v0.1.1/go.mod h1:3HzGvLbx6etZjnFlERyakbaYPdfj github.com/libp2p/go-tcp-transport v0.2.0/go.mod h1:vX2U0CnWimU4h0SGSEsg++AzvBcroCGYw28kh94oLe0= github.com/libp2p/go-tcp-transport v0.2.1 h1:ExZiVQV+h+qL16fzCWtd1HSzPsqWottJ8KXwWaVi8Ns= github.com/libp2p/go-tcp-transport v0.2.1/go.mod h1:zskiJ70MEfWz2MKxvFB/Pv+tPIB1PpPUrHIWQ8aFw7M= +github.com/libp2p/go-tcp-transport v0.2.4/go.mod h1:9dvr03yqrPyYGIEN6Dy5UvdJZjyPFvl1S/igQ5QD1SU= +github.com/libp2p/go-tcp-transport v0.2.7/go.mod h1:lue9p1b3VmZj1MhhEGB/etmvF/nBQ0X9CW2DutBT3MM= github.com/libp2p/go-ws-transport v0.2.0/go.mod h1:9BHJz/4Q5A9ludYWKoGCFC5gUElzlHoKzu0yY9p/klM= github.com/libp2p/go-ws-transport v0.3.0/go.mod h1:bpgTJmRZAvVHrgHybCVyqoBmyLQ1fiZuEaBYusP5zsk= github.com/libp2p/go-ws-transport v0.4.0 h1:9tvtQ9xbws6cA5LvqdE6Ne3vcmGB4f1z9SByggk4s0k= @@ -723,6 +786,7 @@ github.com/logrusorgru/aurora v0.0.0-20200102142835-e9ef32dff381 h1:bqDmpDG49ZRn github.com/logrusorgru/aurora v0.0.0-20200102142835-e9ef32dff381/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= github.com/lucas-clemente/quic-go v0.19.3 h1:eCDQqvGBB+kCTkA0XrAFtNe81FMa0/fn4QSoeAbmiF4= github.com/lucas-clemente/quic-go v0.19.3/go.mod h1:ADXpNbTQjq1hIzCpB+y/k5iz4n4z4IwqoLb94Kh5Hu8= +github.com/lucas-clemente/quic-go v0.21.2/go.mod h1:vF5M1XqhBAHgbjKcJOXY3JZz3GP0T3FQhz/uyOUS38Q= github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= github.com/m4ksio/wal v1.0.0 h1:PucHOZPz58BgWowe+Gf+gZUbgEdd4zFx+He45SGkHG0= @@ -738,6 +802,10 @@ github.com/marten-seemann/qtls v0.10.0/go.mod h1:UvMd1oaYDACI99/oZUYLzMCkBXQVT0a github.com/marten-seemann/qtls-go1-15 v0.1.1/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= github.com/marten-seemann/qtls-go1-15 v0.1.4 h1:RehYMOyRW8hPVEja1KBVsFVNSm35Jj9Mvs5yNoZZ28A= github.com/marten-seemann/qtls-go1-15 v0.1.4/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-15 v0.1.5/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-16 v0.1.4/go.mod h1:gNpI2Ol+lRS3WwSOtIUUtRwZEQMXjYK+dQSBFbethAk= +github.com/marten-seemann/qtls-go1-17 v0.1.0-rc.1/go.mod h1:fz4HIxByo+LlWcreM4CZOYNuz3taBQ8rN2X6FqvaWo8= +github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd/go.mod h1:QuCEs1Nt24+FYQEqAAncTDPJIuGs+LxK1MCiFL25pMU= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.0/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ= @@ -768,6 +836,9 @@ github.com/miekg/dns v1.1.12/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3N github.com/miekg/dns v1.1.28/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/miekg/dns v1.1.41 h1:WMszZWJG0XmzbK9FEmzH2TVcqYzFesusSIB41b8KHxY= github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c/go.mod h1:0SQS9kMwD2VsyFEB++InYyBJroV/FRmBgcydeSUcJms= +github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b/go.mod h1:lxPUiZwKoFL8DUUmalo2yJJUCxbPKtm8OKfqr2/FTNU= +github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc/go.mod h1:cGKTAVKx4SxOuR/czcZ/E2RSJ3sfHs8FpHhQ5CWMf9s= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= @@ -776,6 +847,8 @@ github.com/minio/sha256-simd v0.1.0/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= github.com/minio/sha256-simd v0.1.1 h1:5QHSlgo3nt5yKOJrC7W8w7X+NFl8cMPZm96iu8kKUJU= github.com/minio/sha256-simd v0.1.1/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= +github.com/minio/sha256-simd v1.0.0 h1:v1ta+49hkWZyvaKwrQB8elexRqm6Y0aMLjCNsrYxo6g= +github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db/go.mod h1:l0dey0ia/Uv7NcFFVbCLtqEBQbrT4OCwCSKTEv6enCw= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= @@ -812,6 +885,8 @@ github.com/multiformats/go-multiaddr v0.2.2/go.mod h1:NtfXiOtHvghW9KojvtySjH5y0u github.com/multiformats/go-multiaddr v0.3.0/go.mod h1:dF9kph9wfJ+3VLAaeBqo9Of8x4fJxp6ggJGteB8HQTI= github.com/multiformats/go-multiaddr v0.3.1 h1:1bxa+W7j9wZKTZREySx1vPMs2TqrYWjVZ7zE6/XLG1I= github.com/multiformats/go-multiaddr v0.3.1/go.mod h1:uPbspcUPd5AfaP6ql3ujFY+QWzmBD8uLLL4bXW0XfGc= +github.com/multiformats/go-multiaddr v0.3.3 h1:vo2OTSAqnENB2rLk79pLtr+uhj+VAzSe3uef5q0lRSs= +github.com/multiformats/go-multiaddr v0.3.3/go.mod h1:lCKNGP1EQ1eZ35Za2wlqnabm9xQkib3fyB+nZXHLag0= github.com/multiformats/go-multiaddr-dns v0.0.1/go.mod h1:9kWcqw/Pj6FwxAwW38n/9403szc57zJPs45fmnznu3Q= github.com/multiformats/go-multiaddr-dns v0.0.2/go.mod h1:9kWcqw/Pj6FwxAwW38n/9403szc57zJPs45fmnznu3Q= github.com/multiformats/go-multiaddr-dns v0.2.0/go.mod h1:TJ5pr5bBO7Y1B18djPuRsVkduhQH2YqYSbxWJzYGdK0= @@ -832,13 +907,18 @@ github.com/multiformats/go-multiaddr-net v0.2.0/go.mod h1:gGdH3UXny6U3cKKYCvpXI5 github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= github.com/multiformats/go-multibase v0.0.3 h1:l/B6bJDQjvQ5G52jw4QGSYeOTZoAwIO77RblWplfIqk= github.com/multiformats/go-multibase v0.0.3/go.mod h1:5+1R4eQrT3PkYZ24C3W2Ue2tPwIdYQD509ZjSb5y9Oc= +github.com/multiformats/go-multicodec v0.2.0 h1:MUzKZWxOFagwLLtlx96pub9zwDQAbMAf1k9fXOdc3so= +github.com/multiformats/go-multicodec v0.2.0/go.mod h1:/y4YVwkfMyry5kFbMTbLJKErhycTIftytRV+llXdyS4= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multihash v0.0.5/go.mod h1:lt/HCbqlQwlPBz7lv0sQCdtfcMtlJvakRUn/0Ual8po= github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= +github.com/multiformats/go-multihash v0.0.9/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.10/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.13/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= github.com/multiformats/go-multihash v0.0.14 h1:QoBceQYQQtNUuf6s7wHxnE2c8bhbMqhfGzNI032se/I= github.com/multiformats/go-multihash v0.0.14/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= +github.com/multiformats/go-multihash v0.0.15 h1:hWOPdrNqDjwHDx82vsYGSDZNyktOJJ2dzZJzFkOV1jM= +github.com/multiformats/go-multihash v0.0.15/go.mod h1:D6aZrWNLFTV/ynMpKsNtB40mJzmCl4jb1alC0OvHiHg= github.com/multiformats/go-multistream v0.1.0/go.mod h1:fJTiDfXJVmItycydCnNx4+wSzZ5NwG2FEVAI30fiovg= github.com/multiformats/go-multistream v0.1.1/go.mod h1:KmHZ40hzVxiaiwlj3MEbYgK9JFk2/9UktWZAF54Du38= github.com/multiformats/go-multistream v0.2.0/go.mod h1:5GZPQZbkWOLOn3J2y4Y99vVW7vOfsAflxARk3x14o6k= @@ -866,6 +946,7 @@ github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJE github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= @@ -904,6 +985,8 @@ github.com/onsi/ginkgo v1.12.0/go.mod h1:oUhWkIvk5aDxtKvDDuw8gItl8pKl42LzjC9KZE0 github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= +github.com/onsi/ginkgo v1.16.2/go.mod h1:CObGmKUOKaSC0RjmoAK7tKyn4Azo5P2IWuoMnvwxz1E= +github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= github.com/onsi/gomega v1.4.1/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= @@ -911,6 +994,7 @@ github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7J github.com/onsi/gomega v1.9.0/go.mod h1:Ho0h+IUsWyvy1OpqCwxlQ/21gkhVunqlU8fDGcoTdcA= github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.13.0/go.mod h1:lRk9szgn8TxENtWd0Tp4c3wjlRfMTMH27I+3Je41yGY= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= @@ -942,6 +1026,8 @@ github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6J github.com/pkg/term v1.1.0/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1 h1:CskT+S6Ay54OwxBGB0R3Rsx4Muto6UnEYTyKJbyRIAI= +github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= @@ -951,6 +1037,8 @@ github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5Fsn github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.7.1 h1:NTGy1Ja9pByO+xAeH/qiWnLrKtr3hJPNjaVUwnjpdpA= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.9.0/go.mod h1:FqZLKOZnGdFAhOK4nqGHa7D66IdsO+O441Eve7ptJDU= +github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -967,6 +1055,8 @@ github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt2 github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.14.0 h1:RHRyE8UocrbjU+6UvRzwi6HjiDfxrrBU91TtbKzkGp4= github.com/prometheus/common v0.14.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= +github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= +github.com/prometheus/common v0.18.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= @@ -975,6 +1065,8 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.1.3 h1:F0+tqvhOksq22sc6iCHF5WGlWjdwj92p0udFh1VFBS8= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.2.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/tsdb v0.6.2-0.20190402121629-4f204dcbc150/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/psiemens/graceland v1.0.0/go.mod h1:1Tof+vt1LbmcZFE0lzgdwMN0QBymAChG3FRgDx8XisU= @@ -1110,6 +1202,9 @@ github.com/vmihailenco/msgpack/v4 v4.3.11 h1:Q47CePddpNGNhk4GCnAx9DDtASi2rasatE0 github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= github.com/vmihailenco/tagparser v0.1.1 h1:quXMXlA39OCbd2wAdTsGDlK9RkOk6Wuw+x37wVyIuWY= github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= +github.com/wangjia184/sortedset v0.0.0-20160527075905-f5d03557ba30/go.mod h1:YkocrP2K2tcw938x9gCOmT5G5eCD6jsTz0SZuyAqwIE= +github.com/warpfork/go-wish v0.0.0-20200122115046-b9ea61034e4a/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= +github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc/go.mod h1:bopw91TMyo8J3tvftk8xmU2kPmlrt4nScJQZU2hE5EM= github.com/whyrusleeping/go-logging v0.0.1/go.mod h1:lDPYj54zutzG1XYfHAhcc7oNXEburHQBn+Iqd4yS4vE= @@ -1129,6 +1224,7 @@ github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= @@ -1164,6 +1260,8 @@ go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM= go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= +go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4= +go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -1193,6 +1291,7 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= +golang.org/x/crypto v0.0.0-20210506145944-38f3c27a63bf/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a h1:kr2P4QFmQr29mSLA43kwrOcgcReGTfbE9N577tCTuBc= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1234,6 +1333,7 @@ golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180719180050-a680a1efc54d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1279,9 +1379,14 @@ golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210423184538-5f58ad60dda6 h1:0PC75Fz/kyMGhL0e1QnypqK2kQMqKt9csD1GnMJR+Zk= golang.org/x/net v0.0.0-20210423184538-5f58ad60dda6/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= +golang.org/x/net v0.0.0-20210428140749-89ef3d95e781 h1:DzZ89McO9/gWPsQXS/FVKAlG02ZjaQ6AlZRBimEYOd0= +golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -1300,6 +1405,7 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1367,10 +1473,19 @@ golang.org/x/sys v0.0.0-20200918174421-af09f7315aff/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201008064518-c1f3e3309c71/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201214210602-f9fddec55a1e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210223095934-7937bea0104d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210317225723-c4fcb01b228e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210426080607-c94f62235c83/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603125802-9665404d3644 h1:CA1DEQ4NdKphKeL70tvsWNdT5oFh1lOjihRcEDROi0I= golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= @@ -1448,8 +1563,10 @@ golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200828161849-5deb26317202/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= +golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a h1:CB3a9Nez8M13wwlr/E2YtwoU+qYHKfC+JrDa45RXXoQ= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1602,6 +1719,7 @@ gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJdjuHRquDANNeA4x7B8WQ9o= diff --git a/integration/go.mod b/integration/go.mod index 250d9933ffa..7d39fbbb1f4 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -15,7 +15,12 @@ require ( github.com/go-openapi/strfmt v0.20.1 // indirect github.com/go-test/deep v1.0.7 // indirect github.com/golang/snappy v0.0.3 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/jedib0t/go-pretty v4.3.0+incompatible + github.com/libp2p/go-libp2p v0.14.4 // indirect + github.com/libp2p/go-libp2p-kad-dht v0.13.0 // indirect + github.com/libp2p/go-libp2p-swarm v0.5.3 // indirect github.com/logrusorgru/aurora v2.0.3+incompatible // indirect github.com/onflow/cadence v0.18.1-0.20210729032058-d9eb6683d6ed github.com/onflow/flow-core-contracts/lib/go/contracts v0.7.5 @@ -26,7 +31,6 @@ require ( github.com/onflow/flow-go/crypto v0.18.0 // replaced by version on-disk github.com/onflow/flow/protobuf/go/flow v0.2.0 github.com/plus3it/gorecurcopy v0.0.1 - github.com/prometheus/client_golang v1.10.0 // indirect github.com/prometheus/common v0.20.0 // indirect github.com/rs/zerolog v1.21.0 github.com/stretchr/testify v1.7.0 @@ -34,11 +38,11 @@ require ( github.com/uber/jaeger-lib v2.4.1+incompatible // indirect github.com/vmihailenco/msgpack/v4 v4.3.12 // indirect github.com/vmihailenco/tagparser v0.1.2 // indirect + go.uber.org/zap v1.18.1 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20210405174219-a39eb2f71cb9 // indirect google.golang.org/grpc v1.36.1 gopkg.in/yaml.v2 v2.4.0 - gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect ) // temp fix for MacOS build. See comment https://github.com/ory/dockertest/issues/208#issuecomment-686820414 diff --git a/integration/go.sum b/integration/go.sum index 13b002194b3..5ae89d39a70 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -46,6 +46,7 @@ dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= +github.com/AndreasBriese/bbloom v0.0.0-20190825152654-46b345b51c96/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/Azure/azure-pipeline-go v0.2.1/go.mod h1:UGSo8XybXnIGZ3epmeBw7Jdz+HiUVpqIlpz/HKHylF4= github.com/Azure/azure-pipeline-go v0.2.2/go.mod h1:4rQ/NZncSvGqNkkOsNpOU1tgoNuIlp9AfUH5G1tvCHc= github.com/Azure/azure-storage-blob-go v0.7.0/go.mod h1:f9YQKtsG1nMisotuTPpO0tjNuEjKRYAcJU8/ydDI++4= @@ -113,8 +114,9 @@ github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN github.com/aws/aws-sdk-go v1.34.28/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/benbjohnson/clock v1.0.2/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= -github.com/benbjohnson/clock v1.0.3 h1:vkLuvpK4fmtSCuo60+yC63p7y0BmQ8gm5ZXGuBCJyXg= github.com/benbjohnson/clock v1.0.3/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= +github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= +github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -216,8 +218,9 @@ github.com/decred/dcrd/lru v1.0.0/go.mod h1:mxKOwFd7lFjN2GZYsiz/ecgqR6kkYAl+0pz0 github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= github.com/dgraph-io/badger v1.6.0-rc1/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/badger v1.6.1 h1:w9pSFNSdq/JPM1N12Fz/F/bzo993Is1W+Q7HjPzi7yg= github.com/dgraph-io/badger v1.6.1/go.mod h1:FRmFw3uxvcpa8zG3Rxs0th+hCLIuaQg8HlNV5bjgnuU= +github.com/dgraph-io/badger v1.6.2 h1:mNw0qs90GVgGGWylh0umH5iag1j6n/PeJtNvL6KY/x8= +github.com/dgraph-io/badger v1.6.2/go.mod h1:JW2yswe3V058sS0kZ2h/AXeDSqFjxnZcRrVH//y2UQE= github.com/dgraph-io/badger/v2 v2.0.3/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= github.com/dgraph-io/badger/v2 v2.2007.2 h1:EjjK0KqwaFMlPin1ajhP943VPENHJdEz1KLIegjaI3k= github.com/dgraph-io/badger/v2 v2.2007.2/go.mod h1:26P/7fbL4kUZVEVKLAKXkBXKOydDmM2p1e+NhhnBCAE= @@ -287,6 +290,7 @@ github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJn github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= +github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= @@ -330,6 +334,8 @@ github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 h1:p104kn46Q8WdvHunIJ9dAyjPVtrBPhSr3KT2yUst43I= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/go-test/deep v1.0.5/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= github.com/go-test/deep v1.0.7 h1:/VSMRlnY/JSyqxQUzQLKVMAskpY/NZKFA5j2P+0pP2M= github.com/go-test/deep v1.0.7/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= @@ -385,8 +391,9 @@ github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= -github.com/golang/mock v1.5.0 h1:jlYHihg//f7RRwuPfptm04yp4s7O6Kw8EZiVYIGcH0g= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= +github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= +github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -450,8 +457,9 @@ github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.5 h1:kxhtnfFVi+rYdOALN0B3k9UT86zVJKfBimRaciULW4I= github.com/google/uuid v1.1.5/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= +github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= @@ -492,8 +500,10 @@ github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtng github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-multierror v0.0.0-20161216184304-ed905158d874/go.mod h1:JMRHfdO9jKNzS/+BTlxCjKNQHg/jZAft8U7LloJvN7I= -github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-multierror v1.1.0/go.mod h1:spPvp8C1qA32ftKqdAHm4hHTbPw+vmowP0z+KUhOZdA= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= @@ -545,34 +555,45 @@ github.com/ipfs/go-cid v0.0.6/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqg github.com/ipfs/go-cid v0.0.7 h1:ysQJVJA3fNDF1qigJbsSQOdjhVLsOEoPdh0+R97k3jY= github.com/ipfs/go-cid v0.0.7/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= github.com/ipfs/go-datastore v0.0.1/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= +github.com/ipfs/go-datastore v0.1.0/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= +github.com/ipfs/go-datastore v0.1.1/go.mod h1:w38XXW9kVFNp57Zj5knbKWM2T+KOZCGDRVNdgPHtbHw= github.com/ipfs/go-datastore v0.4.0/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.1/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.4/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= +github.com/ipfs/go-datastore v0.4.5 h1:cwOUcGMLdLPWgu3SlrCckCMznaGADbPqE0r8h768/Dg= github.com/ipfs/go-datastore v0.4.5/go.mod h1:eXTcaaiN6uOlVCLS9GjJUJtlvJfM3xk23w3fyfrmmJs= github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= github.com/ipfs/go-ds-badger v0.0.2/go.mod h1:Y3QpeSFWQf6MopLTiZD+VT6IC1yZqaGmjvRcKeSGij8= github.com/ipfs/go-ds-badger v0.0.5/go.mod h1:g5AuuCGmr7efyzQhLL8MzwqcauPojGPUaHzfGTzuE3s= +github.com/ipfs/go-ds-badger v0.0.7/go.mod h1:qt0/fWzZDoPW6jpQeqUjR5kBfhDNB65jd9YlmAvpQBk= github.com/ipfs/go-ds-badger v0.2.1/go.mod h1:Tx7l3aTph3FMFrRS838dcSJh+jjA7cX9DrGVwx/NOwE= github.com/ipfs/go-ds-badger v0.2.3/go.mod h1:pEYw0rgg3FIrywKKnL+Snr+w/LjJZVMTBRn4FS6UHUk= +github.com/ipfs/go-ds-badger v0.2.7/go.mod h1:02rnztVKA4aZwDuaRPTf8mpqcKmXP7mLl6JPxd14JHA= github.com/ipfs/go-ds-leveldb v0.0.1/go.mod h1:feO8V3kubwsEF22n0YRQCffeb79OOYIykR4L04tMOYc= +github.com/ipfs/go-ds-leveldb v0.1.0/go.mod h1:hqAW8y4bwX5LWcCtku2rFNX3vjDZCy5LZCg+cSZvYb8= github.com/ipfs/go-ds-leveldb v0.4.1/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ds-leveldb v0.4.2/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-util v0.0.1/go.mod h1:spsl5z8KUnrve+73pOhSVZND1SIxPW5RyBCNzQxlJBc= github.com/ipfs/go-ipfs-util v0.0.2 h1:59Sswnk1MFaiq+VcaknX7aYEyGyGDAA73ilhEK2POp8= github.com/ipfs/go-ipfs-util v0.0.2/go.mod h1:CbPtkWJzjLdEcezDns2XYaehFVNXG9zrdrtMecczcsQ= +github.com/ipfs/go-ipns v0.1.2 h1:O/s/0ht+4Jl9+VoxoUo0zaHjnZUS+aBQIKTuzdZ/ucI= +github.com/ipfs/go-ipns v0.1.2/go.mod h1:ioQ0j02o6jdIVW+bmi18f4k2gRf0AV3kZ9KeHYHICnQ= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-log v1.0.2/go.mod h1:1MNjMxe0u6xvJZgeqbJ8vdo2TKaGwZ1a0Bpza+sr2Sk= github.com/ipfs/go-log v1.0.3/go.mod h1:OsLySYkwIbiSUR/yBTdv1qPtcE4FW3WPWk/ewz9Ru+A= -github.com/ipfs/go-log v1.0.4 h1:6nLQdX4W8P9yZZFH7mO+X/PzjN8Laozm/lMJ6esdgzY= github.com/ipfs/go-log v1.0.4/go.mod h1:oDCg2FkjogeFOhqqb+N39l2RpTNPL6F/StPkB3kPgcs= +github.com/ipfs/go-log v1.0.5 h1:2dOuUCB1Z7uoczMWgAyDck5JLb72zHzrMnGnCNNbvY8= +github.com/ipfs/go-log v1.0.5/go.mod h1:j0b8ZoR+7+R99LD9jZ6+AJsrzkPbSXbZfGakb5JPtIo= github.com/ipfs/go-log/v2 v2.0.2/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= github.com/ipfs/go-log/v2 v2.0.3/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= github.com/ipfs/go-log/v2 v2.0.5/go.mod h1:eZs4Xt4ZUJQFM3DlanGhy7TkwwawCZcSByscwkWG+dw= github.com/ipfs/go-log/v2 v2.1.1/go.mod h1:2v2nsGfZsvvAJz13SyFzf9ObaqwHiHxsPLEHntrv9KM= github.com/ipfs/go-log/v2 v2.1.3 h1:1iS3IU7aXRlbgUpN8yTTpJ53NXYjAe37vcI5+5nYrzk= github.com/ipfs/go-log/v2 v2.1.3/go.mod h1:/8d0SH3Su5Ooc31QlL1WysJhvyOTDCjcCZ9Axpmri6g= +github.com/ipld/go-ipld-prime v0.9.0 h1:N2OjJMb+fhyFPwPnVvJcWU/NsumP8etal+d2v3G4eww= +github.com/ipld/go-ipld-prime v0.9.0/go.mod h1:KvBLMr4PX1gWptgkzRjVZCrLmSGcZCb/jioOQwCqZN8= github.com/jackpal/gateway v1.0.5/go.mod h1:lTpwd4ACLXmpyiCTRtfiNyVnUmqT9RivzCDQetPfnjA= github.com/jackpal/go-nat-pmp v1.0.1/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= @@ -634,7 +655,12 @@ github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0 github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.11.7 h1:0hzRabrMN4tSTvMfnL3SCv1ZGeAP23ynzodBgaHeMeg= +github.com/klauspost/compress v1.11.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= +github.com/klauspost/cpuid/v2 v2.0.4 h1:g0I61F2K2DjRHz1cnxlkNSBIaePVoJIjjnHui8QHbiw= +github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPRLkC4NPOvfYeR5KNOrY6TD+/sAC3HXPZgDYg= github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -665,11 +691,14 @@ github.com/lib/pq v0.0.0-20170810061220-e42267488fe3/go.mod h1:5WUZQaWbwv1U+lTRe github.com/lib/pq v1.0.0 h1:X5PMW56eZitiTeO7tKzZxFCSpbFZJtkMMooicw2us9A= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/libp2p/go-addr-util v0.0.1/go.mod h1:4ac6O7n9rIAKB1dnd+s8IbbMXkt+oBpzX4/+RACcnlQ= -github.com/libp2p/go-addr-util v0.0.2 h1:7cWK5cdA5x72jX0g8iLrQWm5TRJZ6CzGdPEhWj7plWU= github.com/libp2p/go-addr-util v0.0.2/go.mod h1:Ecd6Fb3yIuLzq4bD7VcywcVSBtefcAwnUISBM3WG15E= +github.com/libp2p/go-addr-util v0.1.0 h1:acKsntI33w2bTU7tC9a0SaPimJGfSI0bFKC18ChxeVI= +github.com/libp2p/go-addr-util v0.1.0/go.mod h1:6I3ZYuFr2O/9D+SoyM0zEw0EF3YkldtTX406BpdQMqw= github.com/libp2p/go-buffer-pool v0.0.1/go.mod h1:xtyIz9PMobb13WaxR6Zo1Pd1zXJKYg0a8KiIvDp3TzQ= github.com/libp2p/go-buffer-pool v0.0.2 h1:QNK2iAFa8gjAe1SPz6mHSMuCcjs+X1wlHzeOSqcmlfs= github.com/libp2p/go-buffer-pool v0.0.2/go.mod h1:MvaB6xw5vOrDl8rYZGLFdKAuk/hRoRZd1Vi32+RXyFM= +github.com/libp2p/go-cidranger v1.1.0 h1:ewPN8EZ0dd1LSnrtuwd4709PXVcITVeuwbag38yPW7c= +github.com/libp2p/go-cidranger v1.1.0/go.mod h1:KWZTfSr+r9qEo9OkI9/SIEeAtw+NNoU0dXIXt15Okic= github.com/libp2p/go-conn-security-multistream v0.1.0/go.mod h1:aw6eD7LOsHEX7+2hJkDxw1MteijaVcI+/eP2/x3J1xc= github.com/libp2p/go-conn-security-multistream v0.2.0/go.mod h1:hZN4MjlNetKD3Rq5Jb/P5ohUnFLNzEAR4DLSzpn2QLU= github.com/libp2p/go-conn-security-multistream v0.2.1 h1:ft6/POSK7F+vl/2qzegnHDaXFU0iWB4yVTYrioC6Zy0= @@ -678,14 +707,18 @@ github.com/libp2p/go-eventbus v0.1.0/go.mod h1:vROgu5cs5T7cv7POWlWxBaVLxfSegC5UG github.com/libp2p/go-eventbus v0.2.1 h1:VanAdErQnpTioN2TowqNcOijf6YwhuODe4pPKSDpxGc= github.com/libp2p/go-eventbus v0.2.1/go.mod h1:jc2S4SoEVPP48H9Wpzm5aiGwUCBMfGhVhhBjyhhCJs8= github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZxBdp967ls1g+k8= +github.com/libp2p/go-flow-metrics v0.0.2/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-flow-metrics v0.0.3 h1:8tAs/hSdNvUiLgtlSy3mxwxWP4I9y/jlkPFT7epKdeM= github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-libp2p v0.6.1/go.mod h1:CTFnWXogryAHjXAKEbOf1OWY+VeAP3lDMZkfEI5sT54= github.com/libp2p/go-libp2p v0.7.0/go.mod h1:hZJf8txWeCduQRDC/WSqBGMxaTHCOYHt2xSU1ivxn0k= github.com/libp2p/go-libp2p v0.7.4/go.mod h1:oXsBlTLF1q7pxr+9w6lqzS1ILpyHsaBPniVO7zIHGMw= github.com/libp2p/go-libp2p v0.8.1/go.mod h1:QRNH9pwdbEBpx5DTJYg+qxcVaDMAz3Ee/qDKwXujH5o= -github.com/libp2p/go-libp2p v0.14.1 h1:R0vNY7nkU8IISlDuHd2yk4eNAZsVQ0rCr2bPfWU3sXo= github.com/libp2p/go-libp2p v0.14.1/go.mod h1:0PQMADQEjCM2l8cSMYDpTgsb8gr6Zq7i4LUgq1mlW2E= +github.com/libp2p/go-libp2p v0.14.4 h1:QCJE+jGyqxWdrSPuS4jByXCzosgaIg4SJTLCRplJ53w= +github.com/libp2p/go-libp2p v0.14.4/go.mod h1:EIRU0Of4J5S8rkockZM7eJp2S0UrCyi55m2kJVru3rM= +github.com/libp2p/go-libp2p-asn-util v0.0.0-20200825225859-85005c6cf052 h1:BM7aaOF7RpmNn9+9g6uTjGJ0cTzWr5j9i9IKeun2M8U= +github.com/libp2p/go-libp2p-asn-util v0.0.0-20200825225859-85005c6cf052/go.mod h1:nRMRTab+kZuk0LnKZpxhOVH/ndsdr2Nr//Zltc/vwgo= github.com/libp2p/go-libp2p-autonat v0.1.1/go.mod h1:OXqkeGOY2xJVWKAGV2inNF5aKN/djNA3fdpCWloIudE= github.com/libp2p/go-libp2p-autonat v0.2.0/go.mod h1:DX+9teU4pEEoZUqR1PiMlqliONQdNbfzE1C718tcViI= github.com/libp2p/go-libp2p-autonat v0.2.1/go.mod h1:MWtAhV5Ko1l6QBsHQNSuM6b1sRkXrpk0/LqCr+vCVxI= @@ -707,27 +740,36 @@ github.com/libp2p/go-libp2p-core v0.0.4/go.mod h1:jyuCQP356gzfCFtRKyvAbNkyeuxb7O github.com/libp2p/go-libp2p-core v0.2.0/go.mod h1:X0eyB0Gy93v0DZtSYbEM7RnMChm9Uv3j7yRXjO77xSI= github.com/libp2p/go-libp2p-core v0.2.2/go.mod h1:8fcwTbsG2B+lTgRJ1ICZtiM5GWCWZVoVrLaDRvIRng0= github.com/libp2p/go-libp2p-core v0.2.4/go.mod h1:STh4fdfa5vDYr0/SzYYeqnt+E6KfEV5VxfIrm0bcI0g= +github.com/libp2p/go-libp2p-core v0.2.5/go.mod h1:6+5zJmKhsf7yHn1RbmYDu08qDUpIUxGdqHuEZckmZOA= github.com/libp2p/go-libp2p-core v0.3.0/go.mod h1:ACp3DmS3/N64c2jDzcV429ukDpicbL6+TrrxANBjPGw= github.com/libp2p/go-libp2p-core v0.3.1/go.mod h1:thvWy0hvaSBhnVBaW37BvzgVV68OUhgJJLAa6almrII= github.com/libp2p/go-libp2p-core v0.4.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= github.com/libp2p/go-libp2p-core v0.5.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= github.com/libp2p/go-libp2p-core v0.5.1/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= +github.com/libp2p/go-libp2p-core v0.5.3/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= github.com/libp2p/go-libp2p-core v0.5.4/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= github.com/libp2p/go-libp2p-core v0.5.5/go.mod h1:vj3awlOr9+GMZJFH9s4mpt9RHHgGqeHCopzbYKZdRjM= github.com/libp2p/go-libp2p-core v0.5.6/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.5.7/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.6.0/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= +github.com/libp2p/go-libp2p-core v0.6.1/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.7.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.1/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.2/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= -github.com/libp2p/go-libp2p-core v0.8.5 h1:aEgbIcPGsKy6zYcC+5AJivYFedhYa4sW7mIpWpUaLKw= github.com/libp2p/go-libp2p-core v0.8.5/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= +github.com/libp2p/go-libp2p-core v0.8.6 h1:3S8g006qG6Tjpj1JdRK2S+TWc2DJQKX/RG9fdLeiLSU= +github.com/libp2p/go-libp2p-core v0.8.6/go.mod h1:dgHr0l0hIKfWpGpqAMbpo19pen9wJfdCGv51mTmdpmM= github.com/libp2p/go-libp2p-crypto v0.1.0/go.mod h1:sPUokVISZiy+nNuTTH/TY+leRSxnFj/2GLjtOTW90hI= github.com/libp2p/go-libp2p-discovery v0.2.0/go.mod h1:s4VGaxYMbw4+4+tsoQTqh7wfxg97AEdo4GYBt6BadWg= github.com/libp2p/go-libp2p-discovery v0.3.0/go.mod h1:o03drFnz9BVAZdzC/QUQ+NeQOu38Fu7LJGEOK2gQltw= github.com/libp2p/go-libp2p-discovery v0.5.0 h1:Qfl+e5+lfDgwdrXdu4YNCWyEo3fWuP+WgN9mN0iWviQ= github.com/libp2p/go-libp2p-discovery v0.5.0/go.mod h1:+srtPIU9gDaBNu//UHvcdliKBIcr4SfDcm0/PfPJLug= +github.com/libp2p/go-libp2p-kad-dht v0.13.0 h1:qBNYzee8BVS6RkD8ukIAGRG6LmVz8+kkeponyI7W+yA= +github.com/libp2p/go-libp2p-kad-dht v0.13.0/go.mod h1:NkGf28RNhPrcsGYWJHm6EH8ULkiJ2qxsWmpE7VTL3LI= +github.com/libp2p/go-libp2p-kbucket v0.3.1/go.mod h1:oyjT5O7tS9CQurok++ERgc46YLwEpuGoFq9ubvoUOio= +github.com/libp2p/go-libp2p-kbucket v0.4.7 h1:spZAcgxifvFZHBD8tErvppbnNiKA5uokDu3CV7axu70= +github.com/libp2p/go-libp2p-kbucket v0.4.7/go.mod h1:XyVo99AfQH0foSf176k4jY1xUJ2+jUJIZCSDm7r2YKk= github.com/libp2p/go-libp2p-loggables v0.1.0/go.mod h1:EyumB2Y6PrYjr55Q3/tiJ/o3xoDasoRYM7nOzEpoa90= github.com/libp2p/go-libp2p-mplex v0.2.0/go.mod h1:Ejl9IyjvXJ0T9iqUTE1jpYATQ9NM3g+OtR+EMMODbKo= github.com/libp2p/go-libp2p-mplex v0.2.1/go.mod h1:SC99Rxs8Vuzrf/6WhmH41kNn13TiYdAWNYHrwImKLnE= @@ -746,20 +788,27 @@ github.com/libp2p/go-libp2p-noise v0.2.0/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhU github.com/libp2p/go-libp2p-peer v0.2.0/go.mod h1:RCffaCvUyW2CJmG2gAWVqwePwW7JMgxjsHm7+J5kjWY= github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= github.com/libp2p/go-libp2p-peerstore v0.1.3/go.mod h1:BJ9sHlm59/80oSkpWgr1MyY1ciXAXV397W6h1GH/uKI= +github.com/libp2p/go-libp2p-peerstore v0.1.4/go.mod h1:+4BDbDiiKf4PzpANZDAT+knVdLxvqh7hXOujessqdzs= github.com/libp2p/go-libp2p-peerstore v0.2.0/go.mod h1:N2l3eVIeAitSg3Pi2ipSrJYnqhVnMNQZo9nkSCuAbnQ= github.com/libp2p/go-libp2p-peerstore v0.2.1/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= -github.com/libp2p/go-libp2p-peerstore v0.2.7 h1:83JoLxyR9OYTnNfB5vvFqvMUv/xDNa6NoPHnENhBsGw= github.com/libp2p/go-libp2p-peerstore v0.2.7/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= +github.com/libp2p/go-libp2p-peerstore v0.2.8 h1:nJghUlUkFVvyk7ccsM67oFA6kqUkwyCM1G4WPVMCWYA= +github.com/libp2p/go-libp2p-peerstore v0.2.8/go.mod h1:gGiPlXdz7mIHd2vfAsHzBNAMqSDkt2UBFwgcITgw1lA= github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6na5f0/k= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= github.com/libp2p/go-libp2p-pubsub v0.4.1 h1:j4umIg5nyus+sqNfU+FWvb9aeYFQH/A+nDFhWj+8yy8= github.com/libp2p/go-libp2p-pubsub v0.4.1/go.mod h1:izkeMLvz6Ht8yAISXjx60XUQZMq9ZMe5h2ih4dLIBIQ= -github.com/libp2p/go-libp2p-quic-transport v0.10.0 h1:koDCbWD9CCHwcHZL3/WEvP2A+e/o5/W5L3QS/2SPMA0= github.com/libp2p/go-libp2p-quic-transport v0.10.0/go.mod h1:RfJbZ8IqXIhxBRm5hqUEJqjiiY8xmEuq3HUDS993MkA= +github.com/libp2p/go-libp2p-quic-transport v0.11.2 h1:p1YQDZRHH4Cv2LPtHubqlQ9ggz4CKng/REZuXZbZMhM= +github.com/libp2p/go-libp2p-quic-transport v0.11.2/go.mod h1:wlanzKtIh6pHrq+0U3p3DY9PJfGqxMgPaGKaK5LifwQ= +github.com/libp2p/go-libp2p-record v0.1.2/go.mod h1:pal0eNcT5nqZaTV7UGhqeGqxFgGdsU/9W//C8dqjQDk= +github.com/libp2p/go-libp2p-record v0.1.3 h1:R27hoScIhQf/A8XJZ8lYpnqh9LatJ5YbHs28kCIfql0= +github.com/libp2p/go-libp2p-record v0.1.3/go.mod h1:yNUff/adKIfPnYQXgp6FQmNu3gLJ6EMg7+/vv2+9pY4= +github.com/libp2p/go-libp2p-routing-helpers v0.2.3/go.mod h1:795bh+9YeoFl99rMASoiVgHdi5bjack0N1+AFAdbvBw= github.com/libp2p/go-libp2p-secio v0.1.0/go.mod h1:tMJo2w7h3+wN4pgU2LSYeiKPrfqBgkOsdiKK77hE7c8= github.com/libp2p/go-libp2p-secio v0.2.0/go.mod h1:2JdZepB8J5V9mBp79BmwsaPQhRPNN2NrnB2lKQcdy6g= github.com/libp2p/go-libp2p-secio v0.2.1/go.mod h1:cWtZpILJqkqrSkiYcDBh5lA3wbT2Q+hz3rJQq3iftD8= @@ -770,8 +819,9 @@ github.com/libp2p/go-libp2p-swarm v0.2.3/go.mod h1:P2VO/EpxRyDxtChXz/VPVXyTnszHv github.com/libp2p/go-libp2p-swarm v0.2.8/go.mod h1:JQKMGSth4SMqonruY0a8yjlPVIkb0mdNSwckW7OYziM= github.com/libp2p/go-libp2p-swarm v0.3.0/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= github.com/libp2p/go-libp2p-swarm v0.3.1/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= -github.com/libp2p/go-libp2p-swarm v0.5.0 h1:HIK0z3Eqoo8ugmN8YqWAhD2RORgR+3iNXYG4U2PFd1E= github.com/libp2p/go-libp2p-swarm v0.5.0/go.mod h1:sU9i6BoHE0Ve5SKz3y9WfKrh8dUat6JknzUehFx8xW4= +github.com/libp2p/go-libp2p-swarm v0.5.3 h1:hsYaD/y6+kZff1o1Mc56NcuwSg80lIphTS/zDk3mO4M= +github.com/libp2p/go-libp2p-swarm v0.5.3/go.mod h1:NBn7eNW2lu568L7Ns9wdFrOhgRlkRnIDg0FLKbuu3i8= github.com/libp2p/go-libp2p-testing v0.0.2/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.3/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.4/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= @@ -779,15 +829,18 @@ github.com/libp2p/go-libp2p-testing v0.1.0/go.mod h1:xaZWMJrPUM5GlDBxCeGUi7kI4eq github.com/libp2p/go-libp2p-testing v0.1.1/go.mod h1:xaZWMJrPUM5GlDBxCeGUi7kI4eqnjVyavGroI2nxEM0= github.com/libp2p/go-libp2p-testing v0.1.2-0.20200422005655-8775583591d8/go.mod h1:Qy8sAncLKpwXtS2dSnDOP8ktexIAHKu+J+pnZOFZLTc= github.com/libp2p/go-libp2p-testing v0.3.0/go.mod h1:efZkql4UZ7OVsEfaxNHZPzIehtsBXMrXnCfJIgDti5g= -github.com/libp2p/go-libp2p-testing v0.4.0 h1:PrwHRi0IGqOwVQWR3xzgigSlhlLfxgfXgkHxr77EghQ= github.com/libp2p/go-libp2p-testing v0.4.0/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= +github.com/libp2p/go-libp2p-testing v0.4.2 h1:IOiA5mMigi+eEjf4J+B7fepDhsjtsoWA9QbsCqbNp5U= +github.com/libp2p/go-libp2p-testing v0.4.2/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= github.com/libp2p/go-libp2p-tls v0.1.3 h1:twKMhMu44jQO+HgQK9X8NHO5HkeJu2QbhLzLJpa8oNM= github.com/libp2p/go-libp2p-tls v0.1.3/go.mod h1:wZfuewxOndz5RTnCAxFliGjvYSDA40sKitV4c50uI1M= github.com/libp2p/go-libp2p-transport-upgrader v0.1.1/go.mod h1:IEtA6or8JUbsV07qPW4r01GnTenLW4oi3lOPbUMGJJA= github.com/libp2p/go-libp2p-transport-upgrader v0.2.0/go.mod h1:mQcrHj4asu6ArfSoMuyojOdjx73Q47cYD7s5+gZOlns= github.com/libp2p/go-libp2p-transport-upgrader v0.3.0/go.mod h1:i+SKzbRnvXdVbU3D1dwydnTmKRPXiAR/fyvi1dXuL4o= -github.com/libp2p/go-libp2p-transport-upgrader v0.4.2 h1:4JsnbfJzgZeRS9AWN7B9dPqn/LY/HoQTlO9gtdJTIYM= github.com/libp2p/go-libp2p-transport-upgrader v0.4.2/go.mod h1:NR8ne1VwfreD5VIWIU62Agt/J18ekORFU/j1i2y8zvk= +github.com/libp2p/go-libp2p-transport-upgrader v0.4.6 h1:SHt3g0FslnqIkEWF25YOB8UCOCTpGAVvHRWQYJ+veiI= +github.com/libp2p/go-libp2p-transport-upgrader v0.4.6/go.mod h1:JE0WQuQdy+uLZ5zOaI3Nw9dWGYJIA7mywEtP2lMvnyk= +github.com/libp2p/go-libp2p-xor v0.0.0-20210714161855-5c005aca55db/go.mod h1:LSTM5yRnjGZbWNTA/hRwq2gGFrvRIbQJscoIL/u6InY= github.com/libp2p/go-libp2p-yamux v0.2.0/go.mod h1:Db2gU+XfLpm6E4rG5uGCFX6uXA8MEXOxFcRoXUODaK8= github.com/libp2p/go-libp2p-yamux v0.2.2/go.mod h1:lIohaR0pT6mOt0AZ0L2dFze9hds9Req3OfS+B+dv4qw= github.com/libp2p/go-libp2p-yamux v0.2.5/go.mod h1:Zpgj6arbyQrmZ3wxSZxfBmbdnWtbZ48OpsfmQVTErwA= @@ -817,6 +870,7 @@ github.com/libp2p/go-nat v0.0.5 h1:qxnwkco8RLKqVh1NmjQ+tJ8p8khNLFxuElYG/TwqW4Q= github.com/libp2p/go-nat v0.0.5/go.mod h1:B7NxsVNPZmRLvMOwiEO1scOSyjA56zxYAGv1yQgRkEU= github.com/libp2p/go-netroute v0.1.2/go.mod h1:jZLDV+1PE8y5XxBySEBgbuVAXbhtuHSdmLPL2n9MKbk= github.com/libp2p/go-netroute v0.1.3/go.mod h1:jZLDV+1PE8y5XxBySEBgbuVAXbhtuHSdmLPL2n9MKbk= +github.com/libp2p/go-netroute v0.1.5/go.mod h1:V1SR3AaECRkEQCoFFzYwVYWvYIEtlxx89+O3qcpCl4A= github.com/libp2p/go-netroute v0.1.6 h1:ruPJStbYyXVYGQ81uzEDzuvbYRLKRrLvTYd33yomC38= github.com/libp2p/go-netroute v0.1.6/go.mod h1:AqhkMh0VuWmfgtxKPp3Oc1LdU5QSWS7wl0QLhSZqXxQ= github.com/libp2p/go-openssl v0.0.2/go.mod h1:v8Zw2ijCSWBQi8Pq5GAixw6DbFfa9u6VIYDXnvOXkc0= @@ -830,9 +884,11 @@ github.com/libp2p/go-reuseport v0.0.2 h1:XSG94b1FJfGA01BUrT82imejHQyTxO4jEWqheyC github.com/libp2p/go-reuseport v0.0.2/go.mod h1:SPD+5RwGC7rcnzngoYC86GjPzjSywuQyMVAheVBD9nQ= github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA20obEtsW9BLkuOUyQAOCbs= github.com/libp2p/go-reuseport-transport v0.0.3/go.mod h1:Spv+MPft1exxARzP2Sruj2Wb5JSyHNncjf1Oi2dEbzM= -github.com/libp2p/go-reuseport-transport v0.0.4 h1:OZGz0RB620QDGpv300n1zaOcKGGAoGVf8h9txtt/1uM= github.com/libp2p/go-reuseport-transport v0.0.4/go.mod h1:trPa7r/7TJK/d+0hdBLOCGvpQQVOU74OXbNCIMkufGw= +github.com/libp2p/go-reuseport-transport v0.0.5 h1:lJzi+vSYbyJj2faPKLxNGWEIBcaV/uJmyvsUxXy2mLw= +github.com/libp2p/go-reuseport-transport v0.0.5/go.mod h1:TC62hhPc8qs5c/RoXDZG6YmjK+/YWUPC0yYmeUecbjc= github.com/libp2p/go-sockaddr v0.0.2/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= +github.com/libp2p/go-sockaddr v0.1.0/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-sockaddr v0.1.1 h1:yD80l2ZOdGksnOyHrhxDdTDFrf7Oy+v3FMVArIRgZxQ= github.com/libp2p/go-sockaddr v0.1.1/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-stream-muxer v0.0.1/go.mod h1:bAo8x7YkSpadMTbtTaxGVHWUQsR/l5MEaHbKaliuT14= @@ -842,8 +898,10 @@ github.com/libp2p/go-stream-muxer-multistream v0.3.0/go.mod h1:yDh8abSIzmZtqtOt6 github.com/libp2p/go-tcp-transport v0.1.0/go.mod h1:oJ8I5VXryj493DEJ7OsBieu8fcg2nHGctwtInJVpipc= github.com/libp2p/go-tcp-transport v0.1.1/go.mod h1:3HzGvLbx6etZjnFlERyakbaYPdfjg2pWP97dFZworkY= github.com/libp2p/go-tcp-transport v0.2.0/go.mod h1:vX2U0CnWimU4h0SGSEsg++AzvBcroCGYw28kh94oLe0= -github.com/libp2p/go-tcp-transport v0.2.1 h1:ExZiVQV+h+qL16fzCWtd1HSzPsqWottJ8KXwWaVi8Ns= github.com/libp2p/go-tcp-transport v0.2.1/go.mod h1:zskiJ70MEfWz2MKxvFB/Pv+tPIB1PpPUrHIWQ8aFw7M= +github.com/libp2p/go-tcp-transport v0.2.4/go.mod h1:9dvr03yqrPyYGIEN6Dy5UvdJZjyPFvl1S/igQ5QD1SU= +github.com/libp2p/go-tcp-transport v0.2.7 h1:Z8Kc/Kb8tD84WiaH55xAlaEnkqzrp88jSEySCKV4+gg= +github.com/libp2p/go-tcp-transport v0.2.7/go.mod h1:lue9p1b3VmZj1MhhEGB/etmvF/nBQ0X9CW2DutBT3MM= github.com/libp2p/go-ws-transport v0.2.0/go.mod h1:9BHJz/4Q5A9ludYWKoGCFC5gUElzlHoKzu0yY9p/klM= github.com/libp2p/go-ws-transport v0.3.0/go.mod h1:bpgTJmRZAvVHrgHybCVyqoBmyLQ1fiZuEaBYusP5zsk= github.com/libp2p/go-ws-transport v0.4.0 h1:9tvtQ9xbws6cA5LvqdE6Ne3vcmGB4f1z9SByggk4s0k= @@ -863,8 +921,9 @@ github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0U github.com/logrusorgru/aurora v0.0.0-20200102142835-e9ef32dff381/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= github.com/logrusorgru/aurora v2.0.3+incompatible h1:tOpm7WcpBTn4fjmVfgpQq0EfczGlG91VSDkswnjF5A8= github.com/logrusorgru/aurora v2.0.3+incompatible/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= -github.com/lucas-clemente/quic-go v0.19.3 h1:eCDQqvGBB+kCTkA0XrAFtNe81FMa0/fn4QSoeAbmiF4= github.com/lucas-clemente/quic-go v0.19.3/go.mod h1:ADXpNbTQjq1hIzCpB+y/k5iz4n4z4IwqoLb94Kh5Hu8= +github.com/lucas-clemente/quic-go v0.21.2 h1:8LqqL7nBQFDUINadW0fHV/xSaCQJgmJC0Gv+qUnjd78= +github.com/lucas-clemente/quic-go v0.21.2/go.mod h1:vF5M1XqhBAHgbjKcJOXY3JZz3GP0T3FQhz/uyOUS38Q= github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= github.com/m4ksio/wal v1.0.0 h1:PucHOZPz58BgWowe+Gf+gZUbgEdd4zFx+He45SGkHG0= @@ -877,10 +936,17 @@ github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN github.com/markbates/oncer v0.0.0-20181203154359-bf2de49a0be2/go.mod h1:Ld9puTsIW75CHf65OeIOkyKbteujpZVXDpWK6YGZbxE= github.com/markbates/safe v1.0.1/go.mod h1:nAqgmRi7cY2nqMc92/bSEeQA+R4OheNU2T1kNSCBdG0= github.com/marten-seemann/qpack v0.2.1/go.mod h1:F7Gl5L1jIgN1D11ucXefiuJS9UMVP2opoCp2jDKb7wc= -github.com/marten-seemann/qtls v0.10.0 h1:ECsuYUKalRL240rRD4Ri33ISb7kAQ3qGDlrrl55b2pc= github.com/marten-seemann/qtls v0.10.0/go.mod h1:UvMd1oaYDACI99/oZUYLzMCkBXQVT0aGm99sJhbT8hs= -github.com/marten-seemann/qtls-go1-15 v0.1.1 h1:LIH6K34bPVttyXnUWixk0bzH6/N07VxbSabxn5A5gZQ= github.com/marten-seemann/qtls-go1-15 v0.1.1/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-15 v0.1.4/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-15 v0.1.5 h1:Ci4EIUN6Rlb+D6GmLdej/bCQ4nPYNtVXQB+xjiXE1nk= +github.com/marten-seemann/qtls-go1-15 v0.1.5/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-16 v0.1.4 h1:xbHbOGGhrenVtII6Co8akhLEdrawwB2iHl5yhJRpnco= +github.com/marten-seemann/qtls-go1-16 v0.1.4/go.mod h1:gNpI2Ol+lRS3WwSOtIUUtRwZEQMXjYK+dQSBFbethAk= +github.com/marten-seemann/qtls-go1-17 v0.1.0-rc.1 h1:/rpmWuGvceLwwWuaKPdjpR4JJEUH0tq64/I3hvzaNLM= +github.com/marten-seemann/qtls-go1-17 v0.1.0-rc.1/go.mod h1:fz4HIxByo+LlWcreM4CZOYNuz3taBQ8rN2X6FqvaWo8= +github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd h1:br0buuQ854V8u83wA0rVZ8ttrq5CpaPZdvrK0LP2lOk= +github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd/go.mod h1:QuCEs1Nt24+FYQEqAAncTDPJIuGs+LxK1MCiFL25pMU= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.0/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ= @@ -914,14 +980,21 @@ github.com/miekg/dns v1.1.12/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3N github.com/miekg/dns v1.1.28/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/miekg/dns v1.1.41 h1:WMszZWJG0XmzbK9FEmzH2TVcqYzFesusSIB41b8KHxY= github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c h1:bzE/A84HN25pxAuk9Eej1Kz9OUelF97nAc82bDquQI8= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c/go.mod h1:0SQS9kMwD2VsyFEB++InYyBJroV/FRmBgcydeSUcJms= +github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b h1:z78hV3sbSMAUoyUMM0I83AUIT6Hu17AWfgjzIbtrYFc= +github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b/go.mod h1:lxPUiZwKoFL8DUUmalo2yJJUCxbPKtm8OKfqr2/FTNU= +github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc h1:PTfri+PuQmWDqERdnNMiD9ZejrlswWrCpBEZgWOiTrc= +github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc/go.mod h1:cGKTAVKx4SxOuR/czcZ/E2RSJ3sfHs8FpHhQ5CWMf9s= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.0.0-20190328051042-05b4dd3047e5/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.1.0/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= -github.com/minio/sha256-simd v0.1.1 h1:5QHSlgo3nt5yKOJrC7W8w7X+NFl8cMPZm96iu8kKUJU= github.com/minio/sha256-simd v0.1.1/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= +github.com/minio/sha256-simd v1.0.0 h1:v1ta+49hkWZyvaKwrQB8elexRqm6Y0aMLjCNsrYxo6g= +github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db/go.mod h1:l0dey0ia/Uv7NcFFVbCLtqEBQbrT4OCwCSKTEv6enCw= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= @@ -960,8 +1033,9 @@ github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y9 github.com/multiformats/go-multiaddr v0.2.1/go.mod h1:s/Apk6IyxfvMjDafnhJgJ3/46z7tZ04iMk5wP4QMGGE= github.com/multiformats/go-multiaddr v0.2.2/go.mod h1:NtfXiOtHvghW9KojvtySjH5y0u0xW5UouOmQQrn6a3Y= github.com/multiformats/go-multiaddr v0.3.0/go.mod h1:dF9kph9wfJ+3VLAaeBqo9Of8x4fJxp6ggJGteB8HQTI= -github.com/multiformats/go-multiaddr v0.3.1 h1:1bxa+W7j9wZKTZREySx1vPMs2TqrYWjVZ7zE6/XLG1I= github.com/multiformats/go-multiaddr v0.3.1/go.mod h1:uPbspcUPd5AfaP6ql3ujFY+QWzmBD8uLLL4bXW0XfGc= +github.com/multiformats/go-multiaddr v0.3.3 h1:vo2OTSAqnENB2rLk79pLtr+uhj+VAzSe3uef5q0lRSs= +github.com/multiformats/go-multiaddr v0.3.3/go.mod h1:lCKNGP1EQ1eZ35Za2wlqnabm9xQkib3fyB+nZXHLag0= github.com/multiformats/go-multiaddr-dns v0.0.1/go.mod h1:9kWcqw/Pj6FwxAwW38n/9403szc57zJPs45fmnznu3Q= github.com/multiformats/go-multiaddr-dns v0.0.2/go.mod h1:9kWcqw/Pj6FwxAwW38n/9403szc57zJPs45fmnznu3Q= github.com/multiformats/go-multiaddr-dns v0.2.0/go.mod h1:TJ5pr5bBO7Y1B18djPuRsVkduhQH2YqYSbxWJzYGdK0= @@ -977,18 +1051,21 @@ github.com/multiformats/go-multiaddr-net v0.1.2/go.mod h1:QsWt3XK/3hwvNxZJp92iMQ github.com/multiformats/go-multiaddr-net v0.1.3/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= github.com/multiformats/go-multiaddr-net v0.1.4/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= github.com/multiformats/go-multiaddr-net v0.1.5/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= -github.com/multiformats/go-multiaddr-net v0.2.0 h1:MSXRGN0mFymt6B1yo/6BPnIRpLPEnKgQNvVfCX5VDJk= github.com/multiformats/go-multiaddr-net v0.2.0/go.mod h1:gGdH3UXny6U3cKKYCvpXI5rnK7YaOIEOPVDI9tsJbEA= github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= github.com/multiformats/go-multibase v0.0.3 h1:l/B6bJDQjvQ5G52jw4QGSYeOTZoAwIO77RblWplfIqk= github.com/multiformats/go-multibase v0.0.3/go.mod h1:5+1R4eQrT3PkYZ24C3W2Ue2tPwIdYQD509ZjSb5y9Oc= +github.com/multiformats/go-multicodec v0.2.0 h1:MUzKZWxOFagwLLtlx96pub9zwDQAbMAf1k9fXOdc3so= +github.com/multiformats/go-multicodec v0.2.0/go.mod h1:/y4YVwkfMyry5kFbMTbLJKErhycTIftytRV+llXdyS4= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multihash v0.0.5/go.mod h1:lt/HCbqlQwlPBz7lv0sQCdtfcMtlJvakRUn/0Ual8po= github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= +github.com/multiformats/go-multihash v0.0.9/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.10/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.13/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= -github.com/multiformats/go-multihash v0.0.14 h1:QoBceQYQQtNUuf6s7wHxnE2c8bhbMqhfGzNI032se/I= github.com/multiformats/go-multihash v0.0.14/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= +github.com/multiformats/go-multihash v0.0.15 h1:hWOPdrNqDjwHDx82vsYGSDZNyktOJJ2dzZJzFkOV1jM= +github.com/multiformats/go-multihash v0.0.15/go.mod h1:D6aZrWNLFTV/ynMpKsNtB40mJzmCl4jb1alC0OvHiHg= github.com/multiformats/go-multistream v0.1.0/go.mod h1:fJTiDfXJVmItycydCnNx4+wSzZ5NwG2FEVAI30fiovg= github.com/multiformats/go-multistream v0.1.1/go.mod h1:KmHZ40hzVxiaiwlj3MEbYgK9JFk2/9UktWZAF54Du38= github.com/multiformats/go-multistream v0.2.0/go.mod h1:5GZPQZbkWOLOn3J2y4Y99vVW7vOfsAflxARk3x14o6k= @@ -1014,8 +1091,9 @@ github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OS github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= github.com/oklog/ulid v1.3.1 h1:EGfNDEx6MqHz8B3uNV6QAib1UR2Lm97sHi3ocA6ESJ4= @@ -1051,16 +1129,19 @@ github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+W github.com/onsi/ginkgo v1.10.1/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.0/go.mod h1:oUhWkIvk5aDxtKvDDuw8gItl8pKl42LzjC9KZE0HfGg= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= +github.com/onsi/ginkgo v1.16.2/go.mod h1:CObGmKUOKaSC0RjmoAK7tKyn4Azo5P2IWuoMnvwxz1E= +github.com/onsi/ginkgo v1.16.4 h1:29JGrr5oVBm5ulCWet69zQkzWipVXIol6ygQUe/EzNc= +github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= github.com/onsi/gomega v1.4.1/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.9.0/go.mod h1:Ho0h+IUsWyvy1OpqCwxlQ/21gkhVunqlU8fDGcoTdcA= -github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.13.0 h1:7lLHu94wT9Ij0o6EWWclhu0aOh32VxhkwEJvzuWPeak= +github.com/onsi/gomega v1.13.0/go.mod h1:lRk9szgn8TxENtWd0Tp4c3wjlRfMTMH27I+3Je41yGY= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= github.com/opencontainers/go-digest v0.0.0-20180430190053-c9281466c8b2/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s= github.com/opencontainers/go-digest v1.0.0-rc1 h1:WzifXhOVOEOuFYOJAW6aQqW0TooG2iki3E3Ii+WN7gQ= @@ -1110,6 +1191,8 @@ github.com/plus3it/gorecurcopy v0.0.1 h1:H7AgvM0N/uIo7o1PQRlewEGQ92BNr7DqbPy5lnR github.com/plus3it/gorecurcopy v0.0.1/go.mod h1:NvVTm4RX68A1vQbHmHunDO4OtBLVroT6CrsiqAzNyJA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1 h1:CskT+S6Ay54OwxBGB0R3Rsx4Muto6UnEYTyKJbyRIAI= +github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= @@ -1119,6 +1202,7 @@ github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5Fsn github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.9.0/go.mod h1:FqZLKOZnGdFAhOK4nqGHa7D66IdsO+O441Eve7ptJDU= github.com/prometheus/client_golang v1.10.0 h1:/o0BDeWzLWXNZ+4q5gXltUvaMpJqckTa+jTNoB+z4cg= github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= @@ -1137,6 +1221,7 @@ github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+ github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.14.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= +github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/common v0.18.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/common v0.20.0 h1:pfeDeUdQcIxOMutNjCejsEFp7qeP+/iltHSSmLpE+hU= github.com/prometheus/common v0.20.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= @@ -1149,6 +1234,7 @@ github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDa github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.2.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/tsdb v0.6.2-0.20190402121629-4f204dcbc150/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= @@ -1307,6 +1393,9 @@ github.com/vmihailenco/msgpack/v4 v4.3.12/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+ github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= github.com/vmihailenco/tagparser v0.1.2 h1:gnjoVuB/kljJ5wICEEOpx98oXMWPLj22G67Vbd1qPqc= github.com/vmihailenco/tagparser v0.1.2/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= +github.com/wangjia184/sortedset v0.0.0-20160527075905-f5d03557ba30/go.mod h1:YkocrP2K2tcw938x9gCOmT5G5eCD6jsTz0SZuyAqwIE= +github.com/warpfork/go-wish v0.0.0-20200122115046-b9ea61034e4a/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= +github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc/go.mod h1:bopw91TMyo8J3tvftk8xmU2kPmlrt4nScJQZU2hE5EM= github.com/whyrusleeping/go-logging v0.0.1/go.mod h1:lDPYj54zutzG1XYfHAhcc7oNXEburHQBn+Iqd4yS4vE= @@ -1335,6 +1424,7 @@ github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= @@ -1371,8 +1461,9 @@ go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM= go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= +go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4= +go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -1406,6 +1497,7 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= +golang.org/x/crypto v0.0.0-20210506145944-38f3c27a63bf/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a h1:kr2P4QFmQr29mSLA43kwrOcgcReGTfbE9N577tCTuBc= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1448,8 +1540,9 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.1.1-0.20191209134235-331c550502dd/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2 h1:Gz96sIWK3OalVv/I/qNygP42zyoKp3xptRVCWRFEBvo= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180719180050-a680a1efc54d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1496,10 +1589,13 @@ golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= -golang.org/x/net v0.0.0-20210423184538-5f58ad60dda6 h1:0PC75Fz/kyMGhL0e1QnypqK2kQMqKt9csD1GnMJR+Zk= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210423184538-5f58ad60dda6/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= +golang.org/x/net v0.0.0-20210428140749-89ef3d95e781 h1:DzZ89McO9/gWPsQXS/FVKAlG02ZjaQ6AlZRBimEYOd0= +golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -1596,17 +1692,23 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201008064518-c1f3e3309c71/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201101102859-da207088b7d1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201214210602-f9fddec55a1e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210105210732-16f7687f5001/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210223095934-7937bea0104d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210317225723-c4fcb01b228e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210426080607-c94f62235c83 h1:kHSDPqCtsHZOg0nVylfTo20DDhE9gG4Y0jn7hKQ0QAM= golang.org/x/sys v0.0.0-20210426080607-c94f62235c83/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744 h1:yhBbb4IRs2HS9PPlAg6DMC6mUOKexJBNsLf4Z+6En1Q= +golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -1688,9 +1790,11 @@ golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200828161849-5deb26317202/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= +golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.0 h1:po9/4sTYwZU9lPhi1tOrb4hCv3qrhiQ77LZfGa2OjwY= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= +golang.org/x/tools v0.1.1 h1:wGiQel/hW0NnEkJUk8lbzkX2gFJU6PFxf1v5OlCfuOs= +golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1864,7 +1968,6 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4 h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= k8s.io/kubernetes v1.13.0/go.mod h1:ocZa8+6APFNC2tX1DZASIbocyYT5jHzqFVsY5aoB7Jk= pgregory.net/rapid v0.4.7 h1:MTNRktPuv5FNqOO151TM9mDTa+XHcX6ypYeISDVD14g= diff --git a/module/id/id_provider.go b/module/id/id_provider.go new file mode 100644 index 00000000000..430780b6dc8 --- /dev/null +++ b/module/id/id_provider.go @@ -0,0 +1,148 @@ +package identity + +import ( + "sync" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" +) + +type IdentifierProvider interface { + Identifiers() flow.IdentifierList +} + +// TODO: rename to ProtocolStateIDProvider +type ProtocolStateIdentifierProvider struct { + events.Noop + identities flow.IdentityList // TODO: actually we *can* just use identifiers after all + state protocol.State + mu sync.RWMutex + filter flow.IdentityFilter + peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + flowIDs map[peer.ID]flow.Identifier +} + +func WithFilter(filter flow.IdentityFilter) ProtocolStateIdentifierProviderOption { + return func(provider *ProtocolStateIdentifierProvider) { + provider.filter = filter + } +} + +type ProtocolStateIdentifierProviderOption func(*ProtocolStateIdentifierProvider) + +// TODO: this one also implements IDTranslator!!! +func NewProtocolStateIdentifierProvider( + state protocol.State, + eventDistributer *events.Distributor, + opts ...ProtocolStateIdentifierProviderOption, +) (*ProtocolStateIdentifierProvider, error) { + provider := &ProtocolStateIdentifierProvider{ + state: state, + filter: filter.Any, + } + + for _, opt := range opts { + opt(provider) + } + + head, err := state.Final().Head() + if err != nil { + return nil, err // TODO: format the error + } + + provider.update(head.ID()) + eventDistributer.AddConsumer(provider) + + return provider, nil +} + +func (p *ProtocolStateIdentifierProvider) EpochTransition(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) update(blockID flow.Identifier) { + identities, err := p.state.AtBlockID(blockID).Identities(p.filter) + if err != nil { + // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + } + + nIds := identities.Count() + + peerIDs := make(map[flow.Identifier]peer.ID, nIds) + flowIDs := make(map[peer.ID]flow.Identifier, nIds) + + for _, identity := range identities { + pid, err := p2p.IdentityToPeerID(identity) + if err != nil { + // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + } + + flowIDs[pid] = identity.NodeID + peerIDs[identity.NodeID] = pid + } + + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities + p.flowIDs = flowIDs + p.peerIDs = peerIDs +} + +func (p *ProtocolStateIdentifierProvider) Identifiers() flow.IdentifierList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.NodeIDs() +} + +func (p *ProtocolStateIdentifierProvider) GetPeerID(flowID flow.Identifier) (pid peer.ID, found bool) { + p.mu.RLock() + defer p.mu.RUnlock() + + pid, found = p.peerIDs[flowID] + return +} + +func (p *ProtocolStateIdentifierProvider) GetFlowID(peerID peer.ID) (fid flow.Identifier, found bool) { + p.mu.RLock() + defer p.mu.RUnlock() + + fid, found = p.flowIDs[peerID] + return +} + +type PeerstoreIdentifierProvider struct { + store peerstore.Peerstore + idTranslator p2p.IDTranslator +} + +func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator p2p.IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} +} + +func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { + var result flow.IdentifierList + + pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? + for _, pid := range pids { + flowID, success := p.idTranslator.GetFlowID(pid) + if success { + result = append(result, flowID) + } + } + + return result +} diff --git a/network/p2p/idTranslator.go b/network/p2p/idTranslator.go new file mode 100644 index 00000000000..f9f567f1fec --- /dev/null +++ b/network/p2p/idTranslator.go @@ -0,0 +1,33 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type IDTranslator interface { + GetPeerID(flow.Identifier) (peer.ID, bool) + GetFlowID(peer.ID) (flow.Identifier, bool) +} + +type UnstakedNetworkPeerIDProvider struct { + // basically, just do the conversion from flow ID to peer ID using whatever scheme we talked about. + // whether this be, with caching or not +} + +func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { + pk, err := PublicKey(id.NetworkPubKey) + if err != nil { + // TODO: format the error + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + // TODO: format the error + return + } + + return +} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index f1a1b3b51ef..fecc247945e 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -314,16 +314,22 @@ func (n *Node) RemovePeer(ctx context.Context, identity flow.Identity) error { } // CreateStream returns an existing stream connected to identity, if it exists or adds one to identity as a peer and creates a new stream with it. -func (n *Node) CreateStream(ctx context.Context, identity flow.Identity) (libp2pnet.Stream, error) { +func (n *Node) CreateStream(ctx context.Context, nodeID flow.Identifier) (libp2pnet.Stream, error) { // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) - stream, err := n.tryCreateNewStream(ctx, identity, maxConnectAttempt) + stream, err := n.tryCreateNewStream(ctx, nodeID, maxConnectAttempt) if err != nil { + n.host.Peerstore().Addrs() return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (node_id: %s, address: %s): %w", identity.NodeID.String(), identity.Address, err)) } return stream, nil } +func (n *Node) GetPeerIPPort(nodeID flow.Identifier) (string, string, error) { + // TODO: first get peer ID + x := n.host.Peerstore().Addrs() +} + // tryCreateNewStream makes at most maxAttempts to create a stream with the identity. // This was put in as a fix for #2416. PubSub and 1-1 communication compete with each other when trying to connect to // remote nodes and once in a while NewStream returns an error 'both yamux endpoints are clients' From 43971e19c50359523d7dc7d03bed2cc4a2b44d7f Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 14:04:49 -0700 Subject: [PATCH 007/291] Update idTranslator.go --- network/p2p/idTranslator.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/network/p2p/idTranslator.go b/network/p2p/idTranslator.go index f9f567f1fec..b8e36f16e35 100644 --- a/network/p2p/idTranslator.go +++ b/network/p2p/idTranslator.go @@ -16,6 +16,24 @@ type UnstakedNetworkPeerIDProvider struct { // whether this be, with caching or not } +func NewUnstakedNetworkPeerIDProvider() *UnstakedNetworkPeerIDProvider { + return &UnstakedNetworkPeerIDProvider{} +} + +func GetPeerID(flowID flow.Identifier) (peer.ID, bool) { + Flow + return +} + +func GetFlowID(peerID peer.ID) (flow.Identifier, bool) { + Flow + return +} + +func FlowIDToPeerID(flowID flow.Identifier) peer.ID { + +} + func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { pk, err := PublicKey(id.NetworkPubKey) if err != nil { From f12316961e41288df5d9ee6c5544c49c9fb68b9e Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Thu, 12 Aug 2021 18:18:46 -0700 Subject: [PATCH 008/291] testing subscription_filter --- network/p2p/subscriptionFilter_test.go | 142 +++++++++++++++++++++++++ 1 file changed, 142 insertions(+) create mode 100644 network/p2p/subscriptionFilter_test.go diff --git a/network/p2p/subscriptionFilter_test.go b/network/p2p/subscriptionFilter_test.go new file mode 100644 index 00000000000..e73c97eb329 --- /dev/null +++ b/network/p2p/subscriptionFilter_test.go @@ -0,0 +1,142 @@ +package p2p + +import ( + "context" + "fmt" + "testing" + "time" + + golog "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p" + "github.com/libp2p/go-libp2p-core/host" + "github.com/libp2p/go-libp2p-core/peer" + pubsub "github.com/libp2p/go-libp2p-pubsub" + pb "github.com/libp2p/go-libp2p-pubsub/pb" + "github.com/stretchr/testify/require" +) + +func TestBasicSubscriptionFilter(t *testing.T) { + golog.SetAllLoggers(golog.LevelDebug) + ctx := context.Background() + host1, err := libp2p.New(ctx) + require.NoError(t, err) + host2, err := libp2p.New(ctx) + require.NoError(t, err) + host3, err := libp2p.New(ctx) + require.NoError(t, err) + + require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host2))) + require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host3))) + + + topicname1 := "testtopic1" + topicname2 := "testtopic2" + + filter := &Filter{ + allowedIDs: make(map[peer.ID]struct{}), + topic: topicname2, + } + filter.allowedIDs[host1.ID()] = struct{}{} + filter.allowedIDs[host2.ID()] = struct{}{} + + + ps1, err := pubsub.NewGossipSub(ctx, host1, pubsub.WithSubscriptionFilter(filter)) + require.NoError(t, err) + ps2, err := pubsub.NewGossipSub(ctx, host2, pubsub.WithSubscriptionFilter(filter)) + require.NoError(t, err) + ps3, err := pubsub.NewGossipSub(ctx, host3) + require.NoError(t, err) + + + topic1, err := ps1.Join(topicname1) + require.NoError(t, err) + _, err = topic1.Subscribe() + require.NoError(t, err) + topic1, err = ps1.Join(topicname2) + require.NoError(t, err) + _, err = topic1.Subscribe() + require.NoError(t, err) + + topic2, err := ps2.Join(topicname1) + require.NoError(t, err) + _, err = topic2.Subscribe() + require.NoError(t, err) + topic2, err = ps2.Join(topicname2) + require.NoError(t, err) + subscriberHost2Topic2, err := topic2.Subscribe() + require.NoError(t, err) + + topic3, err := ps3.Join(topicname1) + require.NoError(t, err) + _, err = topic3.Subscribe() + require.NoError(t, err) + wrongTopic, err := ps3.Join(topicname2) + require.NoError(t, err) + _, err = wrongTopic.Subscribe() + require.NoError(t, err) + + time.Sleep(2 * time.Second) + + fmt.Printf("host1: %s, host2: %s, host3 :%s\n", host1.ID(), host2.ID(), host3.ID()) + //fmt.Print(host1.Peerstore().Peers()) + //fmt.Print(host2.Peerstore().Peers()) + fmt.Print("host 1 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps1.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps1.ListPeers(topicname2)) + + fmt.Print("host 2 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps2.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps2.ListPeers(topicname2)) + + fmt.Print("host 3 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps3.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps3.ListPeers(topicname2)) + + err = wrongTopic.Publish(ctx, []byte("hello")) + require.NoError(t, err) + + time.Sleep(2 * time.Second) + + msg, err := subscriberHost2Topic2.Next(ctx) + require.NoError(t, err) + fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) + fmt.Println(msg) + + fmt.Print("host 2 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps2.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps2.ListPeers(topicname2)) +} + + +var _ pubsub.SubscriptionFilter = (*Filter)(nil) +type Filter struct { + allowedIDs map[peer.ID]struct{} + topic string +} + +func (filter *Filter) CanSubscribe(topic string) bool { + return true +} + +func (filter *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { + if _, found := filter.allowedIDs[from]; !found { + var newopts []*pb.RPC_SubOpts + for _, opt := range opts { + if *opt.Topicid != filter.topic { + newopts = append(newopts, opt) + } else { + return nil, fmt.Errorf(">>>>>> message received on a topic on which peer %s should not publish", from.String()) + } + } + return newopts, nil + } + return opts, nil +} From 1e631fb3b2b4a40ab6f0beaac1564fd9e59edecd Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Thu, 12 Aug 2021 18:27:21 -0700 Subject: [PATCH 009/291] wip --- network/p2p/subscriptionFilter_test.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/network/p2p/subscriptionFilter_test.go b/network/p2p/subscriptionFilter_test.go index e73c97eb329..0fa1bc83a59 100644 --- a/network/p2p/subscriptionFilter_test.go +++ b/network/p2p/subscriptionFilter_test.go @@ -54,7 +54,7 @@ func TestBasicSubscriptionFilter(t *testing.T) { require.NoError(t, err) topic1, err = ps1.Join(topicname2) require.NoError(t, err) - _, err = topic1.Subscribe() + subscriberHost1Topic2, err := topic1.Subscribe() require.NoError(t, err) topic2, err := ps2.Join(topicname1) @@ -108,6 +108,11 @@ func TestBasicSubscriptionFilter(t *testing.T) { fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) fmt.Println(msg) + msg, err = subscriberHost1Topic2.Next(ctx) + require.NoError(t, err) + fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) + fmt.Println(msg) + fmt.Print("host 2 peers\n") fmt.Printf("\t For %s", topicname1) fmt.Println(ps2.ListPeers(topicname1)) From 1e3f1608367e3a8c4f41df99023a3790b96ce385 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 18:54:13 -0700 Subject: [PATCH 010/291] stuff --- cmd/node_builder.go | 34 +++--- cmd/scaffold.go | 10 +- engine/common/synchronization/engine.go | 64 +++++------ go.mod | 8 +- go.sum | 108 ++++++------------ module/id/filtered_provider.go | 18 +++ module/id/id_provider.go | 141 +----------------------- network/middleware.go | 2 - network/p2p/idTranslator.go | 51 --------- network/p2p/id_translator.go | 12 ++ network/p2p/libp2pNode.go | 1 - network/p2p/network.go | 50 +++------ network/p2p/nodeIDRefresher.go | 82 -------------- network/p2p/peerstore_provider.go | 32 ++++++ network/p2p/protocol_state_provider.go | 129 ++++++++++++++++++++++ network/p2p/unstaked_translator.go | 43 ++++++++ 16 files changed, 336 insertions(+), 449 deletions(-) create mode 100644 module/id/filtered_provider.go delete mode 100644 network/p2p/idTranslator.go create mode 100644 network/p2p/id_translator.go delete mode 100644 network/p2p/nodeIDRefresher.go create mode 100644 network/p2p/peerstore_provider.go create mode 100644 network/p2p/protocol_state_provider.go create mode 100644 network/p2p/unstaked_translator.go diff --git a/cmd/node_builder.go b/cmd/node_builder.go index aecb9934ceb..bb70774390d 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" @@ -114,22 +115,23 @@ type BaseConfig struct { type NodeConfig struct { Cancel context.CancelFunc // cancel function for the context that is passed to the networking layer BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware *p2p.Middleware - Network *p2p.Network - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware *p2p.Middleware + Network *p2p.Network + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey + IdentifierProvider id.IdentityProvider // TODO: initialize these in scaffold and unstaked node // root state information RootBlock *flow.Block diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 8049b9e539e..a96152ba625 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -175,11 +175,6 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { true, fnb.MsgValidators...) - participants, err := fnb.State.Final().Identities(p2p.NetworkingSetFilter) - if err != nil { - return nil, fmt.Errorf("could not get network identities: %w", err) - } - subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) top, err := topology.NewTopicBasedTopology(fnb.NodeID, fnb.Logger, fnb.State) if err != nil { @@ -190,7 +185,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { // creates network instance net, err := p2p.NewNetwork(fnb.Logger, codec, - participants, + fnb.IdentifierProvider, fnb.Me, fnb.Middleware, p2p.DefaultCacheSize, @@ -203,8 +198,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Network = net - idRefresher := p2p.NewNodeIDRefresher(fnb.Logger, fnb.State, net.SetIDs) - idEvents := gadgets.NewIdentityDeltas(idRefresher.OnIdentityTableChanged) + idEvents := gadgets.NewIdentityDeltas(net.RefreshConnectionRules) fnb.ProtocolEvents.AddConsumer(idEvents) return net, err diff --git a/engine/common/synchronization/engine.go b/engine/common/synchronization/engine.go index d067cc714f9..1a348b44bc5 100644 --- a/engine/common/synchronization/engine.go +++ b/engine/common/synchronization/engine.go @@ -15,14 +15,13 @@ import ( "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/events" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module" + identifier "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" synccore "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" ) @@ -43,11 +42,11 @@ type Engine struct { blocks storage.Blocks comp network.Engine // compliance layer engine - pollInterval time.Duration - scanInterval time.Duration - core module.SyncCore - state protocol.State - finalizedHeader *FinalizedHeaderCache + pollInterval time.Duration + scanInterval time.Duration + core module.SyncCore + participantsProvider identifier.IdentifierProvider + finalizedHeader *FinalizedHeaderCache requestHandler *RequestHandlerEngine // component responsible for handling requests @@ -66,7 +65,7 @@ func New( comp network.Engine, core module.SyncCore, finalizedHeader *FinalizedHeaderCache, - state protocol.State, + participantsProvider identifier.IdentifierProvider, opts ...OptionFunc, ) (*Engine, error) { @@ -81,18 +80,18 @@ func New( // initialize the propagation engine with its dependencies e := &Engine{ - unit: engine.NewUnit(), - lm: lifecycle.NewLifecycleManager(), - log: log.With().Str("engine", "synchronization").Logger(), - metrics: metrics, - me: me, - blocks: blocks, - comp: comp, - core: core, - pollInterval: opt.pollInterval, - scanInterval: opt.scanInterval, - finalizedHeader: finalizedHeader, - state: state, + unit: engine.NewUnit(), + lm: lifecycle.NewLifecycleManager(), + log: log.With().Str("engine", "synchronization").Logger(), + metrics: metrics, + me: me, + blocks: blocks, + comp: comp, + core: core, + pollInterval: opt.pollInterval, + scanInterval: opt.scanInterval, + finalizedHeader: finalizedHeader, + participantsProvider: participantsProvider, } err := e.setupResponseMessageHandler() @@ -332,7 +331,7 @@ CheckLoop: e.pollHeight() case <-scan.C: head := e.finalizedHeader.Get() - participants := e.getParticipants(head.ID()) + participants := e.participantsProvider.Identifiers() ranges, batches := e.core.ScanPending(head) e.sendRequests(participants, ranges, batches) } @@ -342,30 +341,17 @@ CheckLoop: scan.Stop() } -// getParticipants gets all of the consensus nodes from the state at the given block ID. -func (e *Engine) getParticipants(blockID flow.Identifier) flow.IdentityList { - participants, err := e.state.AtBlockID(blockID).Identities(filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(e.me.NodeID())), - )) - if err != nil { - e.log.Fatal().Err(err).Msgf("could not get consensus participants at block ID %v", blockID) - } - - return participants -} - // pollHeight will send a synchronization request to three random nodes. func (e *Engine) pollHeight() { head := e.finalizedHeader.Get() - participants := e.getParticipants(head.ID()) + participants := e.participantsProvider.Identifiers() // send the request for synchronization req := &messages.SyncRequest{ Nonce: rand.Uint64(), Height: head.Height, } - err := e.con.Multicast(req, synccore.DefaultPollNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultPollNodes, participants...) if err != nil { e.log.Warn().Err(err).Msg("sending sync request to poll heights failed") return @@ -374,7 +360,7 @@ func (e *Engine) pollHeight() { } // sendRequests sends a request for each range and batch using consensus participants from last finalized snapshot. -func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Range, batches []flow.Batch) { +func (e *Engine) sendRequests(participants flow.IdentifierList, ranges []flow.Range, batches []flow.Batch) { var errs *multierror.Error for _, ran := range ranges { @@ -383,7 +369,7 @@ func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Rang FromHeight: ran.From, ToHeight: ran.To, } - err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants...) if err != nil { errs = multierror.Append(errs, fmt.Errorf("could not submit range request: %w", err)) continue @@ -402,7 +388,7 @@ func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Rang Nonce: rand.Uint64(), BlockIDs: batch.BlockIDs, } - err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants...) if err != nil { errs = multierror.Append(errs, fmt.Errorf("could not submit batch request: %w", err)) continue diff --git a/go.mod b/go.mod index 5383fbc401f..605be87488d 100644 --- a/go.mod +++ b/go.mod @@ -24,16 +24,14 @@ require ( github.com/hashicorp/golang-lru v0.5.4 github.com/improbable-eng/grpc-web v0.12.0 github.com/ipfs/go-log v1.0.5 - github.com/ipfs/go-todocounter v0.0.2 // indirect github.com/jrick/bitset v1.0.0 github.com/kr/text v0.2.0 // indirect github.com/libp2p/go-addr-util v0.1.0 github.com/libp2p/go-libp2p v0.14.4 - github.com/libp2p/go-libp2p-core v0.8.6 + github.com/libp2p/go-libp2p-core v0.9.0 github.com/libp2p/go-libp2p-discovery v0.5.0 - github.com/libp2p/go-libp2p-kad-dht v0.13.0 // indirect + github.com/libp2p/go-libp2p-peerstore v0.2.8 github.com/libp2p/go-libp2p-pubsub v0.4.1 - github.com/libp2p/go-libp2p-routing v0.1.0 // indirect github.com/libp2p/go-libp2p-swarm v0.5.3 github.com/libp2p/go-libp2p-tls v0.1.3 github.com/libp2p/go-libp2p-transport-upgrader v0.4.6 @@ -41,6 +39,7 @@ require ( github.com/m4ksio/wal v1.0.0 github.com/mitchellh/mapstructure v1.3.3 // indirect github.com/multiformats/go-multiaddr v0.3.3 + github.com/multiformats/go-multihash v0.0.15 github.com/onflow/cadence v0.18.1-0.20210729032058-d9eb6683d6ed github.com/onflow/flow-core-contracts/lib/go/contracts v0.7.5 github.com/onflow/flow-core-contracts/lib/go/templates v0.7.5 @@ -62,6 +61,7 @@ require ( github.com/vmihailenco/msgpack v4.0.4+incompatible github.com/vmihailenco/msgpack/v4 v4.3.11 go.uber.org/atomic v1.7.0 + go.uber.org/zap v1.18.1 // indirect golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6 golang.org/x/time v0.0.0-20191024005414-555d28b269f0 diff --git a/go.sum b/go.sum index fa2415cdf7c..73ac55282ef 100644 --- a/go.sum +++ b/go.sum @@ -98,8 +98,8 @@ github.com/aws/aws-sdk-go v1.25.48/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpi github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/benbjohnson/clock v1.0.2/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= -github.com/benbjohnson/clock v1.0.3 h1:vkLuvpK4fmtSCuo60+yC63p7y0BmQ8gm5ZXGuBCJyXg= github.com/benbjohnson/clock v1.0.3/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= +github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= @@ -185,8 +185,8 @@ github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f/go.mod h1:xH/i4TFM github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= github.com/dgraph-io/badger v1.6.0-rc1/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/badger v1.6.1 h1:w9pSFNSdq/JPM1N12Fz/F/bzo993Is1W+Q7HjPzi7yg= github.com/dgraph-io/badger v1.6.1/go.mod h1:FRmFw3uxvcpa8zG3Rxs0th+hCLIuaQg8HlNV5bjgnuU= +github.com/dgraph-io/badger v1.6.2 h1:mNw0qs90GVgGGWylh0umH5iag1j6n/PeJtNvL6KY/x8= github.com/dgraph-io/badger v1.6.2/go.mod h1:JW2yswe3V058sS0kZ2h/AXeDSqFjxnZcRrVH//y2UQE= github.com/dgraph-io/badger/v2 v2.0.3 h1:inzdf6VF/NZ+tJ8RwwYMjJMvsOALTHYdozn0qSl6XJI= github.com/dgraph-io/badger/v2 v2.0.3/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= @@ -233,7 +233,6 @@ github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJn github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= -github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= @@ -265,6 +264,7 @@ github.com/go-sourcemap/sourcemap v2.1.2+incompatible/go.mod h1:F8jJfvm2KbVjc5Nq github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 h1:p104kn46Q8WdvHunIJ9dAyjPVtrBPhSr3KT2yUst43I= github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/go-test/deep v1.0.5 h1:AKODKU3pDH1RzZzm6YZu77YWtEAq6uh1rLIAQlay2qc= github.com/go-test/deep v1.0.5/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= @@ -294,8 +294,8 @@ github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= -github.com/golang/mock v1.5.0 h1:jlYHihg//f7RRwuPfptm04yp4s7O6Kw8EZiVYIGcH0g= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= +github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0= @@ -354,7 +354,6 @@ github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hf github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -400,9 +399,7 @@ github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brv github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-multierror v1.1.0/go.mod h1:spPvp8C1qA32ftKqdAHm4hHTbPw+vmowP0z+KUhOZdA= github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= @@ -445,35 +442,27 @@ github.com/ipfs/go-cid v0.0.6/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqg github.com/ipfs/go-cid v0.0.7 h1:ysQJVJA3fNDF1qigJbsSQOdjhVLsOEoPdh0+R97k3jY= github.com/ipfs/go-cid v0.0.7/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= github.com/ipfs/go-datastore v0.0.1/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= -github.com/ipfs/go-datastore v0.1.0/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= -github.com/ipfs/go-datastore v0.1.1/go.mod h1:w38XXW9kVFNp57Zj5knbKWM2T+KOZCGDRVNdgPHtbHw= github.com/ipfs/go-datastore v0.4.0/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.1/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.4/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= -github.com/ipfs/go-datastore v0.4.5 h1:cwOUcGMLdLPWgu3SlrCckCMznaGADbPqE0r8h768/Dg= github.com/ipfs/go-datastore v0.4.5/go.mod h1:eXTcaaiN6uOlVCLS9GjJUJtlvJfM3xk23w3fyfrmmJs= github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= github.com/ipfs/go-ds-badger v0.0.2/go.mod h1:Y3QpeSFWQf6MopLTiZD+VT6IC1yZqaGmjvRcKeSGij8= github.com/ipfs/go-ds-badger v0.0.5/go.mod h1:g5AuuCGmr7efyzQhLL8MzwqcauPojGPUaHzfGTzuE3s= -github.com/ipfs/go-ds-badger v0.0.7/go.mod h1:qt0/fWzZDoPW6jpQeqUjR5kBfhDNB65jd9YlmAvpQBk= github.com/ipfs/go-ds-badger v0.2.1/go.mod h1:Tx7l3aTph3FMFrRS838dcSJh+jjA7cX9DrGVwx/NOwE= github.com/ipfs/go-ds-badger v0.2.3/go.mod h1:pEYw0rgg3FIrywKKnL+Snr+w/LjJZVMTBRn4FS6UHUk= github.com/ipfs/go-ds-badger v0.2.7/go.mod h1:02rnztVKA4aZwDuaRPTf8mpqcKmXP7mLl6JPxd14JHA= github.com/ipfs/go-ds-leveldb v0.0.1/go.mod h1:feO8V3kubwsEF22n0YRQCffeb79OOYIykR4L04tMOYc= -github.com/ipfs/go-ds-leveldb v0.1.0/go.mod h1:hqAW8y4bwX5LWcCtku2rFNX3vjDZCy5LZCg+cSZvYb8= github.com/ipfs/go-ds-leveldb v0.4.1/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ds-leveldb v0.4.2/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-util v0.0.1/go.mod h1:spsl5z8KUnrve+73pOhSVZND1SIxPW5RyBCNzQxlJBc= github.com/ipfs/go-ipfs-util v0.0.2 h1:59Sswnk1MFaiq+VcaknX7aYEyGyGDAA73ilhEK2POp8= github.com/ipfs/go-ipfs-util v0.0.2/go.mod h1:CbPtkWJzjLdEcezDns2XYaehFVNXG9zrdrtMecczcsQ= -github.com/ipfs/go-ipns v0.1.2 h1:O/s/0ht+4Jl9+VoxoUo0zaHjnZUS+aBQIKTuzdZ/ucI= -github.com/ipfs/go-ipns v0.1.2/go.mod h1:ioQ0j02o6jdIVW+bmi18f4k2gRf0AV3kZ9KeHYHICnQ= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-log v1.0.2/go.mod h1:1MNjMxe0u6xvJZgeqbJ8vdo2TKaGwZ1a0Bpza+sr2Sk= github.com/ipfs/go-log v1.0.3/go.mod h1:OsLySYkwIbiSUR/yBTdv1qPtcE4FW3WPWk/ewz9Ru+A= -github.com/ipfs/go-log v1.0.4 h1:6nLQdX4W8P9yZZFH7mO+X/PzjN8Laozm/lMJ6esdgzY= github.com/ipfs/go-log v1.0.4/go.mod h1:oDCg2FkjogeFOhqqb+N39l2RpTNPL6F/StPkB3kPgcs= github.com/ipfs/go-log v1.0.5 h1:2dOuUCB1Z7uoczMWgAyDck5JLb72zHzrMnGnCNNbvY8= github.com/ipfs/go-log v1.0.5/go.mod h1:j0b8ZoR+7+R99LD9jZ6+AJsrzkPbSXbZfGakb5JPtIo= @@ -483,9 +472,6 @@ github.com/ipfs/go-log/v2 v2.0.5/go.mod h1:eZs4Xt4ZUJQFM3DlanGhy7TkwwawCZcSByscw github.com/ipfs/go-log/v2 v2.1.1/go.mod h1:2v2nsGfZsvvAJz13SyFzf9ObaqwHiHxsPLEHntrv9KM= github.com/ipfs/go-log/v2 v2.1.3 h1:1iS3IU7aXRlbgUpN8yTTpJ53NXYjAe37vcI5+5nYrzk= github.com/ipfs/go-log/v2 v2.1.3/go.mod h1:/8d0SH3Su5Ooc31QlL1WysJhvyOTDCjcCZ9Axpmri6g= -github.com/ipfs/go-todocounter v0.0.2/go.mod h1:l5aErvQc8qKE2r7NDMjmq5UNAvuZy0rC8BHOplkWvZ4= -github.com/ipld/go-ipld-prime v0.9.0 h1:N2OjJMb+fhyFPwPnVvJcWU/NsumP8etal+d2v3G4eww= -github.com/ipld/go-ipld-prime v0.9.0/go.mod h1:KvBLMr4PX1gWptgkzRjVZCrLmSGcZCb/jioOQwCqZN8= github.com/jackpal/gateway v1.0.5/go.mod h1:lTpwd4ACLXmpyiCTRtfiNyVnUmqT9RivzCDQetPfnjA= github.com/jackpal/go-nat-pmp v1.0.1/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= @@ -534,6 +520,7 @@ github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQL github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= +github.com/klauspost/compress v1.11.7 h1:0hzRabrMN4tSTvMfnL3SCv1ZGeAP23ynzodBgaHeMeg= github.com/klauspost/compress v1.11.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/klauspost/cpuid/v2 v2.0.4 h1:g0I61F2K2DjRHz1cnxlkNSBIaePVoJIjjnHui8QHbiw= github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= @@ -554,14 +541,12 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/libp2p/go-addr-util v0.0.1/go.mod h1:4ac6O7n9rIAKB1dnd+s8IbbMXkt+oBpzX4/+RACcnlQ= -github.com/libp2p/go-addr-util v0.0.2 h1:7cWK5cdA5x72jX0g8iLrQWm5TRJZ6CzGdPEhWj7plWU= github.com/libp2p/go-addr-util v0.0.2/go.mod h1:Ecd6Fb3yIuLzq4bD7VcywcVSBtefcAwnUISBM3WG15E= +github.com/libp2p/go-addr-util v0.1.0 h1:acKsntI33w2bTU7tC9a0SaPimJGfSI0bFKC18ChxeVI= github.com/libp2p/go-addr-util v0.1.0/go.mod h1:6I3ZYuFr2O/9D+SoyM0zEw0EF3YkldtTX406BpdQMqw= github.com/libp2p/go-buffer-pool v0.0.1/go.mod h1:xtyIz9PMobb13WaxR6Zo1Pd1zXJKYg0a8KiIvDp3TzQ= github.com/libp2p/go-buffer-pool v0.0.2 h1:QNK2iAFa8gjAe1SPz6mHSMuCcjs+X1wlHzeOSqcmlfs= github.com/libp2p/go-buffer-pool v0.0.2/go.mod h1:MvaB6xw5vOrDl8rYZGLFdKAuk/hRoRZd1Vi32+RXyFM= -github.com/libp2p/go-cidranger v1.1.0 h1:ewPN8EZ0dd1LSnrtuwd4709PXVcITVeuwbag38yPW7c= -github.com/libp2p/go-cidranger v1.1.0/go.mod h1:KWZTfSr+r9qEo9OkI9/SIEeAtw+NNoU0dXIXt15Okic= github.com/libp2p/go-conn-security-multistream v0.1.0/go.mod h1:aw6eD7LOsHEX7+2hJkDxw1MteijaVcI+/eP2/x3J1xc= github.com/libp2p/go-conn-security-multistream v0.2.0/go.mod h1:hZN4MjlNetKD3Rq5Jb/P5ohUnFLNzEAR4DLSzpn2QLU= github.com/libp2p/go-conn-security-multistream v0.2.1 h1:ft6/POSK7F+vl/2qzegnHDaXFU0iWB4yVTYrioC6Zy0= @@ -570,18 +555,15 @@ github.com/libp2p/go-eventbus v0.1.0/go.mod h1:vROgu5cs5T7cv7POWlWxBaVLxfSegC5UG github.com/libp2p/go-eventbus v0.2.1 h1:VanAdErQnpTioN2TowqNcOijf6YwhuODe4pPKSDpxGc= github.com/libp2p/go-eventbus v0.2.1/go.mod h1:jc2S4SoEVPP48H9Wpzm5aiGwUCBMfGhVhhBjyhhCJs8= github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZxBdp967ls1g+k8= -github.com/libp2p/go-flow-metrics v0.0.2/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-flow-metrics v0.0.3 h1:8tAs/hSdNvUiLgtlSy3mxwxWP4I9y/jlkPFT7epKdeM= github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-libp2p v0.6.1/go.mod h1:CTFnWXogryAHjXAKEbOf1OWY+VeAP3lDMZkfEI5sT54= github.com/libp2p/go-libp2p v0.7.0/go.mod h1:hZJf8txWeCduQRDC/WSqBGMxaTHCOYHt2xSU1ivxn0k= github.com/libp2p/go-libp2p v0.7.4/go.mod h1:oXsBlTLF1q7pxr+9w6lqzS1ILpyHsaBPniVO7zIHGMw= github.com/libp2p/go-libp2p v0.8.1/go.mod h1:QRNH9pwdbEBpx5DTJYg+qxcVaDMAz3Ee/qDKwXujH5o= -github.com/libp2p/go-libp2p v0.14.1 h1:R0vNY7nkU8IISlDuHd2yk4eNAZsVQ0rCr2bPfWU3sXo= github.com/libp2p/go-libp2p v0.14.1/go.mod h1:0PQMADQEjCM2l8cSMYDpTgsb8gr6Zq7i4LUgq1mlW2E= +github.com/libp2p/go-libp2p v0.14.4 h1:QCJE+jGyqxWdrSPuS4jByXCzosgaIg4SJTLCRplJ53w= github.com/libp2p/go-libp2p v0.14.4/go.mod h1:EIRU0Of4J5S8rkockZM7eJp2S0UrCyi55m2kJVru3rM= -github.com/libp2p/go-libp2p-asn-util v0.0.0-20200825225859-85005c6cf052 h1:BM7aaOF7RpmNn9+9g6uTjGJ0cTzWr5j9i9IKeun2M8U= -github.com/libp2p/go-libp2p-asn-util v0.0.0-20200825225859-85005c6cf052/go.mod h1:nRMRTab+kZuk0LnKZpxhOVH/ndsdr2Nr//Zltc/vwgo= github.com/libp2p/go-libp2p-autonat v0.1.1/go.mod h1:OXqkeGOY2xJVWKAGV2inNF5aKN/djNA3fdpCWloIudE= github.com/libp2p/go-libp2p-autonat v0.2.0/go.mod h1:DX+9teU4pEEoZUqR1PiMlqliONQdNbfzE1C718tcViI= github.com/libp2p/go-libp2p-autonat v0.2.1/go.mod h1:MWtAhV5Ko1l6QBsHQNSuM6b1sRkXrpk0/LqCr+vCVxI= @@ -603,37 +585,29 @@ github.com/libp2p/go-libp2p-core v0.0.4/go.mod h1:jyuCQP356gzfCFtRKyvAbNkyeuxb7O github.com/libp2p/go-libp2p-core v0.2.0/go.mod h1:X0eyB0Gy93v0DZtSYbEM7RnMChm9Uv3j7yRXjO77xSI= github.com/libp2p/go-libp2p-core v0.2.2/go.mod h1:8fcwTbsG2B+lTgRJ1ICZtiM5GWCWZVoVrLaDRvIRng0= github.com/libp2p/go-libp2p-core v0.2.4/go.mod h1:STh4fdfa5vDYr0/SzYYeqnt+E6KfEV5VxfIrm0bcI0g= -github.com/libp2p/go-libp2p-core v0.2.5/go.mod h1:6+5zJmKhsf7yHn1RbmYDu08qDUpIUxGdqHuEZckmZOA= github.com/libp2p/go-libp2p-core v0.3.0/go.mod h1:ACp3DmS3/N64c2jDzcV429ukDpicbL6+TrrxANBjPGw= github.com/libp2p/go-libp2p-core v0.3.1/go.mod h1:thvWy0hvaSBhnVBaW37BvzgVV68OUhgJJLAa6almrII= github.com/libp2p/go-libp2p-core v0.4.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= github.com/libp2p/go-libp2p-core v0.5.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= github.com/libp2p/go-libp2p-core v0.5.1/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= -github.com/libp2p/go-libp2p-core v0.5.3/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= github.com/libp2p/go-libp2p-core v0.5.4/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= github.com/libp2p/go-libp2p-core v0.5.5/go.mod h1:vj3awlOr9+GMZJFH9s4mpt9RHHgGqeHCopzbYKZdRjM= github.com/libp2p/go-libp2p-core v0.5.6/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.5.7/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.6.0/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= -github.com/libp2p/go-libp2p-core v0.6.1/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.7.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.1/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.2/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= -github.com/libp2p/go-libp2p-core v0.8.5 h1:aEgbIcPGsKy6zYcC+5AJivYFedhYa4sW7mIpWpUaLKw= github.com/libp2p/go-libp2p-core v0.8.5/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= -github.com/libp2p/go-libp2p-core v0.8.6 h1:3S8g006qG6Tjpj1JdRK2S+TWc2DJQKX/RG9fdLeiLSU= github.com/libp2p/go-libp2p-core v0.8.6/go.mod h1:dgHr0l0hIKfWpGpqAMbpo19pen9wJfdCGv51mTmdpmM= +github.com/libp2p/go-libp2p-core v0.9.0 h1:t97Mv0LIBZlP2FXVRNKKVzHJCIjbIWGxYptGId4+htU= +github.com/libp2p/go-libp2p-core v0.9.0/go.mod h1:ESsbz31oC3C1AvMJoGx26RTuCkNhmkSRCqZ0kQtJ2/8= github.com/libp2p/go-libp2p-crypto v0.1.0/go.mod h1:sPUokVISZiy+nNuTTH/TY+leRSxnFj/2GLjtOTW90hI= github.com/libp2p/go-libp2p-discovery v0.2.0/go.mod h1:s4VGaxYMbw4+4+tsoQTqh7wfxg97AEdo4GYBt6BadWg= github.com/libp2p/go-libp2p-discovery v0.3.0/go.mod h1:o03drFnz9BVAZdzC/QUQ+NeQOu38Fu7LJGEOK2gQltw= github.com/libp2p/go-libp2p-discovery v0.5.0 h1:Qfl+e5+lfDgwdrXdu4YNCWyEo3fWuP+WgN9mN0iWviQ= github.com/libp2p/go-libp2p-discovery v0.5.0/go.mod h1:+srtPIU9gDaBNu//UHvcdliKBIcr4SfDcm0/PfPJLug= -github.com/libp2p/go-libp2p-kad-dht v0.13.0 h1:qBNYzee8BVS6RkD8ukIAGRG6LmVz8+kkeponyI7W+yA= -github.com/libp2p/go-libp2p-kad-dht v0.13.0/go.mod h1:NkGf28RNhPrcsGYWJHm6EH8ULkiJ2qxsWmpE7VTL3LI= -github.com/libp2p/go-libp2p-kbucket v0.3.1/go.mod h1:oyjT5O7tS9CQurok++ERgc46YLwEpuGoFq9ubvoUOio= -github.com/libp2p/go-libp2p-kbucket v0.4.7 h1:spZAcgxifvFZHBD8tErvppbnNiKA5uokDu3CV7axu70= -github.com/libp2p/go-libp2p-kbucket v0.4.7/go.mod h1:XyVo99AfQH0foSf176k4jY1xUJ2+jUJIZCSDm7r2YKk= github.com/libp2p/go-libp2p-loggables v0.1.0/go.mod h1:EyumB2Y6PrYjr55Q3/tiJ/o3xoDasoRYM7nOzEpoa90= github.com/libp2p/go-libp2p-mplex v0.2.0/go.mod h1:Ejl9IyjvXJ0T9iqUTE1jpYATQ9NM3g+OtR+EMMODbKo= github.com/libp2p/go-libp2p-mplex v0.2.1/go.mod h1:SC99Rxs8Vuzrf/6WhmH41kNn13TiYdAWNYHrwImKLnE= @@ -652,12 +626,10 @@ github.com/libp2p/go-libp2p-noise v0.2.0/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhU github.com/libp2p/go-libp2p-peer v0.2.0/go.mod h1:RCffaCvUyW2CJmG2gAWVqwePwW7JMgxjsHm7+J5kjWY= github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= github.com/libp2p/go-libp2p-peerstore v0.1.3/go.mod h1:BJ9sHlm59/80oSkpWgr1MyY1ciXAXV397W6h1GH/uKI= -github.com/libp2p/go-libp2p-peerstore v0.1.4/go.mod h1:+4BDbDiiKf4PzpANZDAT+knVdLxvqh7hXOujessqdzs= github.com/libp2p/go-libp2p-peerstore v0.2.0/go.mod h1:N2l3eVIeAitSg3Pi2ipSrJYnqhVnMNQZo9nkSCuAbnQ= github.com/libp2p/go-libp2p-peerstore v0.2.1/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= -github.com/libp2p/go-libp2p-peerstore v0.2.7 h1:83JoLxyR9OYTnNfB5vvFqvMUv/xDNa6NoPHnENhBsGw= github.com/libp2p/go-libp2p-peerstore v0.2.7/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.8 h1:nJghUlUkFVvyk7ccsM67oFA6kqUkwyCM1G4WPVMCWYA= github.com/libp2p/go-libp2p-peerstore v0.2.8/go.mod h1:gGiPlXdz7mIHd2vfAsHzBNAMqSDkt2UBFwgcITgw1lA= @@ -665,14 +637,9 @@ github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6n github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= github.com/libp2p/go-libp2p-pubsub v0.4.1 h1:j4umIg5nyus+sqNfU+FWvb9aeYFQH/A+nDFhWj+8yy8= github.com/libp2p/go-libp2p-pubsub v0.4.1/go.mod h1:izkeMLvz6Ht8yAISXjx60XUQZMq9ZMe5h2ih4dLIBIQ= -github.com/libp2p/go-libp2p-quic-transport v0.10.0 h1:koDCbWD9CCHwcHZL3/WEvP2A+e/o5/W5L3QS/2SPMA0= github.com/libp2p/go-libp2p-quic-transport v0.10.0/go.mod h1:RfJbZ8IqXIhxBRm5hqUEJqjiiY8xmEuq3HUDS993MkA= +github.com/libp2p/go-libp2p-quic-transport v0.11.2 h1:p1YQDZRHH4Cv2LPtHubqlQ9ggz4CKng/REZuXZbZMhM= github.com/libp2p/go-libp2p-quic-transport v0.11.2/go.mod h1:wlanzKtIh6pHrq+0U3p3DY9PJfGqxMgPaGKaK5LifwQ= -github.com/libp2p/go-libp2p-record v0.1.2/go.mod h1:pal0eNcT5nqZaTV7UGhqeGqxFgGdsU/9W//C8dqjQDk= -github.com/libp2p/go-libp2p-record v0.1.3 h1:R27hoScIhQf/A8XJZ8lYpnqh9LatJ5YbHs28kCIfql0= -github.com/libp2p/go-libp2p-record v0.1.3/go.mod h1:yNUff/adKIfPnYQXgp6FQmNu3gLJ6EMg7+/vv2+9pY4= -github.com/libp2p/go-libp2p-routing v0.1.0/go.mod h1:zfLhI1RI8RLEzmEaaPwzonRvXeeSHddONWkcTcB54nE= -github.com/libp2p/go-libp2p-routing-helpers v0.2.3/go.mod h1:795bh+9YeoFl99rMASoiVgHdi5bjack0N1+AFAdbvBw= github.com/libp2p/go-libp2p-secio v0.1.0/go.mod h1:tMJo2w7h3+wN4pgU2LSYeiKPrfqBgkOsdiKK77hE7c8= github.com/libp2p/go-libp2p-secio v0.2.0/go.mod h1:2JdZepB8J5V9mBp79BmwsaPQhRPNN2NrnB2lKQcdy6g= github.com/libp2p/go-libp2p-secio v0.2.1/go.mod h1:cWtZpILJqkqrSkiYcDBh5lA3wbT2Q+hz3rJQq3iftD8= @@ -683,8 +650,8 @@ github.com/libp2p/go-libp2p-swarm v0.2.3/go.mod h1:P2VO/EpxRyDxtChXz/VPVXyTnszHv github.com/libp2p/go-libp2p-swarm v0.2.8/go.mod h1:JQKMGSth4SMqonruY0a8yjlPVIkb0mdNSwckW7OYziM= github.com/libp2p/go-libp2p-swarm v0.3.0/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= github.com/libp2p/go-libp2p-swarm v0.3.1/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= -github.com/libp2p/go-libp2p-swarm v0.5.0 h1:HIK0z3Eqoo8ugmN8YqWAhD2RORgR+3iNXYG4U2PFd1E= github.com/libp2p/go-libp2p-swarm v0.5.0/go.mod h1:sU9i6BoHE0Ve5SKz3y9WfKrh8dUat6JknzUehFx8xW4= +github.com/libp2p/go-libp2p-swarm v0.5.3 h1:hsYaD/y6+kZff1o1Mc56NcuwSg80lIphTS/zDk3mO4M= github.com/libp2p/go-libp2p-swarm v0.5.3/go.mod h1:NBn7eNW2lu568L7Ns9wdFrOhgRlkRnIDg0FLKbuu3i8= github.com/libp2p/go-libp2p-testing v0.0.2/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.3/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= @@ -693,18 +660,17 @@ github.com/libp2p/go-libp2p-testing v0.1.0/go.mod h1:xaZWMJrPUM5GlDBxCeGUi7kI4eq github.com/libp2p/go-libp2p-testing v0.1.1/go.mod h1:xaZWMJrPUM5GlDBxCeGUi7kI4eqnjVyavGroI2nxEM0= github.com/libp2p/go-libp2p-testing v0.1.2-0.20200422005655-8775583591d8/go.mod h1:Qy8sAncLKpwXtS2dSnDOP8ktexIAHKu+J+pnZOFZLTc= github.com/libp2p/go-libp2p-testing v0.3.0/go.mod h1:efZkql4UZ7OVsEfaxNHZPzIehtsBXMrXnCfJIgDti5g= -github.com/libp2p/go-libp2p-testing v0.4.0 h1:PrwHRi0IGqOwVQWR3xzgigSlhlLfxgfXgkHxr77EghQ= github.com/libp2p/go-libp2p-testing v0.4.0/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= +github.com/libp2p/go-libp2p-testing v0.4.2 h1:IOiA5mMigi+eEjf4J+B7fepDhsjtsoWA9QbsCqbNp5U= github.com/libp2p/go-libp2p-testing v0.4.2/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= github.com/libp2p/go-libp2p-tls v0.1.3 h1:twKMhMu44jQO+HgQK9X8NHO5HkeJu2QbhLzLJpa8oNM= github.com/libp2p/go-libp2p-tls v0.1.3/go.mod h1:wZfuewxOndz5RTnCAxFliGjvYSDA40sKitV4c50uI1M= github.com/libp2p/go-libp2p-transport-upgrader v0.1.1/go.mod h1:IEtA6or8JUbsV07qPW4r01GnTenLW4oi3lOPbUMGJJA= github.com/libp2p/go-libp2p-transport-upgrader v0.2.0/go.mod h1:mQcrHj4asu6ArfSoMuyojOdjx73Q47cYD7s5+gZOlns= github.com/libp2p/go-libp2p-transport-upgrader v0.3.0/go.mod h1:i+SKzbRnvXdVbU3D1dwydnTmKRPXiAR/fyvi1dXuL4o= -github.com/libp2p/go-libp2p-transport-upgrader v0.4.2 h1:4JsnbfJzgZeRS9AWN7B9dPqn/LY/HoQTlO9gtdJTIYM= github.com/libp2p/go-libp2p-transport-upgrader v0.4.2/go.mod h1:NR8ne1VwfreD5VIWIU62Agt/J18ekORFU/j1i2y8zvk= +github.com/libp2p/go-libp2p-transport-upgrader v0.4.6 h1:SHt3g0FslnqIkEWF25YOB8UCOCTpGAVvHRWQYJ+veiI= github.com/libp2p/go-libp2p-transport-upgrader v0.4.6/go.mod h1:JE0WQuQdy+uLZ5zOaI3Nw9dWGYJIA7mywEtP2lMvnyk= -github.com/libp2p/go-libp2p-xor v0.0.0-20210714161855-5c005aca55db/go.mod h1:LSTM5yRnjGZbWNTA/hRwq2gGFrvRIbQJscoIL/u6InY= github.com/libp2p/go-libp2p-yamux v0.2.0/go.mod h1:Db2gU+XfLpm6E4rG5uGCFX6uXA8MEXOxFcRoXUODaK8= github.com/libp2p/go-libp2p-yamux v0.2.2/go.mod h1:lIohaR0pT6mOt0AZ0L2dFze9hds9Req3OfS+B+dv4qw= github.com/libp2p/go-libp2p-yamux v0.2.5/go.mod h1:Zpgj6arbyQrmZ3wxSZxfBmbdnWtbZ48OpsfmQVTErwA= @@ -748,8 +714,8 @@ github.com/libp2p/go-reuseport v0.0.2 h1:XSG94b1FJfGA01BUrT82imejHQyTxO4jEWqheyC github.com/libp2p/go-reuseport v0.0.2/go.mod h1:SPD+5RwGC7rcnzngoYC86GjPzjSywuQyMVAheVBD9nQ= github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA20obEtsW9BLkuOUyQAOCbs= github.com/libp2p/go-reuseport-transport v0.0.3/go.mod h1:Spv+MPft1exxARzP2Sruj2Wb5JSyHNncjf1Oi2dEbzM= -github.com/libp2p/go-reuseport-transport v0.0.4 h1:OZGz0RB620QDGpv300n1zaOcKGGAoGVf8h9txtt/1uM= github.com/libp2p/go-reuseport-transport v0.0.4/go.mod h1:trPa7r/7TJK/d+0hdBLOCGvpQQVOU74OXbNCIMkufGw= +github.com/libp2p/go-reuseport-transport v0.0.5 h1:lJzi+vSYbyJj2faPKLxNGWEIBcaV/uJmyvsUxXy2mLw= github.com/libp2p/go-reuseport-transport v0.0.5/go.mod h1:TC62hhPc8qs5c/RoXDZG6YmjK+/YWUPC0yYmeUecbjc= github.com/libp2p/go-sockaddr v0.0.2/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-sockaddr v0.1.0/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= @@ -762,9 +728,9 @@ github.com/libp2p/go-stream-muxer-multistream v0.3.0/go.mod h1:yDh8abSIzmZtqtOt6 github.com/libp2p/go-tcp-transport v0.1.0/go.mod h1:oJ8I5VXryj493DEJ7OsBieu8fcg2nHGctwtInJVpipc= github.com/libp2p/go-tcp-transport v0.1.1/go.mod h1:3HzGvLbx6etZjnFlERyakbaYPdfjg2pWP97dFZworkY= github.com/libp2p/go-tcp-transport v0.2.0/go.mod h1:vX2U0CnWimU4h0SGSEsg++AzvBcroCGYw28kh94oLe0= -github.com/libp2p/go-tcp-transport v0.2.1 h1:ExZiVQV+h+qL16fzCWtd1HSzPsqWottJ8KXwWaVi8Ns= github.com/libp2p/go-tcp-transport v0.2.1/go.mod h1:zskiJ70MEfWz2MKxvFB/Pv+tPIB1PpPUrHIWQ8aFw7M= github.com/libp2p/go-tcp-transport v0.2.4/go.mod h1:9dvr03yqrPyYGIEN6Dy5UvdJZjyPFvl1S/igQ5QD1SU= +github.com/libp2p/go-tcp-transport v0.2.7 h1:Z8Kc/Kb8tD84WiaH55xAlaEnkqzrp88jSEySCKV4+gg= github.com/libp2p/go-tcp-transport v0.2.7/go.mod h1:lue9p1b3VmZj1MhhEGB/etmvF/nBQ0X9CW2DutBT3MM= github.com/libp2p/go-ws-transport v0.2.0/go.mod h1:9BHJz/4Q5A9ludYWKoGCFC5gUElzlHoKzu0yY9p/klM= github.com/libp2p/go-ws-transport v0.3.0/go.mod h1:bpgTJmRZAvVHrgHybCVyqoBmyLQ1fiZuEaBYusP5zsk= @@ -784,8 +750,8 @@ github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-b github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= github.com/logrusorgru/aurora v0.0.0-20200102142835-e9ef32dff381 h1:bqDmpDG49ZRnB5PcgP0RXtQvnMSgIF14M7CBd2shtXs= github.com/logrusorgru/aurora v0.0.0-20200102142835-e9ef32dff381/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= -github.com/lucas-clemente/quic-go v0.19.3 h1:eCDQqvGBB+kCTkA0XrAFtNe81FMa0/fn4QSoeAbmiF4= github.com/lucas-clemente/quic-go v0.19.3/go.mod h1:ADXpNbTQjq1hIzCpB+y/k5iz4n4z4IwqoLb94Kh5Hu8= +github.com/lucas-clemente/quic-go v0.21.2 h1:8LqqL7nBQFDUINadW0fHV/xSaCQJgmJC0Gv+qUnjd78= github.com/lucas-clemente/quic-go v0.21.2/go.mod h1:vF5M1XqhBAHgbjKcJOXY3JZz3GP0T3FQhz/uyOUS38Q= github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= @@ -797,14 +763,16 @@ github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czP github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/marten-seemann/qpack v0.2.1/go.mod h1:F7Gl5L1jIgN1D11ucXefiuJS9UMVP2opoCp2jDKb7wc= -github.com/marten-seemann/qtls v0.10.0 h1:ECsuYUKalRL240rRD4Ri33ISb7kAQ3qGDlrrl55b2pc= github.com/marten-seemann/qtls v0.10.0/go.mod h1:UvMd1oaYDACI99/oZUYLzMCkBXQVT0aGm99sJhbT8hs= github.com/marten-seemann/qtls-go1-15 v0.1.1/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= -github.com/marten-seemann/qtls-go1-15 v0.1.4 h1:RehYMOyRW8hPVEja1KBVsFVNSm35Jj9Mvs5yNoZZ28A= github.com/marten-seemann/qtls-go1-15 v0.1.4/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-15 v0.1.5 h1:Ci4EIUN6Rlb+D6GmLdej/bCQ4nPYNtVXQB+xjiXE1nk= github.com/marten-seemann/qtls-go1-15 v0.1.5/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-16 v0.1.4 h1:xbHbOGGhrenVtII6Co8akhLEdrawwB2iHl5yhJRpnco= github.com/marten-seemann/qtls-go1-16 v0.1.4/go.mod h1:gNpI2Ol+lRS3WwSOtIUUtRwZEQMXjYK+dQSBFbethAk= +github.com/marten-seemann/qtls-go1-17 v0.1.0-rc.1 h1:/rpmWuGvceLwwWuaKPdjpR4JJEUH0tq64/I3hvzaNLM= github.com/marten-seemann/qtls-go1-17 v0.1.0-rc.1/go.mod h1:fz4HIxByo+LlWcreM4CZOYNuz3taBQ8rN2X6FqvaWo8= +github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd h1:br0buuQ854V8u83wA0rVZ8ttrq5CpaPZdvrK0LP2lOk= github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd/go.mod h1:QuCEs1Nt24+FYQEqAAncTDPJIuGs+LxK1MCiFL25pMU= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.0/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= @@ -836,8 +804,11 @@ github.com/miekg/dns v1.1.12/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3N github.com/miekg/dns v1.1.28/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/miekg/dns v1.1.41 h1:WMszZWJG0XmzbK9FEmzH2TVcqYzFesusSIB41b8KHxY= github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c h1:bzE/A84HN25pxAuk9Eej1Kz9OUelF97nAc82bDquQI8= github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c/go.mod h1:0SQS9kMwD2VsyFEB++InYyBJroV/FRmBgcydeSUcJms= +github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b h1:z78hV3sbSMAUoyUMM0I83AUIT6Hu17AWfgjzIbtrYFc= github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b/go.mod h1:lxPUiZwKoFL8DUUmalo2yJJUCxbPKtm8OKfqr2/FTNU= +github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc h1:PTfri+PuQmWDqERdnNMiD9ZejrlswWrCpBEZgWOiTrc= github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc/go.mod h1:cGKTAVKx4SxOuR/czcZ/E2RSJ3sfHs8FpHhQ5CWMf9s= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= @@ -845,7 +816,6 @@ github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16/go.mod h1:2FMWW+ github.com/minio/sha256-simd v0.0.0-20190328051042-05b4dd3047e5/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.1.0/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= -github.com/minio/sha256-simd v0.1.1 h1:5QHSlgo3nt5yKOJrC7W8w7X+NFl8cMPZm96iu8kKUJU= github.com/minio/sha256-simd v0.1.1/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= github.com/minio/sha256-simd v1.0.0 h1:v1ta+49hkWZyvaKwrQB8elexRqm6Y0aMLjCNsrYxo6g= github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM= @@ -883,7 +853,6 @@ github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y9 github.com/multiformats/go-multiaddr v0.2.1/go.mod h1:s/Apk6IyxfvMjDafnhJgJ3/46z7tZ04iMk5wP4QMGGE= github.com/multiformats/go-multiaddr v0.2.2/go.mod h1:NtfXiOtHvghW9KojvtySjH5y0u0xW5UouOmQQrn6a3Y= github.com/multiformats/go-multiaddr v0.3.0/go.mod h1:dF9kph9wfJ+3VLAaeBqo9Of8x4fJxp6ggJGteB8HQTI= -github.com/multiformats/go-multiaddr v0.3.1 h1:1bxa+W7j9wZKTZREySx1vPMs2TqrYWjVZ7zE6/XLG1I= github.com/multiformats/go-multiaddr v0.3.1/go.mod h1:uPbspcUPd5AfaP6ql3ujFY+QWzmBD8uLLL4bXW0XfGc= github.com/multiformats/go-multiaddr v0.3.3 h1:vo2OTSAqnENB2rLk79pLtr+uhj+VAzSe3uef5q0lRSs= github.com/multiformats/go-multiaddr v0.3.3/go.mod h1:lCKNGP1EQ1eZ35Za2wlqnabm9xQkib3fyB+nZXHLag0= @@ -902,20 +871,15 @@ github.com/multiformats/go-multiaddr-net v0.1.2/go.mod h1:QsWt3XK/3hwvNxZJp92iMQ github.com/multiformats/go-multiaddr-net v0.1.3/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= github.com/multiformats/go-multiaddr-net v0.1.4/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= github.com/multiformats/go-multiaddr-net v0.1.5/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= -github.com/multiformats/go-multiaddr-net v0.2.0 h1:MSXRGN0mFymt6B1yo/6BPnIRpLPEnKgQNvVfCX5VDJk= github.com/multiformats/go-multiaddr-net v0.2.0/go.mod h1:gGdH3UXny6U3cKKYCvpXI5rnK7YaOIEOPVDI9tsJbEA= github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= github.com/multiformats/go-multibase v0.0.3 h1:l/B6bJDQjvQ5G52jw4QGSYeOTZoAwIO77RblWplfIqk= github.com/multiformats/go-multibase v0.0.3/go.mod h1:5+1R4eQrT3PkYZ24C3W2Ue2tPwIdYQD509ZjSb5y9Oc= -github.com/multiformats/go-multicodec v0.2.0 h1:MUzKZWxOFagwLLtlx96pub9zwDQAbMAf1k9fXOdc3so= -github.com/multiformats/go-multicodec v0.2.0/go.mod h1:/y4YVwkfMyry5kFbMTbLJKErhycTIftytRV+llXdyS4= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multihash v0.0.5/go.mod h1:lt/HCbqlQwlPBz7lv0sQCdtfcMtlJvakRUn/0Ual8po= github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= -github.com/multiformats/go-multihash v0.0.9/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.10/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.13/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= -github.com/multiformats/go-multihash v0.0.14 h1:QoBceQYQQtNUuf6s7wHxnE2c8bhbMqhfGzNI032se/I= github.com/multiformats/go-multihash v0.0.14/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= github.com/multiformats/go-multihash v0.0.15 h1:hWOPdrNqDjwHDx82vsYGSDZNyktOJJ2dzZJzFkOV1jM= github.com/multiformats/go-multihash v0.0.15/go.mod h1:D6aZrWNLFTV/ynMpKsNtB40mJzmCl4jb1alC0OvHiHg= @@ -944,8 +908,8 @@ github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxzi github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= -github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= @@ -983,17 +947,17 @@ github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+W github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.0/go.mod h1:oUhWkIvk5aDxtKvDDuw8gItl8pKl42LzjC9KZE0HfGg= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= github.com/onsi/ginkgo v1.16.2/go.mod h1:CObGmKUOKaSC0RjmoAK7tKyn4Azo5P2IWuoMnvwxz1E= +github.com/onsi/ginkgo v1.16.4 h1:29JGrr5oVBm5ulCWet69zQkzWipVXIol6ygQUe/EzNc= github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= github.com/onsi/gomega v1.4.1/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.9.0/go.mod h1:Ho0h+IUsWyvy1OpqCwxlQ/21gkhVunqlU8fDGcoTdcA= -github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.13.0 h1:7lLHu94wT9Ij0o6EWWclhu0aOh32VxhkwEJvzuWPeak= github.com/onsi/gomega v1.13.0/go.mod h1:lRk9szgn8TxENtWd0Tp4c3wjlRfMTMH27I+3Je41yGY= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= @@ -1026,8 +990,6 @@ github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6J github.com/pkg/term v1.1.0/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1 h1:CskT+S6Ay54OwxBGB0R3Rsx4Muto6UnEYTyKJbyRIAI= -github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= @@ -1035,9 +997,9 @@ github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.7.1 h1:NTGy1Ja9pByO+xAeH/qiWnLrKtr3hJPNjaVUwnjpdpA= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= github.com/prometheus/client_golang v1.9.0/go.mod h1:FqZLKOZnGdFAhOK4nqGHa7D66IdsO+O441Eve7ptJDU= +github.com/prometheus/client_golang v1.10.0 h1:/o0BDeWzLWXNZ+4q5gXltUvaMpJqckTa+jTNoB+z4cg= github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= @@ -1053,9 +1015,9 @@ github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y8 github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.14.0 h1:RHRyE8UocrbjU+6UvRzwi6HjiDfxrrBU91TtbKzkGp4= github.com/prometheus/common v0.14.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= +github.com/prometheus/common v0.18.0 h1:WCVKW7aL6LEe1uryfI9dnEc2ZqNB1Fn0ok930v0iL1Y= github.com/prometheus/common v0.18.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= @@ -1063,9 +1025,9 @@ github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.1.3 h1:F0+tqvhOksq22sc6iCHF5WGlWjdwj92p0udFh1VFBS8= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.2.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/tsdb v0.6.2-0.20190402121629-4f204dcbc150/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= @@ -1202,9 +1164,6 @@ github.com/vmihailenco/msgpack/v4 v4.3.11 h1:Q47CePddpNGNhk4GCnAx9DDtASi2rasatE0 github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= github.com/vmihailenco/tagparser v0.1.1 h1:quXMXlA39OCbd2wAdTsGDlK9RkOk6Wuw+x37wVyIuWY= github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= -github.com/wangjia184/sortedset v0.0.0-20160527075905-f5d03557ba30/go.mod h1:YkocrP2K2tcw938x9gCOmT5G5eCD6jsTz0SZuyAqwIE= -github.com/warpfork/go-wish v0.0.0-20200122115046-b9ea61034e4a/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= -github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc/go.mod h1:bopw91TMyo8J3tvftk8xmU2kPmlrt4nScJQZU2hE5EM= github.com/whyrusleeping/go-logging v0.0.1/go.mod h1:lDPYj54zutzG1XYfHAhcc7oNXEburHQBn+Iqd4yS4vE= @@ -1258,7 +1217,6 @@ go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM= go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= @@ -1331,8 +1289,8 @@ golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2 h1:Gz96sIWK3OalVv/I/qNygP42zyoKp3xptRVCWRFEBvo= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180719180050-a680a1efc54d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1383,7 +1341,6 @@ golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210423184538-5f58ad60dda6 h1:0PC75Fz/kyMGhL0e1QnypqK2kQMqKt9csD1GnMJR+Zk= golang.org/x/net v0.0.0-20210423184538-5f58ad60dda6/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210428140749-89ef3d95e781 h1:DzZ89McO9/gWPsQXS/FVKAlG02ZjaQ6AlZRBimEYOd0= golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= @@ -1564,8 +1521,8 @@ golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200828161849-5deb26317202/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a h1:CB3a9Nez8M13wwlr/E2YtwoU+qYHKfC+JrDa45RXXoQ= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.1 h1:wGiQel/hW0NnEkJUk8lbzkX2gFJU6PFxf1v5OlCfuOs= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1717,8 +1674,8 @@ gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= @@ -1730,7 +1687,6 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4 h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= pgregory.net/rapid v0.4.7 h1:MTNRktPuv5FNqOO151TM9mDTa+XHcX6ypYeISDVD14g= pgregory.net/rapid v0.4.7/go.mod h1:UYpPVyjFHzYBGHIxLFoupi8vwk6rXNzRY9OMvVxFIOU= diff --git a/module/id/filtered_provider.go b/module/id/filtered_provider.go new file mode 100644 index 00000000000..d935125232f --- /dev/null +++ b/module/id/filtered_provider.go @@ -0,0 +1,18 @@ +package id + +import ( + "github.com/onflow/flow-go/model/flow" +) + +type FilteredIdentifierProvider struct { + filter flow.IdentityFilter + identityProvider IdentityProvider +} + +func NewFilteredIdentifierProvider(filter flow.IdentityFilter, identityProvider IdentityProvider) *FilteredIdentifierProvider { + return &FilteredIdentifierProvider{filter, identityProvider} +} + +func (p *FilteredIdentifierProvider) Identifiers() flow.IdentifierList { + return p.identityProvider.Identities(p.filter).NodeIDs() +} diff --git a/module/id/id_provider.go b/module/id/id_provider.go index 430780b6dc8..a663a8f9215 100644 --- a/module/id/id_provider.go +++ b/module/id/id_provider.go @@ -1,148 +1,13 @@ -package identity +package id import ( - "sync" - - "github.com/libp2p/go-libp2p-core/peer" - "github.com/libp2p/go-libp2p-core/peerstore" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/state/protocol/events" ) type IdentifierProvider interface { Identifiers() flow.IdentifierList } -// TODO: rename to ProtocolStateIDProvider -type ProtocolStateIdentifierProvider struct { - events.Noop - identities flow.IdentityList // TODO: actually we *can* just use identifiers after all - state protocol.State - mu sync.RWMutex - filter flow.IdentityFilter - peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! - flowIDs map[peer.ID]flow.Identifier -} - -func WithFilter(filter flow.IdentityFilter) ProtocolStateIdentifierProviderOption { - return func(provider *ProtocolStateIdentifierProvider) { - provider.filter = filter - } -} - -type ProtocolStateIdentifierProviderOption func(*ProtocolStateIdentifierProvider) - -// TODO: this one also implements IDTranslator!!! -func NewProtocolStateIdentifierProvider( - state protocol.State, - eventDistributer *events.Distributor, - opts ...ProtocolStateIdentifierProviderOption, -) (*ProtocolStateIdentifierProvider, error) { - provider := &ProtocolStateIdentifierProvider{ - state: state, - filter: filter.Any, - } - - for _, opt := range opts { - opt(provider) - } - - head, err := state.Final().Head() - if err != nil { - return nil, err // TODO: format the error - } - - provider.update(head.ID()) - eventDistributer.AddConsumer(provider) - - return provider, nil -} - -func (p *ProtocolStateIdentifierProvider) EpochTransition(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) update(blockID flow.Identifier) { - identities, err := p.state.AtBlockID(blockID).Identities(p.filter) - if err != nil { - // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. - } - - nIds := identities.Count() - - peerIDs := make(map[flow.Identifier]peer.ID, nIds) - flowIDs := make(map[peer.ID]flow.Identifier, nIds) - - for _, identity := range identities { - pid, err := p2p.IdentityToPeerID(identity) - if err != nil { - // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping - } - - flowIDs[pid] = identity.NodeID - peerIDs[identity.NodeID] = pid - } - - p.mu.Lock() - defer p.mu.Unlock() - p.identities = identities - p.flowIDs = flowIDs - p.peerIDs = peerIDs -} - -func (p *ProtocolStateIdentifierProvider) Identifiers() flow.IdentifierList { - p.mu.RLock() - defer p.mu.RUnlock() - return p.identities.NodeIDs() -} - -func (p *ProtocolStateIdentifierProvider) GetPeerID(flowID flow.Identifier) (pid peer.ID, found bool) { - p.mu.RLock() - defer p.mu.RUnlock() - - pid, found = p.peerIDs[flowID] - return -} - -func (p *ProtocolStateIdentifierProvider) GetFlowID(peerID peer.ID) (fid flow.Identifier, found bool) { - p.mu.RLock() - defer p.mu.RUnlock() - - fid, found = p.flowIDs[peerID] - return -} - -type PeerstoreIdentifierProvider struct { - store peerstore.Peerstore - idTranslator p2p.IDTranslator -} - -func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator p2p.IDTranslator) *PeerstoreIdentifierProvider { - return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} -} - -func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { - var result flow.IdentifierList - - pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? - for _, pid := range pids { - flowID, success := p.idTranslator.GetFlowID(pid) - if success { - result = append(result, flowID) - } - } - - return result +type IdentityProvider interface { + Identities(flow.IdentityFilter) flow.IdentityList } diff --git a/network/middleware.go b/network/middleware.go index ed935995e6c..83695402162 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -59,8 +59,6 @@ type Middleware interface { type Overlay interface { // Topology returns an identity list of nodes which this node should be directly connected to as peers Topology() (flow.IdentityList, error) - // Identity returns a map of all identifier to flow identity - Identity() (map[flow.Identifier]flow.Identity, error) Receive(nodeID flow.Identifier, msg *message.Message) error } diff --git a/network/p2p/idTranslator.go b/network/p2p/idTranslator.go deleted file mode 100644 index b8e36f16e35..00000000000 --- a/network/p2p/idTranslator.go +++ /dev/null @@ -1,51 +0,0 @@ -package p2p - -import ( - "github.com/libp2p/go-libp2p-core/peer" - - "github.com/onflow/flow-go/model/flow" -) - -type IDTranslator interface { - GetPeerID(flow.Identifier) (peer.ID, bool) - GetFlowID(peer.ID) (flow.Identifier, bool) -} - -type UnstakedNetworkPeerIDProvider struct { - // basically, just do the conversion from flow ID to peer ID using whatever scheme we talked about. - // whether this be, with caching or not -} - -func NewUnstakedNetworkPeerIDProvider() *UnstakedNetworkPeerIDProvider { - return &UnstakedNetworkPeerIDProvider{} -} - -func GetPeerID(flowID flow.Identifier) (peer.ID, bool) { - Flow - return -} - -func GetFlowID(peerID peer.ID) (flow.Identifier, bool) { - Flow - return -} - -func FlowIDToPeerID(flowID flow.Identifier) peer.ID { - -} - -func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { - pk, err := PublicKey(id.NetworkPubKey) - if err != nil { - // TODO: format the error - return - } - - pid, err = peer.IDFromPublicKey(pk) - if err != nil { - // TODO: format the error - return - } - - return -} diff --git a/network/p2p/id_translator.go b/network/p2p/id_translator.go new file mode 100644 index 00000000000..8edb2d500d5 --- /dev/null +++ b/network/p2p/id_translator.go @@ -0,0 +1,12 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type IDTranslator interface { + GetPeerID(flow.Identifier) (peer.ID, error) + GetFlowID(peer.ID) (flow.Identifier, error) +} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index fecc247945e..5b6ffb52ef7 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -533,7 +533,6 @@ func (n *Node) UpdateAllowList(identities flow.IdentityList) error { // if the node was so far not under allowList if n.connGater == nil { return fmt.Errorf("could not add an allow list, this node was started without allow listing") - } // generates peer address information for all identities diff --git a/network/p2p/network.go b/network/p2p/network.go index fae4a03bd34..9eaca883ee9 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -12,7 +12,9 @@ import ( "github.com/onflow/flow-go/crypto/hash" channels "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" @@ -22,6 +24,14 @@ import ( const DefaultCacheSize = 10e6 +// NetworkingSetFilter is an identity filter that, when applied to the identity +// table at a given snapshot, returns all nodes that we should communicate with +// over the networking layer. +// +// NOTE: The protocol state includes nodes from the previous/next epoch that should +// be included in network communication. We omit any nodes that have been ejected. +var NetworkingSetFilter = filter.Not(filter.Ejected) + type ReadyDoneAwareNetwork interface { module.Network module.ReadyDoneAware @@ -33,7 +43,6 @@ type Network struct { sync.RWMutex logger zerolog.Logger codec network.Codec - ids flow.IdentityList me module.Local mw network.Middleware top network.Topology // used to determine fanout connections @@ -44,7 +53,7 @@ type Network struct { cancel context.CancelFunc subMngr network.SubscriptionManager // used to keep track of subscribed channels lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle - ReadyDoneAwareNetwork + idProvider id.IdentifierProvider } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -54,7 +63,7 @@ type Network struct { func NewNetwork( log zerolog.Logger, codec network.Codec, - ids flow.IdentityList, + idProvider id.IdentifierProvider, me module.Local, mw network.Middleware, csize int, @@ -80,7 +89,6 @@ func NewNetwork( lifecycleManager: lifecycle.NewLifecycleManager(), } o.ctx, o.cancel = context.WithCancel(context.Background()) - o.ids = ids // setup the message queue // create priority queue @@ -154,15 +162,12 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -// Identity returns a map of all flow.Identifier to flow identity by querying the flow state -func (n *Network) Identity() (map[flow.Identifier]flow.Identity, error) { - n.RLock() - defer n.RUnlock() - identifierToID := make(map[flow.Identifier]flow.Identity) - for _, id := range n.ids { - identifierToID[id.NodeID] = *id +func (n *Network) RefreshConnectionRules() { + n.logger.Info().Msg("updating network allow list upon identity table change") + err := n.mw.UpdateAllowList() + if err != nil { + n.logger.Err(err).Msg("failed to update network allow list") } - return identifierToID, nil } // Topology returns the identities of a uniform subset of nodes in protocol state using the topology provided earlier. @@ -172,7 +177,7 @@ func (n *Network) Topology() (flow.IdentityList, error) { defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.ids, subscribedChannels) + top, err := n.top.GenerateFanout(n.idProvider.Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } @@ -187,25 +192,6 @@ func (n *Network) Receive(nodeID flow.Identifier, msg *message.Message) error { return nil } -// SetIDs updates the identity list cached by the network layer -func (n *Network) SetIDs(ids flow.IdentityList) error { - - // remove self from id - ids = ids.Filter(n.me.NotMeFilter()) - - n.Lock() - n.ids = ids - n.Unlock() - - // update the allow list - err := n.mw.UpdateAllowList() - if err != nil { - return fmt.Errorf("failed to update middleware allow list: %w", err) - } - - return nil -} - func (n *Network) processNetworkMessage(senderID flow.Identifier, message *message.Message) error { // checks the cache for deduplication and adds the message if not already present if n.rcache.add(message.EventID, network.Channel(message.ChannelID)) { diff --git a/network/p2p/nodeIDRefresher.go b/network/p2p/nodeIDRefresher.go deleted file mode 100644 index 219951ebbfc..00000000000 --- a/network/p2p/nodeIDRefresher.go +++ /dev/null @@ -1,82 +0,0 @@ -package p2p - -import ( - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/state/protocol" -) - -// NodeIDRefresher derives the latest list of flow identities with which the -// network should be communicating based on identity table changes in the -// protocol state. -type NodeIDRefresher struct { - logger zerolog.Logger - state protocol.State - callBack func(flow.IdentityList) error // callBack to call when the id list has changed -} - -func NewNodeIDRefresher(logger zerolog.Logger, state protocol.State, callBack func(list flow.IdentityList) error) *NodeIDRefresher { - return &NodeIDRefresher{ - logger: logger.With().Str("component", "network-refresher").Logger(), - state: state, - callBack: callBack, - } -} - -func (listener *NodeIDRefresher) getLogger(final protocol.Snapshot) zerolog.Logger { - - log := listener.logger - - // retrieve some contextual information for logging - head, err := final.Head() - if err != nil { - log.Error().Err(err).Msg("failed to get finalized header") - return log - } - log = log.With().Uint64("final_height", head.Height).Logger() - - phase, err := listener.state.Final().Phase() - if err != nil { - log.Error().Err(err).Msg("failed to get epoch phase") - return log - } - log = log.With().Str("epoch_phase", phase.String()).Logger() - - return log -} - -// OnIdentityTableChanged updates the networking layer's list of nodes to connect -// to when the identity table changes in the protocol state. -func (listener *NodeIDRefresher) OnIdentityTableChanged() { - - final := listener.state.Final() - log := listener.getLogger(final) - - log.Info().Msg("updating network ids upon identity table change") - - // get the new set of IDs - newIDs, err := final.Identities(NetworkingSetFilter) - if err != nil { - log.Err(err).Msg("failed to determine new identity table after identity table change") - return - } - - // call the registered callback - err = listener.callBack(newIDs) - if err != nil { - log.Err(err).Msg("failed to update network ids on identity table change") - return - } - - log.Info().Msg("successfully updated network ids upon identity table change") -} - -// NetworkingSetFilter is an identity filter that, when applied to the identity -// table at a given snapshot, returns all nodes that we should communicate with -// over the networking layer. -// -// NOTE: The protocol state includes nodes from the previous/next epoch that should -// be included in network communication. We omit any nodes that have been ejected. -var NetworkingSetFilter = filter.Not(filter.Ejected) diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go new file mode 100644 index 00000000000..d39b2ecf595 --- /dev/null +++ b/network/p2p/peerstore_provider.go @@ -0,0 +1,32 @@ +package p2p + +import ( + peerstore "github.com/libp2p/go-libp2p-peerstore" + + "github.com/onflow/flow-go/model/flow" +) + +type PeerstoreIdentifierProvider struct { + store peerstore.Peerstore + idTranslator IDTranslator +} + +func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} +} + +func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { + var result flow.IdentifierList + + pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? + for _, pid := range pids { + flowID, err := p.idTranslator.GetFlowID(pid) + if err != nil { + // TODO: log error + } else { + result = append(result, flowID) + } + } + + return result +} diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go new file mode 100644 index 00000000000..739958684db --- /dev/null +++ b/network/p2p/protocol_state_provider.go @@ -0,0 +1,129 @@ +package p2p + +import ( + "fmt" + "sync" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" +) + +type ProtocolStateIDCache struct { + events.Noop + identities flow.IdentityList + state protocol.State + mu sync.RWMutex + peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + flowIDs map[peer.ID]flow.Identifier +} + +func NewProtocolStateIDCache( + state protocol.State, + eventDistributer *events.Distributor, +) (*ProtocolStateIDCache, error) { + provider := &ProtocolStateIDCache{ + state: state, + } + + head, err := state.Final().Head() + if err != nil { + return nil, err // TODO: format the error + } + + provider.update(head.ID()) + eventDistributer.AddConsumer(provider) + + return provider, nil +} + +func (p *ProtocolStateIDCache) EpochTransition(_ uint64, header *flow.Header) { + // TODO: maybe we actually want to log the epoch information from the arguments here (epoch phase, etc) + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { + // TODO: log status here + identities, err := p.state.AtBlockID(blockID).Identities(filter.Any) + if err != nil { + // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + } + + nIds := identities.Count() + + peerIDs := make(map[flow.Identifier]peer.ID, nIds) + flowIDs := make(map[peer.ID]flow.Identifier, nIds) + + for _, identity := range identities { + pid, err := IdentityToPeerID(identity) + if err != nil { + // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + } + + flowIDs[pid] = identity.NodeID + peerIDs[identity.NodeID] = pid + } + + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities + p.flowIDs = flowIDs + p.peerIDs = peerIDs +} + +func (p *ProtocolStateIDCache) Identities(filter flow.IdentityFilter) flow.IdentityList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.Filter(filter) +} + +func (p *ProtocolStateIDCache) GetPeerID(flowID flow.Identifier) (pid peer.ID, err error) { + p.mu.RLock() + defer p.mu.RUnlock() + + pid, found := p.peerIDs[flowID] + if !found { + err = fmt.Errorf("flow ID %v was not found in cached identity list", flowID) + } + + return +} + +func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, err error) { + p.mu.RLock() + defer p.mu.RUnlock() + + fid, found := p.flowIDs[peerID] + if !found { + err = fmt.Errorf("peer ID %v was not found in cached identity list", peerID) + } + + return +} + +func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { + pk, err := PublicKey(id.NetworkPubKey) + if err != nil { + // TODO: format the error + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + // TODO: format the error + return + } + + return +} diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go new file mode 100644 index 00000000000..2098824de26 --- /dev/null +++ b/network/p2p/unstaked_translator.go @@ -0,0 +1,43 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + "github.com/multiformats/go-multihash" + "github.com/libp2p/go-libp2p-core/crypto/pb" + + "github.com/onflow/flow-go/model/flow" +) + +type UnstakedNetworkPeerIDProvider struct{} + +func NewUnstakedNetworkPeerIDProvider() *UnstakedNetworkPeerIDProvider { + return &UnstakedNetworkPeerIDProvider{} +} + +func GetPeerID(flowID flow.Identifier) (peer.ID, error) { + data := append([]byte{0x02}, flowID[:]...) + mh, err := multihash.Sum(data, multihash.IDENTITY, -1) + if err != nil { + // TODO: return error + } + + return peer.ID(mh), nil +} + +func GetFlowID(peerID peer.ID) (flow.Identifier, error) { + pk, err := peerID.ExtractPublicKey() + if err != nil { + // return error + } + + if pk.Type() != crypto_pb.KeyType_ECDSA { + // fail + } + + data, err := pk.Raw() + if err != nil || data[0] != 0x02 { // TODO: check if this is the right byte to check + // fail + } + + return flow.HashToID(data[1:]), nil +} From 3b82cfe620d8cc1d2922e358496b8f1bbd4be163 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 19:37:28 -0700 Subject: [PATCH 011/291] fix deps --- go.mod | 4 ++-- go.sum | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 605be87488d..fb6bb13d4a7 100644 --- a/go.mod +++ b/go.mod @@ -27,8 +27,8 @@ require ( github.com/jrick/bitset v1.0.0 github.com/kr/text v0.2.0 // indirect github.com/libp2p/go-addr-util v0.1.0 - github.com/libp2p/go-libp2p v0.14.4 - github.com/libp2p/go-libp2p-core v0.9.0 + github.com/libp2p/go-libp2p v0.14.1 + github.com/libp2p/go-libp2p-core v0.8.5 github.com/libp2p/go-libp2p-discovery v0.5.0 github.com/libp2p/go-libp2p-peerstore v0.2.8 github.com/libp2p/go-libp2p-pubsub v0.4.1 diff --git a/go.sum b/go.sum index 73ac55282ef..56f6f3321f6 100644 --- a/go.sum +++ b/go.sum @@ -623,6 +623,8 @@ github.com/libp2p/go-libp2p-netutil v0.1.0 h1:zscYDNVEcGxyUpMd0JReUZTrpMfia8PmLK github.com/libp2p/go-libp2p-netutil v0.1.0/go.mod h1:3Qv/aDqtMLTUyQeundkKsA+YCThNdbQD54k3TqjpbFU= github.com/libp2p/go-libp2p-noise v0.2.0 h1:wmk5nhB9a2w2RxMOyvsoKjizgJOEaJdfAakr0jN8gds= github.com/libp2p/go-libp2p-noise v0.2.0/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhUeqlO8lVSREYu2Q= +github.com/libp2p/go-libp2p-noise v0.2.2 h1:MRt5XGfYziDXIUy2udtMWfPmzZqUDYoC1FZoKnqPzwk= +github.com/libp2p/go-libp2p-noise v0.2.2/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhUeqlO8lVSREYu2Q= github.com/libp2p/go-libp2p-peer v0.2.0/go.mod h1:RCffaCvUyW2CJmG2gAWVqwePwW7JMgxjsHm7+J5kjWY= github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= github.com/libp2p/go-libp2p-peerstore v0.1.3/go.mod h1:BJ9sHlm59/80oSkpWgr1MyY1ciXAXV397W6h1GH/uKI= From eae8a1209c0c3d8ac4ee6fc64f98e64ecd3271e1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 19:37:46 -0700 Subject: [PATCH 012/291] deps --- go.mod | 2 +- go.sum | 9 ++------- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/go.mod b/go.mod index fb6bb13d4a7..8edf44ffee0 100644 --- a/go.mod +++ b/go.mod @@ -28,7 +28,7 @@ require ( github.com/kr/text v0.2.0 // indirect github.com/libp2p/go-addr-util v0.1.0 github.com/libp2p/go-libp2p v0.14.1 - github.com/libp2p/go-libp2p-core v0.8.5 + github.com/libp2p/go-libp2p-core v0.8.6 github.com/libp2p/go-libp2p-discovery v0.5.0 github.com/libp2p/go-libp2p-peerstore v0.2.8 github.com/libp2p/go-libp2p-pubsub v0.4.1 diff --git a/go.sum b/go.sum index 56f6f3321f6..7c063f01886 100644 --- a/go.sum +++ b/go.sum @@ -561,9 +561,8 @@ github.com/libp2p/go-libp2p v0.6.1/go.mod h1:CTFnWXogryAHjXAKEbOf1OWY+VeAP3lDMZk github.com/libp2p/go-libp2p v0.7.0/go.mod h1:hZJf8txWeCduQRDC/WSqBGMxaTHCOYHt2xSU1ivxn0k= github.com/libp2p/go-libp2p v0.7.4/go.mod h1:oXsBlTLF1q7pxr+9w6lqzS1ILpyHsaBPniVO7zIHGMw= github.com/libp2p/go-libp2p v0.8.1/go.mod h1:QRNH9pwdbEBpx5DTJYg+qxcVaDMAz3Ee/qDKwXujH5o= +github.com/libp2p/go-libp2p v0.14.1 h1:R0vNY7nkU8IISlDuHd2yk4eNAZsVQ0rCr2bPfWU3sXo= github.com/libp2p/go-libp2p v0.14.1/go.mod h1:0PQMADQEjCM2l8cSMYDpTgsb8gr6Zq7i4LUgq1mlW2E= -github.com/libp2p/go-libp2p v0.14.4 h1:QCJE+jGyqxWdrSPuS4jByXCzosgaIg4SJTLCRplJ53w= -github.com/libp2p/go-libp2p v0.14.4/go.mod h1:EIRU0Of4J5S8rkockZM7eJp2S0UrCyi55m2kJVru3rM= github.com/libp2p/go-libp2p-autonat v0.1.1/go.mod h1:OXqkeGOY2xJVWKAGV2inNF5aKN/djNA3fdpCWloIudE= github.com/libp2p/go-libp2p-autonat v0.2.0/go.mod h1:DX+9teU4pEEoZUqR1PiMlqliONQdNbfzE1C718tcViI= github.com/libp2p/go-libp2p-autonat v0.2.1/go.mod h1:MWtAhV5Ko1l6QBsHQNSuM6b1sRkXrpk0/LqCr+vCVxI= @@ -600,9 +599,8 @@ github.com/libp2p/go-libp2p-core v0.8.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJB github.com/libp2p/go-libp2p-core v0.8.1/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.2/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-core v0.8.5/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= +github.com/libp2p/go-libp2p-core v0.8.6 h1:3S8g006qG6Tjpj1JdRK2S+TWc2DJQKX/RG9fdLeiLSU= github.com/libp2p/go-libp2p-core v0.8.6/go.mod h1:dgHr0l0hIKfWpGpqAMbpo19pen9wJfdCGv51mTmdpmM= -github.com/libp2p/go-libp2p-core v0.9.0 h1:t97Mv0LIBZlP2FXVRNKKVzHJCIjbIWGxYptGId4+htU= -github.com/libp2p/go-libp2p-core v0.9.0/go.mod h1:ESsbz31oC3C1AvMJoGx26RTuCkNhmkSRCqZ0kQtJ2/8= github.com/libp2p/go-libp2p-crypto v0.1.0/go.mod h1:sPUokVISZiy+nNuTTH/TY+leRSxnFj/2GLjtOTW90hI= github.com/libp2p/go-libp2p-discovery v0.2.0/go.mod h1:s4VGaxYMbw4+4+tsoQTqh7wfxg97AEdo4GYBt6BadWg= github.com/libp2p/go-libp2p-discovery v0.3.0/go.mod h1:o03drFnz9BVAZdzC/QUQ+NeQOu38Fu7LJGEOK2gQltw= @@ -623,8 +621,6 @@ github.com/libp2p/go-libp2p-netutil v0.1.0 h1:zscYDNVEcGxyUpMd0JReUZTrpMfia8PmLK github.com/libp2p/go-libp2p-netutil v0.1.0/go.mod h1:3Qv/aDqtMLTUyQeundkKsA+YCThNdbQD54k3TqjpbFU= github.com/libp2p/go-libp2p-noise v0.2.0 h1:wmk5nhB9a2w2RxMOyvsoKjizgJOEaJdfAakr0jN8gds= github.com/libp2p/go-libp2p-noise v0.2.0/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhUeqlO8lVSREYu2Q= -github.com/libp2p/go-libp2p-noise v0.2.2 h1:MRt5XGfYziDXIUy2udtMWfPmzZqUDYoC1FZoKnqPzwk= -github.com/libp2p/go-libp2p-noise v0.2.2/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhUeqlO8lVSREYu2Q= github.com/libp2p/go-libp2p-peer v0.2.0/go.mod h1:RCffaCvUyW2CJmG2gAWVqwePwW7JMgxjsHm7+J5kjWY= github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= github.com/libp2p/go-libp2p-peerstore v0.1.3/go.mod h1:BJ9sHlm59/80oSkpWgr1MyY1ciXAXV397W6h1GH/uKI= @@ -731,7 +727,6 @@ github.com/libp2p/go-tcp-transport v0.1.0/go.mod h1:oJ8I5VXryj493DEJ7OsBieu8fcg2 github.com/libp2p/go-tcp-transport v0.1.1/go.mod h1:3HzGvLbx6etZjnFlERyakbaYPdfjg2pWP97dFZworkY= github.com/libp2p/go-tcp-transport v0.2.0/go.mod h1:vX2U0CnWimU4h0SGSEsg++AzvBcroCGYw28kh94oLe0= github.com/libp2p/go-tcp-transport v0.2.1/go.mod h1:zskiJ70MEfWz2MKxvFB/Pv+tPIB1PpPUrHIWQ8aFw7M= -github.com/libp2p/go-tcp-transport v0.2.4/go.mod h1:9dvr03yqrPyYGIEN6Dy5UvdJZjyPFvl1S/igQ5QD1SU= github.com/libp2p/go-tcp-transport v0.2.7 h1:Z8Kc/Kb8tD84WiaH55xAlaEnkqzrp88jSEySCKV4+gg= github.com/libp2p/go-tcp-transport v0.2.7/go.mod h1:lue9p1b3VmZj1MhhEGB/etmvF/nBQ0X9CW2DutBT3MM= github.com/libp2p/go-ws-transport v0.2.0/go.mod h1:9BHJz/4Q5A9ludYWKoGCFC5gUElzlHoKzu0yY9p/klM= From bcdfd2ebae8637b6b034aeeab8b9cd9c173b91d1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 21:01:38 -0700 Subject: [PATCH 013/291] fix middleware --- cmd/scaffold.go | 2 +- network/middleware.go | 4 +- network/p2p/connGater.go | 8 +- network/p2p/libp2pNode.go | 93 ++++++--------------- network/p2p/middleware.go | 108 ++++++++++--------------- network/p2p/network.go | 13 +-- network/p2p/peer | 0 network/p2p/protocol_state_provider.go | 4 +- 8 files changed, 78 insertions(+), 154 deletions(-) create mode 100644 network/p2p/peer diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a96152ba625..400c6fb62f5 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -198,7 +198,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Network = net - idEvents := gadgets.NewIdentityDeltas(net.RefreshConnectionRules) + idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) fnb.ProtocolEvents.AddConsumer(idEvents) return net, err diff --git a/network/middleware.go b/network/middleware.go index 83695402162..ee8ccdfd383 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -7,6 +7,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network/message" + "github.com/onflow/flow-go/module/id" ) // Topic is the internal type of Libp2p which corresponds to the Channel in the network level. @@ -51,12 +52,13 @@ type Middleware interface { // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. - UpdateAllowList() error + UpdateAllowList() } // Overlay represents the interface that middleware uses to interact with the // overlay network layer. type Overlay interface { + id.IdentifierProvider // Topology returns an identity list of nodes which this node should be directly connected to as peers Topology() (flow.IdentityList, error) Receive(nodeID flow.Identifier, msg *message.Message) error diff --git a/network/p2p/connGater.go b/network/p2p/connGater.go index 10a944a5f04..caa05bb1f1a 100644 --- a/network/p2p/connGater.go +++ b/network/p2p/connGater.go @@ -29,14 +29,14 @@ func NewConnGater(log zerolog.Logger) *ConnGater { } // update updates the peer ID map -func (c *ConnGater) update(peerInfos []peer.AddrInfo) { +func (c *ConnGater) update(pids []peer.ID) { // create a new peer.ID map - peerIDs := make(map[peer.ID]struct{}, len(peerInfos)) + peerIDs := make(map[peer.ID]struct{}, len(pids)) // for each peer.AddrInfo, create an entry in the map for the peer.ID - for _, p := range peerInfos { - peerIDs[p.ID] = struct{}{} + for _, pid := range pids { + peerIDs[pid] = struct{}{} } // cache the new map diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 5b6ffb52ef7..596b37752a3 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -285,13 +285,8 @@ func (n *Node) Stop() (chan struct{}, error) { } // AddPeer adds a peer to this node by adding it to this node's peerstore and connecting to it -func (n *Node) AddPeer(ctx context.Context, identity flow.Identity) error { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return fmt.Errorf("failed to add peer %s: %w", identity.String(), err) - } - - err = n.host.Connect(ctx, pInfo) +func (n *Node) AddPeer(ctx context.Context, peerID peer.ID) error { + err := n.host.Connect(ctx, peer.AddrInfo{ID: peerID}) if err != nil { return err } @@ -299,51 +294,29 @@ func (n *Node) AddPeer(ctx context.Context, identity flow.Identity) error { return nil } -// RemovePeer closes the connection with the identity. -func (n *Node) RemovePeer(ctx context.Context, identity flow.Identity) error { - pInfo, err := PeerAddressInfo(identity) +// RemovePeer closes the connection with the peer. +func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { + err := n.host.Network().ClosePeer(peerID) if err != nil { - return fmt.Errorf("failed to remove peer %x: %w", identity, err) - } - - err = n.host.Network().ClosePeer(pInfo.ID) - if err != nil { - return fmt.Errorf("failed to remove peer %s: %w", identity, err) + return fmt.Errorf("failed to remove peer %s: %w", peerID, err) } return nil } -// CreateStream returns an existing stream connected to identity, if it exists or adds one to identity as a peer and creates a new stream with it. -func (n *Node) CreateStream(ctx context.Context, nodeID flow.Identifier) (libp2pnet.Stream, error) { +// CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. +func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) - stream, err := n.tryCreateNewStream(ctx, nodeID, maxConnectAttempt) + stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) if err != nil { - n.host.Peerstore().Addrs() - return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (node_id: %s, address: %s): %w", identity.NodeID.String(), - identity.Address, err)) + return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (peer_id: %s): %w", peerID, err)) } return stream, nil } -func (n *Node) GetPeerIPPort(nodeID flow.Identifier) (string, string, error) { - // TODO: first get peer ID - x := n.host.Peerstore().Addrs() -} - -// tryCreateNewStream makes at most maxAttempts to create a stream with the identity. +// tryCreateNewStream makes at most maxAttempts to create a stream with the peer. // This was put in as a fix for #2416. PubSub and 1-1 communication compete with each other when trying to connect to // remote nodes and once in a while NewStream returns an error 'both yamux endpoints are clients' -func (n *Node) tryCreateNewStream(ctx context.Context, identity flow.Identity, maxAttempts int) (libp2pnet.Stream, error) { - _, _, key, err := networkingInfo(identity) - if err != nil { - return nil, fmt.Errorf("could not get translate identity to networking info %s: %w", identity.NodeID.String(), err) - } - - peerID, err := peer.IDFromPublicKey(key) - if err != nil { - return nil, fmt.Errorf("could not get peer ID: %w", err) - } - +func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttempts int) (libp2pnet.Stream, error) { // protect the underlying connection from being inadvertently pruned by the peer manager while the stream and // connection creation is being attempted n.connMgr.ProtectPeer(peerID) @@ -376,7 +349,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, identity flow.Identity, m time.Sleep(time.Duration(r) * time.Millisecond) } - err = n.AddPeer(ctx, identity) + err := n.AddPeer(ctx, peerID) if err != nil { // if the connection was rejected due to invalid node id, skip the re-attempt @@ -498,23 +471,18 @@ func (n *Node) Publish(ctx context.Context, topic flownet.Topic, data []byte) er } // Ping pings a remote node and returns the time it took to ping the remote node if successful or the error -func (n *Node) Ping(ctx context.Context, identity flow.Identity) (message.PingResponse, time.Duration, error) { - +func (n *Node) Ping(ctx context.Context, peerID peer.ID) (message.PingResponse, time.Duration, error) { pingError := func(err error) error { - return fmt.Errorf("failed to ping %s (%s): %w", identity.NodeID.String(), identity.Address, err) + return fmt.Errorf("failed to ping peer %s: %w", peerID, err) } - // convert the target node address to libp2p peer info - targetInfo, err := PeerAddressInfo(identity) - if err != nil { - return message.PingResponse{}, -1, pingError(err) - } + targetInfo := peer.AddrInfo{ID: peerID} n.connMgr.ProtectPeer(targetInfo.ID) defer n.connMgr.UnprotectPeer(targetInfo.ID) // connect to the target node - err = n.host.Connect(ctx, targetInfo) + err := n.host.Connect(ctx, targetInfo) if err != nil { return message.PingResponse{}, -1, pingError(err) } @@ -529,24 +497,13 @@ func (n *Node) Ping(ctx context.Context, identity flow.Identity) (message.PingRe } // UpdateAllowList allows the peer allow list to be updated. -func (n *Node) UpdateAllowList(identities flow.IdentityList) error { - // if the node was so far not under allowList +func (n *Node) UpdateAllowList(peers []peer.ID) { if n.connGater == nil { - return fmt.Errorf("could not add an allow list, this node was started without allow listing") + n.logger.Debug().Hex("node_id", logging.ID(n.id)).Msg("skipping update allow list, connection gating is not enabled") + return } - // generates peer address information for all identities - allowlist := make([]peer.AddrInfo, len(identities)) - var err error - for i, identity := range identities { - allowlist[i], err = PeerAddressInfo(*identity) - if err != nil { - return fmt.Errorf("could not generate address info: %w", err) - } - } - - n.connGater.update(allowlist) - return nil + n.connGater.update(peers) } // Host returns pointer to host object of node. @@ -565,13 +522,9 @@ func (n *Node) SetPingStreamHandler(handler libp2pnet.StreamHandler) { } // IsConnected returns true is address is a direct peer of this node else false -func (n *Node) IsConnected(identity flow.Identity) (bool, error) { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return false, err - } +func (n *Node) IsConnected(peerID peer.ID) (bool, error) { // query libp2p for connectedness status of this peer - isConnected := n.host.Network().Connectedness(pInfo.ID) == libp2pnet.Connected + isConnected := n.host.Network().Connectedness(peerID) == libp2pnet.Connected return isConnected, nil } diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index ba9fd508eea..35ea77fd07a 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -12,6 +12,7 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" "github.com/rs/zerolog" + "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" @@ -77,6 +78,7 @@ type Middleware struct { unicastMessageTimeout time.Duration connectionGating bool managePeerConnections bool + idTranslator IDTranslator } // NewMiddleware creates a new middleware instance @@ -88,7 +90,8 @@ type Middleware struct { // connectionGating if set to True, restricts this node to only talk to other nodes which are part of the identity list // managePeerConnections if set to True, enables the default PeerManager which continuously updates the node's peer connections // validators are the set of the different message validators that each inbound messages is passed through -func NewMiddleware(log zerolog.Logger, +func NewMiddleware( + log zerolog.Logger, libP2PNodeFactory LibP2PFactoryFunc, flowID flow.Identifier, metrics module.NetworkMetrics, @@ -97,6 +100,7 @@ func NewMiddleware(log zerolog.Logger, unicastMessageTimeout time.Duration, connectionGating bool, managePeerConnections bool, + idTranslator IDTranslator, validators ...network.MessageValidator) *Middleware { if len(validators) == 0 { @@ -125,6 +129,7 @@ func NewMiddleware(log zerolog.Logger, unicastMessageTimeout: unicastMessageTimeout, connectionGating: connectionGating, managePeerConnections: managePeerConnections, + idTranslator: idTranslator, } } @@ -135,6 +140,22 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes } } +func (m *Middleware) peerIDs() []peer.ID { + identifiers := m.ov.Identifiers() + result := make([]peer.ID, len(identifiers)) + + for _, fid := range identifiers { + pid, err := m.idTranslator.GetPeerID(fid) + if err != nil { + // TODO: log here + } + + result = append(result, pid) + } + + return result +} + // Me returns the flow identifier of the this middleware func (m *Middleware) Me() flow.Identifier { return m.me @@ -156,17 +177,8 @@ func (m *Middleware) Start(ov network.Overlay) error { m.libP2PNode = libP2PNode m.libP2PNode.SetFlowProtocolStreamHandler(m.handleIncomingStream) - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return fmt.Errorf("could not get identities: %w", err) - } - if m.connectionGating { - err = m.libP2PNode.UpdateAllowList(identityList(idsMap)) - if err != nil { - return fmt.Errorf("could not update approved peer list: %w", err) - } + m.libP2PNode.UpdateAllowList(m.peerIDs()) } if m.managePeerConnections { @@ -220,10 +232,10 @@ func (m *Middleware) Stop() { // Dispatch should be used whenever guaranteed delivery to a specific target is required. Otherwise, Publish is // a more efficient candidate. func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) error { - // translates identifier to identity - targetIdentity, err := m.identity(targetID) + // translates identifier to peer id + peerID, err := m.idTranslator.GetPeerID(targetID) if err != nil { - return fmt.Errorf("could not find identity for target id: %w", err) + return fmt.Errorf("could not find peer id for target id: %w", err) } maxMsgSize := unicastMaxMsgSize(msg) @@ -243,9 +255,9 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) // (streams don't need to be reused and are fairly inexpensive to be created for each send. // A stream creation does NOT incur an RTT as stream negotiation happens as part of the first message // sent out the the receiver - stream, err := m.libP2PNode.CreateStream(ctx, targetIdentity) + stream, err := m.libP2PNode.CreateStream(ctx, peerID) if err != nil { - return fmt.Errorf("failed to create stream for %s :%w", targetID.String(), err) + return fmt.Errorf("failed to create stream for %s :%w", targetID, err) } // create a gogo protobuf writer @@ -254,19 +266,19 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) err = writer.WriteMsg(msg) if err != nil { - return fmt.Errorf("failed to send message to %s: %w", targetID.String(), err) + return fmt.Errorf("failed to send message to %s: %w", targetID, err) } // flush the stream err = bufw.Flush() if err != nil { - return fmt.Errorf("failed to flush stream for %s: %w", targetIdentity.String(), err) + return fmt.Errorf("failed to flush stream for %s: %w", targetID, err) } // close the stream immediately err = stream.Close() if err != nil { - return fmt.Errorf("failed to close the stream for %s: %w", targetIdentity.String(), err) + return fmt.Errorf("failed to close the stream for %s: %w", targetID, err) } // OneToOne communication metrics are reported with topic OneToOne @@ -275,35 +287,6 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) return nil } -// identity returns corresponding identity of an identifier based on overlay identity list. -func (m *Middleware) identity(identifier flow.Identifier) (flow.Identity, error) { - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return flow.Identity{}, fmt.Errorf("could not get identities: %w", err) - } - - // retrieve the flow.Identity for the give flow.ID - flowIdentity, found := idsMap[identifier] - if !found { - return flow.Identity{}, fmt.Errorf("could not get node identity for %s: %w", identifier.String(), err) - } - - return flowIdentity, nil -} - -// identityList translates an identity map into an identity list. -func identityList(identityMap map[flow.Identifier]flow.Identity) flow.IdentityList { - var identities flow.IdentityList - for _, identity := range identityMap { - // casts identity into a local variable to - // avoid shallow copy of the loop variable - id := identity - identities = append(identities, &id) - - } - return identities -} // handleIncomingStream handles an incoming stream from a remote peer // it is a callback that gets called for each incoming stream by libp2p with a new stream object @@ -412,40 +395,33 @@ func (m *Middleware) Publish(msg *message.Message, channel network.Channel) erro // Ping pings the target node and returns the ping RTT or an error func (m *Middleware) Ping(targetID flow.Identifier) (message.PingResponse, time.Duration, error) { - targetIdentity, err := m.identity(targetID) + peerID, err := m.idTranslator.GetPeerID(targetID) if err != nil { - return message.PingResponse{}, -1, fmt.Errorf("could not find identity for target id: %w", err) + return message.PingResponse{}, -1, fmt.Errorf("could not find peer id for target id: %w", err) } - return m.libP2PNode.Ping(m.ctx, targetIdentity) + return m.libP2PNode.Ping(m.ctx, peerID) } // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. -func (m *Middleware) UpdateAllowList() error { - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return fmt.Errorf("could not get identities: %w", err) - } - +func (m *Middleware) UpdateAllowList() { // update libp2pNode's approve lists if this middleware also does connection gating if m.connectionGating { - err = m.libP2PNode.UpdateAllowList(identityList(idsMap)) - if err != nil { - return fmt.Errorf("failed to update approved peer list: %w", err) - } + m.libP2PNode.UpdateAllowList(m.peerIDs()) } // update peer connections if this middleware also does peer management m.peerManagerUpdate() - - return nil } // IsConnected returns true if this node is connected to the node with id nodeID. -func (m *Middleware) IsConnected(identity flow.Identity) (bool, error) { - return m.libP2PNode.IsConnected(identity) +func (m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { + peerID, err := m.idTranslator.GetPeerID(nodeID) + if err != nil { + return false, fmt.Errorf("could not find peer id for target id: %w", err) + } + return m.libP2PNode.IsConnected(peerID) } // unicastMaxMsgSize returns the max permissible size for a unicast message diff --git a/network/p2p/network.go b/network/p2p/network.go index 9eaca883ee9..ce8c74d0b18 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -41,6 +41,7 @@ type ReadyDoneAwareNetwork interface { // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { sync.RWMutex + id.IdentifierProvider logger zerolog.Logger codec network.Codec me module.Local @@ -53,7 +54,6 @@ type Network struct { cancel context.CancelFunc subMngr network.SubscriptionManager // used to keep track of subscribed channels lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle - idProvider id.IdentifierProvider } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -78,6 +78,7 @@ func NewNetwork( } o := &Network{ + IdentifierProvider: idProvider, logger: log, codec: codec, me: me, @@ -162,14 +163,6 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -func (n *Network) RefreshConnectionRules() { - n.logger.Info().Msg("updating network allow list upon identity table change") - err := n.mw.UpdateAllowList() - if err != nil { - n.logger.Err(err).Msg("failed to update network allow list") - } -} - // Topology returns the identities of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. func (n *Network) Topology() (flow.IdentityList, error) { @@ -177,7 +170,7 @@ func (n *Network) Topology() (flow.IdentityList, error) { defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.idProvider.Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } diff --git a/network/p2p/peer b/network/p2p/peer new file mode 100644 index 00000000000..e69de29bb2d diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 739958684db..6933af0db61 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -66,7 +66,7 @@ func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { flowIDs := make(map[peer.ID]flow.Identifier, nIds) for _, identity := range identities { - pid, err := IdentityToPeerID(identity) + pid, err := ExtractPeerID(identity) if err != nil { // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping } @@ -112,7 +112,7 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e return } -func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { +func ExtractPeerID(id *flow.Identity) (pid peer.ID, err error) { pk, err := PublicKey(id.NetworkPubKey) if err != nil { // TODO: format the error From e854e3cfcac9524dbe42b6195e1f7109665f07d9 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Thu, 12 Aug 2021 21:08:10 -0700 Subject: [PATCH 014/291] adding a topic validator test --- network/p2p/subscriptionFilter_test.go | 1 - network/p2p/topicValidator_test.go | 139 +++++++++++++++++++++++++ 2 files changed, 139 insertions(+), 1 deletion(-) create mode 100644 network/p2p/topicValidator_test.go diff --git a/network/p2p/subscriptionFilter_test.go b/network/p2p/subscriptionFilter_test.go index 0fa1bc83a59..198e02f59c1 100644 --- a/network/p2p/subscriptionFilter_test.go +++ b/network/p2p/subscriptionFilter_test.go @@ -47,7 +47,6 @@ func TestBasicSubscriptionFilter(t *testing.T) { ps3, err := pubsub.NewGossipSub(ctx, host3) require.NoError(t, err) - topic1, err := ps1.Join(topicname1) require.NoError(t, err) _, err = topic1.Subscribe() diff --git a/network/p2p/topicValidator_test.go b/network/p2p/topicValidator_test.go new file mode 100644 index 00000000000..1282306e159 --- /dev/null +++ b/network/p2p/topicValidator_test.go @@ -0,0 +1,139 @@ +package p2p + +import ( + "context" + "fmt" + "testing" + "time" + + golog "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p" + "github.com/libp2p/go-libp2p-core/host" + "github.com/libp2p/go-libp2p-core/peer" + pubsub "github.com/libp2p/go-libp2p-pubsub" + "github.com/stretchr/testify/require" +) + +func TestTopicValidator(t *testing.T) { + golog.SetAllLoggers(golog.LevelDebug) + ctx := context.Background() + host1, err := libp2p.New(ctx) + require.NoError(t, err) + host2, err := libp2p.New(ctx) + require.NoError(t, err) + host3, err := libp2p.New(ctx) + require.NoError(t, err) + + require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host2))) + require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host3))) + + + topicname1 := "testtopic1" + topicname2 := "testtopic2" + + + ps1, err := pubsub.NewGossipSub(ctx, host1) + require.NoError(t, err) + + ps2, err := pubsub.NewGossipSub(ctx, host2) + require.NoError(t, err) + + ps3, err := pubsub.NewGossipSub(ctx, host3) + require.NoError(t, err) + + validator := &topicValidator{ + allowedIDs: make(map[peer.ID]struct{}), + blackListFunc: ps1.BlacklistPeer, + } + validator.allowedIDs[host1.ID()] = struct{}{} + validator.allowedIDs[host2.ID()] = struct{}{} + ps1.RegisterTopicValidator(topicname2, validator.validate) + + + topic1, err := ps1.Join(topicname1) + require.NoError(t, err) + _, err = topic1.Subscribe() + require.NoError(t, err) + topic1, err = ps1.Join(topicname2) + require.NoError(t, err) + subscriberHost1Topic2, err := topic1.Subscribe() + require.NoError(t, err) + + topic2, err := ps2.Join(topicname1) + require.NoError(t, err) + _, err = topic2.Subscribe() + require.NoError(t, err) + topic2, err = ps2.Join(topicname2) + require.NoError(t, err) + subscriberHost2Topic2, err := topic2.Subscribe() + require.NoError(t, err) + + topic3, err := ps3.Join(topicname1) + require.NoError(t, err) + _, err = topic3.Subscribe() + require.NoError(t, err) + wrongTopic, err := ps3.Join(topicname2) + require.NoError(t, err) + _, err = wrongTopic.Subscribe() + require.NoError(t, err) + + time.Sleep(2 * time.Second) + + fmt.Printf("host1: %s, host2: %s, host3 :%s\n", host1.ID(), host2.ID(), host3.ID()) + //fmt.Print(host1.Peerstore().Peers()) + //fmt.Print(host2.Peerstore().Peers()) + fmt.Print("host 1 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps1.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps1.ListPeers(topicname2)) + + fmt.Print("host 2 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps2.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps2.ListPeers(topicname2)) + + fmt.Print("host 3 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps3.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps3.ListPeers(topicname2)) + + err = wrongTopic.Publish(ctx, []byte("hello")) + require.NoError(t, err) + + time.Sleep(2 * time.Second) + + + fmt.Print("host 1 peers\n") + fmt.Printf("\t For %s", topicname1) + fmt.Println(ps1.ListPeers(topicname1)) + fmt.Printf("\t For %s", topicname2) + fmt.Println(ps1.ListPeers(topicname2)) + fmt.Println(host1.Peerstore().Peers()) + + msg, err := subscriberHost2Topic2.Next(ctx) + require.NoError(t, err) + fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) + fmt.Println(msg) + + msg, err = subscriberHost1Topic2.Next(ctx) + require.NoError(t, err) + fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) + fmt.Println(msg) + + +} + +type topicValidator struct { + allowedIDs map[peer.ID]struct{} + blackListFunc func(id peer.ID) +} +func(validator *topicValidator) validate(ctx context.Context, from peer.ID, _ *pubsub.Message) bool { + _, found := validator.allowedIDs[from] + if !found { + validator.blackListFunc(from) + } + return found +} From 915347530026b569a1438eeca24bfc89b4de8a45 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:21:51 -0700 Subject: [PATCH 015/291] update to reflect new design --- cmd/node_builder.go | 33 +++++----- cmd/scaffold.go | 1 - integration/testnet/network.go | 114 +++++++++++---------------------- 3 files changed, 54 insertions(+), 94 deletions(-) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index bfd78fe2f3a..7cfad0d4012 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -114,23 +114,22 @@ type BaseConfig struct { type NodeConfig struct { Cancel context.CancelFunc // cancel function for the context that is passed to the networking layer BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware network.Middleware - Network p2p.ReadyDoneAwareNetwork - SubscriptionManager network.SubscriptionManager - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware network.Middleware + Network p2p.ReadyDoneAwareNetwork + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // root state information RootBlock *flow.Block diff --git a/cmd/scaffold.go b/cmd/scaffold.go index f885cdf17fa..8049b9e539e 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -181,7 +181,6 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { } subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) - node.SubscriptionManager = subscriptionManager top, err := topology.NewTopicBasedTopology(fnb.NodeID, fnb.Logger, fnb.State) if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index b1cc9941529..40225acc9e1 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -14,33 +14,31 @@ import ( "testing" "time" - "github.com/onflow/flow-go/cmd/bootstrap/utils" - "github.com/docker/docker/api/types/container" dockerclient "github.com/docker/docker/client" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/onflow/flow-go-sdk/crypto" - "github.com/onflow/flow-go/model/encodable" - "github.com/onflow/flow-go/model/flow/order" - "github.com/onflow/flow-go/utils/io" - "github.com/dapperlabs/testingdock" "github.com/onflow/cadence" + "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/cmd/bootstrap/run" + "github.com/onflow/flow-go/cmd/bootstrap/utils" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" dkgmod "github.com/onflow/flow-go/model/dkg" + "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" clusterstate "github.com/onflow/flow-go/state/cluster" "github.com/onflow/flow-go/state/protocol/inmem" + "github.com/onflow/flow-go/utils/io" "github.com/onflow/flow-go/utils/unittest" ) @@ -67,19 +65,14 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" - // UnstakedNetworkPort is the name used for the access node unstaked libp2p network port. - UnstakedNetworkPort = "access-unstaked-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" // ExeNodeMetricsPort ExeNodeMetricsPort = "exe-metrics-port" - // default staked network port - DefaultStakedFlowPort = 2137 - - // default unstaked network port - DefaultUnstakedFlowPort = 7312 + // default network port + DefaultFlowPort = 2137 DefaultViewsInStakingAuction uint64 = 5 DefaultViewsInDKGPhase uint64 = 50 @@ -312,17 +305,14 @@ func (n *NetworkConfig) Swap(i, j int) { // NodeConfig defines the input config for a particular node, specified prior // to network creation. type NodeConfig struct { - Role flow.Role - Stake uint64 - Identifier flow.Identifier - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - // Unstaked - only applicable to Access Node. Access nodes can be staked or unstaked. - // Unstaked nodes are not part of the identity table - Unstaked bool // only applicable to Access node - ParticipatesInUnstakedNetwork bool + Role flow.Role + Stake uint64 + Identifier flow.Identifier + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + ParticipatesInPublicNetwork bool // only applicable to Access node } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -398,9 +388,9 @@ func AsGhost() func(config *NodeConfig) { } } -func AsUnstakedNetworkParticipant() func(config *NodeConfig) { +func AsPublicNetworkParticipant() func(config *NodeConfig) { return func(config *NodeConfig) { - config.ParticipatesInUnstakedNetwork = true + config.ParticipatesInPublicNetwork = true } } @@ -499,11 +489,8 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, consensus_follower.WithBootstrapDir(followerBootstrapDir), } - // TODO: eventually we will need upstream node's address - // - // upstreamANPort := net.ContainerByID(followerConf.upstreamNodeID).Ports[testnet.UnstakedNetworkPort] - // upstreamANAddress := fmt.Sprintf("127.0.0.1:%d", upstreamANPort) - + // TODO: update consensus follower to just accept a networking key instead of a node ID + // it should be able to figure out the rest on its own. follower := consensus_follower.NewConsensusFollower( followerConf.nodeID, followerConf.upstreamNodeID, @@ -649,13 +636,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort - if nodeConf.ParticipatesInUnstakedNetwork { - hostUnstakedPort := testingdock.RandomPort(t) - containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) - nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) - nodeContainer.addFlag("unstaked-bind-addr", fmt.Sprintf("%s:%d", nodeContainer.Name(), DefaultUnstakedFlowPort)) - nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort - net.AccessPorts[UnstakedNetworkPort] = hostUnstakedPort + if nodeConf.ParticipatesInPublicNetwork { + // TODO: define this flag for Access node + nodeContainer.addFlag("public-network-participant", "true") } case flow.RoleConsensus: @@ -678,18 +661,12 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort - if nodeConf.ParticipatesInUnstakedNetwork { - hostUnstakedPort := testingdock.RandomPort(t) - containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) - - // TODO: Currently, it is not possible to create a staked ghost AN which - // participates on the unstaked network, because the ghost node only joins - // a single network during startup. The ghost node needs to support the - // "unstaked-bind-addr" flag which can be used to specify a bind address - // for the unstaked network. - - nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) - nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + if nodeConf.ParticipatesInPublicNetwork { + // TODO: Currently, it is not possible to create a ghost AN which participates + // in the public network, because connection gating is enabled by default and + // therefore the ghost node will deny incoming connections from all consensus + // followers. A flag for the ghost node will need to be created to enable + // overriding the default behavior. } } @@ -761,29 +738,19 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo sort.Sort(&networkConf) // generate staking and networking keys for each configured node - // NOTE: this includes unstaked access nodes, which need private keys written - // but should not be included in the identity table - allConfs, err := setupKeys(networkConf) + stakedConfs, err := setupKeys(networkConf) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to setup keys: %w", err) } - // only staked configs - this only includes identity table members - stakedConfs := filterContainerConfigs(allConfs, func(config ContainerConfig) bool { - return !config.Unstaked - }) - followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) } - allNodeInfos := toNodeInfos(allConfs) - allNodeInfos = append(allNodeInfos, followerInfos...) + allNodeInfos = append(toNodeInfos(stakedConfs), followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices - // IMPORTANT: these nodes infos must include exactly the identity table - // members (no unstaked access nodes) stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), order.Canonical) // run DKG for all consensus nodes @@ -963,11 +930,7 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { // define the node's name _ and address : name := fmt.Sprintf("%s_%d", conf.Role.String(), roleCounter[conf.Role]+1) - flowPort := DefaultStakedFlowPort - if conf.Unstaked { - flowPort = DefaultUnstakedFlowPort - } - addr := fmt.Sprintf("%s:%d", name, flowPort) + addr := fmt.Sprintf("%s:%d", name, DefaultFlowPort) roleCounter[conf.Role]++ info := bootstrap.NewPrivateNodeInfo( @@ -980,14 +943,13 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - Unstaked: conf.Unstaked, - ParticipatesInUnstakedNetwork: conf.ParticipatesInUnstakedNetwork, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + ParticipatesInPublicNetwork: conf.ParticipatesInPublicNetwork, } confs = append(confs, containerConf) From abcd7ba7a021f8e04d6c6c975904bb00c913a5c7 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:22:49 -0700 Subject: [PATCH 016/291] Update staked_access_node_builder.go --- .../staked_access_node_builder.go | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index d55d1994d63..307be38bde2 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -52,12 +52,6 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { - if anb.ParticipatesInUnstakedNetwork() { - anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) - return node.Network, nil - }) - } anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -75,23 +69,7 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { } return ping, nil }) - if anb.ParticipatesInUnstakedNetwork() { - // create relay engine - anb.Component("relay engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - channels := node.SubscriptionManager.Channels() - if len(channels) == 0 { - return nil, fmt.Errorf("no subscribed channels to relay") - } - relayEngine, err := relay.New(node.Logger, channels, node.Network, anb.UnstakedNetwork) - - if err != nil { - return nil, fmt.Errorf("could not create relay engine: %w", err) - } - - return relayEngine, nil - }) - } return anb } From 86cd570ffa064b81d02e5062922dd344f978aa93 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:31:53 -0700 Subject: [PATCH 017/291] Update consensus_follower.go --- follower/consensus_follower.go | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index e1051091e35..8bf8399bff1 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -7,6 +7,7 @@ import ( "github.com/onflow/flow-go/cmd" access "github.com/onflow/flow-go/cmd/access/node_builder" "github.com/onflow/flow-go/consensus/hotstuff/notifications/pubsub" + "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" ) @@ -22,11 +23,11 @@ type ConsensusFollower interface { // Config contains the configurable fields for a `ConsensusFollower`. type Config struct { - nodeID flow.Identifier // the node ID of this node - upstreamAccessNodeID flow.Identifier // the node ID of the upstream access node - bindAddr string // address to bind on - dataDir string // directory to store the protocol state - bootstrapDir string // path to the bootstrap directory + networkPubKey crypto.PublicKey // the network public key of this node + upstreamAccessNodeID flow.Identifier // the node ID of the upstream access node + bindAddr string // address to bind on + dataDir string // directory to store the protocol state + bootstrapDir string // path to the bootstrap directory } type Option func(c *Config) @@ -46,14 +47,14 @@ func WithBootstrapDir(bootstrapDir string) Option { func getAccessNodeOptions(config *Config) []access.Option { return []access.Option{ access.WithUpstreamAccessNodeID(config.upstreamAccessNodeID), - access.WithUnstakedNetworkBindAddr(config.bindAddr), + access.WithBindAddr(config.bindAddr), access.WithBaseOptions(getBaseOptions(config)), } } func getBaseOptions(config *Config) []cmd.Option { options := []cmd.Option{ - cmd.WithNodeID(config.nodeID), + cmd.WithNetworkPublicKey(config.networkPubKey), cmd.WithMetricsEnabled(false), } if config.bootstrapDir != "" { @@ -84,13 +85,13 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - nodeID flow.Identifier, + networkPublicKey crypto.PublicKey, upstreamAccessNodeID flow.Identifier, bindAddr string, opts ...Option, ) *ConsensusFollowerImpl { config := &Config{ - nodeID: nodeID, + networkPublicKey: networkPublicKey, upstreamAccessNodeID: upstreamAccessNodeID, bindAddr: bindAddr, } From 48a8b9ae8cc2d3f8b5cde4eb35aa845151fee7e5 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:51:21 -0700 Subject: [PATCH 018/291] f --- follower/consensus_follower.go | 2 +- integration/testnet/network.go | 12 ++++++++---- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 8bf8399bff1..db3517fdbe7 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -85,7 +85,7 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - networkPublicKey crypto.PublicKey, + networkPublicKey crypto.PublicKey, // TODO: make this optional. if not explicitly supplied, we can auto-generate one for them. upstreamAccessNodeID flow.Identifier, bindAddr string, opts ...Option, diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 40225acc9e1..ee1f90c2be7 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -27,6 +27,7 @@ import ( "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/cmd/bootstrap/run" "github.com/onflow/flow-go/cmd/bootstrap/utils" + fcrypto "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" @@ -216,13 +217,13 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier + networkKey fcrypto.PrivateKey upstreamNodeID flow.Identifier } -func NewConsensusFollowerConfig(nodeID flow.Identifier, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - nodeID: nodeID, + networkKey: fcrypto.PrivateKey, upstreamNodeID: upstreamNodeID, } } @@ -492,12 +493,13 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. follower := consensus_follower.NewConsensusFollower( - followerConf.nodeID, + followerConf.networkKey, followerConf.upstreamNodeID, bindAddr, opts..., ) + // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower } @@ -696,6 +698,7 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { var nodeInfos []bootstrap.NodeInfo + // TODO: remove this, networking keys should be provided by the consensus follower config. // get networking keys for all followers networkKeys, err := unittest.NetworkingKeys(len(confs)) if err != nil { @@ -710,6 +713,7 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for i, conf := range confs { info := bootstrap.NewPrivateNodeInfo( + // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address From 0ba7f978fa13b9203ef4e3f6ed7f90cc673b346a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 16 Aug 2021 10:21:30 -0700 Subject: [PATCH 019/291] fix more compilation errors --- network/middleware.go | 4 ++-- network/p2p/libp2pConnector.go | 35 +++++++++++-------------------- network/p2p/middleware.go | 34 ++++++++++++++++++++---------- network/p2p/network.go | 22 ++++++++++---------- network/p2p/peerManager.go | 38 +++++++++++++++++----------------- network/topology.go | 8 +++---- 6 files changed, 71 insertions(+), 70 deletions(-) diff --git a/network/middleware.go b/network/middleware.go index ee8ccdfd383..94fdcbfd9e7 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -59,8 +59,8 @@ type Middleware interface { // overlay network layer. type Overlay interface { id.IdentifierProvider - // Topology returns an identity list of nodes which this node should be directly connected to as peers - Topology() (flow.IdentityList, error) + // Topology returns an identifier list of nodes which this node should be directly connected to as peers + Topology() (flow.IdentifierList, error) Receive(nodeID flow.Identifier, msg *message.Message) error } diff --git a/network/p2p/libp2pConnector.go b/network/p2p/libp2pConnector.go index 985bd117f20..8e38ade7f8a 100644 --- a/network/p2p/libp2pConnector.go +++ b/network/p2p/libp2pConnector.go @@ -65,34 +65,23 @@ func newLibp2pConnector(host host.Host, log zerolog.Logger) (*libp2pConnector, e // UpdatePeers is the implementation of the Connector.UpdatePeers function. It connects to all of the ids and // disconnects from any other connection that the libp2p node might have. -func (l *libp2pConnector) UpdatePeers(ctx context.Context, ids flow.IdentityList) error { - - // derive the peer.AddrInfo from each of the flow.Identity - pInfos, invalidIDs := peerInfosFromIDs(ids) - +func (l *libp2pConnector) UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) { // connect to each of the peer.AddrInfo in pInfos - l.connectToPeers(ctx, pInfos) + l.connectToPeers(ctx, peerIDs) // disconnect from any other peers not in pInfos - l.trimAllConnectionsExcept(pInfos) - - // if some ids didn't translate to peer.AddrInfo, return error - if len(invalidIDs) != 0 { - return NewUnconvertableIdentitiesError(invalidIDs) - } - - return nil + l.trimAllConnectionsExcept(peerIDs) } // connectToPeers connects each of the peer in pInfos -func (l *libp2pConnector) connectToPeers(ctx context.Context, pInfos []peer.AddrInfo) { +func (l *libp2pConnector) connectToPeers(ctx context.Context, peerIDs peer.IDSlice) { // create a channel of peer.AddrInfo as expected by the connector - peerCh := make(chan peer.AddrInfo, len(pInfos)) + peerCh := make(chan peer.AddrInfo, len(peerIDs)) // stuff all the peer.AddrInfo it into the channel - for _, peerInfo := range pInfos { - peerCh <- peerInfo + for _, peerID := range peerIDs { + peerCh <- peer.AddrInfo{ID: peerID} } // close the channel to ensure Connect does not block @@ -102,15 +91,15 @@ func (l *libp2pConnector) connectToPeers(ctx context.Context, pInfos []peer.Addr l.backoffConnector.Connect(ctx, peerCh) } -// trimAllConnectionsExcept trims all connections of the node from peers not part of peerInfos. +// trimAllConnectionsExcept trims all connections of the node from peers not part of peerIDs. // A node would have created such extra connections earlier when the identity list may have been different, or // it may have been target of such connections from node which have now been excluded. -func (l *libp2pConnector) trimAllConnectionsExcept(peerInfos []peer.AddrInfo) { +func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs []peer.ID) { // convert the peerInfos to a peer.ID -> bool map - peersToKeep := make(map[peer.ID]bool, len(peerInfos)) - for _, pInfo := range peerInfos { - peersToKeep[pInfo.ID] = true + peersToKeep := make(map[peer.ID]bool, len(peerIDs)) + for _, pid := range peerIDs { + peersToKeep[pid] = true } // get all current node connections diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 35ea77fd07a..f4678198c15 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -11,8 +11,8 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" - "github.com/rs/zerolog" "github.com/libp2p/go-libp2p-core/peer" + "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" @@ -78,7 +78,7 @@ type Middleware struct { unicastMessageTimeout time.Duration connectionGating bool managePeerConnections bool - idTranslator IDTranslator + idTranslator IDTranslator } // NewMiddleware creates a new middleware instance @@ -129,7 +129,7 @@ func NewMiddleware( unicastMessageTimeout: unicastMessageTimeout, connectionGating: connectionGating, managePeerConnections: managePeerConnections, - idTranslator: idTranslator, + idTranslator: idTranslator, } } @@ -140,11 +140,24 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes } } -func (m *Middleware) peerIDs() []peer.ID { - identifiers := m.ov.Identifiers() - result := make([]peer.ID, len(identifiers)) +func (m *Middleware) topologyPeers() (peer.IDSlice, error) { + identifiers, err := m.ov.Topology() + if err != nil { + // TODO: format error + return nil, err + } + + return m.peerIDs(identifiers), nil +} + +func (m *Middleware) allPeers() peer.IDSlice { + return m.peerIDs(m.ov.Identifiers()) +} + +func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { + result := make([]peer.ID, len(flowIDs)) - for _, fid := range identifiers { + for _, fid := range flowIDs { pid, err := m.idTranslator.GetPeerID(fid) if err != nil { // TODO: log here @@ -152,7 +165,7 @@ func (m *Middleware) peerIDs() []peer.ID { result = append(result, pid) } - + return result } @@ -178,7 +191,7 @@ func (m *Middleware) Start(ov network.Overlay) error { m.libP2PNode.SetFlowProtocolStreamHandler(m.handleIncomingStream) if m.connectionGating { - m.libP2PNode.UpdateAllowList(m.peerIDs()) + m.libP2PNode.UpdateAllowList(m.allPeers()) } if m.managePeerConnections { @@ -187,7 +200,7 @@ func (m *Middleware) Start(ov network.Overlay) error { return fmt.Errorf("failed to create libp2pConnector: %w", err) } - m.peerManager = NewPeerManager(m.log, m.ov.Topology, libp2pConnector, WithInterval(m.peerUpdateInterval)) + m.peerManager = NewPeerManager(m.log, m.topologyPeers, libp2pConnector, WithInterval(m.peerUpdateInterval)) select { case <-m.peerManager.Ready(): m.log.Debug().Msg("peer manager successfully started") @@ -287,7 +300,6 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) return nil } - // handleIncomingStream handles an incoming stream from a remote peer // it is a callback that gets called for each incoming stream by libp2p with a new stream object func (m *Middleware) handleIncomingStream(s libp2pnetwork.Stream) { diff --git a/network/p2p/network.go b/network/p2p/network.go index ce8c74d0b18..9cfd4257d03 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -79,15 +79,15 @@ func NewNetwork( o := &Network{ IdentifierProvider: idProvider, - logger: log, - codec: codec, - me: me, - mw: mw, - rcache: rcache, - top: top, - metrics: metrics, - subMngr: sm, - lifecycleManager: lifecycle.NewLifecycleManager(), + logger: log, + codec: codec, + me: me, + mw: mw, + rcache: rcache, + top: top, + metrics: metrics, + subMngr: sm, + lifecycleManager: lifecycle.NewLifecycleManager(), } o.ctx, o.cancel = context.WithCancel(context.Background()) @@ -163,9 +163,9 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -// Topology returns the identities of a uniform subset of nodes in protocol state using the topology provided earlier. +// Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. -func (n *Network) Topology() (flow.IdentityList, error) { +func (n *Network) Topology() (flow.IdentifierList, error) { n.Lock() defer n.Unlock() diff --git a/network/p2p/peerManager.go b/network/p2p/peerManager.go index 031732caadb..c0c7162048f 100644 --- a/network/p2p/peerManager.go +++ b/network/p2p/peerManager.go @@ -5,20 +5,20 @@ import ( "fmt" "time" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" ) // Connector connects to peer and disconnects from peer using the underlying networking library type Connector interface { - // UpdatePeers connects to the given flow.Identities and returns a map of identifiers which failed. It also + // UpdatePeers connects to the given peer.IDs and returns a map of peers which failed. It also // disconnects from any other peers with which it may have previously established connection. // UpdatePeers implementation should be idempotent such that multiple calls to connect to the same peer should not // return an error or create multiple connections - UpdatePeers(ctx context.Context, ids flow.IdentityList) error + UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) } // DefaultPeerUpdateInterval is default duration for which the peer manager waits in between attempts to update peer connections @@ -28,10 +28,10 @@ var DefaultPeerUpdateInterval = 10 * time.Minute type PeerManager struct { unit *engine.Unit logger zerolog.Logger - idsProvider func() (flow.IdentityList, error) // callback to retrieve list of peers to connect to - peerRequestQ chan struct{} // a channel to queue a peer update request - connector Connector // connector to connect or disconnect from peers - peerUpdateInterval time.Duration // interval the peer manager runs on + peersProvider func() (peer.IDSlice, error) // callback to retrieve list of peers to connect to + peerRequestQ chan struct{} // a channel to queue a peer update request + connector Connector // connector to connect or disconnect from peers + peerUpdateInterval time.Duration // interval the peer manager runs on } // Option represents an option for the peer manager. @@ -43,16 +43,16 @@ func WithInterval(period time.Duration) Option { } } -// NewPeerManager creates a new peer manager which calls the idsProvider callback to get a list of peers to connect to +// NewPeerManager creates a new peer manager which calls the peersProvider callback to get a list of peers to connect to // and it uses the connector to actually connect or disconnect from peers. -func NewPeerManager(logger zerolog.Logger, idsProvider func() (flow.IdentityList, error), +func NewPeerManager(logger zerolog.Logger, peersProvider func() (peer.IDSlice, error), connector Connector, options ...Option) *PeerManager { pm := &PeerManager{ - unit: engine.NewUnit(), - logger: logger, - idsProvider: idsProvider, - connector: connector, - peerRequestQ: make(chan struct{}, 1), + unit: engine.NewUnit(), + logger: logger, + peersProvider: peersProvider, + connector: connector, + peerRequestQ: make(chan struct{}, 1), } // apply options for _, o := range options { @@ -101,23 +101,23 @@ func (pm *PeerManager) RequestPeerUpdate() { } } -// updatePeers updates the peers by connecting to all the nodes provided by the idsProvider callback and disconnecting from +// updatePeers updates the peers by connecting to all the nodes provided by the peersProvider callback and disconnecting from // previous nodes that are no longer in the new list of nodes. func (pm *PeerManager) updatePeers() { - // get all the ids to connect to - ids, err := pm.idsProvider() + // get all the peer ids to connect to + peers, err := pm.peersProvider() if err != nil { pm.logger.Error().Err(err).Msg("failed to update peers") return } pm.logger.Trace(). - Str("peers", fmt.Sprintf("%v", ids.NodeIDs())). + Str("peers", fmt.Sprintf("%v", peers)). Msg("connecting to peers") // ask the connector to connect to all peers in the list - err = pm.connector.UpdatePeers(pm.unit.Ctx(), ids) + err = pm.connector.UpdatePeers(pm.unit.Ctx(), peers) if err == nil { return } diff --git a/network/topology.go b/network/topology.go index 1d746c70896..1d0faf16212 100644 --- a/network/topology.go +++ b/network/topology.go @@ -6,9 +6,9 @@ import ( // Topology provides a subset of nodes which a given node should directly connect to for 1-k messaging. type Topology interface { - // GenerateFanout receives IdentityList of entire network, and list of channels the node is subscribing to. - // It constructs and returns the fanout IdentityList of node. - // A node directly communicates with its fanout IdentityList on epidemic dissemination + // GenerateFanout receives IdentifierList of entire network, and list of channels the node is subscribing to. + // It constructs and returns the fanout IdentifierList of node. + // A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. @@ -19,5 +19,5 @@ type Topology interface { // // GenerateFanout is not concurrency safe. It is responsibility of caller to lock for it. // with the channels argument, it allows the returned topology to be cached, which is necessary for randomized topology. - GenerateFanout(ids flow.IdentityList, channels ChannelList) (flow.IdentityList, error) + GenerateFanout(ids flow.IdentifierList, channels ChannelList) (flow.IdentifierList, error) } From 203dc013f54176e4d37779a8463000b70da5cbb4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 16 Aug 2021 10:28:52 -0700 Subject: [PATCH 020/291] more build fixes --- network/p2p/middleware.go | 2 +- network/p2p/peerManager.go | 5 +---- network/topology/cache.go | 16 ++++++++-------- 3 files changed, 10 insertions(+), 13 deletions(-) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index f4678198c15..802a4d13036 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -420,7 +420,7 @@ func (m *Middleware) Ping(targetID flow.Identifier) (message.PingResponse, time. func (m *Middleware) UpdateAllowList() { // update libp2pNode's approve lists if this middleware also does connection gating if m.connectionGating { - m.libP2PNode.UpdateAllowList(m.peerIDs()) + m.libP2PNode.UpdateAllowList(m.allPeers()) } // update peer connections if this middleware also does peer management diff --git a/network/p2p/peerManager.go b/network/p2p/peerManager.go index c0c7162048f..d7bb3366d98 100644 --- a/network/p2p/peerManager.go +++ b/network/p2p/peerManager.go @@ -117,10 +117,7 @@ func (pm *PeerManager) updatePeers() { Msg("connecting to peers") // ask the connector to connect to all peers in the list - err = pm.connector.UpdatePeers(pm.unit.Ctx(), peers) - if err == nil { - return - } + pm.connector.UpdatePeers(pm.unit.Ctx(), peers) if IsUnconvertibleIdentitiesError(err) { // log conversion error as fatal since it indicates a bad identity table diff --git a/network/topology/cache.go b/network/topology/cache.go index c305140abed..a738e9e9453 100644 --- a/network/topology/cache.go +++ b/network/topology/cache.go @@ -19,10 +19,10 @@ import ( // in a concurrency safe way, i.e., the caller should lock for it. type Cache struct { log zerolog.Logger - top network.Topology // instance of underlying topology. - cachedFanout flow.IdentityList // most recently generated fanout list by invoking underlying topology. - idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. - chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. + top network.Topology // instance of underlying topology. + cachedFanout flow.IdentifierList // most recently generated fanout list by invoking underlying topology. + idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. + chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. } //NewCache creates and returns a topology Cache given an instance of topology implementation. @@ -36,19 +36,19 @@ func NewCache(log zerolog.Logger, top network.Topology) *Cache { } } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList // of this instance. // It caches the most recently generated fanout list, so as long as the input list is the same, it returns // the same output. It invalidates and updates its internal cache the first time input list changes. -// A node directly communicates with its fanout IdentityList on epidemic dissemination +// A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. // // Note that this implementation of GenerateFanout preserves same output as long as input is the same. This // should not be assumed as a 1-1 mapping between input and output. -func (c *Cache) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { - inputIdsFP := ids.Fingerprint() +func (c *Cache) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { + inputIdsFP := flow.MerkleRoot(ids...) inputChansFP := channels.ID() log := c.log.With(). From 293819e4376edba0ba6757460843572970bc7108 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 16 Aug 2021 19:53:41 -0700 Subject: [PATCH 021/291] fix more complication errors --- .../node_builder/access_node_builder.go | 50 ++++++----- .../staked_access_node_builder.go | 36 ++++++-- .../unstaked_access_node_builder.go | 29 ++++++- cmd/collection/main.go | 9 +- cmd/consensus/main.go | 9 +- cmd/execution/main.go | 9 +- cmd/node_builder.go | 41 +++++---- cmd/scaffold.go | 30 ++++++- cmd/verification/main.go | 10 ++- model/flow/identifierList.go | 27 ++++++ network/middleware.go | 7 +- network/p2p/middleware.go | 25 +++--- network/p2p/network.go | 82 ++++++++++++------ network/p2p/peer | 0 network/p2p/peerstore_provider.go | 10 +-- network/p2p/unstaked_translator.go | 12 +-- network/topology/fixedListTopology.go | 10 +-- network/topology/randomizedTopology.go | 71 +++++++++------- network/topology/topicBasedTopology.go | 85 ++++++++++--------- network/topology/topology_utils.go | 4 +- 20 files changed, 374 insertions(+), 182 deletions(-) delete mode 100644 network/p2p/peer diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 1ff698f87a6..45495391454 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -35,6 +35,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" @@ -144,24 +145,25 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + UnstakedNetwork *p2p.Network + unstakedMiddleware *p2p.Middleware + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower + SyncEngineParticipantsProvider id.IdentifierProvider // engines IngestEng *ingestion.Engine @@ -314,7 +316,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - node.State, + builder.SyncEngineParticipantsProvider, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) @@ -628,6 +630,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, unicastMessageTimeout, connectionGating, managerPeerConnections, + builder.IDTranslator, validators...) return builder.unstakedMiddleware } @@ -638,7 +641,6 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, middleware *p2p.Middleware, - participants flow.IdentityList, topology network.Topology) (*p2p.Network, error) { codec := jsoncodec.NewCodec() @@ -646,15 +648,17 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, subscriptionManager := p2p.NewChannelSubscriptionManager(middleware) // creates network instance - net, err := p2p.NewNetwork(builder.Logger, + net, err := p2p.NewNetwork( + builder.Logger, codec, - participants, nodeID, builder.unstakedMiddleware, p2p.DefaultCacheSize, topology, subscriptionManager, - networkMetrics) + networkMetrics, + p2p.WithIdentifierProvider(builder.NetworkingIdentifierProvider), + ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) } diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a4b82e8cfd2..9af1a3ca7ae 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -8,7 +8,9 @@ import ( "github.com/onflow/flow-go/cmd" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" @@ -26,11 +28,39 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui } } +func (fnb *StakedAccessNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + // translator + // networking provider + fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ) + + // TODO: need special providers here + // for network, needs one that recognizes both protocl state and peerstore + // same for translator + + return nil + }) +} + func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) builder.Cancel = cancel + builder.InitIDProviders() + // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node builder.EnqueueNetworkInit(ctx) @@ -104,14 +134,10 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) msgValidators...) - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - // TODO: this list should be the unstaked addresses of all the staked AN that participate in the unstaked network - participants := flow.IdentityList{} - // topology returns empty list since peers are not known upfront top := topology.EmptyListTopology{} - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, top) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, top) builder.MustNot(err) builder.UnstakedNetwork = network diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index c290e077079..81c9f4600f7 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -24,6 +24,13 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } +func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + return nil + }) +} + func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) @@ -31,6 +38,8 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() + builder.InitIDProviders() + builder.enqueueUnstakedNetworkInit(ctx) builder.EnqueueMetricsServerInit() @@ -77,6 +86,16 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N } } +func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { + anb. + Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + anb.SyncEngineParticipantsProvider = node.Network.GetIdentifierProvider() + return nil + }) + anb.FlowAccessNodeBuilder.Build() + return anb +} + // enqueueUnstakedNetworkInit enqueues the unstaked network component initialized for the unstaked node func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { @@ -109,16 +128,18 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) msgValidators...) - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - participants := flow.IdentityList{} - upstreamANIdentifier, err := flow.HexStringToIdentifier(builder.stakedAccessNodeIDHex) builder.MustNot(err) // topology only consist of the upsteam staked AN top := topology.NewFixedListTopology(upstreamANIdentifier) - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, top) + network, err := builder.initNetwork( + builder.Me, + unstakedNetworkMetrics, + middleware, + top, + ) builder.MustNot(err) builder.UnstakedNetwork = network diff --git a/cmd/collection/main.go b/cmd/collection/main.go index 1e055d03a9c..ef364a5421a 100644 --- a/cmd/collection/main.go +++ b/cmd/collection/main.go @@ -35,6 +35,7 @@ import ( builder "github.com/onflow/flow-go/module/builder/collection" "github.com/onflow/flow-go/module/epochs" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/ingress" "github.com/onflow/flow-go/module/mempool" epochpool "github.com/onflow/flow-go/module/mempool/epochs" @@ -264,7 +265,13 @@ func main() { followerEng, mainChainSyncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) 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 524fd11ce4d..30b5d32f285 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -50,6 +50,7 @@ import ( dkgmodule "github.com/onflow/flow-go/module/dkg" "github.com/onflow/flow-go/module/epochs" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -617,7 +618,13 @@ func main() { comp, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/execution/main.go b/cmd/execution/main.go index 2a955b77b5a..5cec46ecd6e 100644 --- a/cmd/execution/main.go +++ b/cmd/execution/main.go @@ -42,6 +42,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" chainsync "github.com/onflow/flow-go/module/synchronization" @@ -455,7 +456,13 @@ func main() { followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index bb70774390d..91672d88485 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -44,6 +44,9 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() + // InitIDProviders initializes the IdentityProvider and IDTranslator + InitIDProviders() + // EnqueueNetworkInit enqueues the default network component with the given context EnqueueNetworkInit(ctx context.Context) @@ -115,23 +118,27 @@ type BaseConfig struct { type NodeConfig struct { Cancel context.CancelFunc // cancel function for the context that is passed to the networking layer BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware *p2p.Middleware - Network *p2p.Network - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey - IdentifierProvider id.IdentityProvider // TODO: initialize these in scaffold and unstaked node + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware *p2p.Middleware + Network *p2p.Network + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey + + // TODO: initialize these in scaffold and unstaked node + IdentityProvider id.IdentityProvider + IDTranslator p2p.IDTranslator + NetworkingIdentifierProvider id.IdentifierProvider // root state information RootBlock *flow.Block diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 400c6fb62f5..2954ce7da3e 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -22,6 +22,7 @@ import ( "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" @@ -173,10 +174,16 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.BaseConfig.UnicastMessageTimeout, true, true, + fnb.IDTranslator, fnb.MsgValidators...) subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) - top, err := topology.NewTopicBasedTopology(fnb.NodeID, fnb.Logger, fnb.State) + top, err := topology.NewTopicBasedTopology( + fnb.NodeID, + fnb.IdentityProvider, + fnb.Logger, + fnb.State, + ) if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) } @@ -185,13 +192,14 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { // creates network instance net, err := p2p.NewNetwork(fnb.Logger, codec, - fnb.IdentifierProvider, fnb.Me, fnb.Middleware, p2p.DefaultCacheSize, topologyCache, subscriptionManager, - fnb.Metrics.Network) + fnb.Metrics.Network, + p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) } @@ -407,6 +415,20 @@ func (fnb *FlowNodeBuilder) initStorage() { } } +func (fnb *FlowNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + fnb.IDTranslator = idCache + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NetworkingSetFilter, idCache) + return nil + }) +} + func (fnb *FlowNodeBuilder) initState() { fnb.ProtocolEvents = events.NewDistributor() @@ -706,6 +728,8 @@ func (fnb *FlowNodeBuilder) Initialize() NodeBuilder { fnb.ParseAndPrintFlags() + fnb.InitIDProviders() + fnb.EnqueueNetworkInit(ctx) if fnb.metricsEnabled { diff --git a/cmd/verification/main.go b/cmd/verification/main.go index 04b4b7b6f9b..419e525c668 100644 --- a/cmd/verification/main.go +++ b/cmd/verification/main.go @@ -24,10 +24,12 @@ import ( "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/encoding" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" @@ -352,7 +354,13 @@ func main() { followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/model/flow/identifierList.go b/model/flow/identifierList.go index cb1d4780b49..224340b7b02 100644 --- a/model/flow/identifierList.go +++ b/model/flow/identifierList.go @@ -2,6 +2,7 @@ package flow import ( "bytes" + "math/rand" "github.com/rs/zerolog/log" ) @@ -70,6 +71,32 @@ func (il IdentifierList) Contains(target Identifier) bool { return false } +// Union returns a new identifier list containing the union of `il` and `other`. +// There are no duplicates in the output. +func (il IdentifierList) Union(other IdentifierList) IdentifierList { + // stores the output, the union of the two lists + union := make(IdentifierList, 0, len(il)+len(other)) + // efficient lookup to avoid duplicates + lookup := make(map[Identifier]struct{}) + + // add all identifiers, omitted duplicates + for _, identifier := range append(il.Copy(), other...) { + if _, exists := lookup[identifier]; exists { + continue + } + union = append(union, identifier) + lookup[identifier] = struct{}{} + } + + return union +} + +// DeterministicSample returns deterministic random sample from the `IdentifierList` using the given seed +func (il IdentifierList) DeterministicSample(size uint, seed int64) IdentifierList { + rand.Seed(seed) + return il.Sample(size) +} + // Sample returns random sample of length 'size' of the ids // [Fisher-Yates shuffle](https://en.wikipedia.org/wiki/Fisher–Yates_shuffle). func (il IdentifierList) Sample(size uint) IdentifierList { diff --git a/network/middleware.go b/network/middleware.go index 94fdcbfd9e7..82928584d43 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -6,8 +6,8 @@ import ( "time" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network/message" ) // Topic is the internal type of Libp2p which corresponds to the Channel in the network level. @@ -52,16 +52,17 @@ type Middleware interface { // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. - UpdateAllowList() + UpdateAllowList() } // Overlay represents the interface that middleware uses to interact with the // overlay network layer. type Overlay interface { - id.IdentifierProvider // Topology returns an identifier list of nodes which this node should be directly connected to as peers Topology() (flow.IdentifierList, error) Receive(nodeID flow.Identifier, msg *message.Message) error + SetDefaultIdentifierProvider(id.IdentifierProvider) + GetIdentifierProvider() id.IdentifierProvider } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 802a4d13036..e25786b1141 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -101,13 +101,8 @@ func NewMiddleware( connectionGating bool, managePeerConnections bool, idTranslator IDTranslator, - validators ...network.MessageValidator) *Middleware { - - if len(validators) == 0 { - // add default validators to filter out unwanted messages received by this node - validators = DefaultValidators(log, flowID) - } - + validators ...network.MessageValidator, +) *Middleware { ctx, cancel := context.WithCancel(context.Background()) if unicastMessageTimeout <= 0 { @@ -115,7 +110,7 @@ func NewMiddleware( } // create the node entity and inject dependencies & config - return &Middleware{ + mw := &Middleware{ ctx: ctx, cancel: cancel, log: log, @@ -124,13 +119,19 @@ func NewMiddleware( libP2PNodeFactory: libP2PNodeFactory, metrics: metrics, rootBlockID: rootBlockID, - validators: validators, + validators: DefaultValidators(log, flowID), peerUpdateInterval: peerUpdateInterval, unicastMessageTimeout: unicastMessageTimeout, connectionGating: connectionGating, managePeerConnections: managePeerConnections, idTranslator: idTranslator, } + + if len(validators) != 0 { + mw.validators = validators + } + + return mw } func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.MessageValidator { @@ -151,7 +152,7 @@ func (m *Middleware) topologyPeers() (peer.IDSlice, error) { } func (m *Middleware) allPeers() peer.IDSlice { - return m.peerIDs(m.ov.Identifiers()) + return m.peerIDs(m.ov.GetIdentifierProvider().Identifiers()) } func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { @@ -184,6 +185,8 @@ func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov libP2PNode, err := m.libP2PNodeFactory() + ov.SetDefaultIdentifierProvider(NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator)) + if err != nil { return fmt.Errorf("could not create libp2p node: %w", err) } @@ -415,7 +418,7 @@ func (m *Middleware) Ping(targetID flow.Identifier) (message.PingResponse, time. return m.libP2PNode.Ping(m.ctx, peerID) } -// UpdateAllowList fetches the most recent identity of the nodes from overlay +// UpdateAllowList fetches the most recent identifiers of the nodes from overlay // and updates the underlying libp2p node. func (m *Middleware) UpdateAllowList() { // update libp2pNode's approve lists if this middleware also does connection gating diff --git a/network/p2p/network.go b/network/p2p/network.go index 9cfd4257d03..b5394b6c3e2 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -41,19 +41,28 @@ type ReadyDoneAwareNetwork interface { // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { sync.RWMutex - id.IdentifierProvider - logger zerolog.Logger - codec network.Codec - me module.Local - mw network.Middleware - top network.Topology // used to determine fanout connections - metrics module.NetworkMetrics - rcache *RcvCache // used to deduplicate incoming messages - queue network.MessageQueue - ctx context.Context - cancel context.CancelFunc - subMngr network.SubscriptionManager // used to keep track of subscribed channels - lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle + idProvider id.IdentifierProvider + defaultIdProvider id.IdentifierProvider + logger zerolog.Logger + codec network.Codec + me module.Local + mw network.Middleware + top network.Topology // used to determine fanout connections + metrics module.NetworkMetrics + rcache *RcvCache // used to deduplicate incoming messages + queue network.MessageQueue + ctx context.Context + cancel context.CancelFunc + subMngr network.SubscriptionManager // used to keep track of subscribed channels + lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle +} + +type NetworkOption func(*Network) + +func WithIdentifierProvider(provider id.IdentifierProvider) NetworkOption { + return func(net *Network) { + net.idProvider = provider + } } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -63,13 +72,13 @@ type Network struct { func NewNetwork( log zerolog.Logger, codec network.Codec, - idProvider id.IdentifierProvider, me module.Local, mw network.Middleware, csize int, top network.Topology, sm network.SubscriptionManager, metrics module.NetworkMetrics, + opts ...NetworkOption, ) (*Network, error) { rcache, err := newRcvCache(csize) @@ -78,16 +87,15 @@ func NewNetwork( } o := &Network{ - IdentifierProvider: idProvider, - logger: log, - codec: codec, - me: me, - mw: mw, - rcache: rcache, - top: top, - metrics: metrics, - subMngr: sm, - lifecycleManager: lifecycle.NewLifecycleManager(), + logger: log, + codec: codec, + me: me, + mw: mw, + rcache: rcache, + top: top, + metrics: metrics, + subMngr: sm, + lifecycleManager: lifecycle.NewLifecycleManager(), } o.ctx, o.cancel = context.WithCancel(context.Background()) @@ -98,6 +106,10 @@ func NewNetwork( // create workers to read from the queue and call queueSubmitFunc queue.CreateQueueWorkers(o.ctx, queue.DefaultNumWorkers, o.queue, o.queueSubmitFunc) + for _, opt := range opts { + opt(o) + } + return o, nil } @@ -163,6 +175,19 @@ func (n *Network) unregister(channel network.Channel) error { return nil } +func (n *Network) GetIdentifierProvider() id.IdentifierProvider { + if n.idProvider != nil { + return n.idProvider + } + n.RLock() + defer n.RUnlock() + if n.defaultIdProvider == nil { + n.logger.Fatal().Msg("TODO") + // TODO + } + return n.defaultIdProvider +} + // Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. func (n *Network) Topology() (flow.IdentifierList, error) { @@ -170,7 +195,7 @@ func (n *Network) Topology() (flow.IdentifierList, error) { defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.GetIdentifierProvider().Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } @@ -277,6 +302,12 @@ func (n *Network) genNetworkMessage(channel network.Channel, event interface{}, return msg, nil } +func (n *Network) SetDefaultIdentifierProvider(provider id.IdentifierProvider) { + n.Lock() + n.defaultIdProvider = provider + n.Unlock() +} + // unicast sends the message in a reliable way to the given recipient. // It uses 1-1 direct messaging over the underlying network to deliver the message. // It returns an error if unicasting fails. @@ -318,7 +349,6 @@ func (n *Network) publish(channel network.Channel, message interface{}, targetID // multicast unreliably sends the specified event over the channel to randomly selected 'num' number of recipients // selected from the specified targetIDs. func (n *Network) multicast(channel network.Channel, message interface{}, num uint, targetIDs ...flow.Identifier) error { - selectedIDs := flow.IdentifierList(targetIDs).Filter(n.removeSelfFilter()).Sample(num) if len(selectedIDs) == 0 { diff --git a/network/p2p/peer b/network/p2p/peer deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go index d39b2ecf595..cea16d466a9 100644 --- a/network/p2p/peerstore_provider.go +++ b/network/p2p/peerstore_provider.go @@ -1,24 +1,24 @@ package p2p import ( - peerstore "github.com/libp2p/go-libp2p-peerstore" + "github.com/libp2p/go-libp2p-core/host" "github.com/onflow/flow-go/model/flow" ) type PeerstoreIdentifierProvider struct { - store peerstore.Peerstore + host host.Host idTranslator IDTranslator } -func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator IDTranslator) *PeerstoreIdentifierProvider { - return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} +func NewPeerstoreIdentifierProvider(host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{host: host, idTranslator: idTranslator} } func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { var result flow.IdentifierList - pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? + pids := p.host.Peerstore().PeersWithAddrs() // TODO: should we just call Peers here? for _, pid := range pids { flowID, err := p.idTranslator.GetFlowID(pid) if err != nil { diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 2098824de26..5e320ccb236 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -1,20 +1,20 @@ package p2p import ( + crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/libp2p/go-libp2p-core/peer" "github.com/multiformats/go-multihash" - "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/onflow/flow-go/model/flow" ) -type UnstakedNetworkPeerIDProvider struct{} +type UnstakedNetworkIDTranslator struct{} -func NewUnstakedNetworkPeerIDProvider() *UnstakedNetworkPeerIDProvider { - return &UnstakedNetworkPeerIDProvider{} +func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { + return &UnstakedNetworkIDTranslator{} } -func GetPeerID(flowID flow.Identifier) (peer.ID, error) { +func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { data := append([]byte{0x02}, flowID[:]...) mh, err := multihash.Sum(data, multihash.IDENTITY, -1) if err != nil { @@ -24,7 +24,7 @@ func GetPeerID(flowID flow.Identifier) (peer.ID, error) { return peer.ID(mh), nil } -func GetFlowID(peerID peer.ID) (flow.Identifier, error) { +func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { pk, err := peerID.ExtractPublicKey() if err != nil { // return error diff --git a/network/topology/fixedListTopology.go b/network/topology/fixedListTopology.go index 1b171ce8919..6091da61d38 100644 --- a/network/topology/fixedListTopology.go +++ b/network/topology/fixedListTopology.go @@ -2,7 +2,7 @@ package topology import ( "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" "github.com/onflow/flow-go/network" ) @@ -17,14 +17,14 @@ func NewFixedListTopology(nodeID flow.Identifier) FixedListTopology { } } -func (r FixedListTopology) GenerateFanout(ids flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { - return ids.Filter(filter.HasNodeID(r.fixedNodeID)), nil +func (r FixedListTopology) GenerateFanout(ids flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { + return ids.Filter(idFilter.Is(r.fixedNodeID)), nil } // EmptyListTopology always returns an empty list as the fanout type EmptyListTopology struct { } -func (r EmptyListTopology) GenerateFanout(_ flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { - return flow.IdentityList{}, nil +func (r EmptyListTopology) GenerateFanout(_ flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { + return flow.IdentifierList{}, nil } diff --git a/network/topology/randomizedTopology.go b/network/topology/randomizedTopology.go index c1bc959ed9e..55be3545025 100644 --- a/network/topology/randomizedTopology.go +++ b/network/topology/randomizedTopology.go @@ -9,6 +9,8 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -17,15 +19,16 @@ import ( // By random topology we mean a node is connected to any other co-channel nodes with some // edge probability. type RandomizedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - chance uint64 // used to translate connectedness probability into a number in [0, 100] - rng random.Rand // used as a stateful random number generator to sample edges - logger zerolog.Logger + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + chance uint64 // used to translate connectedness probability into a number in [0, 100] + rng random.Rand // used as a stateful random number generator to sample edges + logger zerolog.Logger + idProvider id.IdentityProvider } // NewRandomizedTopology returns an instance of the RandomizedTopology. -func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { +func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { // edge probability should be a positive value between 0 and 1. However, // we like it to be strictly greater than zero. Also, at the current scale of // Flow, we need it to be greater than 0.01 to support probabilistic connectedness. @@ -44,33 +47,34 @@ func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgePr } t := &RandomizedTopology{ - myNodeID: nodeID, - state: state, - chance: uint64(100 * edgeProb), - rng: rng, - logger: logger.With().Str("component:", "randomized-topology").Logger(), + myNodeID: nodeID, + state: state, + chance: uint64(100 * edgeProb), + rng: rng, + logger: logger.With().Str("component:", "randomized-topology").Logger(), + idProvider: idProvider, } return t, nil } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList -// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. This should be done with a very high probability // in randomized topology. -func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { +func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. r.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } - var myFanout flow.IdentityList + var myFanout flow.IdentifierList // generates a randomized subgraph per channel for _, myChannel := range myUniqueChannels { @@ -90,12 +94,12 @@ func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels netwo return myFanout, nil } -// subsetChannel returns a random subset of the identity list that is passed. -// Returned identities should all subscribed to the specified `channel`. -// Note: this method should not include identity of its executor. -func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +// subsetChannel returns a random subset of the identifier list that is passed. +// Returned identifiers should all subscribed to the specified `channel`. +// Note: this method should not include identifier of its executor. +func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { // excludes node itself - sampleSpace := ids.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) + sampleSpace := ids.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) // samples a random graph based on whether channel is cluster-based or not. if _, ok := engine.ClusterChannel(channel); ok { @@ -109,12 +113,12 @@ func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network // Independent invocations of this method over different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityList, error) { +func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.IdentifierList, error) { if len(ids) == 0 { - return nil, fmt.Errorf("empty identity list") + return nil, fmt.Errorf("empty identifier list") } - fanout := flow.IdentityList{} + fanout := flow.IdentifierList{} for _, id := range ids { // tosses a biased coin and adds id to fanout accordingly. // biased coin follows the edge probability distribution. @@ -127,7 +131,7 @@ func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityLi } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.IdentityList, error) { +func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow.IdentifierList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(r.myNodeID, r.state) if err != nil { @@ -135,10 +139,10 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.I } // excludes node itself from cluster - clusterPeers = clusterPeers.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) + clusterPeers = clusterPeers.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) + nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } @@ -148,7 +152,7 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.I } // clusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. -func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -160,7 +164,16 @@ func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, chan } // samples fanout among interacting roles - return r.sampleFanout(ids.Filter(filter.HasRole(roles...))) + return r.sampleFanout( + r.idProvider. + Identities( + filter.And( + filter.HasNodeID(ids...), + filter.HasRole(roles...), + ), + ). + NodeIDs(), + ) } // tossBiasedBit returns true with probability equal `r.chance/100`, and returns false otherwise. diff --git a/network/topology/topicBasedTopology.go b/network/topology/topicBasedTopology.go index 9d14e34049b..fdc55f43b1e 100644 --- a/network/topology/topicBasedTopology.go +++ b/network/topology/topicBasedTopology.go @@ -8,6 +8,8 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -15,14 +17,15 @@ import ( // TopicBasedTopology is a deterministic topology mapping that creates a connected graph component among the nodes // involved in each topic. type TopicBasedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - logger zerolog.Logger - seed int64 + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + logger zerolog.Logger + seed int64 + identityProvider id.IdentityProvider } // NewTopicBasedTopology returns an instance of the TopicBasedTopology. -func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { +func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { seed, err := intSeedFromID(nodeID) if err != nil { return nil, fmt.Errorf("could not generate seed from id:%w", err) @@ -38,19 +41,19 @@ func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state return t, nil } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList -// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. -func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { +func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. t.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } // finds all interacting roles with this node @@ -64,7 +67,7 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels netwo } // builds a connected component per role this node interact with, - var myFanout flow.IdentityList + var myFanout flow.IdentifierList for _, role := range myInteractingRoles { if role == flow.RoleCollection { // we do not build connected component for collection nodes based on their role @@ -98,32 +101,36 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels netwo return myFanout, nil } -// subsetChannel returns a random subset of the identity list that is passed. `shouldHave` represents set of -// identities that should be included in the returned subset. -// Returned identities should all subscribed to the specified `channel`. -// Note: this method should not include identity of its executor. -func (t *TopicBasedTopology) subsetChannel(ids flow.IdentityList, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +// subsetChannel returns a random subset of the identifier list that is passed. `shouldHave` represents set of +// identifiers that should be included in the returned subset. +// Returned identifiers should all subscribed to the specified `channel`. +// Note: this method should not include identifier of its executor. +func (t *TopicBasedTopology) subsetChannel(ids flow.IdentifierList, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return t.clusterChannelHandler(ids, shouldHave) } return t.nonClusterChannelHandler(ids, shouldHave, channel) } -// subsetRole returns a random subset of the identity list that is passed. `shouldHave` represents set of -// identities that should be included in the returned subset. -// Returned identities should all be of one of the specified `roles`. -// Note: this method should not include identity of its executor. -func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.IdentityList, roles flow.RoleList) (flow.IdentityList, error) { +// subsetRole returns a random subset of the identifier list that is passed. `shouldHave` represents set of +// identifiers that should be included in the returned subset. +// Returned identifiers should all be of one of the specified `roles`. +// Note: this method should not include the identifier of its executor. +func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow.IdentifierList, roles flow.RoleList) (flow.IdentifierList, error) { // excludes irrelevant roles and the node itself from both should have and ids set - shouldHave = shouldHave.Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )) - - ids = ids.Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )) + shouldHave = t.identityProvider. + Identities(filter.HasNodeID(shouldHave...)). + Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )).NodeIDs() + + ids = t.identityProvider. + Identities(filter.HasNodeID(ids...)). + Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )).NodeIDs() sample, err := t.sampleConnectedGraph(ids, shouldHave) if err != nil { @@ -138,11 +145,11 @@ func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.Id // different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHave flow.IdentityList) (flow.IdentityList, error) { +func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { if len(all) == 0 { t.logger.Debug().Msg("skips sampling connected graph with zero nodes") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } if len(shouldHave) == 0 { @@ -153,9 +160,9 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa } // checks `shouldHave` be a subset of `all` - nonMembers := shouldHave.Filter(filter.Not(filter.In(all))) + nonMembers := shouldHave.Filter(idFilter.Not(idFilter.In(all...))) if len(nonMembers) != 0 { - return nil, fmt.Errorf("should have identities is not a subset of all: %v", nonMembers) + return nil, fmt.Errorf("should have identifiers is not a subset of all: %v", nonMembers) } // total sample size @@ -170,7 +177,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa subsetSize := totalSize - len(shouldHave) // others are all excluding should have ones - others := all.Filter(filter.Not(filter.In(shouldHave))) + others := all.Filter(idFilter.Not(idFilter.In(shouldHave...))) others = others.DeterministicSample(uint(subsetSize), t.seed) return others.Union(shouldHave), nil @@ -178,7 +185,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityList) (flow.IdentityList, error) { +func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(t.myNodeID, t.state) if err != nil { @@ -186,18 +193,18 @@ func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityL } // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) + nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } // samples a connected graph topology from the cluster peers - return t.subsetRole(clusterPeers, shouldHave.Filter(filter.HasNodeID(clusterPeers.NodeIDs()...)), flow.RoleList{flow.RoleCollection}) + return t.subsetRole(clusterPeers, shouldHave.Filter(idFilter.In(clusterPeers...)), flow.RoleList{flow.RoleCollection}) } // nonClusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. // The returned sample contains `shouldHave` ones that also subscribed to `channel`. -func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -209,5 +216,5 @@ func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.Identi } // samples a connected graph topology - return t.subsetRole(ids.Filter(filter.HasRole(roles...)), shouldHave, roles) + return t.subsetRole(ids, shouldHave, roles) } diff --git a/network/topology/topology_utils.go b/network/topology/topology_utils.go index bf1d2b43986..3668d8557a6 100644 --- a/network/topology/topology_utils.go +++ b/network/topology/topology_utils.go @@ -51,7 +51,7 @@ func byteSeedFromID(id flow.Identifier) ([]byte, error) { } // clusterPeers returns the list of other nodes within the same cluster as specified identifier. -func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, error) { +func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList, error) { currentEpoch := state.Final().Epochs().Current() clusterList, err := currentEpoch.Clustering() if err != nil { @@ -63,5 +63,5 @@ func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, return nil, fmt.Errorf("failed to find the cluster for node ID %s", id.String()) } - return myCluster, nil + return myCluster.NodeIDs(), nil } From d0d0cef9899fa03d120319b8b3c695273e3f78de Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 16 Aug 2021 20:30:29 -0700 Subject: [PATCH 022/291] same --- network/p2p/protocol_state_provider.go | 7 ++++--- utils/grpc/grpc.go | 9 ++------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 6933af0db61..d47542c3b18 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -6,6 +6,7 @@ import ( "github.com/libp2p/go-libp2p-core/peer" + "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/state/protocol" @@ -66,7 +67,7 @@ func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { flowIDs := make(map[peer.ID]flow.Identifier, nIds) for _, identity := range identities { - pid, err := ExtractPeerID(identity) + pid, err := ExtractPeerID(identity.NetworkPubKey) if err != nil { // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping } @@ -112,8 +113,8 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e return } -func ExtractPeerID(id *flow.Identity) (pid peer.ID, err error) { - pk, err := PublicKey(id.NetworkPubKey) +func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { + pk, err := PublicKey(networkPubKey) if err != nil { // TODO: format the error return diff --git a/utils/grpc/grpc.go b/utils/grpc/grpc.go index ceb8921289f..6ca684cb615 100644 --- a/utils/grpc/grpc.go +++ b/utils/grpc/grpc.go @@ -7,7 +7,6 @@ import ( "fmt" lcrypto "github.com/libp2p/go-libp2p-core/crypto" - "github.com/libp2p/go-libp2p-core/peer" libp2ptls "github.com/libp2p/go-libp2p-tls" "github.com/onflow/flow-go/crypto" @@ -102,13 +101,9 @@ func DefaultClientTLSConfig(publicKey crypto.PublicKey) (*tls.Config, error) { func verifyPeerCertificateFunc(expectedPublicKey crypto.PublicKey) (func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error, error) { // convert the Flow.crypto key to LibP2P key for easy comparision using LibP2P TLS utils - expectedLibP2PKey, err := p2p.PublicKey(expectedPublicKey) + remotePeerLibP2PID, err := p2p.ExtractPeerID(expectedPublicKey) if err != nil { - return nil, fmt.Errorf("failed to generate a libp2p key from a Flow key: %w", err) - } - remotePeerLibP2PID, err := peer.IDFromPublicKey(expectedLibP2PKey) - if err != nil { - return nil, fmt.Errorf("failed to derive the libp2p Peer ID from the libp2p public key: %w", err) + return nil, fmt.Errorf("failed to derive the libp2p Peer ID from the Flow key: %w", err) } // We're using InsecureSkipVerify, so the verifiedChains parameter will always be empty. From c405a4d8b0a3bddbf53fd91ffa254dc3280b21e4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 13:52:26 -0700 Subject: [PATCH 023/291] add dht peer lookup --- .../node_builder/access_node_builder.go | 2 +- cmd/scaffold.go | 2 + network/p2p/dht.go | 6 +-- network/p2p/libp2pNode.go | 44 ++++++++++++++++--- 4 files changed, 42 insertions(+), 12 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b193a8d8ec2..cfb6e5f71a4 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -597,7 +597,7 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, SetRootBlockID(builder.RootBlock.ID().String()). // unlike the staked network where currently all the node addresses are known upfront, // for the unstaked network the nodes need to discover each other using DHT Discovery. - SetPubsubOptions(p2p.WithDHTDiscovery(dhtOptions...)). + SetDHTOptions(dhtOptions...). SetLogger(builder.Logger). Build(ctx) if err != nil { diff --git a/cmd/scaffold.go b/cmd/scaffold.go index d4b9f2da7ba..906250d4d69 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -208,6 +208,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) fnb.ProtocolEvents.AddConsumer(idEvents) + // TODO: add a consumer which implements setting the permanent peer addresses, + // and expiring the old permanent ones return net, err }) diff --git a/network/p2p/dht.go b/network/p2p/dht.go index 8cd702c5a9a..b21211a9adb 100644 --- a/network/p2p/dht.go +++ b/network/p2p/dht.go @@ -5,7 +5,6 @@ import ( "github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p-core/peer" - discovery "github.com/libp2p/go-libp2p-discovery" dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow-go/model/flow" @@ -13,7 +12,7 @@ import ( // This produces a new IPFS DHT // on the name, see https://github.com/libp2p/go-libp2p-kad-dht/issues/337 -func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*discovery.RoutingDiscovery, error) { +func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*dht.IpfsDHT, error) { defaultOptions := defaultDHTOptions() allOptions := append(defaultOptions, options...) @@ -27,8 +26,7 @@ func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*discov return nil, err } - routingDiscovery := discovery.NewRoutingDiscovery(kdht) - return routingDiscovery, nil + return kdht, nil } // DHT defaults to ModeAuto which will automatically switch the DHT between Server and Client modes based on diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 4a07329953c..43ea260599e 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -17,6 +17,7 @@ import ( libp2pnet "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/protocol" + discovery "github.com/libp2p/go-libp2p-discovery" dht "github.com/libp2p/go-libp2p-kad-dht" pubsub "github.com/libp2p/go-libp2p-pubsub" swarm "github.com/libp2p/go-libp2p-swarm" @@ -73,6 +74,7 @@ type NodeBuilder interface { SetConnectionGater(*ConnGater) NodeBuilder SetPubsubOptions(...PubsubOption) NodeBuilder SetPingInfoProvider(PingInfoProvider) NodeBuilder + SetDHTOptions(...dht.Option) NodeBuilder SetLogger(zerolog.Logger) NodeBuilder Build(context.Context) (*Node, error) } @@ -87,6 +89,7 @@ type DefaultLibP2PNodeBuilder struct { pubSubMaker func(context.Context, host.Host, ...pubsub.Option) (*pubsub.PubSub, error) hostMaker func(context.Context, ...config.Option) (host.Host, error) pubSubOpts []PubsubOption + dhtOpts []dht.Option } func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcrypto.PrivateKey) NodeBuilder { @@ -101,6 +104,11 @@ func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcr } } +func (builder *DefaultLibP2PNodeBuilder) SetDHTOptions(opts ...dht.Option) NodeBuilder { + builder.dhtOpts = opts + return builder +} + func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId string) NodeBuilder { builder.rootBlockID = rootBlockId return builder @@ -179,6 +187,15 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro } node.host = libp2pHost + if len(builder.dhtOpts) != 0 { + kdht, err := NewDHT(ctx, node.host, builder.dhtOpts...) + if err != nil { + return nil, err + } + node.dht = kdht + builder.pubSubOpts = append(builder.pubSubOpts, WithDHTDiscovery(kdht)) + } + if builder.pingInfoProvider != nil { pingLibP2PProtocolID := generatePingProtcolID(builder.rootBlockID) pingService := NewPingService(libp2pHost, pingLibP2PProtocolID, builder.pingInfoProvider, node.logger) @@ -227,6 +244,7 @@ type Node struct { flowLibP2PProtocolID protocol.ID // the unique protocol ID pingService *PingService connMgr TagLessConnManager + dht *dht.IpfsDHT } // Stop stops the libp2p node. @@ -339,7 +357,22 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp } // remove the peer from the peer store if present - n.host.Peerstore().ClearAddrs(peerID) + // TODO: why were we doing this? + // n.host.Peerstore().ClearAddrs(peerID) + + if len(n.host.Peerstore().Addrs(peerID)) == 0 { + if n.dht != nil { + // TODO: adjust timeout + timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) + // try to find the peer + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + } + } + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() @@ -621,12 +654,9 @@ func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { } } -func WithDHTDiscovery(option ...dht.Option) PubsubOption { +func WithDHTDiscovery(kdht *dht.IpfsDHT) PubsubOption { return func(ctx context.Context, host host.Host) (pubsub.Option, error) { - dhtDiscovery, err := NewDHT(ctx, host, option...) - if err != nil { - return nil, err - } - return pubsub.WithDiscovery(dhtDiscovery), nil + routingDiscovery := discovery.NewRoutingDiscovery(kdht) + return pubsub.WithDiscovery(routingDiscovery), nil } } From 14e407e874b12d337a285434913e7ae3ce286339 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 15:42:48 -0700 Subject: [PATCH 024/291] Add dht lookup and default peerstore addresses --- .../unstaked_access_node_builder.go | 9 +++++++++ cmd/scaffold.go | 7 ++++--- network/middleware.go | 5 ++++- network/p2p/libp2pNode.go | 1 + network/p2p/middleware.go | 18 +++++++++++++++++- network/p2p/network.go | 15 +++++++++++++-- 6 files changed, 48 insertions(+), 7 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 886af554298..88ce203fe65 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -8,6 +8,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/p2p" ) type UnstakedAccessNodeBuilder struct { @@ -23,6 +24,14 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + return nil }) } diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 906250d4d69..db04f2dee59 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -206,10 +206,11 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Network = net - idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) + idEvents := gadgets.NewIdentityDeltas(func() { + fnb.Middleware.UpdateNodeAddresses() + fnb.Middleware.UpdateAllowList() + }) fnb.ProtocolEvents.AddConsumer(idEvents) - // TODO: add a consumer which implements setting the permanent peer addresses, - // and expiring the old permanent ones return net, err }) diff --git a/network/middleware.go b/network/middleware.go index 82928584d43..8bd89167549 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -53,6 +53,8 @@ type Middleware interface { // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. UpdateAllowList() + + UpdateNodeAddresses() } // Overlay represents the interface that middleware uses to interact with the @@ -62,7 +64,8 @@ type Overlay interface { Topology() (flow.IdentifierList, error) Receive(nodeID flow.Identifier, msg *message.Message) error SetDefaultIdentifierProvider(id.IdentifierProvider) - GetIdentifierProvider() id.IdentifierProvider + Identifiers() flow.IdentifierList + Identities() flow.IdentityList } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 43ea260599e..d846b78c0dd 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -361,6 +361,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp // n.host.Peerstore().ClearAddrs(peerID) if len(n.host.Peerstore().Addrs(peerID)) == 0 { + // TODO: add bunch of logging here if n.dht != nil { // TODO: adjust timeout timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index e25786b1141..3164f0e8034 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -12,11 +12,13 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" @@ -79,6 +81,7 @@ type Middleware struct { connectionGating bool managePeerConnections bool idTranslator IDTranslator + idProvider id.IdentityProvider } // NewMiddleware creates a new middleware instance @@ -101,6 +104,7 @@ func NewMiddleware( connectionGating bool, managePeerConnections bool, idTranslator IDTranslator, + idProvider id.IdentityProvider, validators ...network.MessageValidator, ) *Middleware { ctx, cancel := context.WithCancel(context.Background()) @@ -125,6 +129,7 @@ func NewMiddleware( connectionGating: connectionGating, managePeerConnections: managePeerConnections, idTranslator: idTranslator, + idProvider: idProvider, } if len(validators) != 0 { @@ -152,7 +157,7 @@ func (m *Middleware) topologyPeers() (peer.IDSlice, error) { } func (m *Middleware) allPeers() peer.IDSlice { - return m.peerIDs(m.ov.GetIdentifierProvider().Identifiers()) + return m.peerIDs(m.ov.Identifiers()) } func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { @@ -180,6 +185,15 @@ func (m *Middleware) GetIPPort() (string, string, error) { return m.libP2PNode.GetIPPort() } +func (m *Middleware) UpdateNodeAddresses() { + ids := m.ov.Identities() + infos, _ := peerInfosFromIDs(ids) + + for _, info := range infos { + m.libP2PNode.host.Peerstore().SetAddrs(info.ID, info.Addrs, peerstore.PermanentAddrTTL) + } +} + // Start will start the middleware. func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov @@ -193,6 +207,8 @@ func (m *Middleware) Start(ov network.Overlay) error { m.libP2PNode = libP2PNode m.libP2PNode.SetFlowProtocolStreamHandler(m.handleIncomingStream) + m.UpdateNodeAddresses() + if m.connectionGating { m.libP2PNode.UpdateAllowList(m.allPeers()) } diff --git a/network/p2p/network.go b/network/p2p/network.go index 0584143e5c5..b87957774c4 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -43,6 +43,7 @@ type Network struct { sync.RWMutex idProvider id.IdentifierProvider defaultIdProvider id.IdentifierProvider + identityProvider id.IdentityProvider logger zerolog.Logger codec network.Codec me module.Local @@ -78,6 +79,7 @@ func NewNetwork( top network.Topology, sm network.SubscriptionManager, metrics module.NetworkMetrics, + identityProvider id.IdentityProvider, opts ...NetworkOption, ) (*Network, error) { @@ -96,6 +98,7 @@ func NewNetwork( metrics: metrics, subMngr: sm, lifecycleManager: lifecycle.NewLifecycleManager(), + identityProvider: identityProvider, } o.ctx, o.cancel = context.WithCancel(context.Background()) @@ -175,7 +178,7 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -func (n *Network) GetIdentifierProvider() id.IdentifierProvider { +func (n *Network) getIdentifierProvider() id.IdentifierProvider { if n.idProvider != nil { return n.idProvider } @@ -188,6 +191,14 @@ func (n *Network) GetIdentifierProvider() id.IdentifierProvider { return n.defaultIdProvider } +func (n *Network) Identifiers() flow.IdentifierList { + return n.getIdentifierProvider().Identifiers() +} + +func (n *Network) Identities() flow.IdentityList { + return n.identityProvider.Identities(NetworkingSetFilter) +} + // Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. func (n *Network) Topology() (flow.IdentifierList, error) { @@ -195,7 +206,7 @@ func (n *Network) Topology() (flow.IdentifierList, error) { defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.GetIdentifierProvider().Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } From c530ff591bf8a6a27348eebff4e90fc965d0e52c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 15:48:37 -0700 Subject: [PATCH 025/291] undo topology changes --- network/topology/cache.go | 16 ++--- network/topology/fixedListTopology.go | 10 +-- network/topology/randomizedTopology.go | 71 +++++++++------------ network/topology/topicBasedTopology.go | 85 ++++++++++++-------------- network/topology/topology_utils.go | 4 +- 5 files changed, 83 insertions(+), 103 deletions(-) diff --git a/network/topology/cache.go b/network/topology/cache.go index a738e9e9453..c305140abed 100644 --- a/network/topology/cache.go +++ b/network/topology/cache.go @@ -19,10 +19,10 @@ import ( // in a concurrency safe way, i.e., the caller should lock for it. type Cache struct { log zerolog.Logger - top network.Topology // instance of underlying topology. - cachedFanout flow.IdentifierList // most recently generated fanout list by invoking underlying topology. - idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. - chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. + top network.Topology // instance of underlying topology. + cachedFanout flow.IdentityList // most recently generated fanout list by invoking underlying topology. + idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. + chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. } //NewCache creates and returns a topology Cache given an instance of topology implementation. @@ -36,19 +36,19 @@ func NewCache(log zerolog.Logger, top network.Topology) *Cache { } } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList // of this instance. // It caches the most recently generated fanout list, so as long as the input list is the same, it returns // the same output. It invalidates and updates its internal cache the first time input list changes. -// A node directly communicates with its fanout IdentifierList on epidemic dissemination +// A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. // // Note that this implementation of GenerateFanout preserves same output as long as input is the same. This // should not be assumed as a 1-1 mapping between input and output. -func (c *Cache) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { - inputIdsFP := flow.MerkleRoot(ids...) +func (c *Cache) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { + inputIdsFP := ids.Fingerprint() inputChansFP := channels.ID() log := c.log.With(). diff --git a/network/topology/fixedListTopology.go b/network/topology/fixedListTopology.go index 6091da61d38..1b171ce8919 100644 --- a/network/topology/fixedListTopology.go +++ b/network/topology/fixedListTopology.go @@ -2,7 +2,7 @@ package topology import ( "github.com/onflow/flow-go/model/flow" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/network" ) @@ -17,14 +17,14 @@ func NewFixedListTopology(nodeID flow.Identifier) FixedListTopology { } } -func (r FixedListTopology) GenerateFanout(ids flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { - return ids.Filter(idFilter.Is(r.fixedNodeID)), nil +func (r FixedListTopology) GenerateFanout(ids flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { + return ids.Filter(filter.HasNodeID(r.fixedNodeID)), nil } // EmptyListTopology always returns an empty list as the fanout type EmptyListTopology struct { } -func (r EmptyListTopology) GenerateFanout(_ flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { - return flow.IdentifierList{}, nil +func (r EmptyListTopology) GenerateFanout(_ flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { + return flow.IdentityList{}, nil } diff --git a/network/topology/randomizedTopology.go b/network/topology/randomizedTopology.go index 55be3545025..c1bc959ed9e 100644 --- a/network/topology/randomizedTopology.go +++ b/network/topology/randomizedTopology.go @@ -9,8 +9,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -19,16 +17,15 @@ import ( // By random topology we mean a node is connected to any other co-channel nodes with some // edge probability. type RandomizedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - chance uint64 // used to translate connectedness probability into a number in [0, 100] - rng random.Rand // used as a stateful random number generator to sample edges - logger zerolog.Logger - idProvider id.IdentityProvider + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + chance uint64 // used to translate connectedness probability into a number in [0, 100] + rng random.Rand // used as a stateful random number generator to sample edges + logger zerolog.Logger } // NewRandomizedTopology returns an instance of the RandomizedTopology. -func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { +func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { // edge probability should be a positive value between 0 and 1. However, // we like it to be strictly greater than zero. Also, at the current scale of // Flow, we need it to be greater than 0.01 to support probabilistic connectedness. @@ -47,34 +44,33 @@ func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvide } t := &RandomizedTopology{ - myNodeID: nodeID, - state: state, - chance: uint64(100 * edgeProb), - rng: rng, - logger: logger.With().Str("component:", "randomized-topology").Logger(), - idProvider: idProvider, + myNodeID: nodeID, + state: state, + chance: uint64(100 * edgeProb), + rng: rng, + logger: logger.With().Str("component:", "randomized-topology").Logger(), } return t, nil } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList -// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. This should be done with a very high probability // in randomized topology. -func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { +func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. r.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } - var myFanout flow.IdentifierList + var myFanout flow.IdentityList // generates a randomized subgraph per channel for _, myChannel := range myUniqueChannels { @@ -94,12 +90,12 @@ func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels net return myFanout, nil } -// subsetChannel returns a random subset of the identifier list that is passed. -// Returned identifiers should all subscribed to the specified `channel`. -// Note: this method should not include identifier of its executor. -func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +// subsetChannel returns a random subset of the identity list that is passed. +// Returned identities should all subscribed to the specified `channel`. +// Note: this method should not include identity of its executor. +func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { // excludes node itself - sampleSpace := ids.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) + sampleSpace := ids.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) // samples a random graph based on whether channel is cluster-based or not. if _, ok := engine.ClusterChannel(channel); ok { @@ -113,12 +109,12 @@ func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel netwo // Independent invocations of this method over different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.IdentifierList, error) { +func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityList, error) { if len(ids) == 0 { - return nil, fmt.Errorf("empty identifier list") + return nil, fmt.Errorf("empty identity list") } - fanout := flow.IdentifierList{} + fanout := flow.IdentityList{} for _, id := range ids { // tosses a biased coin and adds id to fanout accordingly. // biased coin follows the edge probability distribution. @@ -131,7 +127,7 @@ func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.Identifi } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow.IdentifierList, error) { +func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.IdentityList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(r.myNodeID, r.state) if err != nil { @@ -139,10 +135,10 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow } // excludes node itself from cluster - clusterPeers = clusterPeers.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) + clusterPeers = clusterPeers.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) + nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } @@ -152,7 +148,7 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow } // clusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. -func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -164,16 +160,7 @@ func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, ch } // samples fanout among interacting roles - return r.sampleFanout( - r.idProvider. - Identities( - filter.And( - filter.HasNodeID(ids...), - filter.HasRole(roles...), - ), - ). - NodeIDs(), - ) + return r.sampleFanout(ids.Filter(filter.HasRole(roles...))) } // tossBiasedBit returns true with probability equal `r.chance/100`, and returns false otherwise. diff --git a/network/topology/topicBasedTopology.go b/network/topology/topicBasedTopology.go index fdc55f43b1e..9d14e34049b 100644 --- a/network/topology/topicBasedTopology.go +++ b/network/topology/topicBasedTopology.go @@ -8,8 +8,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -17,15 +15,14 @@ import ( // TopicBasedTopology is a deterministic topology mapping that creates a connected graph component among the nodes // involved in each topic. type TopicBasedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - logger zerolog.Logger - seed int64 - identityProvider id.IdentityProvider + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + logger zerolog.Logger + seed int64 } // NewTopicBasedTopology returns an instance of the TopicBasedTopology. -func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { +func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { seed, err := intSeedFromID(nodeID) if err != nil { return nil, fmt.Errorf("could not generate seed from id:%w", err) @@ -41,19 +38,19 @@ func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvide return t, nil } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList -// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. -func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. t.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } // finds all interacting roles with this node @@ -67,7 +64,7 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels net } // builds a connected component per role this node interact with, - var myFanout flow.IdentifierList + var myFanout flow.IdentityList for _, role := range myInteractingRoles { if role == flow.RoleCollection { // we do not build connected component for collection nodes based on their role @@ -101,36 +98,32 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels net return myFanout, nil } -// subsetChannel returns a random subset of the identifier list that is passed. `shouldHave` represents set of -// identifiers that should be included in the returned subset. -// Returned identifiers should all subscribed to the specified `channel`. -// Note: this method should not include identifier of its executor. -func (t *TopicBasedTopology) subsetChannel(ids flow.IdentifierList, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +// subsetChannel returns a random subset of the identity list that is passed. `shouldHave` represents set of +// identities that should be included in the returned subset. +// Returned identities should all subscribed to the specified `channel`. +// Note: this method should not include identity of its executor. +func (t *TopicBasedTopology) subsetChannel(ids flow.IdentityList, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return t.clusterChannelHandler(ids, shouldHave) } return t.nonClusterChannelHandler(ids, shouldHave, channel) } -// subsetRole returns a random subset of the identifier list that is passed. `shouldHave` represents set of -// identifiers that should be included in the returned subset. -// Returned identifiers should all be of one of the specified `roles`. -// Note: this method should not include the identifier of its executor. -func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow.IdentifierList, roles flow.RoleList) (flow.IdentifierList, error) { +// subsetRole returns a random subset of the identity list that is passed. `shouldHave` represents set of +// identities that should be included in the returned subset. +// Returned identities should all be of one of the specified `roles`. +// Note: this method should not include identity of its executor. +func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.IdentityList, roles flow.RoleList) (flow.IdentityList, error) { // excludes irrelevant roles and the node itself from both should have and ids set - shouldHave = t.identityProvider. - Identities(filter.HasNodeID(shouldHave...)). - Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )).NodeIDs() - - ids = t.identityProvider. - Identities(filter.HasNodeID(ids...)). - Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )).NodeIDs() + shouldHave = shouldHave.Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )) + + ids = ids.Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )) sample, err := t.sampleConnectedGraph(ids, shouldHave) if err != nil { @@ -145,11 +138,11 @@ func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow. // different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHave flow.IdentityList) (flow.IdentityList, error) { if len(all) == 0 { t.logger.Debug().Msg("skips sampling connected graph with zero nodes") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } if len(shouldHave) == 0 { @@ -160,9 +153,9 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should } // checks `shouldHave` be a subset of `all` - nonMembers := shouldHave.Filter(idFilter.Not(idFilter.In(all...))) + nonMembers := shouldHave.Filter(filter.Not(filter.In(all))) if len(nonMembers) != 0 { - return nil, fmt.Errorf("should have identifiers is not a subset of all: %v", nonMembers) + return nil, fmt.Errorf("should have identities is not a subset of all: %v", nonMembers) } // total sample size @@ -177,7 +170,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should subsetSize := totalSize - len(shouldHave) // others are all excluding should have ones - others := all.Filter(idFilter.Not(idFilter.In(shouldHave...))) + others := all.Filter(filter.Not(filter.In(shouldHave))) others = others.DeterministicSample(uint(subsetSize), t.seed) return others.Union(shouldHave), nil @@ -185,7 +178,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityList) (flow.IdentityList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(t.myNodeID, t.state) if err != nil { @@ -193,18 +186,18 @@ func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.Identifie } // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) + nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } // samples a connected graph topology from the cluster peers - return t.subsetRole(clusterPeers, shouldHave.Filter(idFilter.In(clusterPeers...)), flow.RoleList{flow.RoleCollection}) + return t.subsetRole(clusterPeers, shouldHave.Filter(filter.HasNodeID(clusterPeers.NodeIDs()...)), flow.RoleList{flow.RoleCollection}) } // nonClusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. // The returned sample contains `shouldHave` ones that also subscribed to `channel`. -func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -216,5 +209,5 @@ func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.Identi } // samples a connected graph topology - return t.subsetRole(ids, shouldHave, roles) + return t.subsetRole(ids.Filter(filter.HasRole(roles...)), shouldHave, roles) } diff --git a/network/topology/topology_utils.go b/network/topology/topology_utils.go index 3668d8557a6..bf1d2b43986 100644 --- a/network/topology/topology_utils.go +++ b/network/topology/topology_utils.go @@ -51,7 +51,7 @@ func byteSeedFromID(id flow.Identifier) ([]byte, error) { } // clusterPeers returns the list of other nodes within the same cluster as specified identifier. -func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList, error) { +func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, error) { currentEpoch := state.Final().Epochs().Current() clusterList, err := currentEpoch.Clustering() if err != nil { @@ -63,5 +63,5 @@ func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList return nil, fmt.Errorf("failed to find the cluster for node ID %s", id.String()) } - return myCluster.NodeIDs(), nil + return myCluster, nil } From f159fa312de87daca10c65fa2a968962a8690ff7 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 16:06:35 -0700 Subject: [PATCH 026/291] fix compilation errors --- cmd/scaffold.go | 10 +++--- network/middleware.go | 7 ++-- network/p2p/middleware.go | 34 +++++++++++++------ network/p2p/network.go | 70 +++++++++------------------------------ network/topology.go | 8 ++--- 5 files changed, 52 insertions(+), 77 deletions(-) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index db04f2dee59..e49d0d05b5d 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -165,7 +165,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) } - fnb.Middleware = p2p.NewMiddleware(fnb.Logger.Level(zerolog.ErrorLevel), + fnb.Middleware = p2p.NewMiddleware( + fnb.Logger.Level(zerolog.ErrorLevel), libP2PNodeFactory, fnb.Me.NodeID(), fnb.Metrics.Network, @@ -175,12 +176,13 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { true, true, fnb.IDTranslator, - fnb.MsgValidators...) + p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + p2p.WithMessageValidators(fnb.MsgValidators...), + ) subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) top, err := topology.NewTopicBasedTopology( fnb.NodeID, - fnb.IdentityProvider, fnb.Logger, fnb.State, ) @@ -198,7 +200,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { topologyCache, subscriptionManager, fnb.Metrics.Network, - p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + fnb.IdentityProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) diff --git a/network/middleware.go b/network/middleware.go index 8bd89167549..b6d538da53a 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -6,7 +6,6 @@ import ( "time" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/message" ) @@ -60,11 +59,9 @@ type Middleware interface { // Overlay represents the interface that middleware uses to interact with the // overlay network layer. type Overlay interface { - // Topology returns an identifier list of nodes which this node should be directly connected to as peers - Topology() (flow.IdentifierList, error) + // Topology returns an identity list of nodes which this node should be directly connected to as peers + Topology() (flow.IdentityList, error) Receive(nodeID flow.Identifier, msg *message.Message) error - SetDefaultIdentifierProvider(id.IdentifierProvider) - Identifiers() flow.IdentifierList Identities() flow.IdentityList } diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 3164f0e8034..3829e9a1e1a 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -81,7 +81,21 @@ type Middleware struct { connectionGating bool managePeerConnections bool idTranslator IDTranslator - idProvider id.IdentityProvider + idProvider id.IdentifierProvider +} + +type MiddlewareOption func(*Middleware) + +func WithIdentifierProvider(provider id.IdentifierProvider) MiddlewareOption { + return func(mw *Middleware) { + mw.idProvider = provider + } +} + +func WithMessageValidators(validators ...network.MessageValidator) MiddlewareOption { + return func(mw *Middleware) { + mw.validators = validators + } } // NewMiddleware creates a new middleware instance @@ -104,8 +118,7 @@ func NewMiddleware( connectionGating bool, managePeerConnections bool, idTranslator IDTranslator, - idProvider id.IdentityProvider, - validators ...network.MessageValidator, + opts ...MiddlewareOption, ) *Middleware { ctx, cancel := context.WithCancel(context.Background()) @@ -129,11 +142,10 @@ func NewMiddleware( connectionGating: connectionGating, managePeerConnections: managePeerConnections, idTranslator: idTranslator, - idProvider: idProvider, } - if len(validators) != 0 { - mw.validators = validators + for _, opt := range opts { + opt(mw) } return mw @@ -147,17 +159,17 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes } func (m *Middleware) topologyPeers() (peer.IDSlice, error) { - identifiers, err := m.ov.Topology() + identities, err := m.ov.Topology() if err != nil { // TODO: format error return nil, err } - return m.peerIDs(identifiers), nil + return m.peerIDs(identities.NodeIDs()), nil } func (m *Middleware) allPeers() peer.IDSlice { - return m.peerIDs(m.ov.Identifiers()) + return m.peerIDs(m.idProvider.Identifiers()) } func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { @@ -199,7 +211,9 @@ func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov libP2PNode, err := m.libP2PNodeFactory() - ov.SetDefaultIdentifierProvider(NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator)) + if m.idProvider == nil { + m.idProvider = NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator) + } if err != nil { return fmt.Errorf("could not create libp2p node: %w", err) diff --git a/network/p2p/network.go b/network/p2p/network.go index b87957774c4..82235bd4938 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -41,29 +41,19 @@ type ReadyDoneAwareNetwork interface { // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { sync.RWMutex - idProvider id.IdentifierProvider - defaultIdProvider id.IdentifierProvider - identityProvider id.IdentityProvider - logger zerolog.Logger - codec network.Codec - me module.Local - mw network.Middleware - top network.Topology // used to determine fanout connections - metrics module.NetworkMetrics - rcache *RcvCache // used to deduplicate incoming messages - queue network.MessageQueue - ctx context.Context - cancel context.CancelFunc - subMngr network.SubscriptionManager // used to keep track of subscribed channels - lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle -} - -type NetworkOption func(*Network) - -func WithIdentifierProvider(provider id.IdentifierProvider) NetworkOption { - return func(net *Network) { - net.idProvider = provider - } + identityProvider id.IdentityProvider + logger zerolog.Logger + codec network.Codec + me module.Local + mw network.Middleware + top network.Topology // used to determine fanout connections + metrics module.NetworkMetrics + rcache *RcvCache // used to deduplicate incoming messages + queue network.MessageQueue + ctx context.Context + cancel context.CancelFunc + subMngr network.SubscriptionManager // used to keep track of subscribed channels + lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -80,7 +70,6 @@ func NewNetwork( sm network.SubscriptionManager, metrics module.NetworkMetrics, identityProvider id.IdentityProvider, - opts ...NetworkOption, ) (*Network, error) { rcache, err := newRcvCache(csize) @@ -109,10 +98,6 @@ func NewNetwork( // create workers to read from the queue and call queueSubmitFunc queue.CreateQueueWorkers(o.ctx, queue.DefaultNumWorkers, o.queue, o.queueSubmitFunc) - for _, opt := range opts { - opt(o) - } - return o, nil } @@ -178,35 +163,18 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -func (n *Network) getIdentifierProvider() id.IdentifierProvider { - if n.idProvider != nil { - return n.idProvider - } - n.RLock() - defer n.RUnlock() - if n.defaultIdProvider == nil { - n.logger.Fatal().Msg("TODO") - // TODO - } - return n.defaultIdProvider -} - -func (n *Network) Identifiers() flow.IdentifierList { - return n.getIdentifierProvider().Identifiers() -} - func (n *Network) Identities() flow.IdentityList { return n.identityProvider.Identities(NetworkingSetFilter) } -// Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. +// Topology returns the identitiess of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. -func (n *Network) Topology() (flow.IdentifierList, error) { +func (n *Network) Topology() (flow.IdentityList, error) { n.Lock() defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.Identities(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } @@ -313,12 +281,6 @@ func (n *Network) genNetworkMessage(channel network.Channel, event interface{}, return msg, nil } -func (n *Network) SetDefaultIdentifierProvider(provider id.IdentifierProvider) { - n.Lock() - n.defaultIdProvider = provider - n.Unlock() -} - // unicast sends the message in a reliable way to the given recipient. // It uses 1-1 direct messaging over the underlying network to deliver the message. // It returns an error if unicasting fails. diff --git a/network/topology.go b/network/topology.go index 1d0faf16212..1d746c70896 100644 --- a/network/topology.go +++ b/network/topology.go @@ -6,9 +6,9 @@ import ( // Topology provides a subset of nodes which a given node should directly connect to for 1-k messaging. type Topology interface { - // GenerateFanout receives IdentifierList of entire network, and list of channels the node is subscribing to. - // It constructs and returns the fanout IdentifierList of node. - // A node directly communicates with its fanout IdentifierList on epidemic dissemination + // GenerateFanout receives IdentityList of entire network, and list of channels the node is subscribing to. + // It constructs and returns the fanout IdentityList of node. + // A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. @@ -19,5 +19,5 @@ type Topology interface { // // GenerateFanout is not concurrency safe. It is responsibility of caller to lock for it. // with the channels argument, it allows the returned topology to be cached, which is necessary for randomized topology. - GenerateFanout(ids flow.IdentifierList, channels ChannelList) (flow.IdentifierList, error) + GenerateFanout(ids flow.IdentityList, channels ChannelList) (flow.IdentityList, error) } From b69484b3e9c48a5c783924cd048909546c05e0f3 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 16:15:02 -0700 Subject: [PATCH 027/291] more fixes --- .github/workflows/ci.yml | 1 - cmd/access/node_builder/access_node_builder.go | 10 +++++++--- .../node_builder/unstaked_access_node_builder.go | 5 +---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index ae24b5627d5..b1ec01599fe 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -11,7 +11,6 @@ on: branches: - master - 'auto-cadence-upgrade/**' - - test jobs: golangci: diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index cfb6e5f71a4..2cd73e0a3ba 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -150,6 +150,7 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components + UnstakedLibP2PNode *p2p.Node UnstakedNetwork *p2p.Network unstakedMiddleware *p2p.Middleware FollowerState protocol.MutableState @@ -614,7 +615,8 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, validators ...network.MessageValidator) *p2p.Middleware { - builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, + builder.unstakedMiddleware = p2p.NewMiddleware( + builder.Logger, factoryFunc, nodeID, networkMetrics, @@ -624,7 +626,9 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, false, // no connection gating for the unstaked network false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) builder.IDTranslator, - validators...) + p2p.WithMessageValidators(validators...), + // use default identifier provider + ) return builder.unstakedMiddleware } @@ -650,7 +654,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, topology, subscriptionManager, networkMetrics, - p2p.WithIdentifierProvider(builder.NetworkingIdentifierProvider), + builder.IdentityProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 88ce203fe65..856a10482ba 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -134,11 +134,8 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - participants := flow.IdentityList{} - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, nil) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) builder.MustNot(err) builder.UnstakedNetwork = network From 9f8b80ee5607e5ac6534053c0b78c5fe0b39c7f8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 16:27:46 -0700 Subject: [PATCH 028/291] more fixes --- cmd/access/node_builder/staked_access_node_builder.go | 1 + cmd/access/node_builder/unstaked_access_node_builder.go | 3 ++- cmd/access/node_builder/upstream_connector.go | 7 ++++++- network/p2p/middleware.go | 4 ++++ 4 files changed, 13 insertions(+), 2 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index d022c886700..a8e5a9731c1 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -11,6 +11,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" ) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 856a10482ba..d2f9d8e02d0 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -105,7 +105,8 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - anb.SyncEngineParticipantsProvider = node.Network.GetIdentifierProvider() + // use the default identifier provider + anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() return nil }) anb.FlowAccessNodeBuilder.Build() diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index 0a4c3405dbd..aabfa446426 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -105,8 +105,13 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f return default: } + peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) + if err != nil { + // TODO: return formatted error + } + // try and connect to the bootstrap server - err := connector.unstakedNode.AddPeer(ctx, bootstrapPeer) + err = connector.unstakedNode.AddPeer(ctx, peerID) resultChan <- result{ id: bootstrapPeer, err: err, diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 3829e9a1e1a..3732e16baab 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -460,6 +460,10 @@ func (m *Middleware) UpdateAllowList() { m.peerManagerUpdate() } +func (m *Middleware) IdentifierProvider() id.IdentifierProvider { + return m.idProvider +} + // IsConnected returns true if this node is connected to the node with id nodeID. func (m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { peerID, err := m.idTranslator.GetPeerID(nodeID) From e3305e901097bca56dcdd486fd5ebf983ee1fe39 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 16:50:41 -0700 Subject: [PATCH 029/291] add custom id translator --- .../staked_access_node_builder.go | 11 ++++-- network/p2p/hierarchical_translator.go | 37 +++++++++++++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) create mode 100644 network/p2p/hierarchical_translator.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a8e5a9731c1..b7bb142d921 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -36,18 +36,20 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache // translator - // networking provider fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NetworkingSetFilter, ), idCache, ) + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) // TODO: need special providers here - // for network, needs one that recognizes both protocl state and peerstore - // same for translator + // NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // doesn't participate in unstaked network. + // If it does, then we can just use the default one (peerstoreProvider) return nil }) @@ -62,7 +64,8 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - builder.EnqueueNetworkInit(ctx) + // TODO: we should remove this call since we are no longer instantiating two networks + // builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network if builder.ParticipatesInUnstakedNetwork() { diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go new file mode 100644 index 00000000000..28dbdb03c31 --- /dev/null +++ b/network/p2p/hierarchical_translator.go @@ -0,0 +1,37 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type HierarchicalIDTranslator struct { + translators []IDTranslator +} + +func NewHierarchicalIDTranslator(translators ...IDTranslator) *HierarchicalIDTranslator { + return &HierarchicalIDTranslator{translators} +} + +func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + for _, translator := range t.translators { + pid, err := translator.GetPeerID(flowID) + if err == nil { + return pid, nil + } + } + return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v", flowID) +} + +func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { + for _, translator := range t.translators { + fid, err := translator.GetFlowID(peerID) + if err == nil { + return fid, nil + } + } + return flow.ZeroID, fmt.Errorf("could not find corresponding flow ID for peer ID %v", peerID) +} From 4aa3432fac7d019dbae85db72035e517ff0eb1b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 18 Aug 2021 11:10:53 -0400 Subject: [PATCH 030/291] validate ID translation --- go.mod | 4 +- go.sum | 6 +- network/p2p/unstaked_translator.go | 12 +-- network/p2p/unstaked_translator_test.go | 98 +++++++++++++++++++++++++ 4 files changed, 111 insertions(+), 9 deletions(-) create mode 100644 network/p2p/unstaked_translator_test.go diff --git a/go.mod b/go.mod index 6c068d789e3..0b941c99d87 100644 --- a/go.mod +++ b/go.mod @@ -46,7 +46,7 @@ require ( github.com/onflow/flow-emulator v0.20.3 github.com/onflow/flow-go-sdk v0.21.0 github.com/onflow/flow-go/crypto v0.18.0 - github.com/onflow/flow/protobuf/go/flow v0.2.0 + github.com/onflow/flow/protobuf/go/flow v0.2.2 github.com/opentracing/opentracing-go v1.2.0 github.com/pelletier/go-toml v1.7.0 // indirect github.com/pkg/errors v0.9.1 @@ -66,7 +66,7 @@ require ( golang.org/x/time v0.0.0-20191024005414-555d28b269f0 google.golang.org/api v0.31.0 google.golang.org/grpc v1.36.1 - google.golang.org/protobuf v1.26.0 + google.golang.org/protobuf v1.27.1 gotest.tools v2.2.0+incompatible pgregory.net/rapid v0.4.7 ) diff --git a/go.sum b/go.sum index 9a4f32df42b..8c3ecca07d6 100644 --- a/go.sum +++ b/go.sum @@ -969,8 +969,9 @@ github.com/onflow/flow-go-sdk v0.20.0/go.mod h1:52QZyLwU3p3UZ2FXOy+sRl4JPdtvJoae github.com/onflow/flow-go-sdk v0.21.0 h1:KRU6F80KZLD+CJLj57S2EaAPNJUx4qpFTw1Ok0AJZ1M= github.com/onflow/flow-go-sdk v0.21.0/go.mod h1:2xhtzwRAeItwbHQzHiIK2gPgLDw1hNPa0xYlpvx8Gx4= github.com/onflow/flow/protobuf/go/flow v0.1.9/go.mod h1:kRugbzZjwQqvevJhrnnCFMJZNmoSJmxlKt6hTGXZojM= -github.com/onflow/flow/protobuf/go/flow v0.2.0 h1:a4Cg0ekoqb76zeOEo1wtSWtlnhGXwcxebp0itFwGtlE= github.com/onflow/flow/protobuf/go/flow v0.2.0/go.mod h1:kRugbzZjwQqvevJhrnnCFMJZNmoSJmxlKt6hTGXZojM= +github.com/onflow/flow/protobuf/go/flow v0.2.2 h1:EVhA0w3lu+BG7RK39ojIJVghLH998iP7YC0V/Op0KnU= +github.com/onflow/flow/protobuf/go/flow v0.2.2/go.mod h1:gQxYqCfkI8lpnKsmIjwtN2mV/N2PIwc1I+RUK4HPIc8= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -1673,8 +1674,9 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= +google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 5e320ccb236..a3211c1fc68 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -1,6 +1,8 @@ package p2p import ( + "fmt" + crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/libp2p/go-libp2p-core/peer" "github.com/multiformats/go-multihash" @@ -27,16 +29,16 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { pk, err := peerID.ExtractPublicKey() if err != nil { - // return error + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) } - if pk.Type() != crypto_pb.KeyType_ECDSA { - // fail + if pk.Type() != crypto_pb.KeyType_Secp256k1 { + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) } data, err := pk.Raw() - if err != nil || data[0] != 0x02 { // TODO: check if this is the right byte to check - // fail + if err != nil || data[0] != 0x02 { + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) } return flow.HashToID(data[1:]), nil diff --git a/network/p2p/unstaked_translator_test.go b/network/p2p/unstaked_translator_test.go new file mode 100644 index 00000000000..46a3b1dd58c --- /dev/null +++ b/network/p2p/unstaked_translator_test.go @@ -0,0 +1,98 @@ +package p2p + +import ( + "crypto/rand" + "math" + "testing" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/stretchr/testify/require" + + fcrypto "github.com/onflow/flow-go/crypto" +) + +// This test shows we can't use ECDSA P-256 keys for libp2p and expect PeerID <=> PublicKey bijections +func TestIDTranslationP256(t *testing.T) { + loops := 50 + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSAP256) + + // check that we can not extract the public key back + // This makes sense: the x509 serialization of ECDSA P-256 keys in uncompressed form is 64 + 2 bytes, + // and libp2p uses multihash.IDENTITY only on serializations of less than 42 bytes + _, err := pID.ExtractPublicKey() + require.NotNil(t, err) + + } +} + +// This test shows we can use ECDSA Secp256k1 keys for libp2p and expect PeerID <=> PublicKey bijections +func TestIDTranslationSecp256k1(t *testing.T) { + loops := 50 + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) + + // check that we can extract the public key back + // This makes sense: the compressed serialization of ECDSA Secp256k1 keys is 33 + 2 bytes, + // and libp2p uses multihash.IDENTITY on serializations of less than 42 bytes + _, err := pID.ExtractPublicKey() + require.NoError(t, err) + + } +} + +func TestUnstakedTranslationRoundTrip(t *testing.T) { + loops := 50 + unstakedTranslator := NewUnstakedNetworkIDTranslator() + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) + + pk, err := pID.ExtractPublicKey() + require.NoError(t, err) + + // for a secp256k1 key, this is compressed representation preceded by 00 bits + // indicating the multihash.IDENTITY + pkBytes, err := pk.Raw() + require.NoError(t, err) + + // key is positive, roundtrip should be possible + if pkBytes[0] == 0x02 { + flowID, err := unstakedTranslator.GetFlowID(pID) + require.NoError(t, err) + retrievedPeerID, err := unstakedTranslator.GetPeerID(flowID) + require.NoError(t, err) + require.Equal(t, pID, retrievedPeerID) + } + + } +} + +func createPeerIDFromAlgo(t *testing.T, sa fcrypto.SigningAlgorithm) peer.ID { + seed := createSeed(t) + + // this matches GenerateNetworkingKeys, and is intended to validate the choices in cmd/bootstrap + key, err := fcrypto.GeneratePrivateKey(sa, seed) + require.NoError(t, err) + + // get the public key + pubKey := key.PublicKey() + + // extract the corresponding libp2p public Key + libp2pPubKey, err := LibP2PPublicKeyFromFlow(pubKey) + require.NoError(t, err) + + // obtain the PeerID based on libp2p's own rules + pID, err := peer.IDFromPublicKey(libp2pPubKey) + require.NoError(t, err) + + return pID +} + +func createSeed(t *testing.T) []byte { + seedLen := int(math.Max(fcrypto.KeyGenSeedMinLenECDSAP256, fcrypto.KeyGenSeedMinLenECDSASecp256k1)) + seed := make([]byte, seedLen) + n, err := rand.Read(seed) + require.NoError(t, err) + require.Equal(t, n, seedLen) + return seed +} From 8c999e2de80a46b4da050bf598e7d59a474fb0cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 18 Aug 2021 13:14:42 -0400 Subject: [PATCH 031/291] [network] make sure TestUnstakedTranslationRoundTrip runs 50 times --- network/p2p/unstaked_translator_test.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/network/p2p/unstaked_translator_test.go b/network/p2p/unstaked_translator_test.go index 46a3b1dd58c..9850cfa775a 100644 --- a/network/p2p/unstaked_translator_test.go +++ b/network/p2p/unstaked_translator_test.go @@ -11,6 +11,9 @@ import ( fcrypto "github.com/onflow/flow-go/crypto" ) +// For these test, refer to https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md for libp2p +// PeerID specifications and how they relate to keys. + // This test shows we can't use ECDSA P-256 keys for libp2p and expect PeerID <=> PublicKey bijections func TestIDTranslationP256(t *testing.T) { loops := 50 @@ -42,21 +45,25 @@ func TestIDTranslationSecp256k1(t *testing.T) { } func TestUnstakedTranslationRoundTrip(t *testing.T) { - loops := 50 + max_iterations := 50 unstakedTranslator := NewUnstakedNetworkIDTranslator() - for i := 0; i < loops; i++ { + + tested_vectors := 0 + + for ok := true; ok; ok = tested_vectors < max_iterations { pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) pk, err := pID.ExtractPublicKey() require.NoError(t, err) - // for a secp256k1 key, this is compressed representation preceded by 00 bits - // indicating the multihash.IDENTITY + // for a secp256k1 key, this is just the compressed representation pkBytes, err := pk.Raw() require.NoError(t, err) // key is positive, roundtrip should be possible if pkBytes[0] == 0x02 { + tested_vectors++ + flowID, err := unstakedTranslator.GetFlowID(pID) require.NoError(t, err) retrievedPeerID, err := unstakedTranslator.GetPeerID(flowID) From ccc042b9da6d35d9ecf9ee29f1f27fbe288efd43 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 11:47:34 -0700 Subject: [PATCH 032/291] do todos --- .../staked_access_node_builder.go | 3 +- cmd/access/node_builder/upstream_connector.go | 6 +++- cmd/node_builder.go | 2 +- network/p2p/libp2pNode.go | 15 ++++++--- network/p2p/middleware.go | 8 +++-- network/p2p/peerstore_provider.go | 19 +++++++---- network/p2p/protocol_state_provider.go | 33 ++++++++++++------- 7 files changed, 56 insertions(+), 30 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index b7bb142d921..7e6daedb1b8 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -46,8 +46,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { ) fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: need special providers here - // NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // TODO: NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN // doesn't participate in unstaked network. // If it does, then we can just use the default one (peerstoreProvider) diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index aabfa446426..00d4f7a7f3d 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -105,9 +105,13 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f return default: } + peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) if err != nil { - // TODO: return formatted error + resultChan <- result{ + id: flow.Identity{}, + err: err, + } } // try and connect to the bootstrap server diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 91672d88485..f57248bab95 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -135,7 +135,7 @@ type NodeConfig struct { StakingKey crypto.PrivateKey NetworkKey crypto.PrivateKey - // TODO: initialize these in scaffold and unstaked node + // ID providers IdentityProvider id.IdentityProvider IDTranslator p2p.IDTranslator NetworkingIdentifierProvider id.IdentifierProvider diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index d846b78c0dd..faeb74d2b87 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -42,6 +42,10 @@ const ( // maximum number of attempts to be made to connect to a remote node for 1-1 direct communication maxConnectAttempt = 3 + + // timeout for FindPeer queries to the DHT + // TODO: is this a sensible value? + findPeerQueryTimeout = 15 * time.Second ) // LibP2PFactoryFunc is a factory function type for generating libp2p Node instances. @@ -356,21 +360,22 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp default: } + // TODO: why were we doing this? Is it okay to remove? // remove the peer from the peer store if present - // TODO: why were we doing this? // n.host.Peerstore().ClearAddrs(peerID) if len(n.host.Peerstore().Addrs(peerID)) == 0 { - // TODO: add bunch of logging here + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") if n.dht != nil { - // TODO: adjust timeout - timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) - // try to find the peer + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + // try to find the peer using the dht _, err := n.dht.FindPeer(timedCtx, peerID) cancel() if err != nil { return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) } + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") } return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) } diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 3732e16baab..54d4fd8579d 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -161,7 +161,6 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes func (m *Middleware) topologyPeers() (peer.IDSlice, error) { identities, err := m.ov.Topology() if err != nil { - // TODO: format error return nil, err } @@ -178,7 +177,10 @@ func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { for _, fid := range flowIDs { pid, err := m.idTranslator.GetPeerID(fid) if err != nil { - // TODO: log here + // We probably don't need to fail the entire function here, since the other + // translations may still succeed + m.log.Err(err).Str("flowID", fid.String()).Msg("failed to translate to peer ID") + continue } result = append(result, pid) @@ -212,7 +214,7 @@ func (m *Middleware) Start(ov network.Overlay) error { libP2PNode, err := m.libP2PNodeFactory() if m.idProvider == nil { - m.idProvider = NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator) + m.idProvider = NewPeerstoreIdentifierProvider(m.log, libP2PNode.host, m.idTranslator) } if err != nil { diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go index cea16d466a9..c31b2cf6917 100644 --- a/network/p2p/peerstore_provider.go +++ b/network/p2p/peerstore_provider.go @@ -2,6 +2,7 @@ package p2p import ( "github.com/libp2p/go-libp2p-core/host" + "github.com/rs/zerolog" "github.com/onflow/flow-go/model/flow" ) @@ -9,23 +10,29 @@ import ( type PeerstoreIdentifierProvider struct { host host.Host idTranslator IDTranslator + logger zerolog.Logger } -func NewPeerstoreIdentifierProvider(host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { - return &PeerstoreIdentifierProvider{host: host, idTranslator: idTranslator} +func NewPeerstoreIdentifierProvider(logger zerolog.Logger, host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{ + logger: logger.With().Str("component", "peerstore-id-provider").Logger(), + host: host, + idTranslator: idTranslator, + } } func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { var result flow.IdentifierList - pids := p.host.Peerstore().PeersWithAddrs() // TODO: should we just call Peers here? + pids := p.host.Peerstore().PeersWithAddrs() for _, pid := range pids { flowID, err := p.idTranslator.GetFlowID(pid) if err != nil { - // TODO: log error - } else { - result = append(result, flowID) + p.logger.Err(err).Str("peerID", pid.Pretty()).Msg("failed to translate to Flow ID") + continue } + + result = append(result, flowID) } return result diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index d47542c3b18..9bcfb8b18b4 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -5,6 +5,7 @@ import ( "sync" "github.com/libp2p/go-libp2p-core/peer" + "github.com/rs/zerolog" "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" @@ -18,21 +19,24 @@ type ProtocolStateIDCache struct { identities flow.IdentityList state protocol.State mu sync.RWMutex - peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + peerIDs map[flow.Identifier]peer.ID flowIDs map[peer.ID]flow.Identifier + logger zerolog.Logger } func NewProtocolStateIDCache( + logger zerolog.Logger, state protocol.State, eventDistributer *events.Distributor, ) (*ProtocolStateIDCache, error) { provider := &ProtocolStateIDCache{ - state: state, + state: state, + logger: logger.With().Str("component", "protocol-state-id-cache").Logger(), } head, err := state.Final().Head() if err != nil { - return nil, err // TODO: format the error + return nil, fmt.Errorf("failed to get latest state header: %w", err) } provider.update(head.ID()) @@ -41,24 +45,28 @@ func NewProtocolStateIDCache( return provider, nil } -func (p *ProtocolStateIDCache) EpochTransition(_ uint64, header *flow.Header) { - // TODO: maybe we actually want to log the epoch information from the arguments here (epoch phase, etc) +func (p *ProtocolStateIDCache) EpochTransition(newEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("newEpochCounter", newEpochCounter).Msg("epoch transition") p.update(header.ID()) } -func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { +func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(currentEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("currentEpochCounter", currentEpochCounter).Msg("epoch setup phase started") p.update(header.ID()) } -func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { +func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(currentEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("currentEpochCounter", currentEpochCounter).Msg("epoch committed phase started") p.update(header.ID()) } func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { - // TODO: log status here + p.logger.Info().Str("blockID", blockID.String()).Msg("updating cached identities") + identities, err := p.state.AtBlockID(blockID).Identities(filter.Any) if err != nil { - // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + // We don't want to continue with an expired identity list. + p.logger.Fatal().Err(err).Msg("failed to fetch new identities") } nIds := identities.Count() @@ -69,7 +77,8 @@ func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { for _, identity := range identities { pid, err := ExtractPeerID(identity.NetworkPubKey) if err != nil { - // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + p.logger.Err(err).Interface("identity", identity).Msg("failed to extract peer ID from network key") + continue } flowIDs[pid] = identity.NodeID @@ -116,13 +125,13 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { pk, err := PublicKey(networkPubKey) if err != nil { - // TODO: format the error + err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) return } pid, err = peer.IDFromPublicKey(pk) if err != nil { - // TODO: format the error + err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) return } From ba220e37a5031fbce84e6723934eca4307141216 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 11:54:24 -0700 Subject: [PATCH 033/291] fix todos --- cmd/access/node_builder/staked_access_node_builder.go | 2 +- cmd/access/node_builder/unstaked_access_node_builder.go | 2 +- cmd/scaffold.go | 2 +- network/p2p/unstaked_translator.go | 8 ++++---- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 7e6daedb1b8..233cb56819d 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -29,7 +29,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index d2f9d8e02d0..f6fd01e63d2 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -25,7 +25,7 @@ func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } diff --git a/cmd/scaffold.go b/cmd/scaffold.go index e49d0d05b5d..d0a41b48170 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -422,7 +422,7 @@ func (fnb *FlowNodeBuilder) initStorage() { func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index a3211c1fc68..62f5bde9668 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -20,7 +20,7 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID data := append([]byte{0x02}, flowID[:]...) mh, err := multihash.Sum(data, multihash.IDENTITY, -1) if err != nil { - // TODO: return error + return "", fmt.Errorf("failed to compute multihash: %w", err) } return peer.ID(mh), nil @@ -29,16 +29,16 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { pk, err := peerID.ExtractPublicKey() if err != nil { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) } if pk.Type() != crypto_pb.KeyType_Secp256k1 { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) } data, err := pk.Raw() if err != nil || data[0] != 0x02 { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) } return flow.HashToID(data[1:]), nil From 8d513b33f4c8aaeb675bdd522df6f79ea465302b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 12:45:03 -0700 Subject: [PATCH 034/291] update mocks, fix a test --- consensus/integration/nodes_test.go | 25 +++++++++++++++++++-- engine/access/mock/access_api_client.go | 30 +++++++++++++++++++++++++ engine/access/mock/access_api_server.go | 23 +++++++++++++++++++ network/mocknetwork/connector.go | 18 +++++---------- network/mocknetwork/middleware.go | 16 +++++-------- network/mocknetwork/overlay.go | 19 +++++----------- 6 files changed, 93 insertions(+), 38 deletions(-) diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index fe2dbd68ae7..789d48ec79f 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -24,6 +24,7 @@ import ( "github.com/onflow/flow-go/module/buffer" builder "github.com/onflow/flow-go/module/builder/consensus" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -31,6 +32,7 @@ import ( synccore "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/network/mocknetwork" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol" bprotocol "github.com/onflow/flow-go/state/protocol/badger" "github.com/onflow/flow-go/state/protocol/blocktimer" @@ -127,7 +129,7 @@ func createNode( setupsDB := storage.NewEpochSetups(metrics, db) commitsDB := storage.NewEpochCommits(metrics, db) statusesDB := storage.NewEpochStatuses(metrics, db) - consumer := events.NewNoop() + consumer := events.NewDistributor() state, err := bprotocol.Bootstrap(metrics, db, headersDB, sealsDB, resultsDB, blocksDB, setupsDB, commitsDB, statusesDB, rootSnapshot) require.NoError(t, err) @@ -230,8 +232,27 @@ func createNode( finalizedHeader, err := synceng.NewFinalizedHeaderCache(log, state, pubsub.NewFinalizationDistributor()) require.NoError(t, err) + idCache, err := p2p.NewProtocolStateIDCache(log, state, consumer) + require.NoError(t, err) + // initialize the synchronization engine - sync, err := synceng.New(log, metrics, net, me, blocksDB, comp, syncCore, finalizedHeader, state) + sync, err := synceng.New( + log, + metrics, + net, + me, + blocksDB, + comp, + syncCore, + finalizedHeader, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(me.NodeID())), + ), + idCache, + ), + ) require.NoError(t, err) pending := []*flow.Header{} diff --git a/engine/access/mock/access_api_client.go b/engine/access/mock/access_api_client.go index d38e98fdadf..884cd38b1a6 100644 --- a/engine/access/mock/access_api_client.go +++ b/engine/access/mock/access_api_client.go @@ -407,6 +407,36 @@ func (_m *AccessAPIClient) GetEventsForHeightRange(ctx context.Context, in *acce return r0, r1 } +// GetExecutionResultForBlockID provides a mock function with given fields: ctx, in, opts +func (_m *AccessAPIClient) GetExecutionResultForBlockID(ctx context.Context, in *access.GetExecutionResultForBlockIDRequest, opts ...grpc.CallOption) (*access.ExecutionResultForBlockIDResponse, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + var r0 *access.ExecutionResultForBlockIDResponse + if rf, ok := ret.Get(0).(func(context.Context, *access.GetExecutionResultForBlockIDRequest, ...grpc.CallOption) *access.ExecutionResultForBlockIDResponse); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*access.ExecutionResultForBlockIDResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *access.GetExecutionResultForBlockIDRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // GetLatestBlock provides a mock function with given fields: ctx, in, opts func (_m *AccessAPIClient) GetLatestBlock(ctx context.Context, in *access.GetLatestBlockRequest, opts ...grpc.CallOption) (*access.BlockResponse, error) { _va := make([]interface{}, len(opts)) diff --git a/engine/access/mock/access_api_server.go b/engine/access/mock/access_api_server.go index b475f874263..bc2f0026e52 100644 --- a/engine/access/mock/access_api_server.go +++ b/engine/access/mock/access_api_server.go @@ -314,6 +314,29 @@ func (_m *AccessAPIServer) GetEventsForHeightRange(_a0 context.Context, _a1 *acc return r0, r1 } +// GetExecutionResultForBlockID provides a mock function with given fields: _a0, _a1 +func (_m *AccessAPIServer) GetExecutionResultForBlockID(_a0 context.Context, _a1 *access.GetExecutionResultForBlockIDRequest) (*access.ExecutionResultForBlockIDResponse, error) { + ret := _m.Called(_a0, _a1) + + var r0 *access.ExecutionResultForBlockIDResponse + if rf, ok := ret.Get(0).(func(context.Context, *access.GetExecutionResultForBlockIDRequest) *access.ExecutionResultForBlockIDResponse); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*access.ExecutionResultForBlockIDResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *access.GetExecutionResultForBlockIDRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // GetLatestBlock provides a mock function with given fields: _a0, _a1 func (_m *AccessAPIServer) GetLatestBlock(_a0 context.Context, _a1 *access.GetLatestBlockRequest) (*access.BlockResponse, error) { ret := _m.Called(_a0, _a1) diff --git a/network/mocknetwork/connector.go b/network/mocknetwork/connector.go index 02f12add1b5..3f32fad44f6 100644 --- a/network/mocknetwork/connector.go +++ b/network/mocknetwork/connector.go @@ -5,8 +5,9 @@ package mocknetwork import ( context "context" - flow "github.com/onflow/flow-go/model/flow" mock "github.com/stretchr/testify/mock" + + peer "github.com/libp2p/go-libp2p-core/peer" ) // Connector is an autogenerated mock type for the Connector type @@ -14,16 +15,7 @@ type Connector struct { mock.Mock } -// UpdatePeers provides a mock function with given fields: ctx, ids -func (_m *Connector) UpdatePeers(ctx context.Context, ids flow.IdentityList) error { - ret := _m.Called(ctx, ids) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, flow.IdentityList) error); ok { - r0 = rf(ctx, ids) - } else { - r0 = ret.Error(0) - } - - return r0 +// UpdatePeers provides a mock function with given fields: ctx, peerIDs +func (_m *Connector) UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) { + _m.Called(ctx, peerIDs) } diff --git a/network/mocknetwork/middleware.go b/network/mocknetwork/middleware.go index 232990ff4cf..8d0a57067f6 100644 --- a/network/mocknetwork/middleware.go +++ b/network/mocknetwork/middleware.go @@ -122,15 +122,11 @@ func (_m *Middleware) Unsubscribe(channel network.Channel) error { } // UpdateAllowList provides a mock function with given fields: -func (_m *Middleware) UpdateAllowList() error { - ret := _m.Called() - - var r0 error - if rf, ok := ret.Get(0).(func() error); ok { - r0 = rf() - } else { - r0 = ret.Error(0) - } +func (_m *Middleware) UpdateAllowList() { + _m.Called() +} - return r0 +// UpdateNodeAddresses provides a mock function with given fields: +func (_m *Middleware) UpdateNodeAddresses() { + _m.Called() } diff --git a/network/mocknetwork/overlay.go b/network/mocknetwork/overlay.go index c096adf32b1..7c98d0b8fe5 100644 --- a/network/mocknetwork/overlay.go +++ b/network/mocknetwork/overlay.go @@ -14,27 +14,20 @@ type Overlay struct { mock.Mock } -// Identity provides a mock function with given fields: -func (_m *Overlay) Identity() (map[flow.Identifier]flow.Identity, error) { +// Identities provides a mock function with given fields: +func (_m *Overlay) Identities() flow.IdentityList { ret := _m.Called() - var r0 map[flow.Identifier]flow.Identity - if rf, ok := ret.Get(0).(func() map[flow.Identifier]flow.Identity); ok { + var r0 flow.IdentityList + if rf, ok := ret.Get(0).(func() flow.IdentityList); ok { r0 = rf() } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(map[flow.Identifier]flow.Identity) + r0 = ret.Get(0).(flow.IdentityList) } } - var r1 error - if rf, ok := ret.Get(1).(func() error); ok { - r1 = rf() - } else { - r1 = ret.Error(1) - } - - return r0, r1 + return r0 } // Receive provides a mock function with given fields: nodeID, msg From a9ad2b253ae3408867ca465f44b07f6e02d8e076 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 13:03:45 -0700 Subject: [PATCH 035/291] fix test --- consensus/integration/nodes_test.go | 15 ++++++--------- module/id/fixed_provider.go | 17 +++++++++++++++++ network/p2p/protocol_state_provider.go | 2 ++ 3 files changed, 25 insertions(+), 9 deletions(-) create mode 100644 module/id/fixed_provider.go diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index 789d48ec79f..caa537fb41b 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -32,7 +32,6 @@ import ( synccore "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/network/mocknetwork" - "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol" bprotocol "github.com/onflow/flow-go/state/protocol/badger" "github.com/onflow/flow-go/state/protocol/blocktimer" @@ -232,8 +231,12 @@ func createNode( finalizedHeader, err := synceng.NewFinalizedHeaderCache(log, state, pubsub.NewFinalizationDistributor()) require.NoError(t, err) - idCache, err := p2p.NewProtocolStateIDCache(log, state, consumer) + identities, err := state.Final().Identities(filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(me.NodeID())), + )) require.NoError(t, err) + idProvider := id.NewFixedIdentifierProvider(identities.NodeIDs()) // initialize the synchronization engine sync, err := synceng.New( @@ -245,13 +248,7 @@ func createNode( comp, syncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(me.NodeID())), - ), - idCache, - ), + idProvider, ) require.NoError(t, err) diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go new file mode 100644 index 00000000000..dbdf10b9403 --- /dev/null +++ b/module/id/fixed_provider.go @@ -0,0 +1,17 @@ +package id + +import ( + "github.com/onflow/flow-go/model/flow" +) + +type FixedIdentifierProvider struct { + identifiers flow.IdentifierList +} + +func NewFixedIdentifierProvider(identifiers flow.IdentifierList) *FixedIdentifierProvider { + return &FixedIdentifierProvider{identifiers} +} + +func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { + return p.identifiers +} diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 9bcfb8b18b4..9290a860e06 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -75,6 +75,8 @@ func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { flowIDs := make(map[peer.ID]flow.Identifier, nIds) for _, identity := range identities { + p.logger.Debug().Interface("identity", identity).Msg("extracting peer ID from network key") + pid, err := ExtractPeerID(identity.NetworkPubKey) if err != nil { p.logger.Err(err).Interface("identity", identity).Msg("failed to extract peer ID from network key") From 1da09e317ad81de58dd02968ed8f79f187f54550 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 15:56:21 -0700 Subject: [PATCH 036/291] fix some tests --- network/p2p/connGater.go | 2 +- network/p2p/dht_test.go | 2 +- network/p2p/libp2pConnector.go | 2 +- network/p2p/libp2pNode.go | 39 ++++++++++---------- network/p2p/libp2pNode_test.go | 65 +++++++++++++++------------------- 5 files changed, 50 insertions(+), 60 deletions(-) diff --git a/network/p2p/connGater.go b/network/p2p/connGater.go index caa05bb1f1a..64ebe02eed0 100644 --- a/network/p2p/connGater.go +++ b/network/p2p/connGater.go @@ -29,7 +29,7 @@ func NewConnGater(log zerolog.Logger) *ConnGater { } // update updates the peer ID map -func (c *ConnGater) update(pids []peer.ID) { +func (c *ConnGater) update(pids peer.IDSlice) { // create a new peer.ID map peerIDs := make(map[peer.ID]struct{}, len(pids)) diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index 8d2134f0591..dc300338ac4 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -180,7 +180,7 @@ func (suite *DHTTestSuite) CreateNodes(count int, dhtServer bool) (nodes []*Node n, err := NewDefaultLibP2PNodeBuilder(flow.Identifier{}, "0.0.0.0:0", key). SetRootBlockID(rootBlockID). SetConnectionManager(connManager). - SetPubsubOptions(WithDHTDiscovery(AsServer(dhtServer))). + SetDHTOptions(AsServer(dhtServer)). SetPingInfoProvider(pingInfoProvider). SetLogger(logger). Build(suite.ctx) diff --git a/network/p2p/libp2pConnector.go b/network/p2p/libp2pConnector.go index 8e38ade7f8a..f27a1931764 100644 --- a/network/p2p/libp2pConnector.go +++ b/network/p2p/libp2pConnector.go @@ -94,7 +94,7 @@ func (l *libp2pConnector) connectToPeers(ctx context.Context, peerIDs peer.IDSli // trimAllConnectionsExcept trims all connections of the node from peers not part of peerIDs. // A node would have created such extra connections earlier when the identity list may have been different, or // it may have been target of such connections from node which have now been excluded. -func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs []peer.ID) { +func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs peer.IDSlice) { // convert the peerInfos to a peer.ID -> bool map peersToKeep := make(map[peer.ID]bool, len(peerIDs)) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index faeb74d2b87..3059ba408e2 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -312,8 +312,8 @@ func (n *Node) Stop() (chan struct{}, error) { } // AddPeer adds a peer to this node by adding it to this node's peerstore and connecting to it -func (n *Node) AddPeer(ctx context.Context, peerID peer.ID) error { - err := n.host.Connect(ctx, peer.AddrInfo{ID: peerID}) +func (n *Node) AddPeer(ctx context.Context, peerInfo peer.AddrInfo) error { + err := n.host.Connect(ctx, peerInfo) if err != nil { return err } @@ -332,6 +332,21 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { // CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { + if len(n.host.Peerstore().Addrs(peerID)) == 0 { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") + if n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + // try to find the peer using the dht + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + } + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + } + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) if err != nil { @@ -364,22 +379,6 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp // remove the peer from the peer store if present // n.host.Peerstore().ClearAddrs(peerID) - if len(n.host.Peerstore().Addrs(peerID)) == 0 { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - if n.dht != nil { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - // try to find the peer using the dht - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) - } - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") - } - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) - } - // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() if swm, ok := network.(*swarm.Swarm); ok { @@ -393,7 +392,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp time.Sleep(time.Duration(r) * time.Millisecond) } - err := n.AddPeer(ctx, peerID) + err := n.AddPeer(ctx, peer.AddrInfo{ID: peerID}) if err != nil { // if the connection was rejected due to invalid node id, skip the re-attempt @@ -541,7 +540,7 @@ func (n *Node) Ping(ctx context.Context, peerID peer.ID) (message.PingResponse, } // UpdateAllowList allows the peer allow list to be updated. -func (n *Node) UpdateAllowList(peers []peer.ID) { +func (n *Node) UpdateAllowList(peers peer.IDSlice) { if n.connGater == nil { n.logger.Debug().Hex("node_id", logging.ID(n.id)).Msg("skipping update allow list, connection gating is not enabled") return diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index f7f5540a51c..c11c2a787e6 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -140,22 +140,13 @@ func (suite *LibP2PNodeTestSuite) TestAddPeers() { // add the remaining nodes to the first node as its set of peers for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, *identity)) + peerInfo, err := PeerAddressInfo(*identity) + require.NoError(suite.T(), err) + require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, peerInfo)) } // Checks if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Peerstore().Peers(), count) - - // Checks whether the first node is connected to the rest - for _, peer := range nodes[0].host.Peerstore().Peers() { - // A node is also a peer to itself but not marked as connected, hence skip checking that. - if nodes[0].host.ID().String() == peer.String() { - continue - } - assert.Eventuallyf(suite.T(), func() bool { - return network.Connected == nodes[0].host.Network().Connectedness(peer) - }, 2*time.Second, tickForAssertEventually, fmt.Sprintf(" first node is not connected to %s", peer.String())) - } + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) } // TestAddPeers checks if nodes can be added as peers to a given node @@ -165,39 +156,27 @@ func (suite *LibP2PNodeTestSuite) TestRemovePeers() { // create nodes nodes, identities := suite.NodesFixture(count, nil, false) + peerInfos, errs := peerInfosFromIDs(identities) + assert.Len(suite.T(), errs, 0) defer StopNodes(suite.T(), nodes) // add nodes two and three to the first node as its peers - for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, *identity)) + for _, pInfo := range peerInfos[1:] { + require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, pInfo)) } // check if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Peerstore().Peers(), count) - - // check whether the first node is connected to the rest - for _, peer := range nodes[0].host.Peerstore().Peers() { - // A node is also a peer to itself but not marked as connected, hence skip checking that. - if nodes[0].host.ID().String() == peer.String() { - continue - } - assert.Eventually(suite.T(), func() bool { - return network.Connected == nodes[0].host.Network().Connectedness(peer) - }, 2*time.Second, tickForAssertEventually) - } + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) // disconnect from each peer and assert that the connection no longer exists - for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].RemovePeer(suite.ctx, *identity)) - pInfo, err := PeerAddressInfo(*identity) - assert.NoError(suite.T(), err) + for _, pInfo := range peerInfos[1:] { + require.NoError(suite.T(), nodes[0].RemovePeer(suite.ctx, pInfo.ID)) assert.Equal(suite.T(), network.NotConnected, nodes[0].host.Network().Connectedness(pInfo.ID)) } } // TestCreateStreams checks if a new streams is created each time when CreateStream is called and an existing stream is not reused func (suite *LibP2PNodeTestSuite) TestCreateStream() { - count := 2 // Creates nodes @@ -213,7 +192,10 @@ func (suite *LibP2PNodeTestSuite) TestCreateStream() { // Now attempt to create another 100 outbound stream to the same destination by calling CreateStream var streams []network.Stream for i := 0; i < 100; i++ { - anotherStream, err := nodes[0].CreateStream(context.Background(), *id2) + pInfo, err := PeerAddressInfo(*id2) + require.NoError(suite.T(), err) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo)) + anotherStream, err := nodes[0].CreateStream(context.Background(), pInfo.ID) // Assert that a stream was returned without error require.NoError(suite.T(), err) require.NotNil(suite.T(), anotherStream) @@ -264,9 +246,14 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { id1 := *identities[0] id2 := *identities[1] + pInfo1, err := PeerAddressInfo(id1) + require.NoError(suite.T(), err) + pInfo2, err := PeerAddressInfo(id2) + require.NoError(suite.T(), err) // Create stream from node 1 to node 2 - s1, err := nodes[0].CreateStream(context.Background(), id2) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo2)) + s1, err := nodes[0].CreateStream(context.Background(), pInfo2.ID) assert.NoError(suite.T(), err) rw := bufio.NewReadWriter(bufio.NewReader(s1), bufio.NewWriter(s1)) @@ -287,7 +274,8 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } // Create stream from node 2 to node 1 - s2, err := nodes[1].CreateStream(context.Background(), id1) + require.NoError(suite.T(), nodes[1].AddPeer(context.Background(), pInfo1)) + s2, err := nodes[1].CreateStream(context.Background(), pInfo1.ID) assert.NoError(suite.T(), err) rw = bufio.NewReadWriter(bufio.NewReader(s2), bufio.NewWriter(s2)) @@ -322,16 +310,19 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() require.NoError(suite.T(), listener.Close()) }() + silentNodeInfo, err := PeerAddressInfo(silentNodeId) + require.NoError(suite.T(), err) + // setup the context to expire after the default timeout ctx, cancel := context.WithTimeout(context.Background(), DefaultUnicastTimeout) defer cancel() // attempt to create a stream from node 1 to node 2 and assert that it fails after timeout grace := 1 * time.Second - var err error unittest.AssertReturnsBefore(suite.T(), func() { - _, err = nodes[0].CreateStream(ctx, silentNodeId) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), silentNodeInfo)) + _, err = nodes[0].CreateStream(ctx, silentNodeInfo.ID) }, DefaultUnicastTimeout+grace) assert.Error(suite.T(), err) From c3d403d942adc84428bd277ae5da16858c758686 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 16:56:15 -0700 Subject: [PATCH 037/291] fix a test --- network/p2p/libp2pNode_test.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index c11c2a787e6..eade36e0d7f 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -335,17 +335,22 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { goodNodes, goodNodeIds := suite.NodesFixture(2, nil, false) defer StopNodes(suite.T(), goodNodes) require.Len(suite.T(), goodNodeIds, 2) + goodNodeInfo1, err := PeerAddressInfo(*goodNodeIds[1]) + require.NoError(suite.T(), err) // create a silent node which never replies listener, silentNodeId := silentNodeFixture(suite.T()) defer func() { require.NoError(suite.T(), listener.Close()) }() + silentNodeInfo, err := PeerAddressInfo(silentNodeId) + require.NoError(suite.T(), err) // creates a stream to unresponsive node and makes sure that the stream creation is blocked blockedCallCh := unittest.RequireNeverReturnBefore(suite.T(), func() { - _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeId) // this call will block + require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), silentNodeInfo)) + _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeInfo.ID) // this call will block }, 1*time.Second, "CreateStream attempt to the unresponsive peer did not block") @@ -353,7 +358,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // requires same peer can still connect to the other regular peer without being blocked unittest.RequireReturnsBefore(suite.T(), func() { - _, err := goodNodes[0].CreateStream(suite.ctx, *goodNodeIds[1]) + require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), goodNodeInfo1)) + _, err := goodNodes[0].CreateStream(suite.ctx, goodNodeInfo1.ID) require.NoError(suite.T(), err) }, 1*time.Second, "creating stream to a responsive node failed while concurrently blocked on unresponsive node") From fa02119c6996740afa485e156c3fb32a78db4d35 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 12:44:25 -0700 Subject: [PATCH 038/291] fix test build --- network/p2p/libp2pNode_test.go | 49 ++++++++++++++++++++++------------ 1 file changed, 32 insertions(+), 17 deletions(-) diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index eade36e0d7f..2497d3f3f76 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -15,6 +15,7 @@ import ( golog "github.com/ipfs/go-log" addrutil "github.com/libp2p/go-addr-util" "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" swarm "github.com/libp2p/go-libp2p-swarm" "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr/net" @@ -378,6 +379,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { nodes, identities := suite.NodesFixture(2, nil, false) defer StopNodes(suite.T(), nodes) require.Len(suite.T(), identities, 2) + nodeInfo1, err := PeerAddressInfo(*identities[1]) + require.NoError(suite.T(), err) wg := sync.WaitGroup{} @@ -386,7 +389,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { createStream := func() { <-gate - _, err := nodes[0].CreateStream(suite.ctx, *identities[1]) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + _, err := nodes[0].CreateStream(suite.ctx, nodeInfo1.ID) assert.NoError(suite.T(), err) // assert that stream was successfully created wg.Done() } @@ -446,10 +450,13 @@ func (suite *LibP2PNodeTestSuite) TestStreamClosing() { // Creates nodes nodes, identities := suite.NodesFixture(2, handler, false) defer StopNodes(suite.T(), nodes) + nodeInfo1, err := PeerAddressInfo(*identities[1]) + require.NoError(suite.T(), err) for i := 0; i < count; i++ { // Create stream from node 1 to node 2 (reuse if one already exists) - s, err := nodes[0].CreateStream(context.Background(), *identities[1]) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + s, err := nodes[0].CreateStream(context.Background(), nodeInfo1.ID) assert.NoError(suite.T(), err) w := bufio.NewWriter(s) @@ -506,7 +513,9 @@ func (suite *LibP2PNodeTestSuite) TestPing() { func testPing(t *testing.T, source *Node, target flow.Identity, expectedVersion string, expectedHeight uint64) { pctx, cancel := context.WithCancel(context.Background()) defer cancel() - resp, rtt, err := source.Ping(pctx, target) + pid, err := ExtractPeerID(target.NetworkPubKey) + assert.NoError(t, err) + resp, rtt, err := source.Ping(pctx, pid) assert.NoError(t, err) assert.NotZero(t, rtt) assert.Equal(t, expectedVersion, resp.Version) @@ -522,10 +531,14 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { node1 := nodes[0] node1Id := *identities[0] defer StopNode(suite.T(), node1) + node1Info, err := PeerAddressInfo(node1Id) + assert.NoError(suite.T(), err) node2 := nodes[1] node2Id := *identities[1] defer StopNode(suite.T(), node2) + node2Info, err := PeerAddressInfo(node2Id) + assert.NoError(suite.T(), err) requireError := func(err error) { require.Error(suite.T(), err) @@ -534,38 +547,40 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { suite.Run("outbound connection to a not-allowed node is rejected", func() { // node1 and node2 both have no allowListed peers - _, err := node1.CreateStream(suite.ctx, node2Id) + requireError(node1.AddPeer(context.Background(), node2Info)) + _, err := node1.CreateStream(suite.ctx, node2Info.ID) requireError(err) - _, err = node2.CreateStream(suite.ctx, node1Id) + requireError(node2.AddPeer(context.Background(), node1Info)) + _, err = node2.CreateStream(suite.ctx, node1Info.ID) requireError(err) }) suite.Run("inbound connection from an allowed node is rejected", func() { // node1 allowlists node2 but node2 does not allowlists node1 - err := node1.UpdateAllowList(flow.IdentityList{&node2Id}) - require.NoError(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node2Info.ID}) // node1 attempts to connect to node2 // node2 should reject the inbound connection - _, err = node1.CreateStream(suite.ctx, node2Id) + require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.Error(suite.T(), err) }) suite.Run("outbound connection to an approved node is allowed", func() { // node1 allowlists node2 - err := node1.UpdateAllowList(flow.IdentityList{&node2Id}) - require.NoError(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node2Info.ID}) // node2 allowlists node1 - err = node2.UpdateAllowList(flow.IdentityList{&node1Id}) - require.NoError(suite.T(), err) + node2.UpdateAllowList(peer.IDSlice{node1Info.ID}) // node1 should be allowed to connect to node2 - _, err = node1.CreateStream(suite.ctx, node2Id) + require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.NoError(suite.T(), err) // node2 should be allowed to connect to node1 - _, err = node2.CreateStream(suite.ctx, node1Id) + require.NoError(suite.T(), node2.AddPeer(context.Background(), node1Info)) + _, err = node2.CreateStream(suite.ctx, node1Info.ID) require.NoError(suite.T(), err) }) } @@ -576,12 +591,12 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGatingBootstrap() { node1 := node[0] node1Id := identity[0] defer StopNode(suite.T(), node1) + node1Info, err := PeerAddressInfo(*node1Id) + assert.NoError(suite.T(), err) suite.Run("updating allowlist of node w/o ConnGater does not crash", func() { - // node1 allowlists node1 - err := node1.UpdateAllowList(flow.IdentityList{node1Id}) - require.Error(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node1Info.ID}) }) } From 5213e9fe6d8b5b43ef0eb2d03a1edbffde4758eb Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 13:01:56 -0700 Subject: [PATCH 039/291] Update peerManager_test.go --- network/p2p/peerManager_test.go | 42 ++++++++++++++++++++++++++------- 1 file changed, 34 insertions(+), 8 deletions(-) diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 65cfa184c71..571a4a8108b 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -7,9 +7,11 @@ import ( "time" "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/model/flow" @@ -41,8 +43,14 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { currentIDs := unittest.IdentityListFixture(10) // setup a ID provider callback to return currentIDs - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } // track IDs that should be disconnected @@ -105,8 +113,14 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } connector := new(mocknetwork.Connector) @@ -137,8 +151,14 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } // chooses peer interval rate deliberately long to capture on demand peer update @@ -180,8 +200,14 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } connector := new(mocknetwork.Connector) From 17f8f05a9e5ea5a33f73b8a70c22f44a4a6b60a8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 14:10:46 -0700 Subject: [PATCH 040/291] Update sporking_test.go --- network/p2p/sporking_test.go | 30 +++++++++++++++++++----------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 935ac173a54..429f522bb0f 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -6,6 +6,7 @@ import ( "time" golog "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" @@ -13,7 +14,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/utils/unittest" ) @@ -50,9 +50,11 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { // create and start node 2 on localhost and random port node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootBlockID, nil, false, defaultAddress) + peerInfo2, err := PeerAddressInfo(id2) + require.NoError(suite.T(), err) // create stream from node 1 to node 2 - testOneToOneMessagingSucceeds(suite.T(), node1, id2) + testOneToOneMessagingSucceeds(suite.T(), node1, peerInfo2) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain @@ -71,7 +73,7 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { // attempt to create a stream from node 1 (old chain) to node 2 (new chain) // this time it should fail since node 2 is using a different public key // (and therefore has a different libp2p node id) - testOneToOneMessagingFails(suite.T(), node1, id2) + testOneToOneMessagingFails(suite.T(), node1, peerInfo2) } // TestOneToOneCrosstalkPrevention tests that a node from the old chain cannot talk directly to a node in the new chain @@ -85,13 +87,15 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { node1key := generateNetworkingKey(suite.T()) node1, id1 := NodeFixture(suite.T(), suite.logger, node1key, rootID1, nil, false, defaultAddress) defer StopNode(suite.T(), node1) + peerInfo1, err := PeerAddressInfo(id1) + require.NoError(suite.T(), err) // create and start node 2 on localhost and random port node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootID1, nil, false, defaultAddress) // create stream from node 2 to node 1 - testOneToOneMessagingSucceeds(suite.T(), node2, id1) + testOneToOneMessagingSucceeds(suite.T(), node2, peerInfo1) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain @@ -110,7 +114,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { // attempt to create a stream from node 2 (new chain) to node 1 (old chain) // this time it should fail since node 2 is listening on a different protocol - testOneToOneMessagingFails(suite.T(), node2, id1) + testOneToOneMessagingFails(suite.T(), node2, peerInfo1) } // TestOneToKCrosstalkPrevention tests that a node from the old chain cannot talk to a node in the new chain via PubSub @@ -128,7 +132,9 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { // create and start node 2 on localhost and random port with the same root block ID node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootIDBeforeSpork, nil, false, defaultAddress) + pInfo2, err := PeerAddressInfo(id2) defer StopNode(suite.T(), node2) + require.NoError(suite.T(), err) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -137,13 +143,13 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { topicBeforeSpork := engine.TopicFromChannel(engine.TestNetwork, rootIDBeforeSpork) // both nodes are initially on the same spork and subscribed to the same topic - _, err := node1.Subscribe(ctx, topicBeforeSpork) + _, err = node1.Subscribe(ctx, topicBeforeSpork) require.NoError(suite.T(), err) sub2, err := node2.Subscribe(ctx, topicBeforeSpork) require.NoError(suite.T(), err) // add node 2 as a peer of node 1 - err = node1.AddPeer(ctx, id2) + err = node1.AddPeer(ctx, pInfo2) require.NoError(suite.T(), err) // let the two nodes form the mesh @@ -170,17 +176,19 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { testOneToKMessagingFails(ctx, suite.T(), node1, sub2, topicAfterSpork) } -func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, targetId flow.Identity) { +func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { + require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from node 1 to node 2 - s, err := sourceNode.CreateStream(context.Background(), targetId) + s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation succeeded require.NoError(t, err) assert.NotNil(t, s) } -func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, targetId flow.Identity) { +func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { + require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from source node to destination address - _, err := sourceNode.CreateStream(context.Background(), targetId) + _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation failed assert.Error(t, err) // assert that it failed with the expected error From 929b9905c9b76d877c9cdb76ad563ea33539dc65 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 16:19:07 -0700 Subject: [PATCH 041/291] Update protocol_state_provider.go --- network/p2p/protocol_state_provider.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 9290a860e06..6f223a4d0fe 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -125,7 +125,7 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e } func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { - pk, err := PublicKey(networkPubKey) + pk, err := LibP2PPublicKeyFromFlow(networkPubKey) if err != nil { err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) return From 70079bc6109a632d2194f9d687a5d85258c290e0 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 19:29:35 -0700 Subject: [PATCH 042/291] Add new DHT test --- module/mocks/network.go | 63 ++++++------ network/p2p/dht.go | 2 +- network/p2p/dht_test.go | 46 +++++++++ network/p2p/libp2pNode.go | 23 +++-- storage/mocks/storage.go | 207 +++++++++++++++++++------------------- 5 files changed, 195 insertions(+), 146 deletions(-) diff --git a/module/mocks/network.go b/module/mocks/network.go index 38f65a552ce..b680ae44c89 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -5,38 +5,39 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" network "github.com/onflow/flow-go/network" - reflect "reflect" ) -// MockNetwork is a mock of Network interface +// MockNetwork is a mock of Network interface. type MockNetwork struct { ctrl *gomock.Controller recorder *MockNetworkMockRecorder } -// MockNetworkMockRecorder is the mock recorder for MockNetwork +// MockNetworkMockRecorder is the mock recorder for MockNetwork. type MockNetworkMockRecorder struct { mock *MockNetwork } -// NewMockNetwork creates a new mock instance +// NewMockNetwork creates a new mock instance. func NewMockNetwork(ctrl *gomock.Controller) *MockNetwork { mock := &MockNetwork{ctrl: ctrl} mock.recorder = &MockNetworkMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockNetwork) EXPECT() *MockNetworkMockRecorder { return m.recorder } -// Register mocks base method +// Register mocks base method. func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (network.Conduit, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Register", arg0, arg1) @@ -45,36 +46,36 @@ func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (netwo return ret0, ret1 } -// Register indicates an expected call of Register +// Register indicates an expected call of Register. func (mr *MockNetworkMockRecorder) Register(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Register", reflect.TypeOf((*MockNetwork)(nil).Register), arg0, arg1) } -// MockLocal is a mock of Local interface +// MockLocal is a mock of Local interface. type MockLocal struct { ctrl *gomock.Controller recorder *MockLocalMockRecorder } -// MockLocalMockRecorder is the mock recorder for MockLocal +// MockLocalMockRecorder is the mock recorder for MockLocal. type MockLocalMockRecorder struct { mock *MockLocal } -// NewMockLocal creates a new mock instance +// NewMockLocal creates a new mock instance. func NewMockLocal(ctrl *gomock.Controller) *MockLocal { mock := &MockLocal{ctrl: ctrl} mock.recorder = &MockLocalMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockLocal) EXPECT() *MockLocalMockRecorder { return m.recorder } -// Address mocks base method +// Address mocks base method. func (m *MockLocal) Address() string { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Address") @@ -82,13 +83,13 @@ func (m *MockLocal) Address() string { return ret0 } -// Address indicates an expected call of Address +// Address indicates an expected call of Address. func (mr *MockLocalMockRecorder) Address() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Address", reflect.TypeOf((*MockLocal)(nil).Address)) } -// NodeID mocks base method +// NodeID mocks base method. func (m *MockLocal) NodeID() flow.Identifier { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NodeID") @@ -96,13 +97,13 @@ func (m *MockLocal) NodeID() flow.Identifier { return ret0 } -// NodeID indicates an expected call of NodeID +// NodeID indicates an expected call of NodeID. func (mr *MockLocalMockRecorder) NodeID() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NodeID", reflect.TypeOf((*MockLocal)(nil).NodeID)) } -// NotMeFilter mocks base method +// NotMeFilter mocks base method. func (m *MockLocal) NotMeFilter() flow.IdentityFilter { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NotMeFilter") @@ -110,13 +111,13 @@ func (m *MockLocal) NotMeFilter() flow.IdentityFilter { return ret0 } -// NotMeFilter indicates an expected call of NotMeFilter +// NotMeFilter indicates an expected call of NotMeFilter. func (mr *MockLocalMockRecorder) NotMeFilter() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NotMeFilter", reflect.TypeOf((*MockLocal)(nil).NotMeFilter)) } -// Sign mocks base method +// Sign mocks base method. func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Sign", arg0, arg1) @@ -125,13 +126,13 @@ func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error return ret0, ret1 } -// Sign indicates an expected call of Sign +// Sign indicates an expected call of Sign. func (mr *MockLocalMockRecorder) Sign(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sign", reflect.TypeOf((*MockLocal)(nil).Sign), arg0, arg1) } -// SignFunc mocks base method +// SignFunc mocks base method. func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.PrivateKey, []byte, hash.Hasher) (crypto.Signature, error)) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SignFunc", arg0, arg1, arg2) @@ -140,66 +141,66 @@ func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.Pri return ret0, ret1 } -// SignFunc indicates an expected call of SignFunc +// SignFunc indicates an expected call of SignFunc. func (mr *MockLocalMockRecorder) SignFunc(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignFunc", reflect.TypeOf((*MockLocal)(nil).SignFunc), arg0, arg1, arg2) } -// MockRequester is a mock of Requester interface +// MockRequester is a mock of Requester interface. type MockRequester struct { ctrl *gomock.Controller recorder *MockRequesterMockRecorder } -// MockRequesterMockRecorder is the mock recorder for MockRequester +// MockRequesterMockRecorder is the mock recorder for MockRequester. type MockRequesterMockRecorder struct { mock *MockRequester } -// NewMockRequester creates a new mock instance +// NewMockRequester creates a new mock instance. func NewMockRequester(ctrl *gomock.Controller) *MockRequester { mock := &MockRequester{ctrl: ctrl} mock.recorder = &MockRequesterMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockRequester) EXPECT() *MockRequesterMockRecorder { return m.recorder } -// EntityByID mocks base method +// EntityByID mocks base method. func (m *MockRequester) EntityByID(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "EntityByID", arg0, arg1) } -// EntityByID indicates an expected call of EntityByID +// EntityByID indicates an expected call of EntityByID. func (mr *MockRequesterMockRecorder) EntityByID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EntityByID", reflect.TypeOf((*MockRequester)(nil).EntityByID), arg0, arg1) } -// Force mocks base method +// Force mocks base method. func (m *MockRequester) Force() { m.ctrl.T.Helper() m.ctrl.Call(m, "Force") } -// Force indicates an expected call of Force +// Force indicates an expected call of Force. func (mr *MockRequesterMockRecorder) Force() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Force", reflect.TypeOf((*MockRequester)(nil).Force)) } -// Query mocks base method +// Query mocks base method. func (m *MockRequester) Query(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Query", arg0, arg1) } -// Query indicates an expected call of Query +// Query indicates an expected call of Query. func (mr *MockRequesterMockRecorder) Query(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockRequester)(nil).Query), arg0, arg1) diff --git a/network/p2p/dht.go b/network/p2p/dht.go index b21211a9adb..a5b82657cb9 100644 --- a/network/p2p/dht.go +++ b/network/p2p/dht.go @@ -39,7 +39,7 @@ func AsServer(enable bool) dht.Option { if enable { return dht.Mode(dht.ModeServer) } - return dht.Mode(dht.ModeAuto) + return dht.Mode(dht.ModeClient) } func WithBootstrapPeers(bootstrapNodes flow.IdentityList) (dht.Option, error) { diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index dc300338ac4..cc01eedaafb 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -43,6 +43,52 @@ func (suite *DHTTestSuite) TearDownTest() { suite.cancel() } +func (suite *DHTTestSuite) TestFindPeerWithDHT() { + count := 10 + golog.SetAllLoggers(golog.LevelFatal) // change this to Debug if libp2p logs are needed + + dhtServerNodes := suite.CreateNodes(2, true) + require.Len(suite.T(), dhtServerNodes, 2) + + dhtClientNodes := suite.CreateNodes(count-2, false) + + nodes := append(dhtServerNodes, dhtClientNodes...) + defer suite.StopNodes(nodes) + + getDhtServerAddr := func(i uint) peer.AddrInfo { + return peer.AddrInfo{ID: dhtServerNodes[i].host.ID(), Addrs: dhtServerNodes[i].host.Addrs()} + } + + for i, clientNode := range dhtClientNodes { + err := clientNode.host.Connect(suite.ctx, getDhtServerAddr(uint(i%2))) + require.NoError(suite.T(), err) + } + + require.Eventually(suite.T(), func() bool { + for i, clientNode := range dhtClientNodes { + if clientNode.dht.RoutingTable().Find(getDhtServerAddr(uint(i%2)).ID) == "" { + return false + } + } + return true + }, time.Second*5, tickForAssertEventually, "nodes failed to connect") + + err := dhtServerNodes[0].host.Connect(suite.ctx, getDhtServerAddr(1)) + require.NoError(suite.T(), err) + + require.Eventually(suite.T(), func() bool { + return dhtServerNodes[0].dht.RoutingTable().Find(getDhtServerAddr(1).ID) != "" + }, time.Second*5, tickForAssertEventually, "dht servers failed to connect") + + for i := 0; i < len(dhtClientNodes); i += 2 { + for j := 1; j < len(dhtClientNodes); j += 2 { + dhtClientNodes[i].host.Peerstore().ClearAddrs(dhtClientNodes[j].host.ID()) + _, err = dhtClientNodes[i].CreateStream(suite.ctx, dhtClientNodes[j].host.ID()) + require.NoError(suite.T(), err) + } + } +} + // TestPubSub checks if nodes can subscribe to a topic and send and receive a message on that topic. The DHT discovery // mechanism is used for nodes to find each other. func (suite *DHTTestSuite) TestPubSubWithDHTDiscovery() { diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index cae93c5daa7..40a682795f5 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -334,18 +334,19 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { if len(n.host.Peerstore().Addrs(peerID)) == 0 { n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - if n.dht != nil { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - // try to find the peer using the dht - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) - } - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + if n.dht == nil { + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } + + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + // try to find the peer using the dht + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) } - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 2967a3e5cce..7a829429fc4 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -5,37 +5,38 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" - reflect "reflect" ) -// MockBlocks is a mock of Blocks interface +// MockBlocks is a mock of Blocks interface. type MockBlocks struct { ctrl *gomock.Controller recorder *MockBlocksMockRecorder } -// MockBlocksMockRecorder is the mock recorder for MockBlocks +// MockBlocksMockRecorder is the mock recorder for MockBlocks. type MockBlocksMockRecorder struct { mock *MockBlocks } -// NewMockBlocks creates a new mock instance +// NewMockBlocks creates a new mock instance. func NewMockBlocks(ctrl *gomock.Controller) *MockBlocks { mock := &MockBlocks{ctrl: ctrl} mock.recorder = &MockBlocksMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockBlocks) EXPECT() *MockBlocksMockRecorder { return m.recorder } -// ByCollectionID mocks base method +// ByCollectionID mocks base method. func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByCollectionID", arg0) @@ -44,13 +45,13 @@ func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByCollectionID indicates an expected call of ByCollectionID +// ByCollectionID indicates an expected call of ByCollectionID. func (mr *MockBlocksMockRecorder) ByCollectionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByCollectionID", reflect.TypeOf((*MockBlocks)(nil).ByCollectionID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -59,13 +60,13 @@ func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockBlocksMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockBlocks)(nil).ByHeight), arg0) } -// ByID mocks base method +// ByID mocks base method. func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -74,13 +75,13 @@ func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockBlocksMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockBlocks)(nil).ByID), arg0) } -// GetLastFullBlockHeight mocks base method +// GetLastFullBlockHeight mocks base method. func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetLastFullBlockHeight") @@ -89,13 +90,13 @@ func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { return ret0, ret1 } -// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight +// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight. func (mr *MockBlocksMockRecorder) GetLastFullBlockHeight() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).GetLastFullBlockHeight)) } -// IndexBlockForCollections mocks base method +// IndexBlockForCollections mocks base method. func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexBlockForCollections", arg0, arg1) @@ -103,13 +104,13 @@ func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow. return ret0 } -// IndexBlockForCollections indicates an expected call of IndexBlockForCollections +// IndexBlockForCollections indicates an expected call of IndexBlockForCollections. func (mr *MockBlocksMockRecorder) IndexBlockForCollections(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexBlockForCollections", reflect.TypeOf((*MockBlocks)(nil).IndexBlockForCollections), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockBlocks) Store(arg0 *flow.Block) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -117,13 +118,13 @@ func (m *MockBlocks) Store(arg0 *flow.Block) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockBlocksMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockBlocks)(nil).Store), arg0) } -// StoreTx mocks base method +// StoreTx mocks base method. func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreTx", arg0) @@ -131,13 +132,13 @@ func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { return ret0 } -// StoreTx indicates an expected call of StoreTx +// StoreTx indicates an expected call of StoreTx. func (mr *MockBlocksMockRecorder) StoreTx(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreTx", reflect.TypeOf((*MockBlocks)(nil).StoreTx), arg0) } -// UpdateLastFullBlockHeight mocks base method +// UpdateLastFullBlockHeight mocks base method. func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "UpdateLastFullBlockHeight", arg0) @@ -145,36 +146,36 @@ func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { return ret0 } -// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight +// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight. func (mr *MockBlocksMockRecorder) UpdateLastFullBlockHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).UpdateLastFullBlockHeight), arg0) } -// MockHeaders is a mock of Headers interface +// MockHeaders is a mock of Headers interface. type MockHeaders struct { ctrl *gomock.Controller recorder *MockHeadersMockRecorder } -// MockHeadersMockRecorder is the mock recorder for MockHeaders +// MockHeadersMockRecorder is the mock recorder for MockHeaders. type MockHeadersMockRecorder struct { mock *MockHeaders } -// NewMockHeaders creates a new mock instance +// NewMockHeaders creates a new mock instance. func NewMockHeaders(ctrl *gomock.Controller) *MockHeaders { mock := &MockHeaders{ctrl: ctrl} mock.recorder = &MockHeadersMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockHeaders) EXPECT() *MockHeadersMockRecorder { return m.recorder } -// BatchIndexByChunkID mocks base method +// BatchIndexByChunkID mocks base method. func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchIndexByChunkID", arg0, arg1, arg2) @@ -182,13 +183,13 @@ func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 stora return ret0 } -// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID +// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID. func (mr *MockHeadersMockRecorder) BatchIndexByChunkID(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchIndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).BatchIndexByChunkID), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -197,13 +198,13 @@ func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockHeadersMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockHeaders)(nil).ByBlockID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -212,13 +213,13 @@ func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockHeadersMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockHeaders)(nil).ByHeight), arg0) } -// ByParentID mocks base method +// ByParentID mocks base method. func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByParentID", arg0) @@ -227,13 +228,13 @@ func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { return ret0, ret1 } -// ByParentID indicates an expected call of ByParentID +// ByParentID indicates an expected call of ByParentID. func (mr *MockHeadersMockRecorder) ByParentID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByParentID", reflect.TypeOf((*MockHeaders)(nil).ByParentID), arg0) } -// IDByChunkID mocks base method +// IDByChunkID mocks base method. func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IDByChunkID", arg0) @@ -242,13 +243,13 @@ func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) return ret0, ret1 } -// IDByChunkID indicates an expected call of IDByChunkID +// IDByChunkID indicates an expected call of IDByChunkID. func (mr *MockHeadersMockRecorder) IDByChunkID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDByChunkID", reflect.TypeOf((*MockHeaders)(nil).IDByChunkID), arg0) } -// IndexByChunkID mocks base method +// IndexByChunkID mocks base method. func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexByChunkID", arg0, arg1) @@ -256,13 +257,13 @@ func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { return ret0 } -// IndexByChunkID indicates an expected call of IndexByChunkID +// IndexByChunkID indicates an expected call of IndexByChunkID. func (mr *MockHeadersMockRecorder) IndexByChunkID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).IndexByChunkID), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockHeaders) Store(arg0 *flow.Header) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -270,36 +271,36 @@ func (m *MockHeaders) Store(arg0 *flow.Header) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockHeadersMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockHeaders)(nil).Store), arg0) } -// MockPayloads is a mock of Payloads interface +// MockPayloads is a mock of Payloads interface. type MockPayloads struct { ctrl *gomock.Controller recorder *MockPayloadsMockRecorder } -// MockPayloadsMockRecorder is the mock recorder for MockPayloads +// MockPayloadsMockRecorder is the mock recorder for MockPayloads. type MockPayloadsMockRecorder struct { mock *MockPayloads } -// NewMockPayloads creates a new mock instance +// NewMockPayloads creates a new mock instance. func NewMockPayloads(ctrl *gomock.Controller) *MockPayloads { mock := &MockPayloads{ctrl: ctrl} mock.recorder = &MockPayloadsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockPayloads) EXPECT() *MockPayloadsMockRecorder { return m.recorder } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -308,13 +309,13 @@ func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockPayloadsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockPayloads)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -322,36 +323,36 @@ func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockPayloadsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockPayloads)(nil).Store), arg0, arg1) } -// MockCollections is a mock of Collections interface +// MockCollections is a mock of Collections interface. type MockCollections struct { ctrl *gomock.Controller recorder *MockCollectionsMockRecorder } -// MockCollectionsMockRecorder is the mock recorder for MockCollections +// MockCollectionsMockRecorder is the mock recorder for MockCollections. type MockCollectionsMockRecorder struct { mock *MockCollections } -// NewMockCollections creates a new mock instance +// NewMockCollections creates a new mock instance. func NewMockCollections(ctrl *gomock.Controller) *MockCollections { mock := &MockCollections{ctrl: ctrl} mock.recorder = &MockCollectionsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCollections) EXPECT() *MockCollectionsMockRecorder { return m.recorder } -// ByID mocks base method +// ByID mocks base method. func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -360,13 +361,13 @@ func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockCollectionsMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockCollections)(nil).ByID), arg0) } -// LightByID mocks base method +// LightByID mocks base method. func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByID", arg0) @@ -375,13 +376,13 @@ func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection return ret0, ret1 } -// LightByID indicates an expected call of LightByID +// LightByID indicates an expected call of LightByID. func (mr *MockCollectionsMockRecorder) LightByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByID", reflect.TypeOf((*MockCollections)(nil).LightByID), arg0) } -// LightByTransactionID mocks base method +// LightByTransactionID mocks base method. func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByTransactionID", arg0) @@ -390,13 +391,13 @@ func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.Ligh return ret0, ret1 } -// LightByTransactionID indicates an expected call of LightByTransactionID +// LightByTransactionID indicates an expected call of LightByTransactionID. func (mr *MockCollectionsMockRecorder) LightByTransactionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByTransactionID", reflect.TypeOf((*MockCollections)(nil).LightByTransactionID), arg0) } -// Remove mocks base method +// Remove mocks base method. func (m *MockCollections) Remove(arg0 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Remove", arg0) @@ -404,13 +405,13 @@ func (m *MockCollections) Remove(arg0 flow.Identifier) error { return ret0 } -// Remove indicates an expected call of Remove +// Remove indicates an expected call of Remove. func (mr *MockCollectionsMockRecorder) Remove(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockCollections)(nil).Remove), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCollections) Store(arg0 *flow.Collection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -418,13 +419,13 @@ func (m *MockCollections) Store(arg0 *flow.Collection) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCollectionsMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCollections)(nil).Store), arg0) } -// StoreLight mocks base method +// StoreLight mocks base method. func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLight", arg0) @@ -432,13 +433,13 @@ func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { return ret0 } -// StoreLight indicates an expected call of StoreLight +// StoreLight indicates an expected call of StoreLight. func (mr *MockCollectionsMockRecorder) StoreLight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLight", reflect.TypeOf((*MockCollections)(nil).StoreLight), arg0) } -// StoreLightAndIndexByTransaction mocks base method +// StoreLightAndIndexByTransaction mocks base method. func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLightAndIndexByTransaction", arg0) @@ -446,36 +447,36 @@ func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollec return ret0 } -// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction +// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction. func (mr *MockCollectionsMockRecorder) StoreLightAndIndexByTransaction(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLightAndIndexByTransaction", reflect.TypeOf((*MockCollections)(nil).StoreLightAndIndexByTransaction), arg0) } -// MockCommits is a mock of Commits interface +// MockCommits is a mock of Commits interface. type MockCommits struct { ctrl *gomock.Controller recorder *MockCommitsMockRecorder } -// MockCommitsMockRecorder is the mock recorder for MockCommits +// MockCommitsMockRecorder is the mock recorder for MockCommits. type MockCommitsMockRecorder struct { mock *MockCommits } -// NewMockCommits creates a new mock instance +// NewMockCommits creates a new mock instance. func NewMockCommits(ctrl *gomock.Controller) *MockCommits { mock := &MockCommits{ctrl: ctrl} mock.recorder = &MockCommitsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -483,13 +484,13 @@ func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockCommitsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockCommits)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -498,13 +499,13 @@ func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, err return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockCommitsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockCommits)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -512,36 +513,36 @@ func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) err return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCommitsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCommits)(nil).Store), arg0, arg1) } -// MockEvents is a mock of Events interface +// MockEvents is a mock of Events interface. type MockEvents struct { ctrl *gomock.Controller recorder *MockEventsMockRecorder } -// MockEventsMockRecorder is the mock recorder for MockEvents +// MockEventsMockRecorder is the mock recorder for MockEvents. type MockEventsMockRecorder struct { mock *MockEvents } -// NewMockEvents creates a new mock instance +// NewMockEvents creates a new mock instance. func NewMockEvents(ctrl *gomock.Controller) *MockEvents { mock := &MockEvents{ctrl: ctrl} mock.recorder = &MockEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockEvents) EXPECT() *MockEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -549,13 +550,13 @@ func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, ar return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -564,13 +565,13 @@ func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockEvents)(nil).ByBlockID), arg0) } -// ByBlockIDEventType mocks base method +// ByBlockIDEventType mocks base method. func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventType) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDEventType", arg0, arg1) @@ -579,13 +580,13 @@ func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventTyp return ret0, ret1 } -// ByBlockIDEventType indicates an expected call of ByBlockIDEventType +// ByBlockIDEventType indicates an expected call of ByBlockIDEventType. func (mr *MockEventsMockRecorder) ByBlockIDEventType(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDEventType", reflect.TypeOf((*MockEvents)(nil).ByBlockIDEventType), arg0, arg1) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -594,36 +595,36 @@ func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow. return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockEventsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockEvents)(nil).ByBlockIDTransactionID), arg0, arg1) } -// MockServiceEvents is a mock of ServiceEvents interface +// MockServiceEvents is a mock of ServiceEvents interface. type MockServiceEvents struct { ctrl *gomock.Controller recorder *MockServiceEventsMockRecorder } -// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents +// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents. type MockServiceEventsMockRecorder struct { mock *MockServiceEvents } -// NewMockServiceEvents creates a new mock instance +// NewMockServiceEvents creates a new mock instance. func NewMockServiceEvents(ctrl *gomock.Controller) *MockServiceEvents { mock := &MockServiceEvents{ctrl: ctrl} mock.recorder = &MockServiceEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -631,13 +632,13 @@ func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockServiceEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockServiceEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -646,36 +647,36 @@ func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockServiceEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockServiceEvents)(nil).ByBlockID), arg0) } -// MockTransactionResults is a mock of TransactionResults interface +// MockTransactionResults is a mock of TransactionResults interface. type MockTransactionResults struct { ctrl *gomock.Controller recorder *MockTransactionResultsMockRecorder } -// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults +// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults. type MockTransactionResultsMockRecorder struct { mock *MockTransactionResults } -// NewMockTransactionResults creates a new mock instance +// NewMockTransactionResults creates a new mock instance. func NewMockTransactionResults(ctrl *gomock.Controller) *MockTransactionResults { mock := &MockTransactionResults{ctrl: ctrl} mock.recorder = &MockTransactionResultsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -683,13 +684,13 @@ func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.Tr return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockTransactionResultsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockTransactionResults)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) (*flow.TransactionResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -698,7 +699,7 @@ func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifi return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockTransactionResultsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockTransactionResults)(nil).ByBlockIDTransactionID), arg0, arg1) From c768ac7e40b642150ae4b057bfac73d26375e81a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 19:44:20 -0700 Subject: [PATCH 043/291] fix test --- engine/common/synchronization/engine_test.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 093fa9cc2a0..4ce42c3ea69 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -18,12 +18,15 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" synccore "github.com/onflow/flow-go/module/synchronization" netint "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/mocknetwork" + "github.com/onflow/flow-go/network/p2p" protocolint "github.com/onflow/flow-go/state/protocol" + protocolEvents "github.com/onflow/flow-go/state/protocol/events" protocol "github.com/onflow/flow-go/state/protocol/mock" storerr "github.com/onflow/flow-go/storage" storage "github.com/onflow/flow-go/storage/mock" @@ -161,7 +164,15 @@ func (ss *SyncSuite) SetupTest() { finalizedHeader, err := NewFinalizedHeaderCache(log, ss.state, pubsub.NewFinalizationDistributor()) require.NoError(ss.T(), err, "could not create finalized snapshot cache") - e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, ss.state) + idCache, err := p2p.NewProtocolStateIDCache(log, ss.state, protocolEvents.NewDistributor()) + e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(ss.me.NodeID())), + ), + idCache, + )) require.NoError(ss.T(), err, "should pass engine initialization") ss.e = e From 3415ea89bddebf2095e07032551387e8154ff624 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 19:55:16 -0700 Subject: [PATCH 044/291] fix more testssss --- engine/common/synchronization/engine_test.go | 4 ++-- engine/testutil/nodes.go | 11 ++++++++++- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 4ce42c3ea69..92d488c787c 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -426,7 +426,7 @@ func (ss *SyncSuite) TestSendRequests() { ss.core.On("BatchRequested", batches[0]) // exclude my node ID - ss.e.sendRequests(ss.participants[1:], ranges, batches) + ss.e.sendRequests(ss.participants[1:].NodeIDs(), ranges, batches) ss.con.AssertExpectations(ss.T()) } @@ -483,6 +483,6 @@ func (ss *SyncSuite) TestOnFinalizedBlock() { err := ss.e.finalizedHeader.updateHeader() require.NoError(ss.T(), err) actualHeader := ss.e.finalizedHeader.Get() - require.ElementsMatch(ss.T(), ss.e.getParticipants(actualHeader.ID()), ss.participants[1:]) + require.ElementsMatch(ss.T(), ss.e.participantsProvider.Identifiers(), ss.participants[1:].NodeIDs()) require.Equal(ss.T(), actualHeader, &finalizedBlock) } diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index eedd08a0f8a..3688e2c6727 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -53,6 +53,7 @@ import ( "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" @@ -64,6 +65,7 @@ import ( chainsync "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/module/validation" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/stub" "github.com/onflow/flow-go/state/protocol" badgerstate "github.com/onflow/flow-go/state/protocol/badger" @@ -625,6 +627,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit finalizedHeader, err := synchronization.NewFinalizedHeaderCache(node.Log, node.State, finalizationDistributor) require.NoError(t, err) + idCache, err := p2p.NewProtocolStateIDCache(node.Log, node.State, events.NewDistributor()) syncEngine, err := synchronization.New( node.Log, node.Metrics, @@ -634,7 +637,13 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ), synchronization.WithPollInterval(time.Duration(0)), ) require.NoError(t, err) From bfd67a0ab48929b3baf0fbda360d286dcba1f316 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 20:03:22 -0700 Subject: [PATCH 045/291] Update testUtil.go --- network/test/testUtil.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 8f46326bcca..868f957027f 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -142,7 +142,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultPeerUpdateInterval, p2p.DefaultUnicastTimeout, true, - true) + true, + ) } return mws } From d00b68eb1591c092469a41e703ae83807465026f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Thu, 19 Aug 2021 12:01:03 -0400 Subject: [PATCH 046/291] go mod tidy --- integration/go.mod | 6 ------ 1 file changed, 6 deletions(-) diff --git a/integration/go.mod b/integration/go.mod index 076cd6f1e29..38d63d1e76a 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -15,12 +15,7 @@ require ( github.com/go-openapi/strfmt v0.20.1 // indirect github.com/go-test/deep v1.0.7 // indirect github.com/golang/snappy v0.0.3 // indirect - github.com/google/uuid v1.3.0 // indirect - github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/jedib0t/go-pretty v4.3.0+incompatible - github.com/libp2p/go-libp2p v0.14.4 // indirect - github.com/libp2p/go-libp2p-kad-dht v0.13.0 // indirect - github.com/libp2p/go-libp2p-swarm v0.5.3 // indirect github.com/logrusorgru/aurora v2.0.3+incompatible // indirect github.com/onflow/cadence v0.18.1-0.20210730161646-b891a21c51fd github.com/onflow/flow-core-contracts/lib/go/contracts v0.7.5 @@ -38,7 +33,6 @@ require ( github.com/uber/jaeger-lib v2.4.1+incompatible // indirect github.com/vmihailenco/msgpack/v4 v4.3.12 // indirect github.com/vmihailenco/tagparser v0.1.2 // indirect - go.uber.org/zap v1.18.1 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20210405174219-a39eb2f71cb9 // indirect google.golang.org/grpc v1.36.1 From 5a2528ce356310d6ac49f5715de2b6e7c85fd242 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Fri, 20 Aug 2021 12:51:30 -0400 Subject: [PATCH 047/291] [network] Create a simple FixedTableIdentityTranslator, fix compilation --- network/test/testUtil.go | 50 +++++++++++++++++++++++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 8f46326bcca..72686913292 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -89,6 +89,49 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics } } +type FixedTableIdentityTranslator struct { + flow2p2p map[flow.Identifier]peer.ID + p2p2flow map[peer.ID]flow.Identifier +} + +func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + nodeID, ok := t.p2p2flow[p] + if !ok { + return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) + } + return nodeID, nil +} + +func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + peerID, ok := t.flow2p2p[n] + if !ok { + return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) + } + return peerID, nil +} + +func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableIdentityTranslator { + flow2p2p := make(map[flow.Identifier]peer.ID) + p2p2flow := make(map[peer.ID]flow.Identifier) + + for identity := range identities { + nodeID := identity.ID() + networkKey := identity.NetworkPublicKey + peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + if err != nil { + panic("could not interpret a network public key from Flow, test identities setup problem") + } + peerID, err := peer.IDFromPublicKey(peerPK) + if err != nil { + panic("could not generate a PeerID from public Key, test identities setup problem") + } + + flow2p2p[nodeID] = peerID + p2p2flow[peerID] = nodeID + } + return &FixedTableIdentityTranslator{flow2p2p, p2p2flow} +} + // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. @@ -133,6 +176,9 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } + // create a fixed id translator for the identities + tableTranslator := NewFixedTableIdentityTranslator(identities) + // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, factory, @@ -142,7 +188,9 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultPeerUpdateInterval, p2p.DefaultUnicastTimeout, true, - true) + true, + tableTranslator, + ) } return mws } From 9df52325f50f1e9315a33af50d339f2b06af70ae Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 10:13:31 -0700 Subject: [PATCH 048/291] Update libp2pNode_test.go --- network/p2p/libp2pNode_test.go | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index 2497d3f3f76..82b5a3b92b5 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -16,6 +16,7 @@ import ( addrutil "github.com/libp2p/go-addr-util" "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" swarm "github.com/libp2p/go-libp2p-swarm" "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr/net" @@ -146,11 +147,11 @@ func (suite *LibP2PNodeTestSuite) TestAddPeers() { require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, peerInfo)) } - // Checks if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) + // Checks if both of the other nodes have been added as peers to the first node + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count-1) } -// TestAddPeers checks if nodes can be added as peers to a given node +// TestRemovePeers checks if nodes can be removed as peers from a given node func (suite *LibP2PNodeTestSuite) TestRemovePeers() { count := 3 @@ -166,8 +167,8 @@ func (suite *LibP2PNodeTestSuite) TestRemovePeers() { require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, pInfo)) } - // check if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) + // check if all other nodes have been added as peers to the first node + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count-1) // disconnect from each peer and assert that the connection no longer exists for _, pInfo := range peerInfos[1:] { @@ -195,7 +196,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStream() { for i := 0; i < 100; i++ { pInfo, err := PeerAddressInfo(*id2) require.NoError(suite.T(), err) - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo)) + nodes[0].host.Peerstore().AddAddrs(pInfo.ID, pInfo.Addrs, peerstore.AddressTTL) anotherStream, err := nodes[0].CreateStream(context.Background(), pInfo.ID) // Assert that a stream was returned without error require.NoError(suite.T(), err) @@ -253,7 +254,7 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { require.NoError(suite.T(), err) // Create stream from node 1 to node 2 - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo2)) + nodes[0].host.Peerstore().AddAddrs(pInfo2.ID, pInfo2.Addrs, peerstore.AddressTTL) s1, err := nodes[0].CreateStream(context.Background(), pInfo2.ID) assert.NoError(suite.T(), err) rw := bufio.NewReadWriter(bufio.NewReader(s1), bufio.NewWriter(s1)) @@ -275,7 +276,7 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } // Create stream from node 2 to node 1 - require.NoError(suite.T(), nodes[1].AddPeer(context.Background(), pInfo1)) + nodes[1].host.Peerstore().AddAddrs(pInfo1.ID, pInfo1.Addrs, peerstore.AddressTTL) s2, err := nodes[1].CreateStream(context.Background(), pInfo1.ID) assert.NoError(suite.T(), err) rw = bufio.NewReadWriter(bufio.NewReader(s2), bufio.NewWriter(s2)) @@ -322,7 +323,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() grace := 1 * time.Second unittest.AssertReturnsBefore(suite.T(), func() { - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), silentNodeInfo)) + nodes[0].host.Peerstore().AddAddrs(silentNodeInfo.ID, silentNodeInfo.Addrs, peerstore.AddressTTL) _, err = nodes[0].CreateStream(ctx, silentNodeInfo.ID) }, DefaultUnicastTimeout+grace) @@ -350,7 +351,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // creates a stream to unresponsive node and makes sure that the stream creation is blocked blockedCallCh := unittest.RequireNeverReturnBefore(suite.T(), func() { - require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), silentNodeInfo)) + goodNodes[0].host.Peerstore().AddAddrs(silentNodeInfo.ID, silentNodeInfo.Addrs, peerstore.AddressTTL) _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeInfo.ID) // this call will block }, 1*time.Second, @@ -359,7 +360,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // requires same peer can still connect to the other regular peer without being blocked unittest.RequireReturnsBefore(suite.T(), func() { - require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), goodNodeInfo1)) + goodNodes[0].host.Peerstore().AddAddrs(goodNodeInfo1.ID, goodNodeInfo1.Addrs, peerstore.AddressTTL) _, err := goodNodes[0].CreateStream(suite.ctx, goodNodeInfo1.ID) require.NoError(suite.T(), err) }, @@ -389,7 +390,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { createStream := func() { <-gate - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + nodes[0].host.Peerstore().AddAddrs(nodeInfo1.ID, nodeInfo1.Addrs, peerstore.AddressTTL) _, err := nodes[0].CreateStream(suite.ctx, nodeInfo1.ID) assert.NoError(suite.T(), err) // assert that stream was successfully created wg.Done() @@ -455,7 +456,7 @@ func (suite *LibP2PNodeTestSuite) TestStreamClosing() { for i := 0; i < count; i++ { // Create stream from node 1 to node 2 (reuse if one already exists) - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + nodes[0].host.Peerstore().AddAddrs(nodeInfo1.ID, nodeInfo1.Addrs, peerstore.AddressTTL) s, err := nodes[0].CreateStream(context.Background(), nodeInfo1.ID) assert.NoError(suite.T(), err) w := bufio.NewWriter(s) From 4729166aabc5ad4fe6c5f9380c61e4c9830e725a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 10:28:59 -0700 Subject: [PATCH 049/291] fix tests --- network/p2p/dht_test.go | 2 ++ network/p2p/libp2pNode.go | 13 +++++++++---- network/p2p/libp2pNode_test.go | 15 ++++++++------- network/p2p/sporking_test.go | 5 +++-- 4 files changed, 22 insertions(+), 13 deletions(-) diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index cc01eedaafb..6643a7652b0 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -29,6 +29,8 @@ type DHTTestSuite struct { cancel context.CancelFunc // used to cancel the context } +// TODO: test that dht findPeer times out + // TestDHTTestSuite test the libp2p pubsub with DHT for discovery func TestDHTTestSuite(t *testing.T) { suite.Run(t, new(DHTTestSuite)) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 40a682795f5..cae25e70292 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -339,10 +339,15 @@ func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stre } n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - // try to find the peer using the dht - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + if err != nil { return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) } diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index 82b5a3b92b5..a2bd6246735 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -514,9 +514,10 @@ func (suite *LibP2PNodeTestSuite) TestPing() { func testPing(t *testing.T, source *Node, target flow.Identity, expectedVersion string, expectedHeight uint64) { pctx, cancel := context.WithCancel(context.Background()) defer cancel() - pid, err := ExtractPeerID(target.NetworkPubKey) + pInfo, err := PeerAddressInfo(target) assert.NoError(t, err) - resp, rtt, err := source.Ping(pctx, pid) + source.host.Peerstore().AddAddrs(pInfo.ID, pInfo.Addrs, peerstore.AddressTTL) + resp, rtt, err := source.Ping(pctx, pInfo.ID) assert.NoError(t, err) assert.NotZero(t, rtt) assert.Equal(t, expectedVersion, resp.Version) @@ -548,10 +549,10 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { suite.Run("outbound connection to a not-allowed node is rejected", func() { // node1 and node2 both have no allowListed peers - requireError(node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err := node1.CreateStream(suite.ctx, node2Info.ID) requireError(err) - requireError(node2.AddPeer(context.Background(), node1Info)) + node2.host.Peerstore().AddAddrs(node1Info.ID, node1Info.Addrs, peerstore.AddressTTL) _, err = node2.CreateStream(suite.ctx, node1Info.ID) requireError(err) }) @@ -563,7 +564,7 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { // node1 attempts to connect to node2 // node2 should reject the inbound connection - require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.Error(suite.T(), err) }) @@ -576,11 +577,11 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { node2.UpdateAllowList(peer.IDSlice{node1Info.ID}) // node1 should be allowed to connect to node2 - require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.NoError(suite.T(), err) // node2 should be allowed to connect to node1 - require.NoError(suite.T(), node2.AddPeer(context.Background(), node1Info)) + node2.host.Peerstore().AddAddrs(node1Info.ID, node1Info.Addrs, peerstore.AddressTTL) _, err = node2.CreateStream(suite.ctx, node1Info.ID) require.NoError(suite.T(), err) }) diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 429f522bb0f..b3462e05ccb 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -7,6 +7,7 @@ import ( golog "github.com/ipfs/go-log" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" @@ -177,8 +178,8 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { } func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { - require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from node 1 to node 2 + sourceNode.host.Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation succeeded require.NoError(t, err) @@ -186,8 +187,8 @@ func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer } func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { - require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from source node to destination address + sourceNode.host.Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation failed assert.Error(t, err) From 4f9950b2d31cdbb094ea34c2c230cc5c3c0754a9 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 10:56:24 -0700 Subject: [PATCH 050/291] fix test --- network/p2p/libp2pNode.go | 2 +- network/p2p/libp2pNode_test.go | 12 +++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index cae25e70292..0fcb148ccd2 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -45,7 +45,7 @@ const ( // timeout for FindPeer queries to the DHT // TODO: is this a sensible value? - findPeerQueryTimeout = 15 * time.Second + findPeerQueryTimeout = 10 * time.Second ) // LibP2PFactoryFunc is a factory function type for generating libp2p Node instances. diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index a2bd6246735..f6ad627c76c 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -297,8 +297,8 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } } -// TestCreateStreamTimeoutWithUnresponsiveNode tests that the CreateStream call does not block longer than the default -// unicast timeout interval +// TestCreateStreamTimeoutWithUnresponsiveNode tests that the CreateStream call does not block longer than the +// timeout interval func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() { // creates a regular node @@ -315,18 +315,20 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() silentNodeInfo, err := PeerAddressInfo(silentNodeId) require.NoError(suite.T(), err) + timeout := 1 * time.Second + // setup the context to expire after the default timeout - ctx, cancel := context.WithTimeout(context.Background(), DefaultUnicastTimeout) + ctx, cancel := context.WithTimeout(context.Background(), timeout) defer cancel() // attempt to create a stream from node 1 to node 2 and assert that it fails after timeout - grace := 1 * time.Second + grace := 100 * time.Millisecond unittest.AssertReturnsBefore(suite.T(), func() { nodes[0].host.Peerstore().AddAddrs(silentNodeInfo.ID, silentNodeInfo.Addrs, peerstore.AddressTTL) _, err = nodes[0].CreateStream(ctx, silentNodeInfo.ID) }, - DefaultUnicastTimeout+grace) + timeout+grace) assert.Error(suite.T(), err) } From 1dbe017697fbfbdeec427e7fe1b93527c1059951 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 10:58:22 -0700 Subject: [PATCH 051/291] Update middleware_test.go --- network/test/middleware_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 62715056af9..17aa8ca2eca 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -105,8 +105,7 @@ func (m *MiddlewareTestSuite) SetupTest() { } for i, mw := range m.mws { assert.NoError(m.T(), mw.Start(m.ov[i])) - err := mw.UpdateAllowList() - require.NoError(m.T(), err) + mw.UpdateAllowList() } } From 1d639135179095c07423d212512ea68c26e72db8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 11:03:39 -0700 Subject: [PATCH 052/291] Update testUtil.go --- network/test/testUtil.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 72686913292..6bdc563808b 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -114,10 +114,10 @@ func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableId flow2p2p := make(map[flow.Identifier]peer.ID) p2p2flow := make(map[peer.ID]flow.Identifier) - for identity := range identities { + for _, identity := range identities { nodeID := identity.ID() - networkKey := identity.NetworkPublicKey - peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + networkKey := identity.NetworkPubKey + peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) if err != nil { panic("could not interpret a network public key from Flow, test identities setup problem") } From bee1f9d386f59b3a6225b5c8b4330cf0146f0c0d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 11:08:07 -0700 Subject: [PATCH 053/291] fixed identity provider --- module/id/fixed_provider.go | 12 ++++++++++++ network/test/testUtil.go | 13 ++++++++++++- 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index dbdf10b9403..106047510b6 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -15,3 +15,15 @@ func NewFixedIdentifierProvider(identifiers flow.IdentifierList) *FixedIdentifie func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { return p.identifiers } + +type FixedIdentityProvider struct { + identities flow.IdentityList +} + +func NewFixedIdentityProvider(identities flow.IdentityList) *FixedIdentityProvider { + return &FixedIdentityProvider{identities} +} + +func (p *FixedIdentityProvider) Identities(flow.IdentityFilter) flow.IdentityList { + return p.identities +} diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 6bdc563808b..c396b88cd7c 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,6 +20,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" @@ -232,7 +233,17 @@ func GenerateNetworks(t *testing.T, me.On("Address").Return(ids[i].Address) // create the network - net, err := p2p.NewNetwork(log, cbor.NewCodec(), ids, me, mws[i], csize, tops[i], sms[i], metrics) + net, err := p2p.NewNetwork( + log, + cbor.NewCodec(), + me, + mws[i], + csize, + tops[i], + sms[i], + metrics, + id.NewFixedIdentityProvider(ids), + ) require.NoError(t, err) nets = append(nets, net) From fe37be8a9c747ec018fc4ba2084b905ac633d2f5 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 11:08:59 -0700 Subject: [PATCH 054/291] Update testUtil.go --- network/test/testUtil.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index c396b88cd7c..bef17b84e6a 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -253,8 +253,6 @@ func GenerateNetworks(t *testing.T, if !dryRunMode { for _, net := range nets { <-net.Ready() - err := net.SetIDs(ids) - require.NoError(t, err) } } return nets From 39772637f48a1fe8e3ee63fb36cc715543c7ede6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Fri, 20 Aug 2021 14:00:53 -0400 Subject: [PATCH 055/291] [network] Fix the compilation of a bunch of remaining tests --- cmd/access/node_builder/upstream_connector.go | 5 +++-- network/test/testUtil.go | 1 - 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index 00d4f7a7f3d..fdc608cfadb 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -106,7 +106,8 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f default: } - peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) + peerAddrInfo, err := p2p.PeerAddressInfo(bootstrapPeer) + if err != nil { resultChan <- result{ id: flow.Identity{}, @@ -115,7 +116,7 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f } // try and connect to the bootstrap server - err = connector.unstakedNode.AddPeer(ctx, peerID) + err = connector.unstakedNode.AddPeer(ctx, peerAddrInfo) resultChan <- result{ id: bootstrapPeer, err: err, diff --git a/network/test/testUtil.go b/network/test/testUtil.go index bef17b84e6a..5d3b824c1a6 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,7 +20,6 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" From bc6b4016173dab0d32aa6b415a620a06edb05a58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Fri, 20 Aug 2021 14:00:53 -0400 Subject: [PATCH 056/291] Last testUtil fixes --- engine/testutil/nodes.go | 2 -- network/test/testUtil.go | 15 ++++++--------- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index 3688e2c6727..eb93482fcca 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -53,7 +53,6 @@ import ( "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" @@ -65,7 +64,6 @@ import ( chainsync "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/module/validation" - "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/stub" "github.com/onflow/flow-go/state/protocol" badgerstate "github.com/onflow/flow-go/state/protocol/badger" diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 5d3b824c1a6..d4c5edb88e6 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -110,21 +110,18 @@ func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, er return peerID, nil } -func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableIdentityTranslator { +func NewFixedTableIdentityTranslator(t *testing.T, identities flow.IdentityList) *FixedTableIdentityTranslator { flow2p2p := make(map[flow.Identifier]peer.ID) p2p2flow := make(map[peer.ID]flow.Identifier) for _, identity := range identities { - nodeID := identity.ID() + nodeID := identity.NodeID networkKey := identity.NetworkPubKey peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) - if err != nil { - panic("could not interpret a network public key from Flow, test identities setup problem") - } + require.NoError(t, err) + peerID, err := peer.IDFromPublicKey(peerPK) - if err != nil { - panic("could not generate a PeerID from public Key, test identities setup problem") - } + require.NoError(t, err) flow2p2p[nodeID] = peerID p2p2flow[peerID] = nodeID @@ -177,7 +174,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id } // create a fixed id translator for the identities - tableTranslator := NewFixedTableIdentityTranslator(identities) + tableTranslator := NewFixedTableIdentityTranslator(t, identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, From c740c212520afa8474c698d4746d55a623d0387b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 12:02:55 -0700 Subject: [PATCH 057/291] Update peerManager_test.go --- network/p2p/peerManager_test.go | 62 +++++++++++++++++---------------- 1 file changed, 32 insertions(+), 30 deletions(-) diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 571a4a8108b..07e64b06063 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -35,21 +35,32 @@ func (suite *PeerManagerTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) } +func (suite *PeerManagerTestSuite) generateIdentities(n int) (flow.IdentityList, peer.IDSlice) { + // create some test ids + currentIDs := unittest.IdentityListFixture(n) + + // setup a ID provider callback to return currentIDs + pids := peer.IDSlice{} + for _, id := range currentIDs { + key, err := generateFlowNetworkingKey(id.NodeID) + require.NoError(suite.T(), err) + id.NetworkPubKey = key.PublicKey() + pid, err := ExtractPeerID(key.PublicKey()) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + + return currentIDs, pids +} + // TestUpdatePeers tests that updatePeers calls the connector with the expected list of ids to connect and disconnect // from. The tests are cumulative and ordered. func (suite *PeerManagerTestSuite) TestUpdatePeers() { - // create some test ids - currentIDs := unittest.IdentityListFixture(10) + currentIDs, pids := suite.generateIdentities(10) - // setup a ID provider callback to return currentIDs + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -112,14 +123,11 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -150,14 +158,11 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -199,14 +204,11 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } From e764103b080457b469376f715eeec769a32ea9f4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 13:17:35 -0700 Subject: [PATCH 058/291] fixed peer manager test --- network/p2p/middleware.go | 8 +--- network/p2p/peerManager_test.go | 68 ++++++++++++++++++++++----------- 2 files changed, 48 insertions(+), 28 deletions(-) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 17836bee941..1fecf2b2cad 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -395,16 +395,12 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { // The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` // this requirement is fulfilled by e.g. the output of readConnection and readSubscription func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { - identities, err := m.ov.Identity() - if err != nil { - m.log.Error().Err(err).Msg("failed to retrieve identities list while delivering a message") - return - } + identities := m.ov.Identities() // check the origin of the message corresponds to the one claimed in the OriginID originID := flow.HashToID(msg.OriginID) - originIdentity, found := identities[originID] + originIdentity, found := identities.ByNodeID(originID) if !found { m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) return diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 07e64b06063..5c6632fbc76 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -1,6 +1,7 @@ package p2p import ( + "math/rand" "os" "sync" "testing" @@ -15,8 +16,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/utils/unittest" ) @@ -57,22 +56,19 @@ func (suite *PeerManagerTestSuite) generateIdentities(n int) (flow.IdentityList, // from. The tests are cumulative and ordered. func (suite *PeerManagerTestSuite) TestUpdatePeers() { // create some test ids - currentIDs, pids := suite.generateIdentities(10) + _, pids := suite.generateIdentities(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { return pids, nil } - // track IDs that should be disconnected - var extraIDs flow.IdentityList - // create the connector mock to check ids requested for connect and disconnect connector := new(mocknetwork.Connector) - connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("flow.IdentityList")). + connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("peer.IDSlice")). Run(func(args mock.Arguments) { - idArg := args[1].(flow.IdentityList) - assertListsEqual(suite.T(), currentIDs, idArg) + idArg := args[1].(peer.IDSlice) + assertListsEqual(suite.T(), pids, idArg) }). Return(nil) @@ -88,8 +84,8 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // a subsequent call to updatePeers should request a connector.UpdatePeers to existing ids and new ids suite.Run("updatePeers connects to old and new peers", func() { // create a new id - newIDs := unittest.IdentityListFixture(1) - currentIDs = append(currentIDs, newIDs...) + _, newPIDs := suite.generateIdentities(1) + pids = append(pids, newPIDs...) pm.updatePeers() connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 2) @@ -98,8 +94,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // when ids are only excluded, connector.UpdatePeers should be called suite.Run("updatePeers disconnects from extra peers", func() { // delete an id - extraIDs = currentIDs.Sample(1) - currentIDs = currentIDs.Filter(filter.Not(filter.In(extraIDs))) + pids = removeRandomElement(pids) pm.updatePeers() connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 3) @@ -108,12 +103,12 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // addition and deletion of ids should result in a call to connector.UpdatePeers suite.Run("updatePeers connects to new peers and disconnects from extra peers", func() { // remove a couple of ids - extraIDs = currentIDs.Sample(2) - currentIDs = currentIDs.Filter(filter.Not(filter.In(extraIDs))) + pids = removeRandomElement(pids) + pids = removeRandomElement(pids) // add a couple of new ids - newIDs := unittest.IdentityListFixture(2) - currentIDs = append(currentIDs, newIDs...) + _, newPIDs := suite.generateIdentities(2) + pids = append(pids, newPIDs...) pm.updatePeers() @@ -121,6 +116,12 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { }) } +func removeRandomElement(pids peer.IDSlice) peer.IDSlice { + i := rand.Intn(len(pids)) + pids[i] = pids[len(pids)-1] + return pids[:len(pids)-1] +} + // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // create some test ids @@ -246,9 +247,32 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { }, 10*time.Second, 100*time.Millisecond) } -// assertListsEqual asserts that two identity list are equal ignoring the order -func assertListsEqual(t *testing.T, list1, list2 flow.IdentityList) { - list1 = list1.Sort(order.ByNodeIDAsc) - list2 = list2.Sort(order.ByNodeIDAsc) - assert.Equal(t, list1, list2) +// assertListsEqual asserts that two peer ID slices are equal ignoring the order +func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { + listsEqual := func() bool { + if len(list1) != len(list2) { + return false + } + + map1 := make(map[peer.ID]int) + map2 := make(map[peer.ID]int) + + for _, e1 := range list1 { + map1[e1]++ + } + for _, e2 := range list2 { + map2[e2]++ + } + + for key, val := range map1 { + if map2[key] != val { + return false + } + } + return true + } + + if !listsEqual() { + assert.Failf(t, "peer ID slices not equal", "list1: %v, list2: %v", list1, list2) + } } From 97ec7d7333e777eb1e3f78fef77bf10f16ab6e50 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 13:33:20 -0700 Subject: [PATCH 059/291] fix epoch transition test --- engine/testutil/nodes.go | 2 + network/test/epochtransition_test.go | 57 +++++++++------------------- network/test/testUtil.go | 1 + 3 files changed, 20 insertions(+), 40 deletions(-) diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index eb93482fcca..3688e2c6727 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -53,6 +53,7 @@ import ( "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" @@ -64,6 +65,7 @@ import ( chainsync "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/module/validation" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/stub" "github.com/onflow/flow-go/state/protocol" badgerstate "github.com/onflow/flow-go/state/protocol/badger" diff --git a/network/test/epochtransition_test.go b/network/test/epochtransition_test.go index af1c8ee7001..26de777dfa7 100644 --- a/network/test/epochtransition_test.go +++ b/network/test/epochtransition_test.go @@ -43,11 +43,10 @@ type MutableIdentityTableSuite struct { // testNode encapsulates the node state which includes its identity, middleware, network, // mesh engine and the id refresher type testNode struct { - id *flow.Identity - mw *p2p.Middleware - net *p2p.Network - engine *MeshEngine - idRefresher *p2p.NodeIDRefresher + id *flow.Identity + mw *p2p.Middleware + net *p2p.Network + engine *MeshEngine } // testNodeList encapsulates a list of test node and @@ -106,16 +105,6 @@ func (t *testNodeList) engines() []*MeshEngine { return engs } -func (t *testNodeList) idRefreshers() []*p2p.NodeIDRefresher { - t.RLock() - defer t.RUnlock() - idRefreshers := make([]*p2p.NodeIDRefresher, len(t.nodes)) - for i, node := range t.nodes { - idRefreshers[i] = node.idRefresher - } - return idRefreshers -} - func (t *testNodeList) networks() []*p2p.Network { t.RLock() defer t.RUnlock() @@ -131,6 +120,13 @@ func TestEpochTransitionTestSuite(t *testing.T) { t.Skip(fmt.Sprintf("test is flaky: %v", &MutableIdentityTableSuite{})) } +func (suite *MutableIdentityTableSuite) signalIdentityChanged() { + for _, n := range suite.testNodes.nodes { + n.mw.UpdateNodeAddresses() + n.mw.UpdateAllowList() + } +} + func (suite *MutableIdentityTableSuite) SetupTest() { suite.testNodes = newTestNodeList() suite.removedTestNodes = newTestNodeList() @@ -180,17 +176,13 @@ func (suite *MutableIdentityTableSuite) addNodes(count int) { // create the engines for the new nodes engines := GenerateEngines(suite.T(), nets) - // create the node refreshers - idRefereshers := suite.generateNodeIDRefreshers(nets) - // create the test engines for i := 0; i < count; i++ { node := testNode{ - id: ids[i], - mw: mws[i], - net: nets[i], - engine: engines[i], - idRefresher: idRefereshers[i], + id: ids[i], + mw: mws[i], + net: nets[i], + engine: engines[i], } suite.testNodes.append(node) } @@ -302,13 +294,6 @@ func (suite *MutableIdentityTableSuite) TestNodesAddedAndRemoved() { suite.assertNetworkPrimitives(remainingIDs, remainingEngs, removedIDs, removedEngines) } -// signalIdentityChanged update IDs for all the current set of nodes (simulating an epoch) -func (suite *MutableIdentityTableSuite) signalIdentityChanged() { - for _, r := range suite.testNodes.idRefreshers() { - r.OnIdentityTableChanged() - } -} - // assertConnected checks that the middleware of a node is directly connected // to at least half of the other nodes. func (suite *MutableIdentityTableSuite) assertConnected(mw *p2p.Middleware, ids flow.IdentityList) { @@ -317,7 +302,7 @@ func (suite *MutableIdentityTableSuite) assertConnected(mw *p2p.Middleware, ids require.Eventuallyf(t, func() bool { connections := 0 for _, id := range ids { - connected, err := mw.IsConnected(*id) + connected, err := mw.IsConnected(id.NodeID) require.NoError(t, err) if connected { connections++ @@ -337,7 +322,7 @@ func (suite *MutableIdentityTableSuite) assertDisconnected(mw *p2p.Middleware, i t := suite.T() require.Eventuallyf(t, func() bool { for _, id := range ids { - connected, err := mw.IsConnected(*id) + connected, err := mw.IsConnected(id.NodeID) require.NoError(t, err) if connected { return false @@ -452,11 +437,3 @@ func (suite *MutableIdentityTableSuite) sendMessage(fromID flow.Identifier, return send(event, fromEngine.con, toIDs.NodeIDs()...) } - -func (suite *MutableIdentityTableSuite) generateNodeIDRefreshers(nets []*p2p.Network) []*p2p.NodeIDRefresher { - refreshers := make([]*p2p.NodeIDRefresher, len(nets)) - for i, net := range nets { - refreshers[i] = p2p.NewNodeIDRefresher(suite.logger, suite.state, net.SetIDs) - } - return refreshers -} diff --git a/network/test/testUtil.go b/network/test/testUtil.go index d4c5edb88e6..329bc1b2e68 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,6 +20,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" From af12d175ed3f2922f51554e7eaa14766dee1e452 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 13:53:04 -0700 Subject: [PATCH 060/291] Update unstaked_translator.go --- network/p2p/unstaked_translator.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 62f5bde9668..400d77fd04c 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -3,9 +3,9 @@ package p2p import ( "fmt" + lcrypto "github.com/libp2p/go-libp2p-core/crypto" crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/libp2p/go-libp2p-core/peer" - "github.com/multiformats/go-multihash" "github.com/onflow/flow-go/model/flow" ) @@ -18,12 +18,19 @@ func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { data := append([]byte{0x02}, flowID[:]...) - mh, err := multihash.Sum(data, multihash.IDENTITY, -1) + + um := lcrypto.PubKeyUnmarshallers[crypto_pb.KeyType_Secp256k1] + key, err := um(data) + if err != nil { + return "", fmt.Errorf("failed to convert flow ID to libp2p public key: %w", err) + } + + pid, err := peer.IDFromPublicKey(key) if err != nil { - return "", fmt.Errorf("failed to compute multihash: %w", err) + return "", fmt.Errorf("failed to get peer ID from libp2p public key: %w", err) } - return peer.ID(mh), nil + return pid, nil } func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { From 6340b0fe5a557e86072c1f4697d5e7963e976a23 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 15:03:38 -0700 Subject: [PATCH 061/291] fixed middleware_test --- engine/common/synchronization/engine_test.go | 6 ++++++ network/p2p/hierarchical_translator_test.go | 15 +++++++++++++++ network/p2p/middleware.go | 1 - network/test/middleware_test.go | 8 ++------ 4 files changed, 23 insertions(+), 7 deletions(-) create mode 100644 network/p2p/hierarchical_translator_test.go diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 92d488c787c..4e79938fab7 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -61,6 +61,12 @@ func (ss *SyncSuite) SetupTest() { // generate own ID ss.participants = unittest.IdentityListFixture(3, unittest.WithRole(flow.RoleConsensus)) + keys, err := unittest.NetworkingKeys(len(ss.participants)) + require.NoError(ss.T(), err) + + for i, p := range ss.participants { + p.NetworkPubKey = keys[i].PublicKey() + } ss.myID = ss.participants[0].NodeID // generate a header for the final state diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go new file mode 100644 index 00000000000..e111ecd0e85 --- /dev/null +++ b/network/p2p/hierarchical_translator_test.go @@ -0,0 +1,15 @@ +package p2p + +import ( + "testing" + + "github.com/stretchr/testify/suite" +) + +type HierarchicalTranslatorTestSuite struct { + suite.Suite +} + +func TestHierarchicalTranslator(t *testing.T) { + suite.Run(t, new(HierarchicalTranslatorTestSuite)) +} diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 1fecf2b2cad..68614e59ea8 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -250,7 +250,6 @@ func (m *Middleware) Start(ov network.Overlay) error { // Stop will end the execution of the middleware and wait for it to end. func (m *Middleware) Stop() { - mgr, found := m.peerMgr() if found { // stops peer manager diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 2996d4c17ef..0685339d6fb 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -65,7 +65,7 @@ type MiddlewareTestSuite struct { } // TestMiddlewareTestSuit runs all the test methods in this test suit -func TestMiddlewareTestSuit(t *testing.T) { +func TestMiddlewareTestSuite(t *testing.T) { suite.Run(t, new(MiddlewareTestSuite)) } @@ -101,11 +101,7 @@ func (m *MiddlewareTestSuite) SetupTest() { overlay := &mocknetwork.Overlay{} m.ov = append(m.ov, overlay) - identifierToID := make(map[flow.Identifier]flow.Identity) - for _, id := range m.ids { - identifierToID[id.NodeID] = *id - } - overlay.On("Identity").Maybe().Return(identifierToID, nil) + overlay.On("Identities").Maybe().Return(flow.IdentityList(m.ids), nil) overlay.On("Topology").Maybe().Return(flow.IdentityList(m.ids), nil) } for i, mw := range m.mws { From 3f1c48c42514c8c3870758c54652bf4f24219547 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 15:20:27 -0700 Subject: [PATCH 062/291] Update execution_test.go --- engine/execution/execution_test.go | 68 ++++++++++++++++++++++++------ 1 file changed, 55 insertions(+), 13 deletions(-) diff --git a/engine/execution/execution_test.go b/engine/execution/execution_test.go index c7201df0e15..76d2ae13137 100644 --- a/engine/execution/execution_test.go +++ b/engine/execution/execution_test.go @@ -37,10 +37,24 @@ func TestExecutionFlow(t *testing.T) { chainID := flow.Testnet - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exeID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - verID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) + keys, errs := unittest.NetworkingKeys(4) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exeID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) + verID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[3].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID) @@ -286,12 +300,23 @@ func TestExecutionStateSyncMultipleExecutionNodes(t *testing.T) { chainID := flow.Emulator - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exe1ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - // exe2ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) + keys, errs := unittest.NetworkingKeys(4) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exe1ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exe1ID) + identities[3].NetworkPubKey = keys[3].PublicKey() collectionNode := testutil.GenericNodeFromParticipants(t, hub, colID, identities, chainID) defer collectionNode.Done() @@ -424,11 +449,28 @@ func TestBroadcastToMultipleVerificationNodes(t *testing.T) { chainID := flow.Emulator - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exeID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - ver1ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - ver2ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) + keys, errs := unittest.NetworkingKeys(5) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exeID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) + ver1ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[3].PublicKey()), + ) + ver2ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[4].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, ver1ID, ver2ID) From 2263b6a58e74b2235c2fb98980af9470284acc38 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 16:05:51 -0700 Subject: [PATCH 063/291] add test for hierchical translator --- network/p2p/fixed_translator.go | 53 +++++++++++++++++++++ network/p2p/hierarchical_translator.go | 5 +- network/p2p/hierarchical_translator_test.go | 43 +++++++++++++++++ network/test/testUtil.go | 43 +---------------- 4 files changed, 102 insertions(+), 42 deletions(-) create mode 100644 network/p2p/fixed_translator.go diff --git a/network/p2p/fixed_translator.go b/network/p2p/fixed_translator.go new file mode 100644 index 00000000000..ecfff75fbb1 --- /dev/null +++ b/network/p2p/fixed_translator.go @@ -0,0 +1,53 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type FixedTableIdentityTranslator struct { + flow2p2p map[flow.Identifier]peer.ID + p2p2flow map[peer.ID]flow.Identifier +} + +func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + nodeID, ok := t.p2p2flow[p] + if !ok { + return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) + } + return nodeID, nil +} + +func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + peerID, ok := t.flow2p2p[n] + if !ok { + return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) + } + return peerID, nil +} + +func NewFixedTableIdentityTranslator(identities flow.IdentityList) (*FixedTableIdentityTranslator, error) { + flow2p2p := make(map[flow.Identifier]peer.ID) + p2p2flow := make(map[peer.ID]flow.Identifier) + + for _, identity := range identities { + nodeID := identity.NodeID + networkKey := identity.NetworkPubKey + peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + if err != nil { + return nil, err + } + + peerID, err := peer.IDFromPublicKey(peerPK) + if err != nil { + return nil, err + } + + flow2p2p[nodeID] = peerID + p2p2flow[peerID] = nodeID + } + return &FixedTableIdentityTranslator{flow2p2p, p2p2flow}, nil +} diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go index 28dbdb03c31..4e4d0e77cd5 100644 --- a/network/p2p/hierarchical_translator.go +++ b/network/p2p/hierarchical_translator.go @@ -3,6 +3,7 @@ package p2p import ( "fmt" + "github.com/hashicorp/go-multierror" "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/model/flow" @@ -17,13 +18,15 @@ func NewHierarchicalIDTranslator(translators ...IDTranslator) *HierarchicalIDTra } func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + var errs *multierror.Error for _, translator := range t.translators { pid, err := translator.GetPeerID(flowID) if err == nil { return pid, nil } + errs = multierror.Append(errs, err) } - return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v", flowID) + return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v: %w", flowID, errs) } func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go index e111ecd0e85..6866547f378 100644 --- a/network/p2p/hierarchical_translator_test.go +++ b/network/p2p/hierarchical_translator_test.go @@ -3,13 +3,56 @@ package p2p import ( "testing" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "gotest.tools/assert" ) type HierarchicalTranslatorTestSuite struct { suite.Suite + translator *HierarchicalIDTranslator + ids flow.IdentityList +} + +func (suite *HierarchicalTranslatorTestSuite) SetupTest() { + suite.ids = unittest.IdentityListFixture(2, unittest.WithKeys) + t1, err := NewFixedTableIdentityTranslator(suite.ids[:1]) + require.NoError(suite.T(), err) + t2, err := NewFixedTableIdentityTranslator(suite.ids[1:]) + require.NoError(suite.T(), err) + + suite.translator = NewHierarchicalIDTranslator(t1, t2) } func TestHierarchicalTranslator(t *testing.T) { suite.Run(t, new(HierarchicalTranslatorTestSuite)) } + +func (suite *HierarchicalTranslatorTestSuite) TestFirstTranslatorSuccess() { + pid, err := suite.translator.GetPeerID(suite.ids[0].NodeID) + require.NoError(suite.T(), err) + fid, err := suite.translator.GetFlowID(pid) + assert.Equal(suite.T(), fid, suite.ids[0].NodeID) +} + +func (suite *HierarchicalTranslatorTestSuite) TestSecondTranslatorSuccess() { + pid, err := suite.translator.GetPeerID(suite.ids[1].NodeID) + require.NoError(suite.T(), err) + fid, err := suite.translator.GetFlowID(pid) + assert.Equal(suite.T(), fid, suite.ids[1].NodeID) +} + +func (suite *HierarchicalTranslatorTestSuite) TestTranslationFailure() { + fid := unittest.IdentifierFixture() + _, err := suite.translator.GetPeerID(fid) + require.Error(suite.T(), err) + + key, err := LibP2PPrivKeyFromFlow(generateNetworkingKey(suite.T())) + require.NoError(suite.T(), err) + pid, err := peer.IDFromPrivateKey(key) + _, err = suite.translator.GetFlowID(pid) + require.Error(suite.T(), err) +} diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 329bc1b2e68..d101779b899 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -90,46 +90,6 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics } } -type FixedTableIdentityTranslator struct { - flow2p2p map[flow.Identifier]peer.ID - p2p2flow map[peer.ID]flow.Identifier -} - -func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { - nodeID, ok := t.p2p2flow[p] - if !ok { - return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) - } - return nodeID, nil -} - -func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { - peerID, ok := t.flow2p2p[n] - if !ok { - return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) - } - return peerID, nil -} - -func NewFixedTableIdentityTranslator(t *testing.T, identities flow.IdentityList) *FixedTableIdentityTranslator { - flow2p2p := make(map[flow.Identifier]peer.ID) - p2p2flow := make(map[peer.ID]flow.Identifier) - - for _, identity := range identities { - nodeID := identity.NodeID - networkKey := identity.NetworkPubKey - peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) - require.NoError(t, err) - - peerID, err := peer.IDFromPublicKey(peerPK) - require.NoError(t, err) - - flow2p2p[nodeID] = peerID - p2p2flow[peerID] = nodeID - } - return &FixedTableIdentityTranslator{flow2p2p, p2p2flow} -} - // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. @@ -175,7 +135,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id } // create a fixed id translator for the identities - tableTranslator := NewFixedTableIdentityTranslator(t, identities) + tableTranslator, err := p2p.NewFixedTableIdentityTranslator(identities) + require.NoError(t, err) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, From 122b826006bf027a7ae439b23c5717fd53083ac1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 16:09:17 -0700 Subject: [PATCH 064/291] Update peerManager_test.go --- network/p2p/peerManager_test.go | 27 ++++++++++----------------- 1 file changed, 10 insertions(+), 17 deletions(-) diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 5c6632fbc76..bdaa6aca015 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -15,7 +15,6 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/utils/unittest" ) @@ -34,29 +33,23 @@ func (suite *PeerManagerTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) } -func (suite *PeerManagerTestSuite) generateIdentities(n int) (flow.IdentityList, peer.IDSlice) { - // create some test ids - currentIDs := unittest.IdentityListFixture(n) - - // setup a ID provider callback to return currentIDs +func (suite *PeerManagerTestSuite) generatePeerIDs(n int) peer.IDSlice { pids := peer.IDSlice{} - for _, id := range currentIDs { - key, err := generateFlowNetworkingKey(id.NodeID) - require.NoError(suite.T(), err) - id.NetworkPubKey = key.PublicKey() + for i := 0; i < n; i++ { + key := generateNetworkingKey(suite.T()) pid, err := ExtractPeerID(key.PublicKey()) require.NoError(suite.T(), err) pids = append(pids, pid) } - return currentIDs, pids + return pids } // TestUpdatePeers tests that updatePeers calls the connector with the expected list of ids to connect and disconnect // from. The tests are cumulative and ordered. func (suite *PeerManagerTestSuite) TestUpdatePeers() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -84,7 +77,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // a subsequent call to updatePeers should request a connector.UpdatePeers to existing ids and new ids suite.Run("updatePeers connects to old and new peers", func() { // create a new id - _, newPIDs := suite.generateIdentities(1) + newPIDs := suite.generatePeerIDs(1) pids = append(pids, newPIDs...) pm.updatePeers() @@ -107,7 +100,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { pids = removeRandomElement(pids) // add a couple of new ids - _, newPIDs := suite.generateIdentities(2) + newPIDs := suite.generatePeerIDs(2) pids = append(pids, newPIDs...) pm.updatePeers() @@ -125,7 +118,7 @@ func removeRandomElement(pids peer.IDSlice) peer.IDSlice { // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -160,7 +153,7 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -206,7 +199,7 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { From c2614af674c06f3de34ad2a2d28bc03c7bb66328 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 18:42:15 -0700 Subject: [PATCH 065/291] Update execution_test.go --- engine/execution/execution_test.go | 34 +++++++++++++----------------- 1 file changed, 15 insertions(+), 19 deletions(-) diff --git a/engine/execution/execution_test.go b/engine/execution/execution_test.go index 76d2ae13137..d46d69a8dd7 100644 --- a/engine/execution/execution_test.go +++ b/engine/execution/execution_test.go @@ -37,23 +37,21 @@ func TestExecutionFlow(t *testing.T) { chainID := flow.Testnet - keys, errs := unittest.NetworkingKeys(4) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exeID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) verID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[3].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID) @@ -300,23 +298,23 @@ func TestExecutionStateSyncMultipleExecutionNodes(t *testing.T) { chainID := flow.Emulator - keys, errs := unittest.NetworkingKeys(4) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exe1ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exe1ID) - identities[3].NetworkPubKey = keys[3].PublicKey() + key, err := unittest.NetworkingKey() + require.NoError(t, err) + identities[3].NetworkPubKey = key.PublicKey() collectionNode := testutil.GenericNodeFromParticipants(t, hub, colID, identities, chainID) defer collectionNode.Done() @@ -449,27 +447,25 @@ func TestBroadcastToMultipleVerificationNodes(t *testing.T) { chainID := flow.Emulator - keys, errs := unittest.NetworkingKeys(5) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exeID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) ver1ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[3].PublicKey()), + unittest.WithKeys, ) ver2ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[4].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, ver1ID, ver2ID) From 01eef9e039846bb80d227b85a0231ab3aa223c47 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 19:32:39 -0700 Subject: [PATCH 066/291] update middleware --- cmd/scaffold.go | 1 + network/p2p/middleware.go | 57 +++++++++++++++++++++++++-------------- 2 files changed, 38 insertions(+), 20 deletions(-) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index e12dd98d456..1445dd1630d 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -736,6 +736,7 @@ func (fnb *FlowNodeBuilder) Initialize() NodeBuilder { fnb.ParseAndPrintFlags() + // ID providers must be initialized before the network fnb.InitIDProviders() fnb.EnqueueNetworkInit(ctx) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 68614e59ea8..3ef13d47f5a 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -65,24 +65,25 @@ const ( // our neighbours on the peer-to-peer network. type Middleware struct { sync.Mutex - ctx context.Context - cancel context.CancelFunc - log zerolog.Logger - ov network.Overlay - wg *sync.WaitGroup - libP2PNode *Node - libP2PNodeFactory LibP2PFactoryFunc - me flow.Identifier - metrics module.NetworkMetrics - rootBlockID string - validators []network.MessageValidator - peerManager *PeerManager - peerUpdateInterval time.Duration - unicastMessageTimeout time.Duration - connectionGating bool - managePeerConnections bool - idTranslator IDTranslator - idProvider id.IdentifierProvider + ctx context.Context + cancel context.CancelFunc + log zerolog.Logger + ov network.Overlay + wg *sync.WaitGroup + libP2PNode *Node + libP2PNodeFactory LibP2PFactoryFunc + me flow.Identifier + metrics module.NetworkMetrics + rootBlockID string + validators []network.MessageValidator + peerManager *PeerManager + peerUpdateInterval time.Duration + unicastMessageTimeout time.Duration + connectionGating bool + managePeerConnections bool + idTranslator IDTranslator + idProvider id.IdentifierProvider + previousProtocolStatePeers []peer.AddrInfo } type MiddlewareOption func(*Middleware) @@ -201,12 +202,28 @@ func (m *Middleware) GetIPPort() (string, string, error) { } func (m *Middleware) UpdateNodeAddresses() { + m.log.Info().Msg("Updating protocol state node addresses") + ids := m.ov.Identities() - infos, _ := peerInfosFromIDs(ids) + newInfos, invalid := peerInfosFromIDs(ids) + + for id, err := range invalid { + m.log.Err(err).Str("node_id", id.String()).Msg("failed to extract peer info from identity") + } + + m.Lock() + defer m.Unlock() - for _, info := range infos { + // set old addresses to expire + for _, oldInfo := range m.previousProtocolStatePeers { + m.libP2PNode.host.Peerstore().SetAddrs(oldInfo.ID, oldInfo.Addrs, peerstore.TempAddrTTL) + } + + for _, info := range newInfos { m.libP2PNode.host.Peerstore().SetAddrs(info.ID, info.Addrs, peerstore.PermanentAddrTTL) } + + m.previousProtocolStatePeers = newInfos } // Start will start the middleware. From 2f7b17833ef12a062059f6025466cd9d6b29debe Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 20:00:32 -0700 Subject: [PATCH 067/291] Address comments --- .../staked_access_node_builder.go | 2 +- cmd/scaffold.go | 2 +- network/p2p/libp2pUtils.go | 17 +++++++++++ network/p2p/network.go | 6 ++-- network/p2p/peerManager_test.go | 28 ++----------------- network/p2p/protocol_state_provider.go | 17 ----------- 6 files changed, 24 insertions(+), 48 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 233cb56819d..a2fcec2fa82 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -40,7 +40,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), - p2p.NetworkingSetFilter, + p2p.NotEjectedFilter, ), idCache, ) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 1445dd1630d..a5ddb71fbf7 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -432,7 +432,7 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache fnb.IDTranslator = idCache - fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NetworkingSetFilter, idCache) + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) return nil }) } diff --git a/network/p2p/libp2pUtils.go b/network/p2p/libp2pUtils.go index ad43adbc877..04a2996c2aa 100644 --- a/network/p2p/libp2pUtils.go +++ b/network/p2p/libp2pUtils.go @@ -16,6 +16,7 @@ import ( "github.com/multiformats/go-multiaddr" "github.com/rs/zerolog" + fcrypto "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" ) @@ -231,3 +232,19 @@ func flowStream(conn network.Conn) network.Stream { } return nil } + +func ExtractPeerID(networkPubKey fcrypto.PublicKey) (pid peer.ID, err error) { + pk, err := LibP2PPublicKeyFromFlow(networkPubKey) + if err != nil { + err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) + return + } + + return +} diff --git a/network/p2p/network.go b/network/p2p/network.go index 82235bd4938..1c0d010a16f 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -24,13 +24,13 @@ import ( const DefaultCacheSize = 10e6 -// NetworkingSetFilter is an identity filter that, when applied to the identity +// NotEjectedFilter is an identity filter that, when applied to the identity // table at a given snapshot, returns all nodes that we should communicate with // over the networking layer. // // NOTE: The protocol state includes nodes from the previous/next epoch that should // be included in network communication. We omit any nodes that have been ejected. -var NetworkingSetFilter = filter.Not(filter.Ejected) +var NotEjectedFilter = filter.Not(filter.Ejected) type ReadyDoneAwareNetwork interface { module.Network @@ -164,7 +164,7 @@ func (n *Network) unregister(channel network.Channel) error { } func (n *Network) Identities() flow.IdentityList { - return n.identityProvider.Identities(NetworkingSetFilter) + return n.identityProvider.Identities(NotEjectedFilter) } // Topology returns the identitiess of a uniform subset of nodes in protocol state using the topology provided earlier. diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index bdaa6aca015..ad8471f8be5 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -242,30 +242,6 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { // assertListsEqual asserts that two peer ID slices are equal ignoring the order func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { - listsEqual := func() bool { - if len(list1) != len(list2) { - return false - } - - map1 := make(map[peer.ID]int) - map2 := make(map[peer.ID]int) - - for _, e1 := range list1 { - map1[e1]++ - } - for _, e2 := range list2 { - map2[e2]++ - } - - for key, val := range map1 { - if map2[key] != val { - return false - } - } - return true - } - - if !listsEqual() { - assert.Failf(t, "peer ID slices not equal", "list1: %v, list2: %v", list1, list2) - } + assert.Subset(t, list1, list2) + assert.Subset(t, list2, list1) } diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 6f223a4d0fe..b1a5bdcf834 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -7,7 +7,6 @@ import ( "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/state/protocol" @@ -123,19 +122,3 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e return } - -func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { - pk, err := LibP2PPublicKeyFromFlow(networkPubKey) - if err != nil { - err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) - return - } - - pid, err = peer.IDFromPublicKey(pk) - if err != nil { - err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) - return - } - - return -} From 112c27d1764edbff04a2e07407485b8667e74fd4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 21:29:16 -0700 Subject: [PATCH 068/291] protocol state provider test --- network/p2p/peerManager_test.go | 8 +- network/p2p/protocol_state_provider_test.go | 138 ++++++++++++++++++++ 2 files changed, 139 insertions(+), 7 deletions(-) create mode 100644 network/p2p/protocol_state_provider_test.go diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index ad8471f8be5..08b3df05193 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -61,7 +61,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("peer.IDSlice")). Run(func(args mock.Arguments) { idArg := args[1].(peer.IDSlice) - assertListsEqual(suite.T(), pids, idArg) + assert.ElementsMatch(suite.T(), pids, idArg) }). Return(nil) @@ -239,9 +239,3 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { return connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 2) }, 10*time.Second, 100*time.Millisecond) } - -// assertListsEqual asserts that two peer ID slices are equal ignoring the order -func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { - assert.Subset(t, list1, list2) - assert.Subset(t, list2, list1) -} diff --git a/network/p2p/protocol_state_provider_test.go b/network/p2p/protocol_state_provider_test.go new file mode 100644 index 00000000000..55ac4074a5e --- /dev/null +++ b/network/p2p/protocol_state_provider_test.go @@ -0,0 +1,138 @@ +package p2p + +import ( + "testing" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" + mockprotocol "github.com/onflow/flow-go/state/protocol/mock" + "github.com/onflow/flow-go/utils/unittest" +) + +type ProtocolStateProviderTestSuite struct { + suite.Suite + provider *ProtocolStateIDCache + distributor *events.Distributor + state protocol.State + snapshot protocol.Snapshot + head *flow.Header + participants flow.IdentityList + epochNum uint64 +} + +func (suite *ProtocolStateProviderTestSuite) SetupTest() { + suite.distributor = events.NewDistributor() + + // set up protocol state mock + state := &mockprotocol.State{} + state.On("Final").Return( + func() protocol.Snapshot { + return suite.snapshot + }, + ) + state.On("AtBlockID", mock.Anything).Return( + func(blockID flow.Identifier) protocol.Snapshot { + if suite.head.ID() == blockID { + return suite.snapshot + } else { + return unittest.StateSnapshotForUnknownBlock() + } + }, + ) + suite.state = state + suite.epochNum = 0 + + suite.triggerUpdate() + + provider, err := NewProtocolStateIDCache(zerolog.Logger{}, state, suite.distributor) + require.NoError(suite.T(), err) + + suite.provider = provider +} + +func (suite *ProtocolStateProviderTestSuite) triggerUpdate() { + suite.participants = unittest.IdentityListFixture(5, unittest.WithAllRoles(), unittest.WithKeys) + + block := unittest.BlockFixture() + suite.head = block.Header + + // set up protocol snapshot mock + snapshot := &mockprotocol.Snapshot{} + snapshot.On("Identities", mock.Anything).Return( + func(filter flow.IdentityFilter) flow.IdentityList { + return suite.participants.Filter(filter) + }, + nil, + ) + snapshot.On("Identity", mock.Anything).Return(func(id flow.Identifier) *flow.Identity { + for _, n := range suite.participants { + if n.ID() == id { + return n + } + } + return nil + }, nil) + snapshot.On("Head").Return( + func() *flow.Header { + return suite.head + }, + nil, + ) + suite.snapshot = snapshot + suite.epochNum += 1 + + suite.distributor.EpochTransition(suite.epochNum, suite.head) +} + +func TestProtocolStateProvider(t *testing.T) { + suite.Run(t, new(ProtocolStateProviderTestSuite)) +} + +func (suite *ProtocolStateProviderTestSuite) checkStateTransition() { + oldParticipants := suite.participants + + suite.triggerUpdate() + + assert.ElementsMatch(suite.T(), suite.participants, suite.provider.Identities(filter.Any)) + for _, participant := range suite.participants { + pid, err := suite.provider.GetPeerID(participant.NodeID) + require.NoError(suite.T(), err) + fid, err := suite.provider.GetFlowID(pid) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), fid, participant.NodeID) + } + for _, participant := range oldParticipants { + _, err := suite.provider.GetPeerID(participant.NodeID) + require.Error(suite.T(), err) + } +} + +func (suite *ProtocolStateProviderTestSuite) TestUpdateState() { + for i := 0; i < 10; i++ { + suite.checkStateTransition() + } +} + +func (suite *ProtocolStateProviderTestSuite) TestIDTranslation() { + for _, participant := range suite.participants { + pid, err := suite.provider.GetPeerID(participant.NodeID) + require.NoError(suite.T(), err) + key, err := LibP2PPublicKeyFromFlow(participant.NetworkPubKey) + require.NoError(suite.T(), err) + expectedPid, err := peer.IDFromPublicKey(key) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), expectedPid, pid) + fid, err := suite.provider.GetFlowID(pid) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), fid, participant.NodeID) + } +} From 96cf07f73f57827f92bb0de566b070aaca16ebe4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 22:59:39 -0700 Subject: [PATCH 069/291] updating origin ID checks to use idtranslator --- network/p2p/middleware.go | 26 +++++++-------- network/p2p/readConnection.go | 20 ++++-------- network/p2p/readSubscription.go | 58 +++------------------------------ 3 files changed, 22 insertions(+), 82 deletions(-) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 3ef13d47f5a..72866bd977d 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -15,7 +15,6 @@ import ( "github.com/libp2p/go-libp2p-core/peerstore" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -406,25 +405,22 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { return nil } -// processAuthenticatedMessage processes a message and a source (indicated by its PublicKey) and eventually passes it to the overlay -// In particular, it checks the claim of protocol authorship situated in the message against `originKey` -// The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` +// processAuthenticatedMessage processes a message and a source (indicated by its peer ID) and eventually passes it to the overlay +// In particular, it checks the claim of protocol authorship situated in the message against `peerID` +// The assumption is that the message has been authenticated at the network level (libp2p) to originate from the peer with ID `peerID` // this requirement is fulfilled by e.g. the output of readConnection and readSubscription -func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { - identities := m.ov.Identities() +func (m *Middleware) processAuthenticatedMessage(msg *message.Message, peerID peer.ID) { + flowID, err := m.idTranslator.GetFlowID(peerID) + if err != nil { + m.log.Warn().Err(err).Msgf("received message from unknown peer %v, and was dropped", peerID.String()) + return + } // check the origin of the message corresponds to the one claimed in the OriginID originID := flow.HashToID(msg.OriginID) - originIdentity, found := identities.ByNodeID(originID) - if !found { - m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) - return - } else if originIdentity.NetworkPubKey == nil { - m.log.Warn().Msgf("received message with claimed originID %x, wich has no network identifiers at this node, and was dropped", originID) - return - } else if !originIdentity.NetworkPubKey.Equals(originKey) { - m.log.Warn().Msgf("received message claiming to be from nodeID %v with key %x was actually signed by %x and dropped", originID, originIdentity.NetworkPubKey, originKey) + if flowID != originID { + m.log.Warn().Msgf("received message claiming to be from nodeID %v was actually from %v and dropped", originID, flowID) return } diff --git a/network/p2p/readConnection.go b/network/p2p/readConnection.go index 5d6a0711872..c71b8bb1e25 100644 --- a/network/p2p/readConnection.go +++ b/network/p2p/readConnection.go @@ -7,10 +7,10 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/message" @@ -21,17 +21,17 @@ import ( type readConnection struct { ctx context.Context stream libp2pnetwork.Stream - remoteKey crypto.PublicKey + remoteID peer.ID log zerolog.Logger metrics module.NetworkMetrics maxMsgSize int - callback func(msg *message.Message, pk crypto.PublicKey) + callback func(msg *message.Message, peerID peer.ID) } // newReadConnection creates a new readConnection func newReadConnection(ctx context.Context, stream libp2pnetwork.Stream, - callback func(msg *message.Message, pubKey crypto.PublicKey), + callback func(msg *message.Message, peerID peer.ID), log zerolog.Logger, metrics module.NetworkMetrics, maxMsgSize int) *readConnection { @@ -40,18 +40,10 @@ func newReadConnection(ctx context.Context, maxMsgSize = DefaultMaxUnicastMsgSize } - remoteKey := stream.Conn().RemotePublicKey() - flowKey, err := FlowPublicKeyFromLibP2P(remoteKey) - // this should not happen if the stream was setup properly - if err != nil { - log.Err(err).Msg("failed to extract flow public key of stream libp2p key") - return nil - } - c := readConnection{ ctx: ctx, stream: stream, - remoteKey: flowKey, + remoteID: stream.Conn().RemotePeer(), callback: callback, log: log, metrics: metrics, @@ -113,7 +105,7 @@ func (rc *readConnection) receiveLoop(wg *sync.WaitGroup) { rc.metrics.NetworkMessageReceived(msg.Size(), metrics.ChannelOneToOne, msg.Type) // call the callback - rc.callback(&msg, rc.remoteKey) + rc.callback(&msg, rc.remoteID) } } diff --git a/network/p2p/readSubscription.go b/network/p2p/readSubscription.go index 3b1b66f8e69..db87b097afa 100644 --- a/network/p2p/readSubscription.go +++ b/network/p2p/readSubscription.go @@ -2,17 +2,14 @@ package p2p import ( "context" - "fmt" "strings" "sync" - lcrypto "github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/network/message" _ "github.com/onflow/flow-go/utils/binstat" @@ -25,13 +22,13 @@ type readSubscription struct { log zerolog.Logger sub *pubsub.Subscription metrics module.NetworkMetrics - callback func(msg *message.Message, pubKey crypto.PublicKey) + callback func(msg *message.Message, peerID peer.ID) } // newReadSubscription reads the messages coming in on the subscription func newReadSubscription(ctx context.Context, sub *pubsub.Subscription, - callback func(msg *message.Message, pubKey crypto.PublicKey), + callback func(msg *message.Message, peerID peer.ID), log zerolog.Logger, metrics module.NetworkMetrics) *readSubscription { @@ -82,16 +79,9 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { return } - // if pubsub.WithMessageSigning(true) and pubsub.WithStrictSignatureVerification(true), - // the emitter is authenticated - emitterKey, err := messagePubKey(rawMsg) + pid, err := peer.IDFromBytes(rawMsg.From) if err != nil { - r.log.Err(err).Msg("failed to extract libp2p public key of message") - return - } - flowKey, err := FlowPublicKeyFromLibP2P(emitterKey) - if err != nil { - r.log.Err(err).Msg("failed to extract flow public key of libp2p key") + r.log.Err(err).Msg("failed to validate peer ID of incoming message") return } @@ -109,44 +99,6 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { r.metrics.NetworkMessageReceived(msg.Size(), msg.ChannelID, msg.Type) // call the callback - r.callback(&msg, flowKey) - } -} - -// messagePubKey extracts the public key of the envelope signer from a libp2p message. -// The location of that key depends on the type of the key, see: -// https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md -// This reproduces the exact logic of the private function doing the same decoding in libp2p: -// https://github.com/libp2p/go-libp2p-pubsub/blob/ba28f8ecfc551d4d916beb748d3384951bce3ed0/sign.go#L77 -func messagePubKey(m *pubsub.Message) (lcrypto.PubKey, error) { - var pubk lcrypto.PubKey - - // m.From is the original sender of the message (versus `m.ReceivedFrom` which is the last hop which sent us this message) - pid, err := peer.IDFromBytes(m.From) - if err != nil { - return nil, err - } - - if m.Key == nil { - // no attached key, it must be extractable from the source ID - pubk, err = pid.ExtractPublicKey() - if err != nil { - return nil, fmt.Errorf("cannot extract signing key: %s", err.Error()) - } - if pubk == nil { - return nil, fmt.Errorf("cannot extract signing key") - } - } else { - pubk, err = lcrypto.UnmarshalPublicKey(m.Key) - if err != nil { - return nil, fmt.Errorf("cannot unmarshal signing key: %s", err.Error()) - } - - // verify that the source ID matches the attached key - if !pid.MatchesPublicKey(pubk) { - return nil, fmt.Errorf("bad signing key; source ID %s doesn't match key", pid) - } + r.callback(&msg, pid) } - - return pubk, nil } From 93a573ad22d110d6b123f71f1b5477c9a76fe720 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 14:12:50 -0700 Subject: [PATCH 070/291] new providers --- model/flow/filter/identity.go | 12 +++++ module/id/fixed_provider.go | 4 +- module/id/updatable_provider.go | 34 ++++++++++++++ network/p2p/identity_provider_translator.go | 51 +++++++++++++++++++++ network/p2p/libp2pNode.go | 32 ++++++------- 5 files changed, 115 insertions(+), 18 deletions(-) create mode 100644 module/id/updatable_provider.go create mode 100644 network/p2p/identity_provider_translator.go diff --git a/model/flow/filter/identity.go b/model/flow/filter/identity.go index f55b2e5227b..bc8e680d82a 100644 --- a/model/flow/filter/identity.go +++ b/model/flow/filter/identity.go @@ -3,6 +3,7 @@ package filter import ( + "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" ) @@ -61,6 +62,17 @@ func HasNodeID(nodeIDs ...flow.Identifier) flow.IdentityFilter { } } +func HasNetworkingKey(keys ...crypto.PublicKey) flow.IdentityFilter { + lookup := make(map[crypto.PublicKey]struct{}) + for _, key := range keys { + lookup[key] = struct{}{} + } + return func(identity *flow.Identity) bool { + _, ok := lookup[identity.NetworkPubKey] + return ok + } +} + // HasStake returns a filter for nodes with non-zero stake. func HasStake(hasStake bool) flow.IdentityFilter { return func(identity *flow.Identity) bool { diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index 106047510b6..39aac7567a1 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -24,6 +24,6 @@ func NewFixedIdentityProvider(identities flow.IdentityList) *FixedIdentityProvid return &FixedIdentityProvider{identities} } -func (p *FixedIdentityProvider) Identities(flow.IdentityFilter) flow.IdentityList { - return p.identities +func (p *FixedIdentityProvider) Identities(filter flow.IdentityFilter) flow.IdentityList { + return p.identities.Filter(filter) } diff --git a/module/id/updatable_provider.go b/module/id/updatable_provider.go new file mode 100644 index 00000000000..2288b9e6834 --- /dev/null +++ b/module/id/updatable_provider.go @@ -0,0 +1,34 @@ +package id + +import ( + "sync" + + "github.com/onflow/flow-go/model/flow" +) + +type UpdatableIDProvider struct { + mu sync.RWMutex + identities flow.IdentityList +} + +func NewUpdatableIDProvider(identities flow.IdentityList) *UpdatableIDProvider { + return &UpdatableIDProvider{identities: identities} +} + +func (p *UpdatableIDProvider) SetIdentities(identities flow.IdentityList) { + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities +} + +func (p *UpdatableIDProvider) Identities(filter flow.IdentityFilter) flow.IdentityList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.Filter(filter) +} + +func (p *UpdatableIDProvider) Identifiers() flow.IdentifierList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.NodeIDs() +} diff --git a/network/p2p/identity_provider_translator.go b/network/p2p/identity_provider_translator.go new file mode 100644 index 00000000000..61504b1ef05 --- /dev/null +++ b/network/p2p/identity_provider_translator.go @@ -0,0 +1,51 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module/id" +) + +type IdentityProviderIdentityTranslator struct { + idProvider id.IdentityProvider +} + +func (t *IdentityProviderIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + key, err := p.ExtractPublicKey() + if err != nil { + return flow.ZeroID, err + } + flowKey, err := FlowPublicKeyFromLibP2P(key) + if err != nil { + return flow.ZeroID, err + } + ids := t.idProvider.Identities(filter.HasNetworkingKey(flowKey)) + if len(ids) == 0 { + return flow.ZeroID, fmt.Errorf("could not find identity corresponding to peer id %v", p.Pretty()) + } + return ids[0].NodeID, nil +} + +func (t *IdentityProviderIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + ids := t.idProvider.Identities(filter.HasNodeID(n)) + if len(ids) == 0 { + return "", fmt.Errorf("could not find identity with id %v", n.String()) + } + key, err := LibP2PPublicKeyFromFlow(ids[0].NetworkPubKey) + if err != nil { + return "", err + } + pid, err := peer.IDFromPublicKey(key) + if err != nil { + return "", err + } + return pid, nil +} + +func NewIdentityProviderIdentityTranslator(provider id.IdentityProvider) *IdentityProviderIdentityTranslator { + return &IdentityProviderIdentityTranslator{provider} +} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 0fcb148ccd2..9268091fe35 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -334,24 +334,24 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { if len(n.host.Peerstore().Addrs(peerID)) == 0 { n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - if n.dht == nil { - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) - } - - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - - var err error - func() { - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - defer cancel() - // try to find the peer using the dht - _, err = n.dht.FindPeer(timedCtx, peerID) - }() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + if n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + + if err != nil { + n.logger.Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") + } else { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + } } - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) From 9dd6d5e94f39571a4bb71db0e16a3e1bd5608683 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 14:37:28 -0700 Subject: [PATCH 071/291] fix test --- model/flow/filter/identity.go | 12 +++--- network/test/middleware_test.go | 73 +++++++++++++++++++++++++++------ network/test/testUtil.go | 25 ++++++----- 3 files changed, 81 insertions(+), 29 deletions(-) diff --git a/model/flow/filter/identity.go b/model/flow/filter/identity.go index bc8e680d82a..dc39e4ce475 100644 --- a/model/flow/filter/identity.go +++ b/model/flow/filter/identity.go @@ -63,13 +63,13 @@ func HasNodeID(nodeIDs ...flow.Identifier) flow.IdentityFilter { } func HasNetworkingKey(keys ...crypto.PublicKey) flow.IdentityFilter { - lookup := make(map[crypto.PublicKey]struct{}) - for _, key := range keys { - lookup[key] = struct{}{} - } return func(identity *flow.Identity) bool { - _, ok := lookup[identity.NetworkPubKey] - return ok + for _, key := range keys { + if key.Equals(identity.NetworkPubKey) { + return true + } + } + return false } } diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 7f97b9ad9a1..3b33df7f312 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -8,8 +8,10 @@ import ( "time" "github.com/ipfs/go-log" + swarm "github.com/libp2p/go-libp2p-swarm" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" mockery "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -17,6 +19,7 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/observable" "github.com/onflow/flow-go/network/codec/cbor" @@ -56,12 +59,14 @@ func (co *tagsObserver) OnComplete() { type MiddlewareTestSuite struct { suite.Suite - size int // used to determine number of middlewares under test - mws []*p2p.Middleware // used to keep track of middlewares under test - ov []*mocknetwork.Overlay - obs chan string // used to keep track of Protect events tagged by pubsub messages - ids []*flow.Identity - metrics *metrics.NoopCollector // no-op performance monitoring simulation + size int // used to determine number of middlewares under test + mws []*p2p.Middleware // used to keep track of middlewares under test + ov []*mocknetwork.Overlay + obs chan string // used to keep track of Protect events tagged by pubsub messages + ids []*flow.Identity + metrics *metrics.NoopCollector // no-op performance monitoring simulation + logger zerolog.Logger + providers []*id.UpdatableIDProvider } // TestMiddlewareTestSuit runs all the test methods in this test suit @@ -73,6 +78,7 @@ func TestMiddlewareTestSuite(t *testing.T) { func (m *MiddlewareTestSuite) SetupTest() { logger := zerolog.New(os.Stderr).Level(zerolog.ErrorLevel) log.SetAllLoggers(log.LevelError) + m.logger = logger m.size = 2 // operates on two middlewares m.metrics = metrics.NewNoopCollector() @@ -85,7 +91,7 @@ func (m *MiddlewareTestSuite) SetupTest() { log: logger, } - m.ids, m.mws, obs = GenerateIDsAndMiddlewares(m.T(), m.size, !DryRun, logger) + m.ids, m.mws, obs, m.providers = GenerateIDsAndMiddlewares(m.T(), m.size, !DryRun, logger) for _, observableConnMgr := range obs { observableConnMgr.Subscribe(&ob) @@ -98,11 +104,7 @@ func (m *MiddlewareTestSuite) SetupTest() { // create the mock overlays for i := 0; i < m.size; i++ { - overlay := &mocknetwork.Overlay{} - m.ov = append(m.ov, overlay) - - overlay.On("Identities").Maybe().Return(flow.IdentityList(m.ids), nil) - overlay.On("Topology").Maybe().Return(flow.IdentityList(m.ids), nil) + m.ov = append(m.ov, m.createOverlay()) } for i, mw := range m.mws { assert.NoError(m.T(), mw.Start(m.ov[i])) @@ -110,6 +112,18 @@ func (m *MiddlewareTestSuite) SetupTest() { } } +func (m *MiddlewareTestSuite) createOverlay() *mocknetwork.Overlay { + overlay := &mocknetwork.Overlay{} + + overlay.On("Identities").Maybe().Return(func() flow.IdentityList { + return flow.IdentityList(m.ids) + }, nil) + overlay.On("Topology").Maybe().Return(func() flow.IdentityList { + return flow.IdentityList(m.ids) + }, nil) + return overlay +} + func (m *MiddlewareTestSuite) TearDownTest() { m.stopMiddlewares() } @@ -462,6 +476,41 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { require.Error(m.Suite.T(), err) } +func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { + ids, mws, _, providers := GenerateIDsAndMiddlewares(m.T(), 1, false, m.logger) + require.Len(m.T(), ids, 1) + require.Len(m.T(), providers, 1) + require.Len(m.T(), mws, 1) + newId := ids[0] + newMw := mws[0] + newProvider := providers[0] + + idList := flow.IdentityList(append(m.ids, newId)) + + newProvider.SetIdentities(idList) + overlay := m.createOverlay() + overlay.On("Receive", + m.ids[0].NodeID, + mock.AnythingOfType("*message.Message"), + ).Return(nil) + assert.NoError(m.T(), newMw.Start(overlay)) + + // needed to enable ID translation + m.providers[0].SetIdentities(idList) + m.mws[0].UpdateAllowList() + + msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") + + err := m.mws[0].SendDirect(msg, newId.NodeID) + require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) + + m.ids = idList + m.mws[0].UpdateNodeAddresses() + + err = m.mws[0].SendDirect(msg, newId.NodeID) + require.NoError(m.T(), err) +} + // TestUnsubscribe tests that an engine can unsubscribe from a topic it was earlier subscribed to and stop receiving // messages. func (m *MiddlewareTestSuite) TestUnsubscribe() { diff --git a/network/test/testUtil.go b/network/test/testUtil.go index d101779b899..9f8c1ef063a 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -21,6 +21,7 @@ import ( message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" + idModule "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" @@ -121,9 +122,10 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) []*p2p.Middleware { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*id.UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) + idProviders := make([]*id.UpdatableIDProvider, len(identities)) for i, id := range identities { // casts libP2PNode instance to a local variable to avoid closure @@ -134,9 +136,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } - // create a fixed id translator for the identities - tableTranslator, err := p2p.NewFixedTableIdentityTranslator(identities) - require.NoError(t, err) + idProviders[i] = idModule.NewUpdatableIDProvider(identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, @@ -148,10 +148,13 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultUnicastTimeout, true, true, - tableTranslator, + p2p.NewIdentityProviderIdentityTranslator(idProviders[i]), + p2p.WithIdentifierProvider( + idProviders[i], + ), ) } - return mws + return mws, idProviders } // GenerateNetworks generates the network for the given middlewares @@ -200,7 +203,7 @@ func GenerateNetworks(t *testing.T, tops[i], sms[i], metrics, - id.NewFixedIdentityProvider(ids), + idModule.NewFixedIdentityProvider(ids), ) require.NoError(t, err) @@ -220,11 +223,11 @@ func GenerateNetworks(t *testing.T, func GenerateIDsAndMiddlewares(t *testing.T, n int, dryRunMode bool, - logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable) { + logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*id.UpdatableIDProvider) { ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) - mws := GenerateMiddlewares(t, logger, ids, libP2PNodes) - return ids, mws, protectObservables + mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) + return ids, mws, protectObservables, providers } func GenerateIDsMiddlewaresNetworks(t *testing.T, @@ -234,7 +237,7 @@ func GenerateIDsMiddlewaresNetworks(t *testing.T, tops []network.Topology, dryRun bool, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []*p2p.Network, []observable.Observable) { - ids, mws, observables := GenerateIDsAndMiddlewares(t, n, dryRun, log, opts...) + ids, mws, observables, _ := GenerateIDsAndMiddlewares(t, n, dryRun, log, opts...) sms := GenerateSubscriptionManagers(t, mws) networks := GenerateNetworks(t, log, ids, mws, csize, tops, sms, dryRun) return ids, mws, networks, observables From c7b4ba658a8a0e75d46eebfb09a1405098a59b84 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 14:49:17 -0700 Subject: [PATCH 072/291] remove comments --- cmd/access/node_builder/staked_access_node_builder.go | 3 +-- network/p2p/dht_test.go | 2 -- network/p2p/libp2pNode.go | 4 ---- 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a2fcec2fa82..8e5b5285e3f 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -63,8 +63,7 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - // TODO: we should remove this call since we are no longer instantiating two networks - // builder.EnqueueNetworkInit(ctx) + builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network if builder.ParticipatesInUnstakedNetwork() { diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index 6643a7652b0..cc01eedaafb 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -29,8 +29,6 @@ type DHTTestSuite struct { cancel context.CancelFunc // used to cancel the context } -// TODO: test that dht findPeer times out - // TestDHTTestSuite test the libp2p pubsub with DHT for discovery func TestDHTTestSuite(t *testing.T) { suite.Run(t, new(DHTTestSuite)) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 9268091fe35..7d0d2a5fbd6 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -381,10 +381,6 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp default: } - // TODO: why were we doing this? Is it okay to remove? - // remove the peer from the peer store if present - // n.host.Peerstore().ClearAddrs(peerID) - // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() if swm, ok := network.(*swarm.Swarm); ok { From 7a994ced23076efabccb220f154c777e9f5ea56b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 14:59:26 -0700 Subject: [PATCH 073/291] Update go.mod --- go.mod | 1 - 1 file changed, 1 deletion(-) diff --git a/go.mod b/go.mod index c5580c08792..c1a054e36c7 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,6 @@ require ( github.com/m4ksio/wal v1.0.0 github.com/mitchellh/mapstructure v1.3.3 // indirect github.com/multiformats/go-multiaddr v0.3.3 - github.com/multiformats/go-multihash v0.0.15 github.com/onflow/cadence v0.18.1-0.20210730161646-b891a21c51fd github.com/onflow/flow-core-contracts/lib/go/contracts v0.7.6 github.com/onflow/flow-core-contracts/lib/go/templates v0.7.6 From 9c0c1c890dd30a59ca569ff5a9f5b3809aca425a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 15:01:31 -0700 Subject: [PATCH 074/291] Update hierarchical_translator_test.go --- network/p2p/hierarchical_translator_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go index 6866547f378..d51e7735ff5 100644 --- a/network/p2p/hierarchical_translator_test.go +++ b/network/p2p/hierarchical_translator_test.go @@ -4,11 +4,12 @@ import ( "testing" "github.com/libp2p/go-libp2p-core/peer" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/utils/unittest" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "gotest.tools/assert" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" ) type HierarchicalTranslatorTestSuite struct { From 24ec4a16bcfb8c6af9fa26eae2dc3d4f77e7b3d9 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 15:08:13 -0700 Subject: [PATCH 075/291] fix int --- engine/common/synchronization/engine_test.go | 1 + engine/testutil/nodes.go | 1 + network/p2p/hierarchical_translator_test.go | 3 +++ 3 files changed, 5 insertions(+) diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 4e79938fab7..5dce4ea28a5 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -171,6 +171,7 @@ func (ss *SyncSuite) SetupTest() { require.NoError(ss.T(), err, "could not create finalized snapshot cache") idCache, err := p2p.NewProtocolStateIDCache(log, ss.state, protocolEvents.NewDistributor()) + require.NoError(ss.T(), err, "could not create protocol state identity cache") e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, id.NewFilteredIdentifierProvider( filter.And( diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index 3688e2c6727..ecaad824601 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -628,6 +628,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit require.NoError(t, err) idCache, err := p2p.NewProtocolStateIDCache(node.Log, node.State, events.NewDistributor()) + require.NoError(t, err, "could not create finalized snapshot cache") syncEngine, err := synchronization.New( node.Log, node.Metrics, diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go index d51e7735ff5..90245d82a97 100644 --- a/network/p2p/hierarchical_translator_test.go +++ b/network/p2p/hierarchical_translator_test.go @@ -36,6 +36,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestFirstTranslatorSuccess() { pid, err := suite.translator.GetPeerID(suite.ids[0].NodeID) require.NoError(suite.T(), err) fid, err := suite.translator.GetFlowID(pid) + require.NoError(suite.T(), err) assert.Equal(suite.T(), fid, suite.ids[0].NodeID) } @@ -43,6 +44,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestSecondTranslatorSuccess() { pid, err := suite.translator.GetPeerID(suite.ids[1].NodeID) require.NoError(suite.T(), err) fid, err := suite.translator.GetFlowID(pid) + require.NoError(suite.T(), err) assert.Equal(suite.T(), fid, suite.ids[1].NodeID) } @@ -54,6 +56,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestTranslationFailure() { key, err := LibP2PPrivKeyFromFlow(generateNetworkingKey(suite.T())) require.NoError(suite.T(), err) pid, err := peer.IDFromPrivateKey(key) + require.NoError(suite.T(), err) _, err = suite.translator.GetFlowID(pid) require.Error(suite.T(), err) } From ca95f68256e9291e7268f8e084a5ef9fdb1882da Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Sun, 22 Aug 2021 21:19:41 -0700 Subject: [PATCH 076/291] removing unstaked network --- .../node_builder/access_node_builder.go | 49 ++++++------ .../staked_access_node_builder.go | 79 ++++++++++++++----- .../unstaked_access_node_builder.go | 18 ++--- cmd/node_builder.go | 4 +- cmd/scaffold.go | 12 ++- follower/consensus_follower.go | 4 +- 6 files changed, 106 insertions(+), 60 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 64c8480d79c..9fddc9abe2d 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -74,9 +74,9 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // ParticipatesInUnstakedNetwork returns True if this is a staked Access node which also participates - // in the unstaked network acting as an upstream for other unstaked access nodes, False otherwise. - ParticipatesInUnstakedNetwork() bool + // SupportsUnstakedNetwork returns True if this is a staked Access node which also supports + // unstaked access nodes/unstaked consensus follower engines, False otherwise. + SupportsUnstakedNetwork() bool // Build defines all of the Access node's components and modules. Build() AccessNodeBuilder @@ -90,7 +90,7 @@ type AccessNodeConfig struct { bootstrapNodeAddresses []string bootstrapNodePublicKeys []string bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes - unstakedNetworkBindAddr string + supportsUnstakedFollower bool collectionGRPCPort uint executionGRPCPort uint pingEnabled bool @@ -138,7 +138,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { staked: true, bootstrapNodeAddresses: []string{}, bootstrapNodePublicKeys: []string{}, - unstakedNetworkBindAddr: cmd.NotSet, + supportsUnstakedFollower: false, } } @@ -150,9 +150,7 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedLibP2PNode *p2p.Node - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware + LibP2PNode *p2p.Node FollowerState protocol.MutableState SyncCore *synchronization.Core RpcEng *rpc.Engine @@ -498,9 +496,9 @@ func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { } } -func WithUnstakedNetworkBindAddr(bindAddr string) Option { +func SupportsUnstakedFollower(enable bool) Option { return func(config *AccessNodeConfig) { - config.unstakedNetworkBindAddr = bindAddr + config.supportsUnstakedFollower = enable } } @@ -526,14 +524,14 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { return builder.staked } -func (builder *FlowAccessNodeBuilder) ParticipatesInUnstakedNetwork() bool { +func (builder *FlowAccessNodeBuilder) SupportsUnstakedNetwork() bool { // unstaked access nodes can't be upstream of other unstaked access nodes for now if !builder.IsStaked() { return false } - // if an unstaked network bind address is provided, then this staked access node will act as the upstream for - // unstaked access nodes - return builder.unstakedNetworkBindAddr != cmd.NotSet + + // a staked access node may or may not support unstaked follower + return builder.supportsUnstakedFollower } func (builder *FlowAccessNodeBuilder) ParseFlags() { @@ -574,7 +572,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") - flags.StringVar(&builder.unstakedNetworkBindAddr, "unstaked-bind-addr", defaultConfig.unstakedNetworkBindAddr, "address to bind on for the unstaked network") + flags.BoolVar(&builder.supportsUnstakedFollower, "supports-unstaked-follower", defaultConfig.supportsUnstakedFollower, "true if this staked access node supports unstaked follower (not applicable for unstaked nodes)") }) } @@ -594,19 +592,24 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, dhtOptions = append(dhtOptions, bootstrapPeersOpt) } + myAddr := builder.NodeConfig.Me.Address() + if builder.BaseConfig.BindAddr != cmd.NotSet { + myAddr = builder.BaseConfig.BindAddr + } + return func() (*p2p.Node, error) { - libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.unstakedNetworkBindAddr, networkKey). + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). SetRootBlockID(builder.RootBlock.ID().String()). - // unlike the staked network where currently all the node addresses are known upfront, - // for the unstaked network the nodes need to discover each other using DHT Discovery. + // unlike the staked side of the network where currently all the node addresses are known upfront, + // for the unstaked side of the network, the nodes need to discover each other using DHT Discovery. SetDHTOptions(dhtOptions...). SetLogger(builder.Logger). Build(ctx) if err != nil { return nil, err } - builder.UnstakedLibP2PNode = libp2pNode - return builder.UnstakedLibP2PNode, nil + builder.LibP2PNode = libp2pNode + return builder.LibP2PNode, nil }, nil } @@ -616,7 +619,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, validators ...network.MessageValidator) *p2p.Middleware { - builder.unstakedMiddleware = p2p.NewMiddleware( + builder.Middleware = p2p.NewMiddleware( builder.Logger, factoryFunc, nodeID, @@ -630,7 +633,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, p2p.WithMessageValidators(validators...), // use default identifier provider ) - return builder.unstakedMiddleware + return builder.Middleware } // initNetwork creates the network.Network implementation with the given metrics, middleware, initial list of network @@ -650,7 +653,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, builder.Logger, codec, nodeID, - builder.unstakedMiddleware, + builder.Middleware, p2p.DefaultCacheSize, topology, subscriptionManager, diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 8e5b5285e3f..6565971d109 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -4,7 +4,10 @@ import ( "context" "fmt" + dht "github.com/libp2p/go-libp2p-kad-dht" + "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/crypto" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" @@ -29,6 +32,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err @@ -44,9 +48,10 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { ), idCache, ) + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // TODO: NetworkingIdentifierProvidzer should be the same as the one used in scaffold.go if this AN // doesn't participate in unstaked network. // If it does, then we can just use the default one (peerstoreProvider) @@ -66,8 +71,10 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network - if builder.ParticipatesInUnstakedNetwork() { + if builder.SupportsUnstakedNetwork() { builder.enqueueUnstakedNetworkInit(ctx) + } else { + builder.EnqueueNetworkInit(ctx) } builder.EnqueueMetricsServerInit() @@ -105,36 +112,70 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // NodeID for the staked node on the unstaked network - // TODO: set a different node ID of the staked access node on the unstaked network - unstakedNodeID := builder.NodeID // currently set the same as the staked NodeID - - // Networking key - // TODO: set a different networking key of the staked access node on the unstaked network - unstakedNetworkKey := builder.NetworkKey - - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) + msgValidators := unstakedNetworkMsgValidators(builder.NodeID) // Network Metrics // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics // TODO: define new network metrics for the unstaked network unstakedNetworkMetrics := metrics.NewNoopCollector() - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + middleware := builder.initMiddleware(builder.NodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront - top := topology.EmptyListTopology{} + top, err := topology.NewTopicBasedTopology( + builder.NodeID, + builder.Logger, + builder.State, + ) + if err != nil { + return nil, fmt.Errorf("could not create topology: %w", err) + } + topologyCache := topology.NewCache(builder.Logger, top) - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, top) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, topologyCache) builder.MustNot(err) - builder.UnstakedNetwork = network - builder.unstakedMiddleware = middleware + builder.Network = network + builder.Middleware = middleware - node.Logger.Info().Msgf("unstaked network will run on address: %s", builder.unstakedNetworkBindAddr) - return builder.UnstakedNetwork, err + node.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + return builder.Network, err }) } + +// initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. +// The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance +func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, + nodeID flow.Identifier, + networkKey crypto.PrivateKey) (p2p.LibP2PFactoryFunc, error) { + + // The staked nodes act as the DHT servers + dhtOptions := []dht.Option{p2p.AsServer(builder.IsStaked())} + + myAddr := builder.NodeConfig.Me.Address() + if builder.BaseConfig.BindAddr != cmd.NotSet { + myAddr = builder.BaseConfig.BindAddr + } + + connManager := p2p.NewConnManager(builder.Logger, builder.Metrics.Network) + + return func() (*p2p.Node, error) { + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). + SetRootBlockID(builder.RootBlock.ID().String()). + // no connection gater + SetConnectionManager(connManager). + // act as a DHT server + SetDHTOptions(dhtOptions...). + SetPubsubOptions(p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize)...). + SetLogger(builder.Logger). + Build(ctx) + if err != nil { + return nil, err + } + builder.LibP2PNode = libp2pNode + return builder.LibP2PNode, nil + }, nil +} diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index f6fd01e63d2..2e9f121a84b 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -63,12 +63,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (builder *UnstakedAccessNodeBuilder) validateParams() { - - // for an unstaked access node, the unstaked network bind address must be provided - if builder.unstakedNetworkBindAddr == cmd.NotSet { - builder.Logger.Fatal().Msg("unstaked bind address not set") - } - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") } @@ -93,7 +87,7 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N NetworkPubKey: node.NetworkKey.PublicKey(), StakingPubKey: nil, // no staking key needed for the unstaked node Role: flow.RoleAccess, // unstaked node can only run as an access node - Address: builder.unstakedNetworkBindAddr, + Address: builder.BindAddr, } me, err := local.New(self, nil) @@ -139,16 +133,16 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) builder.MustNot(err) - builder.UnstakedNetwork = network - builder.unstakedMiddleware = middleware + builder.Network = network + builder.Middleware = middleware // for an unstaked node, the staked network and middleware is set to the same as the unstaked network and middlware builder.Network = network builder.Middleware = middleware - builder.Logger.Info().Msgf("unstaked network will run on address: %s", builder.unstakedNetworkBindAddr) + builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) - return builder.UnstakedNetwork, err + return builder.Network, err }) } @@ -160,6 +154,6 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.UnstakedLibP2PNode, builder.Logger), nil + return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil }) } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index ab027fceda8..5fafe350dae 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -95,7 +95,7 @@ type NodeBuilder interface { // while for a node running as a library, the config fields are expected to be initialized by the caller. type BaseConfig struct { nodeIDHex string - bindAddr string + BindAddr string NodeRole string timeout time.Duration datadir string @@ -155,7 +155,7 @@ func DefaultBaseConfig() *BaseConfig { datadir := filepath.Join(homedir, ".flow", "database") return &BaseConfig{ nodeIDHex: NotSet, - bindAddr: NotSet, + BindAddr: NotSet, BootstrapDir: "bootstrap", timeout: 1 * time.Minute, datadir: datadir, diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a5ddb71fbf7..3ef01a87313 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -110,7 +110,7 @@ func (fnb *FlowNodeBuilder) BaseFlags() { // bind configuration parameters fnb.flags.StringVar(&fnb.BaseConfig.nodeIDHex, "nodeid", defaultConfig.nodeIDHex, "identity of our node") - fnb.flags.StringVar(&fnb.BaseConfig.bindAddr, "bind", defaultConfig.bindAddr, "address to bind on") + fnb.flags.StringVar(&fnb.BaseConfig.BindAddr, "bind", defaultConfig.BindAddr, "address to bind on") fnb.flags.StringVarP(&fnb.BaseConfig.BootstrapDir, "bootstrapdir", "b", defaultConfig.BootstrapDir, "path to the bootstrap directory") fnb.flags.DurationVarP(&fnb.BaseConfig.timeout, "timeout", "t", defaultConfig.timeout, "node startup / shutdown timeout") fnb.flags.StringVarP(&fnb.BaseConfig.datadir, "datadir", "d", defaultConfig.datadir, "directory to store the protocol state") @@ -137,8 +137,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { codec := cborcodec.NewCodec() myAddr := fnb.NodeConfig.Me.Address() - if fnb.BaseConfig.bindAddr != NotSet { - myAddr = fnb.BaseConfig.bindAddr + if fnb.BaseConfig.BindAddr != NotSet { + myAddr = fnb.BaseConfig.BindAddr } // setup the Ping provider to return the software version and the sealed block height @@ -694,6 +694,12 @@ func WithNodeID(nodeID flow.Identifier) Option { } } +func WithBindAddress(bindAddress string) Option { + return func(config *BaseConfig) { + config.BindAddr = bindAddress + } +} + func WithDataDir(dataDir string) Option { return func(config *BaseConfig) { config.datadir = dataDir diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index a34b207537e..2d73a73f17a 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -69,7 +69,6 @@ func getAccessNodeOptions(config *Config) []access.Option { ids := bootstrapIdentities(config.bootstrapNodes) return []access.Option{ access.WithBootStrapPeers(ids...), - access.WithUnstakedNetworkBindAddr(config.bindAddr), access.WithBaseOptions(getBaseOptions(config)), } } @@ -85,6 +84,9 @@ func getBaseOptions(config *Config) []cmd.Option { if config.dataDir != "" { options = append(options, cmd.WithDataDir(config.dataDir)) } + if config.bindAddr != "" { + options = append(options, cmd.WithBindAddress(config.bindAddr)) + } return options } From 9c6136aec0ed3f99620ba90b42bea0da373e352a Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Sun, 22 Aug 2021 21:44:52 -0700 Subject: [PATCH 077/291] wip --- cmd/access/node_builder/staked_access_node_builder.go | 7 ++----- cmd/access/node_builder/unstaked_access_node_builder.go | 4 +++- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 6565971d109..19367514d2c 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -66,14 +66,11 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() - // for the staked access node, initialize the network used to communicate with the other staked flow nodes - // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - builder.EnqueueNetworkInit(ctx) - - // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network + // if this is an access node that supports unstaked followers, enqueue the unstaked network if builder.SupportsUnstakedNetwork() { builder.enqueueUnstakedNetworkInit(ctx) } else { + // otherwise, enqueue the regular network builder.EnqueueNetworkInit(ctx) } diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 2e9f121a84b..ca6d04457d5 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -96,6 +96,8 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N } } +// Build enqueues the sync engine and the follower engine for the unstaked access node. +// Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { @@ -103,7 +105,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() return nil }) - anb.FlowAccessNodeBuilder.Build() + anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } From b432ad6d5cc3d372ae74272d5f8a5d73836faa3a Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 12:40:17 -0700 Subject: [PATCH 078/291] wip - integration tests for consensus follower --- .../node_builder/access_node_builder.go | 21 ++++++---- .../staked_access_node_builder.go | 4 +- .../unstaked_access_node_builder.go | 8 +++- cmd/node_builder.go | 3 ++ cmd/scaffold.go | 10 +---- follower/consensus_follower.go | 10 ++--- integration/testnet/network.go | 39 ++++++++----------- .../tests/access/unstaked_node_test.go | 10 +++-- 8 files changed, 56 insertions(+), 49 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 9fddc9abe2d..b4159c97445 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -74,9 +74,9 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // SupportsUnstakedNetwork returns True if this is a staked Access node which also supports + // SupportsUnstakedNode returns True if this is a staked Access node which also supports // unstaked access nodes/unstaked consensus follower engines, False otherwise. - SupportsUnstakedNetwork() bool + SupportsUnstakedNode() bool // Build defines all of the Access node's components and modules. Build() AccessNodeBuilder @@ -90,6 +90,7 @@ type AccessNodeConfig struct { bootstrapNodeAddresses []string bootstrapNodePublicKeys []string bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes + NetworkKey crypto.PrivateKey supportsUnstakedFollower bool collectionGRPCPort uint executionGRPCPort uint @@ -496,12 +497,18 @@ func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { } } -func SupportsUnstakedFollower(enable bool) Option { +func SupportsUnstakedNode(enable bool) Option { return func(config *AccessNodeConfig) { config.supportsUnstakedFollower = enable } } +func WithNetworkKey(key crypto.PrivateKey) Option { + return func(config *AccessNodeConfig) { + config.NetworkKey = key + } +} + func WithBaseOptions(baseOptions []cmd.Option) Option { return func(config *AccessNodeConfig) { config.baseOptions = baseOptions @@ -524,7 +531,7 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { return builder.staked } -func (builder *FlowAccessNodeBuilder) SupportsUnstakedNetwork() bool { +func (builder *FlowAccessNodeBuilder) SupportsUnstakedNode() bool { // unstaked access nodes can't be upstream of other unstaked access nodes for now if !builder.IsStaked() { return false @@ -572,7 +579,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") - flags.BoolVar(&builder.supportsUnstakedFollower, "supports-unstaked-follower", defaultConfig.supportsUnstakedFollower, "true if this staked access node supports unstaked follower (not applicable for unstaked nodes)") + flags.BoolVar(&builder.supportsUnstakedFollower, "supports-unstaked-node", defaultConfig.supportsUnstakedFollower, "true if this staked access node supports unstaked node") }) } @@ -618,7 +625,7 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - validators ...network.MessageValidator) *p2p.Middleware { + validators ...network.MessageValidator) network.Middleware { builder.Middleware = p2p.NewMiddleware( builder.Logger, factoryFunc, @@ -641,7 +648,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, // updated by calling network.SetIDs. func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, - middleware *p2p.Middleware, + middleware network.Middleware, topology network.Topology) (*p2p.Network, error) { codec := jsoncodec.NewCodec() diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 8843d250ff6..7ce262d6c54 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -9,8 +9,6 @@ import ( "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/crypto" pingeng "github.com/onflow/flow-go/engine/access/ping" - "github.com/onflow/flow-go/engine/access/relay" - splitternetwork "github.com/onflow/flow-go/engine/common/splitter/network" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -69,7 +67,7 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() // if this is an access node that supports unstaked followers, enqueue the unstaked network - if builder.SupportsUnstakedNetwork() { + if builder.SupportsUnstakedNode() { builder.enqueueUnstakedNetworkInit(ctx) } else { // otherwise, enqueue the regular network diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index ca6d04457d5..5e27f7702bf 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -21,6 +21,12 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } +func (fnb *UnstakedAccessNodeBuilder) InitNodeInfo() { + fnb.NodeID = flow.ZeroID // TODO: extract node id from networking key + fnb.NodeConfig.NetworkKey = fnb.NetworkKey // use the networking that has been passed in + fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node +} + func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() @@ -55,7 +61,7 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.RegisterBadgerMetrics() - builder.EnqueueTracer() + builder.EnqxueueTracer() builder.PreInit(builder.initUnstakedLocal()) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 8bdd0f0dfcd..7ea3736df0d 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -44,6 +44,9 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() + // InitNodeInfo initializes the node id, staking key and networking key + InitNodeInfo() + // InitIDProviders initializes the IdentityProvider and IDTranslator InitIDProviders() diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 3ef01a87313..0bbb94f3b7f 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -256,7 +256,7 @@ func (fnb *FlowNodeBuilder) PrintBuildVersionDetails() { fnb.Logger.Info().Str("version", build.Semver()).Str("commit", build.Commit()).Msg("build details") } -func (fnb *FlowNodeBuilder) initNodeInfo() { +func (fnb *FlowNodeBuilder) InitNodeInfo() { if fnb.BaseConfig.nodeIDHex == NotSet { fnb.Logger.Fatal().Msg("cannot start without node ID") } @@ -688,12 +688,6 @@ func WithBootstrapDir(bootstrapDir string) Option { } } -func WithNodeID(nodeID flow.Identifier) Option { - return func(config *BaseConfig) { - config.nodeIDHex = nodeID.String() - } -} - func WithBindAddress(bindAddress string) Option { return func(config *BaseConfig) { config.BindAddr = bindAddress @@ -801,7 +795,7 @@ func (fnb *FlowNodeBuilder) Ready() <-chan struct{} { // seed random generator rand.Seed(time.Now().UnixNano()) - fnb.initNodeInfo() + fnb.InitNodeInfo() fnb.initLogger() diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 221e8b91de7..33a52064bf6 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -24,7 +24,7 @@ type ConsensusFollower interface { // Config contains the configurable fields for a `ConsensusFollower`. type Config struct { - nodeID flow.Identifier // the node ID of this node + networkPrivKey crypto.PrivateKey // the network private key of this node bootstrapNodes []BootstrapNodeInfo // the bootstrap nodes to use bindAddr string // address to bind on dataDir string // directory to store the protocol state @@ -70,12 +70,12 @@ func getAccessNodeOptions(config *Config) []access.Option { return []access.Option{ access.WithBootStrapPeers(ids...), access.WithBaseOptions(getBaseOptions(config)), + access.WithNetworkKey(config.networkPrivKey), } } func getBaseOptions(config *Config) []cmd.Option { options := []cmd.Option{ - cmd.WithNetworkPublicKey(config.networkPubKey), cmd.WithMetricsEnabled(false), } if config.bootstrapDir != "" { @@ -109,13 +109,13 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - nodeID flow.Identifier, - bootstapIdentities []BootstrapNodeInfo, + networkPrivKey crypto.PrivateKey, bindAddr string, + bootstapIdentities []BootstrapNodeInfo, opts ...Option, ) (*ConsensusFollowerImpl, error) { config := &Config{ - nodeID: nodeID, + networkPrivKey: networkPrivKey, bootstrapNodes: bootstapIdentities, bindAddr: bindAddr, } diff --git a/integration/testnet/network.go b/integration/testnet/network.go index ee1f90c2be7..bfdfa4c70b1 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -218,13 +218,13 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { type ConsensusFollowerConfig struct { networkKey fcrypto.PrivateKey - upstreamNodeID flow.Identifier + bootstrapNodes []consensus_follower.BootstrapNodeInfo } -func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, bootstrapNodes []consensus_follower.BootstrapNodeInfo) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - networkKey: fcrypto.PrivateKey, - upstreamNodeID: upstreamNodeID, + networkKey: networkKey, + bootstrapNodes: bootstrapNodes, } } @@ -306,14 +306,14 @@ func (n *NetworkConfig) Swap(i, j int) { // NodeConfig defines the input config for a particular node, specified prior // to network creation. type NodeConfig struct { - Role flow.Role - Stake uint64 - Identifier flow.Identifier - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - ParticipatesInPublicNetwork bool // only applicable to Access node + Role flow.Role + Stake uint64 + Identifier flow.Identifier + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + SupportsUnstakedNodes bool // only applicable to Access node } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -389,9 +389,9 @@ func AsGhost() func(config *NodeConfig) { } } -func AsPublicNetworkParticipant() func(config *NodeConfig) { +func SupportsUnstakedNodes() func(config *NodeConfig) { return func(config *NodeConfig) { - config.ParticipatesInPublicNetwork = true + config.SupportsUnstakedNodes = true } } @@ -492,12 +492,7 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower := consensus_follower.NewConsensusFollower( - followerConf.networkKey, - followerConf.upstreamNodeID, - bindAddr, - opts..., - ) + follower, err := consensus_follower.NewConsensusFollower(followerConf.networkKey, bindAddr, followerConf.bootstrapNodes, opts...) // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower @@ -752,7 +747,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) } - allNodeInfos = append(toNodeInfos(stakedConfs), followerInfos...) + allNodeInfos := append(toNodeInfos(stakedConfs), followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), order.Canonical) @@ -953,7 +948,7 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { Ghost: conf.Ghost, AdditionalFlags: conf.AdditionalFlags, Debug: conf.Debug, - ParticipatesInPublicNetwork: conf.ParticipatesInPublicNetwork, + ParticipatesInPublicNetwork: conf.SupportsUnstakedNodes, } confs = append(confs, containerConf) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index f5259c853aa..16de0d53db4 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/rs/zerolog" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" consensus_follower "github.com/onflow/flow-go/follower" @@ -49,7 +50,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { stakedConfig := testnet.NewNodeConfig( flow.RoleAccess, testnet.WithID(suite.stakedID), - testnet.AsUnstakedNetworkParticipant(), + testnet.SupportsUnstakedNodes(), testnet.WithLogLevel(zerolog.InfoLevel), ) nodeConfigs = append(nodeConfigs, stakedConfig) @@ -89,9 +90,12 @@ func (suite *UnstakedAccessSuite) SetupTest() { nodeConfigs = append(nodeConfigs, collConfig) // consensus follower - suite.unstakedID = unittest.IdentifierFixture() + unstakedKey, err := unittest.NetworkingKey() + require.NoError(suite.T(), err) + + followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(suite.unstakedID, suite.stakedID), + testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) From dde1c4e7f997a936c25400235e5e100f365f40a5 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 15:48:03 -0700 Subject: [PATCH 079/291] wip --- .../unstaked_access_node_builder.go | 20 ++--- integration/testnet/container.go | 14 +-- integration/testnet/network.go | 87 ++++++++++++------- .../tests/access/unstaked_node_test.go | 4 +- 4 files changed, 72 insertions(+), 53 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 5e27f7702bf..fd12b4e3237 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -57,12 +57,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() - builder.EnqueueMetricsServerInit() - - builder.RegisterBadgerMetrics() - - builder.EnqxueueTracer() - builder.PreInit(builder.initUnstakedLocal()) return builder @@ -105,13 +99,13 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { - anb. - Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - // use the default identifier provider - anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() - return nil - }) - anb.FlowAccessNodeBuilder.BuildConsensusFollower() + //anb. + // Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + // // use the default identifier provider + // anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() + // return nil + // }) + //anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } diff --git a/integration/testnet/container.go b/integration/testnet/container.go index 666c54d349a..8b60384ef58 100644 --- a/integration/testnet/container.go +++ b/integration/testnet/container.go @@ -38,13 +38,13 @@ func init() { // ContainerConfig represents configuration for a node container in the network. type ContainerConfig struct { bootstrap.NodeInfo - ContainerName string - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - Unstaked bool - ParticipatesInUnstakedNetwork bool + ContainerName string + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + Unstaked bool + SupportsUnstakedNodes bool } // ImageName returns the Docker image name for the given config. diff --git a/integration/testnet/network.go b/integration/testnet/network.go index bfdfa4c70b1..bccc9c80c86 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -25,9 +25,9 @@ import ( "github.com/onflow/cadence" "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go/cmd/bootstrap/run" "github.com/onflow/flow-go/cmd/bootstrap/utils" - fcrypto "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" @@ -217,14 +217,16 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - networkKey fcrypto.PrivateKey - bootstrapNodes []consensus_follower.BootstrapNodeInfo + nodeID flow.Identifier + networkingPrivKey crypto.PrivateKey + stakedNodeID flow.Identifier } -func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, bootstrapNodes []consensus_follower.BootstrapNodeInfo) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier, nodeID flow.Identifier) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - networkKey: networkKey, - bootstrapNodes: bootstrapNodes, + networkingPrivKey: networkingPrivKey, + stakedNodeID: stakedNodeID, + nodeID: nodeID, // TODO: remove this and derive it from the key instead } } @@ -458,13 +460,13 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { // add each follower to the network for _, followerConf := range networkConf.ConsensusFollowers { - flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) + flowNetwork.addConsensusFollower(t, bootstrapDir, followerConf, confs) } return flowNetwork } -func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) { +func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig, containers []ContainerConfig) { tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") require.NoError(t, err) @@ -490,9 +492,35 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, consensus_follower.WithBootstrapDir(followerBootstrapDir), } + var stakedANContainer *ContainerConfig + // find the upstream Access node container for this follower engine + for _, cont := range containers { + if cont.NodeID == followerConf.stakedNodeID { + stakedANContainer = &cont + break + } + } + require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) + + hostPort := strings.Split(stakedANContainer.Address, ":") + require.Len(t, hostPort, 2, "invalid address for staked AN %s", stakedANContainer.Address) + + host := hostPort[0] + portStr := hostPort[1] + portU64, err := strconv.ParseUint(portStr, 10, 32) + require.NoError(t, err) + port := uint(portU64) + + bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ + Host: host, + Port: port, + NetworkPublicKey: stakedANContainer.NetworkPubKey(), + } + // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower, err := consensus_follower.NewConsensusFollower(followerConf.networkKey, bindAddr, followerConf.bootstrapNodes, opts...) + follower, err := consensus_follower.NewConsensusFollower(followerConf.networkingPrivKey, bindAddr, + []consensus_follower.BootstrapNodeInfo{bootstrapNodeInfo}, opts...) // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower @@ -633,9 +661,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort - if nodeConf.ParticipatesInPublicNetwork { + if nodeConf.SupportsUnstakedNodes { // TODO: define this flag for Access node - nodeContainer.addFlag("public-network-participant", "true") + nodeContainer.addFlag("supports-unstaked-node", "true") } case flow.RoleConsensus: @@ -658,7 +686,7 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort - if nodeConf.ParticipatesInPublicNetwork { + if nodeConf.SupportsUnstakedNodes { // TODO: Currently, it is not possible to create a ghost AN which participates // in the public network, because connection gating is enabled by default and // therefore the ghost node will deny incoming connections from all consensus @@ -693,28 +721,22 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { var nodeInfos []bootstrap.NodeInfo - // TODO: remove this, networking keys should be provided by the consensus follower config. - // get networking keys for all followers - networkKeys, err := unittest.NetworkingKeys(len(confs)) + // TODO: currently just stashing a dummy key as staking key to prevent the nodeinfo.Type() function from + // returning an error. Eventually, a new key type NodeInfoTypePrivateUnstaked needs to be defined + dummyStakingKey, err := unittest.StakingKey() if err != nil { return nil, err } - // get staking keys for all followers - stakingKeys, err := unittest.StakingKeys(len(confs)) - if err != nil { - return nil, err - } - - for i, conf := range confs { + for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address 0, // no stake - networkKeys[i], - stakingKeys[i], + conf.networkingPrivKey, + dummyStakingKey, ) nodeInfos = append(nodeInfos, info) @@ -742,6 +764,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, fmt.Errorf("failed to setup keys: %w", err) } + // generate the follower node keys (follow nodes do not run as docker containers) followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) @@ -897,7 +920,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, err } - return root, result, seal, allConfs, nil + return root, result, seal, stakedConfs, nil } // setupKeys generates private staking and networking keys for each configured @@ -942,13 +965,13 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - ParticipatesInPublicNetwork: conf.SupportsUnstakedNodes, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + SupportsUnstakedNodes: conf.SupportsUnstakedNodes, } confs = append(confs, containerConf) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 16de0d53db4..cc1033c4288 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -92,10 +92,12 @@ func (suite *UnstakedAccessSuite) SetupTest() { // consensus follower unstakedKey, err := unittest.NetworkingKey() require.NoError(suite.T(), err) + // TODO: derive node id from the key + suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID), + testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) From 1c8bb501058fd5086591a3ac5c7096c3be37fcd4 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 17:10:13 -0700 Subject: [PATCH 080/291] wip --- cmd/access/node_builder/access_node_builder.go | 6 +++--- .../node_builder/staked_access_node_builder.go | 12 +++--------- .../unstaked_access_node_builder.go | 17 +++++++++++++---- cmd/node_builder.go | 3 --- cmd/scaffold.go | 7 +++++-- 5 files changed, 24 insertions(+), 21 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b4159c97445..aee41359d8a 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -42,7 +42,7 @@ import ( "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/network" - jsoncodec "github.com/onflow/flow-go/network/codec/json" + cborcodec "github.com/onflow/flow-go/network/codec/cbor" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/validator" "github.com/onflow/flow-go/state/protocol" @@ -90,7 +90,7 @@ type AccessNodeConfig struct { bootstrapNodeAddresses []string bootstrapNodePublicKeys []string bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes - NetworkKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // the networking key passed in by the caller when being used as a library supportsUnstakedFollower bool collectionGRPCPort uint executionGRPCPort uint @@ -651,7 +651,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, middleware network.Middleware, topology network.Topology) (*p2p.Network, error) { - codec := jsoncodec.NewCodec() + codec := cborcodec.NewCodec() subscriptionManager := p2p.NewChannelSubscriptionManager(middleware) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 7ce262d6c54..93f24ca9f0f 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -13,7 +13,6 @@ import ( "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" - "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" ) @@ -110,17 +109,12 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NetworkKey) + libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NodeConfig.NetworkKey) builder.MustNot(err) msgValidators := unstakedNetworkMsgValidators(builder.NodeID) - // Network Metrics - // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics - // TODO: define new network metrics for the unstaked network - unstakedNetworkMetrics := metrics.NewNoopCollector() - - middleware := builder.initMiddleware(builder.NodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront top, err := topology.NewTopicBasedTopology( @@ -133,7 +127,7 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C } topologyCache := topology.NewCache(builder.Logger, top) - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, topologyCache) + network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, topologyCache) builder.MustNot(err) builder.Network = network diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index fd12b4e3237..b684c26565b 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -21,9 +21,13 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } -func (fnb *UnstakedAccessNodeBuilder) InitNodeInfo() { - fnb.NodeID = flow.ZeroID // TODO: extract node id from networking key - fnb.NodeConfig.NetworkKey = fnb.NetworkKey // use the networking that has been passed in +func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { + // use the networking key that has been passed in the config + networkingKey := fnb.AccessNodeConfig.NetworkKey + nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + fnb.MustNot(err) + fnb.NodeID = nodeID + fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } @@ -49,6 +53,11 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() + // if a network key has been passed in the init node info here + if builder.AccessNodeConfig.NetworkKey != nil { + builder.initNodeInfo() + } + builder.InitIDProviders() builder.deriveBootstrapPeerIdentities() @@ -105,7 +114,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { // anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() // return nil // }) - //anb.FlowAccessNodeBuilder.BuildConsensusFollower() + anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 7ea3736df0d..8bdd0f0dfcd 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -44,9 +44,6 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() - // InitNodeInfo initializes the node id, staking key and networking key - InitNodeInfo() - // InitIDProviders initializes the IdentityProvider and IDTranslator InitIDProviders() diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 0bbb94f3b7f..d0b0ede1213 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -256,7 +256,7 @@ func (fnb *FlowNodeBuilder) PrintBuildVersionDetails() { fnb.Logger.Info().Str("version", build.Semver()).Str("commit", build.Commit()).Msg("build details") } -func (fnb *FlowNodeBuilder) InitNodeInfo() { +func (fnb *FlowNodeBuilder) initNodeInfo() { if fnb.BaseConfig.nodeIDHex == NotSet { fnb.Logger.Fatal().Msg("cannot start without node ID") } @@ -795,7 +795,10 @@ func (fnb *FlowNodeBuilder) Ready() <-chan struct{} { // seed random generator rand.Seed(time.Now().UnixNano()) - fnb.InitNodeInfo() + // init nodeinfo by reading the private bootstrap file if not already set + if fnb.NodeID == flow.ZeroID { + fnb.initNodeInfo() + } fnb.initLogger() From 867ee250bb01e77a6244ad9ba0180f5e3297a7b8 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 17:21:26 -0700 Subject: [PATCH 081/291] fixing unstaked_node_test integration test --- cmd/access/node_builder/unstaked_access_node_builder.go | 6 +++--- integration/testnet/network.go | 9 +++++++-- integration/tests/access/unstaked_node_test.go | 2 +- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index b684c26565b..6285c0b3529 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -24,11 +24,11 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config networkingKey := fnb.AccessNodeConfig.NetworkKey - nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this fnb.MustNot(err) fnb.NodeID = nodeID - fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig - fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node + fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig + fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { diff --git a/integration/testnet/network.go b/integration/testnet/network.go index bccc9c80c86..98f5b660184 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -243,10 +243,9 @@ type NetworkConfig struct { type NetworkConfigOpt func(*NetworkConfig) -func NewNetworkConfig(name string, nodes []NodeConfig, followers []ConsensusFollowerConfig, opts ...NetworkConfigOpt) NetworkConfig { +func NewNetworkConfig(name string, nodes []NodeConfig, opts ...NetworkConfigOpt) NetworkConfig { c := NetworkConfig{ Nodes: nodes, - ConsensusFollowers: followers, Name: name, NClusters: 1, // default to 1 cluster ViewsInStakingAuction: DefaultViewsInStakingAuction, @@ -285,6 +284,12 @@ func WithClusters(n uint) func(*NetworkConfig) { } } +func WithConsensusFollowers(followers ...ConsensusFollowerConfig) func(*NetworkConfig) { + return func(conf *NetworkConfig) { + conf.ConsensusFollowers = followers + } +} + func (n *NetworkConfig) Len() int { return len(n.Nodes) } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index cc1033c4288..680050bdc07 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -100,7 +100,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) From d674ac9dd90d51e11a8feb633606a0957080f9a6 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 18:04:50 -0700 Subject: [PATCH 082/291] making the unstaked node test similar to the mvp test to allow block generation --- .../tests/access/unstaked_node_test.go | 90 ++++++++++--------- 1 file changed, 46 insertions(+), 44 deletions(-) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 680050bdc07..0a18d01617a 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -2,7 +2,9 @@ package access import ( "context" + "fmt" "testing" + "time" "github.com/rs/zerolog" "github.com/stretchr/testify/require" @@ -43,7 +45,23 @@ func (suite *UnstakedAccessSuite) TearDownTest() { } func (suite *UnstakedAccessSuite) SetupTest() { - nodeConfigs := []testnet.NodeConfig{} + suite.buildNetworkConfig() + // start the network + suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.net.Start(suite.ctx) +} + +func (suite *UnstakedAccessSuite) TestReceiveBlocks() { + //go suite.follower.Run(suite.ctx) + // TODO: to be implemented later + time.Sleep(time.Second * 30) +} + +func (suite *UnstakedAccessSuite) OnBlockFinalizedConsumer(finalizedBlockID flow.Identifier) { + fmt.Println(finalizedBlockID.String()) +} + +func (suite *UnstakedAccessSuite) buildNetworkConfig() { // staked access node suite.stakedID = unittest.IdentifierFixture() @@ -53,41 +71,34 @@ func (suite *UnstakedAccessSuite) SetupTest() { testnet.SupportsUnstakedNodes(), testnet.WithLogLevel(zerolog.InfoLevel), ) - nodeConfigs = append(nodeConfigs, stakedConfig) - - // consensus node (ghost) - suite.conID = unittest.IdentifierFixture() - conConfig := testnet.NewNodeConfig( - flow.RoleConsensus, - testnet.WithID(suite.conID), - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, conConfig) - // execution node (unused) - exeConfig := testnet.NewNodeConfig( - flow.RoleExecution, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, exeConfig) + collectionConfigs := []func(*testnet.NodeConfig){ + testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), + testnet.WithAdditionalFlag("--block-rate-delay=100ms"), + testnet.WithLogLevel(zerolog.WarnLevel), + // TODO replace these with actual values + testnet.WithAdditionalFlag("--access-address=null"), + } - // verification node (unused) - verConfig := testnet.NewNodeConfig( - flow.RoleVerification, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, verConfig) + consensusConfigs := []func(config *testnet.NodeConfig){ + testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), + testnet.WithAdditionalFlag("--block-rate-delay=100ms"), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 1)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 1)), + testnet.WithLogLevel(zerolog.WarnLevel), + } - // collection node (unused) - collConfig := testnet.NewNodeConfig( - flow.RoleCollection, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, collConfig) + net := []testnet.NodeConfig{ + testnet.NewNodeConfig(flow.RoleCollection, collectionConfigs...), + testnet.NewNodeConfig(flow.RoleCollection, collectionConfigs...), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.WarnLevel)), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.WarnLevel)), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.WarnLevel), testnet.WithDebugImage(false)), + stakedConfig, + } // consensus follower unstakedKey, err := unittest.NetworkingKey() @@ -95,22 +106,13 @@ func (suite *UnstakedAccessSuite) SetupTest() { // TODO: derive node id from the key suite.unstakedID = unittest.IdentifierFixture() - followerConfigs := []testnet.ConsensusFollowerConfig{ testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, testnet.WithConsensusFollowers(followerConfigs...)) + conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) - - // start the network - suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.net.Start(suite.ctx) -} - -func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - go suite.follower.Run(suite.ctx) - // TODO: to be implemented later + suite.follower.AddOnBlockFinalizedConsumer(suite.OnBlockFinalizedConsumer) } From 80a4ad917c434c6dabd9fefedf6f6e79e56eeae4 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 18:30:12 -0700 Subject: [PATCH 083/291] adding default size for cache --- cmd/access/node_builder/unstaked_access_node_builder.go | 6 ++++++ cmd/node_builder.go | 3 +++ integration/tests/access/unstaked_node_test.go | 2 +- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 6285c0b3529..6066d138068 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -66,6 +66,12 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() + //builder.EnqueueMetricsServerInit() + // + //builder.RegisterBadgerMetrics() + // + //builder.EnqueueTracer() + builder.PreInit(builder.initUnstakedLocal()) return builder diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 8bdd0f0dfcd..20cedbab152 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -21,6 +21,7 @@ import ( "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/state/protocol/events" + bstorage "github.com/onflow/flow-go/storage/badger" ) const NotSet = "not set" @@ -169,5 +170,7 @@ func DefaultBaseConfig() *BaseConfig { profilerDuration: 10 * time.Second, tracerEnabled: false, metricsEnabled: true, + receiptsCacheSize: bstorage.DefaultCacheSize, + guaranteesCacheSize: bstorage.DefaultCacheSize, } } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 0a18d01617a..daddce23936 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -52,7 +52,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - //go suite.follower.Run(suite.ctx) + go suite.follower.Run(suite.ctx) // TODO: to be implemented later time.Sleep(time.Second * 30) } From 6416191166bc73fc12492cb1fdf7bc2004993dc0 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 19:42:07 -0700 Subject: [PATCH 084/291] moving builder.deriveBootstrapPeerIdentities to access_node_builder from unstaked_access_node_builder --- cmd/access/node_builder/access_node_builder.go | 10 ++++++++++ .../node_builder/unstaked_access_node_builder.go | 12 +----------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index aee41359d8a..2cd0d8073f4 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -548,6 +548,8 @@ func (builder *FlowAccessNodeBuilder) ParseFlags() { builder.extraFlags() builder.ParseAndPrintFlags() + + builder.deriveBootstrapPeerIdentities() } func (builder *FlowAccessNodeBuilder) extraFlags() { @@ -583,6 +585,14 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { }) } +// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstrap peers from the parameters. +// These are the identities of the staked and unstaked ANs also acting as the DHT bootstrap server +func (builder *FlowAccessNodeBuilder) deriveBootstrapPeerIdentities() { + ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) + builder.MustNot(err) + builder.bootstrapIdentites = ids +} + // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. // The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 6066d138068..8ff04d6a55a 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -60,8 +60,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() - builder.deriveBootstrapPeerIdentities() - builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() @@ -83,14 +81,6 @@ func (builder *UnstakedAccessNodeBuilder) validateParams() { } } -// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstreap peers from the parameters. -// These are the identity of the staked and unstaked AN also acting as the DHT bootstrap server -func (builder *UnstakedAccessNodeBuilder) deriveBootstrapPeerIdentities() { - ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) - builder.MustNot(err) - builder.bootstrapIdentites = ids -} - // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files @@ -170,7 +160,7 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil }) } From f53b07c616bcc4595c4b87cc97d71d480e3ea7ce Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 22:14:53 -0700 Subject: [PATCH 085/291] exposing libp2p port externally in integration test docker container for consensus follower to access --- cmd/access/node_builder/upstream_connector.go | 2 +- integration/testnet/network.go | 15 ++++++++------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index fdc608cfadb..233614d1f87 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -40,7 +40,7 @@ func (connector *upstreamConnector) Ready() <-chan struct{} { defer close(resultChan) // a shorter context for the connection worker - workerCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + workerCtx, cancel := context.WithTimeout(ctx, 30*time.Second) defer cancel() // spawn a connect worker for each bootstrap node diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 98f5b660184..9b14f56bdc4 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -66,6 +66,8 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" + // AccessNodeExternalNetworkPort is the name used for the access node network port accessible from outside any docker container + AccessNodeExternalNetworkPort = "access-external-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" @@ -507,17 +509,13 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, } require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) - hostPort := strings.Split(stakedANContainer.Address, ":") - require.Len(t, hostPort, 2, "invalid address for staked AN %s", stakedANContainer.Address) - - host := hostPort[0] - portStr := hostPort[1] + portStr := net.AccessPorts[AccessNodeExternalNetworkPort] portU64, err := strconv.ParseUint(portStr, 10, 32) require.NoError(t, err) port := uint(portU64) bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ - Host: host, + Host: "localhost", Port: port, NetworkPublicKey: stakedANContainer.NetworkPubKey(), } @@ -667,7 +665,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort if nodeConf.SupportsUnstakedNodes { - // TODO: define this flag for Access node + hostExternalNetworkPort := testingdock.RandomPort(t) + nodeContainer.bindPort(hostExternalNetworkPort, fmt.Sprintf("%s/tcp", strconv.Itoa(DefaultFlowPort))) + net.AccessPorts[AccessNodeExternalNetworkPort] = hostExternalNetworkPort nodeContainer.addFlag("supports-unstaked-node", "true") } @@ -697,6 +697,7 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont // therefore the ghost node will deny incoming connections from all consensus // followers. A flag for the ghost node will need to be created to enable // overriding the default behavior. + return fmt.Errorf("currently ghost node for an access node which supports unstaked node is not implemented") } } From b9de82c2820cdec5092b71f5d1fe03c37f6ef815 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:23:29 -0700 Subject: [PATCH 086/291] update node id logic --- cmd/access/node_builder/access_node_builder.go | 8 ++++---- .../unstaked_access_node_builder.go | 18 +++++++++++------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 2cd0d8073f4..e01b571acf1 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -89,7 +89,7 @@ type AccessNodeConfig struct { staked bool bootstrapNodeAddresses []string bootstrapNodePublicKeys []string - bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes + bootstrapIdentities flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes NetworkKey crypto.PrivateKey // the networking key passed in by the caller when being used as a library supportsUnstakedFollower bool collectionGRPCPort uint @@ -493,7 +493,7 @@ type Option func(*AccessNodeConfig) func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { return func(config *AccessNodeConfig) { - config.bootstrapIdentites = bootstrapNodes + config.bootstrapIdentities = bootstrapNodes } } @@ -590,7 +590,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { func (builder *FlowAccessNodeBuilder) deriveBootstrapPeerIdentities() { ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) builder.MustNot(err) - builder.bootstrapIdentites = ids + builder.bootstrapIdentities = ids } // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. @@ -604,7 +604,7 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, // if this is an unstaked access node, then seed the DHT with the boostrap identities if !builder.IsStaked() { - bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentites) + bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentities) builder.MustNot(err) dhtOptions = append(dhtOptions, bootstrapPeersOpt) } diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 8ff04d6a55a..83d1fd0b6a9 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -3,6 +3,7 @@ package node_builder import ( "context" + "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -24,17 +25,20 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config networkingKey := fnb.AccessNodeConfig.NetworkKey - nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + pubKey, err := p2p.LibP2PPublicKeyFromFlow(networkingKey.PublicKey()) + fnb.MustNot(err) + peerID, err := peer.IDFromPublicKey(pubKey) + fnb.MustNot(err) + fnb.NodeID, err = fnb.IDTranslator.GetFlowID(peerID) fnb.MustNot(err) - fnb.NodeID = nodeID fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err @@ -53,13 +57,13 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() + builder.InitIDProviders() + // if a network key has been passed in the init node info here if builder.AccessNodeConfig.NetworkKey != nil { builder.initNodeInfo() } - builder.InitIDProviders() - builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() @@ -161,6 +165,6 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil + return newUpstreamConnector(builder.bootstrapIdentities, builder.LibP2PNode, builder.Logger), nil }) } From 6622214c3f8df07cd06f2646ebe48d0dfb23ae2a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:31:44 -0700 Subject: [PATCH 087/291] Add NetworkingIdentifierProider to staked node --- cmd/access/node_builder/staked_access_node_builder.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 93f24ca9f0f..649ababf928 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -50,9 +50,9 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: NetworkingIdentifierProvidzer should be the same as the one used in scaffold.go if this AN - // doesn't participate in unstaked network. - // If it does, then we can just use the default one (peerstoreProvider) + if !fnb.SupportsUnstakedNode() { + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) + } return nil }) From dfcd82108039d9796135d335e8a3094a29af4711 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:35:19 -0700 Subject: [PATCH 088/291] Use hierarchical translator for unstaked node --- .../node_builder/unstaked_access_node_builder.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 83d1fd0b6a9..854a2ba4b53 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -29,15 +29,13 @@ func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { fnb.MustNot(err) peerID, err := peer.IDFromPublicKey(pubKey) fnb.MustNot(err) - fnb.NodeID, err = fnb.IDTranslator.GetFlowID(peerID) + fnb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) fnb.MustNot(err) fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { @@ -46,6 +44,8 @@ func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) + return nil }) } @@ -57,13 +57,13 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() - builder.InitIDProviders() - // if a network key has been passed in the init node info here if builder.AccessNodeConfig.NetworkKey != nil { builder.initNodeInfo() } + builder.InitIDProviders() + builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() From 03492e2395392c66ae95feab6581698d30dddc2b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:38:46 -0700 Subject: [PATCH 089/291] Add identity delta for staked and unstaked AN's --- cmd/access/node_builder/staked_access_node_builder.go | 4 ++++ cmd/access/node_builder/unstaked_access_node_builder.go | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 649ababf928..1cb94bec256 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" + "github.com/onflow/flow-go/state/protocol/events/gadgets" ) // StakedAccessNodeBuilder builds a staked access node. The staked access node can optionally participate in the @@ -133,6 +134,9 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Network = network builder.Middleware = middleware + idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) + builder.ProtocolEvents.AddConsumer(idEvents) + node.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) return builder.Network, err }) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 854a2ba4b53..f496f48fd8a 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/state/protocol/events/gadgets" ) type UnstakedAccessNodeBuilder struct { @@ -153,6 +154,9 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) + builder.ProtocolEvents.AddConsumer(idEvents) + return builder.Network, err }) } From e2a315f7aa795fb13a9b45977b42655e8831bdbb Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 00:19:57 -0700 Subject: [PATCH 090/291] Implement proper message validators for unstaked network. --- .../node_builder/access_node_builder.go | 9 +++++- .../staked_access_node_builder.go | 2 +- .../unstaked_access_node_builder.go | 2 +- network/validator/anyValidator.go | 28 +++++++++++++++++++ network/validator/originValidator.go | 24 ++++++++++++++++ 5 files changed, 62 insertions(+), 3 deletions(-) create mode 100644 network/validator/anyValidator.go create mode 100644 network/validator/originValidator.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index e01b571acf1..87dd23db409 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -9,6 +9,7 @@ import ( dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow/protobuf/go/flow/access" + "github.com/rs/zerolog" "github.com/spf13/pflag" "google.golang.org/grpc" "google.golang.org/grpc/credentials" @@ -684,10 +685,16 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, return net, nil } -func unstakedNetworkMsgValidators(selfID flow.Identifier) []network.MessageValidator { +func unstakedNetworkMsgValidators(log zerolog.Logger, idProvider id.IdentityProvider, selfID flow.Identifier) []network.MessageValidator { return []network.MessageValidator{ // filter out messages sent by this node itself validator.ValidateNotSender(selfID), + validator.NewAnyValidator( + validator.NewOriginValidator( + id.NewFilteredIdentifierProvider(filter.IsValidCurrentEpochParticipant, idProvider), + ), + validator.ValidateTarget(log, selfID), + ), } } diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 1cb94bec256..1d001cd5bac 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -113,7 +113,7 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NodeConfig.NetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(builder.NodeID) + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, builder.NodeID) middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index f496f48fd8a..843670f9531 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -137,7 +137,7 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) diff --git a/network/validator/anyValidator.go b/network/validator/anyValidator.go new file mode 100644 index 00000000000..0639de95bbb --- /dev/null +++ b/network/validator/anyValidator.go @@ -0,0 +1,28 @@ +package validator + +import ( + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" +) + +var _ network.MessageValidator = &AnyValidator{} + +// AnyValidator returns true if any of the given validators returns true +type AnyValidator struct { + validators []network.MessageValidator +} + +func NewAnyValidator(validators ...network.MessageValidator) network.MessageValidator { + return &AnyValidator{ + validators: validators, + } +} + +func (v AnyValidator) Validate(msg message.Message) bool { + for _, validator := range v.validators { + if validator.Validate(msg) { + return true + } + } + return false +} diff --git a/network/validator/originValidator.go b/network/validator/originValidator.go new file mode 100644 index 00000000000..250f464d6d5 --- /dev/null +++ b/network/validator/originValidator.go @@ -0,0 +1,24 @@ +package validator + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" +) + +var _ network.MessageValidator = &OriginValidator{} + +// OriginValidator returns true if the sender of the message is among the set of identifiers +// returned by the given IdentifierProvider +type OriginValidator struct { + idProvider id.IdentifierProvider +} + +func NewOriginValidator(provider id.IdentifierProvider) network.MessageValidator { + return &OriginValidator{provider} +} + +func (v OriginValidator) Validate(msg message.Message) bool { + return v.idProvider.Identifiers().Contains(flow.HashToID(msg.OriginID)) +} From e979066f62f57c803096fb6a1a02c85407ed5c83 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 00:31:13 -0700 Subject: [PATCH 091/291] recover sync engine participants provider --- .../unstaked_access_node_builder.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 843670f9531..6d25c040015 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -2,6 +2,7 @@ package node_builder import ( "context" + "errors" "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" @@ -109,12 +110,16 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { - //anb. - // Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - // // use the default identifier provider - // anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() - // return nil - // }) + anb. + Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + middleware, ok := node.Middleware.(*p2p.Middleware) + if !ok { + return errors.New("middleware was of unexpected type") + } + // use the default identifier provider + anb.SyncEngineParticipantsProvider = middleware.IdentifierProvider() + return nil + }) anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } From f84d953191893a178435a336b445652bdd3a3239 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 00:47:01 -0700 Subject: [PATCH 092/291] add sync provider to staked AN --- .../staked_access_node_builder.go | 73 ++++++++++++++----- engine/channels.go | 3 + engine/common/splitter/engine.go | 9 +-- engine/common/splitter/engine_test.go | 27 ++----- 4 files changed, 66 insertions(+), 46 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 1d001cd5bac..612c63a2bda 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -8,11 +8,15 @@ import ( "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/engine" pingeng "github.com/onflow/flow-go/engine/access/ping" + "github.com/onflow/flow-go/engine/common/splitter" + synceng "github.com/onflow/flow-go/engine/common/synchronization" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" "github.com/onflow/flow-go/state/protocol/events/gadgets" @@ -84,23 +88,58 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { - anb.FlowAccessNodeBuilder. - Build(). - Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - ping, err := pingeng.New( - node.Logger, - node.State, - node.Me, - anb.PingMetrics, - anb.pingEnabled, - node.Middleware, - anb.nodeInfoFile, - ) - if err != nil { - return nil, fmt.Errorf("could not create ping engine: %w", err) - } - return ping, nil - }) + anb.FlowAccessNodeBuilder.Build() + + if anb.SupportsUnstakedNode() { + var unstakedNetworkConduit network.Conduit + var proxyEngine *splitter.Engine + + anb. + Component("unstaked sync request proxy", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + proxyEngine := splitter.New(node.Logger, engine.SyncCommittee) + + // register the proxy engine with the unstaked network + var err error + unstakedNetworkConduit, err = node.Network.Register(engine.SyncCommittee, proxyEngine) + if err != nil { + return nil, fmt.Errorf("could not register unstaked sync request proxy: %w", err) + } + + return proxyEngine, nil + }). + Component("unstaked sync request handler", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + syncRequestHandler := synceng.NewRequestHandlerEngine( + node.Logger, // TODO: use different logger for unstaked network? + node.Metrics.Engine, // TODO: use different metrics for unstaked network? + unstakedNetworkConduit, + node.Me, // TODO: does staked node use same Node ID on unstaked network? + node.Storage.Blocks, + anb.SyncCore, + anb.FinalizedHeader, + ) + + // register the sync request handler with the proxy engine + proxyEngine.RegisterEngine(syncRequestHandler) + + return syncRequestHandler, nil + }) + } + + anb.Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + ping, err := pingeng.New( + node.Logger, + node.State, + node.Me, + anb.PingMetrics, + anb.pingEnabled, + node.Middleware, + anb.nodeInfoFile, + ) + if err != nil { + return nil, fmt.Errorf("could not create ping engine: %w", err) + } + return ping, nil + }) return anb } diff --git a/engine/channels.go b/engine/channels.go index 0e673601a7e..b121e534e72 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -133,6 +133,9 @@ const ( ProvideChunks = RequestChunks ProvideReceiptsByBlockID = RequestReceiptsByBlockID ProvideApprovalsByChunk = RequestApprovalsByChunk + + // Unstaked network channels + UnstakedSyncCommittee = network.Channel("unstaked-sync-committee") ) // initializeChannelRoleMap initializes an instance of channelRoleMap and populates it with the channels and their diff --git a/engine/common/splitter/engine.go b/engine/common/splitter/engine.go index beb5b576978..280018924c9 100644 --- a/engine/common/splitter/engine.go +++ b/engine/common/splitter/engine.go @@ -1,7 +1,6 @@ package splitter import ( - "errors" "fmt" "sync" @@ -40,17 +39,11 @@ func New( // RegisterEngine registers a new engine with the splitter. Events // that are received by the splitter after the engine has registered // will be passed down to it. -func (e *Engine) RegisterEngine(engine module.Engine) error { +func (e *Engine) RegisterEngine(engine module.Engine) { e.enginesMu.Lock() defer e.enginesMu.Unlock() - if _, ok := e.engines[engine]; ok { - return errors.New("engine already registered with splitter") - } - e.engines[engine] = struct{}{} - - return nil } // UnregisterEngine unregisters an engine with the splitter. After diff --git a/engine/common/splitter/engine_test.go b/engine/common/splitter/engine_test.go index 0997e232630..ba31803e24d 100644 --- a/engine/common/splitter/engine_test.go +++ b/engine/common/splitter/engine_test.go @@ -48,10 +48,8 @@ func (suite *Suite) TestDownstreamEngineFailure() { engine1 := new(mockmodule.Engine) engine2 := new(mockmodule.Engine) - err := suite.engine.RegisterEngine(engine1) - suite.Assert().Nil(err) - err = suite.engine.RegisterEngine(engine2) - suite.Assert().Nil(err) + suite.engine.RegisterEngine(engine1) + suite.engine.RegisterEngine(engine2) processError := errors.New("Process Error!") @@ -60,7 +58,7 @@ func (suite *Suite) TestDownstreamEngineFailure() { engine1.On("Process", suite.channel, id, event).Return(processError).Once() engine2.On("Process", suite.channel, id, event).Return(nil).Once() - err = suite.engine.Process(suite.channel, id, event) + err := suite.engine.Process(suite.channel, id, event) merr, ok := err.(*multierror.Error) suite.Assert().True(ok) suite.Assert().Len(merr.Errors, 1) @@ -100,26 +98,14 @@ func (suite *Suite) TestProcessUnknownChannel() { engine := new(mockmodule.Engine) - err := suite.engine.RegisterEngine(engine) - suite.Assert().Nil(err) + suite.engine.RegisterEngine(engine) - err = suite.engine.Process(unknownChannel, id, event) + err := suite.engine.Process(unknownChannel, id, event) suite.Assert().Error(err) engine.AssertNumberOfCalls(suite.T(), "Process", 0) } -// TestDuplicateRegistrations tests that an engine cannot register for the same channel twice. -func (suite *Suite) TestDuplicateRegistrations() { - engine := new(mockmodule.Engine) - - err := suite.engine.RegisterEngine(engine) - suite.Assert().Nil(err) - - err = suite.engine.RegisterEngine(engine) - suite.Assert().Error(err) -} - // TestConcurrentEvents tests that sending multiple messages concurrently, results in each engine // receiving every message. func (suite *Suite) TestConcurrentEvents() { @@ -131,8 +117,7 @@ func (suite *Suite) TestConcurrentEvents() { for i := 0; i < numEngines; i++ { engine := new(mockmodule.Engine) - err := suite.engine.RegisterEngine(engine) - suite.Assert().Nil(err) + suite.engine.RegisterEngine(engine) engines[i] = engine } From 1b56de4512d48d3d350b0f6a09fa60398ea87275 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 01:14:20 -0700 Subject: [PATCH 093/291] implement reassignment of sync engine channel --- .../staked_access_node_builder.go | 2 +- .../unstaked_access_node_builder.go | 8 +++--- cmd/node_builder.go | 2 +- engine/common/splitter/network/network.go | 18 +++---------- .../common/splitter/network/network_test.go | 2 +- module/network.go | 5 ++++ network/channel_reassigner/network.go | 25 +++++++++++++++++++ network/p2p/network.go | 5 ---- utils/unittest/network/network.go | 4 +-- 9 files changed, 42 insertions(+), 29 deletions(-) create mode 100644 network/channel_reassigner/network.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 612c63a2bda..580e304134f 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -100,7 +100,7 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { // register the proxy engine with the unstaked network var err error - unstakedNetworkConduit, err = node.Network.Register(engine.SyncCommittee, proxyEngine) + unstakedNetworkConduit, err = node.Network.Register(engine.UnstakedSyncCommittee, proxyEngine) if err != nil { return nil, fmt.Errorf("could not register unstaked sync request proxy: %w", err) } diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 6d25c040015..56909cf44c3 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -6,10 +6,12 @@ import ( "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/channel_reassigner" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol/events/gadgets" ) @@ -150,11 +152,7 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) builder.MustNot(err) - builder.Network = network - builder.Middleware = middleware - - // for an unstaked node, the staked network and middleware is set to the same as the unstaked network and middlware - builder.Network = network + builder.Network = channel_reassigner.NewChannelReassignerNetwork(network, engine.SyncCommittee, engine.UnstakedSyncCommittee) builder.Middleware = middleware builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 20cedbab152..d7dbceed3f1 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -132,7 +132,7 @@ type NodeConfig struct { ProtocolEvents *events.Distributor State protocol.State Middleware network.Middleware - Network p2p.ReadyDoneAwareNetwork + Network module.ReadyDoneAwareNetwork MsgValidators []network.MessageValidator FvmOptions []fvm.Option StakingKey crypto.PrivateKey diff --git a/engine/common/splitter/network/network.go b/engine/common/splitter/network/network.go index e8ed2c395e4..09743c3a130 100644 --- a/engine/common/splitter/network/network.go +++ b/engine/common/splitter/network/network.go @@ -11,7 +11,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/network/p2p" ) // Network is the splitter network. It is a wrapper around the default network implementation @@ -21,7 +20,7 @@ import ( // splitter engine. As a result, multiple engines can register with the splitter network on // the same channel and will each receive all events on that channel. type Network struct { - p2p.ReadyDoneAwareNetwork + module.ReadyDoneAwareNetwork mu sync.RWMutex log zerolog.Logger splitters map[network.Channel]*splitterEngine.Engine // stores splitters for each channel @@ -31,7 +30,7 @@ type Network struct { // NewNetwork returns a new splitter network. func NewNetwork( - net p2p.ReadyDoneAwareNetwork, + net module.ReadyDoneAwareNetwork, log zerolog.Logger, ) *Network { return &Network{ @@ -76,19 +75,10 @@ func (n *Network) Register(channel network.Channel, e network.Engine) (network.C } // register engine with splitter - err := splitter.RegisterEngine(engine) - - if err != nil { - // remove the splitter engine if this was the first time the given channel was registered - if !channelRegistered { - delete(n.splitters, channel) - } - - return nil, fmt.Errorf("failed to register engine with splitter: %w", err) - } + splitter.RegisterEngine(engine) if !channelRegistered { - conduit, err = n.ReadyDoneAwareNetwork.Register(channel, splitter) + conduit, err := n.ReadyDoneAwareNetwork.Register(channel, splitter) if err != nil { // undo previous steps diff --git a/engine/common/splitter/network/network_test.go b/engine/common/splitter/network/network_test.go index b2817a187a9..eb756c6c4a8 100644 --- a/engine/common/splitter/network/network_test.go +++ b/engine/common/splitter/network/network_test.go @@ -32,7 +32,7 @@ type Suite struct { } func (suite *Suite) SetupTest() { - net := new(mocknetwork.ReadyDoneAwareNetwork) + net := new(mockmodule.ReadyDoneAwareNetwork) suite.con = new(mocknetwork.Conduit) suite.engines = make(map[network.Channel]module.Engine) diff --git a/module/network.go b/module/network.go index 53da8e4bee0..60040028328 100644 --- a/module/network.go +++ b/module/network.go @@ -18,3 +18,8 @@ type Network interface { // On a single node, only one engine can be subscribed to a channel at any given time. Register(channel network.Channel, engine network.Engine) (network.Conduit, error) } + +type ReadyDoneAwareNetwork interface { + Network + ReadyDoneAware +} diff --git a/network/channel_reassigner/network.go b/network/channel_reassigner/network.go new file mode 100644 index 00000000000..3daedbbccd0 --- /dev/null +++ b/network/channel_reassigner/network.go @@ -0,0 +1,25 @@ +package channel_reassigner + +import ( + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/network" +) + +type ChannelReassignerNetwork struct { + module.ReadyDoneAwareNetwork + from network.Channel + to network.Channel +} + +func NewChannelReassignerNetwork(net module.ReadyDoneAwareNetwork, from, to network.Channel) *ChannelReassignerNetwork { + return &ChannelReassignerNetwork{net, from, to} +} + +func (n *ChannelReassignerNetwork) Register(channel network.Channel, engine network.Engine) (network.Conduit, error) { + reassignedChannel := channel + if channel == n.from { + reassignedChannel = n.to + } + + return n.ReadyDoneAwareNetwork.Register(reassignedChannel, engine) +} diff --git a/network/p2p/network.go b/network/p2p/network.go index 1c0d010a16f..cbdd02e7345 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -32,11 +32,6 @@ const DefaultCacheSize = 10e6 // be included in network communication. We omit any nodes that have been ejected. var NotEjectedFilter = filter.Not(filter.Ejected) -type ReadyDoneAwareNetwork interface { - module.Network - module.ReadyDoneAware -} - // Network represents the overlay network of our peer-to-peer network, including // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { diff --git a/utils/unittest/network/network.go b/utils/unittest/network/network.go index 9db8cbe7a43..bb247681f9a 100644 --- a/utils/unittest/network/network.go +++ b/utils/unittest/network/network.go @@ -32,7 +32,7 @@ func (c *Conduit) Publish(event interface{}, targetIDs ...flow.Identifier) error // Network represents a mock network. The implementation is not concurrency-safe. type Network struct { - mocknetwork.ReadyDoneAwareNetwork + mockmodule.ReadyDoneAwareNetwork conduits map[network.Channel]*Conduit engines map[network.Channel]network.Engine publishFunc NetworkPublishFunc @@ -41,7 +41,7 @@ type Network struct { // NewNetwork returns a new mock network. func NewNetwork() *Network { return &Network{ - ReadyDoneAwareNetwork: mocknetwork.ReadyDoneAwareNetwork{}, + ReadyDoneAwareNetwork: mockmodule.ReadyDoneAwareNetwork{}, conduits: make(map[network.Channel]*Conduit), engines: make(map[network.Channel]network.Engine), } From 823db49b21b4ed39921c9ca1a387fb69f2f992b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Tue, 24 Aug 2021 11:29:36 -0400 Subject: [PATCH 094/291] [bootstrap] define & doc consts for magic numbers in decompression --- cmd/bootstrap/utils/key_generation.go | 63 ++++++++++++++++++++++ cmd/bootstrap/utils/key_generation_test.go | 16 ++++++ 2 files changed, 79 insertions(+) diff --git a/cmd/bootstrap/utils/key_generation.go b/cmd/bootstrap/utils/key_generation.go index 73cc80ba2e3..7c157727dd8 100644 --- a/cmd/bootstrap/utils/key_generation.go +++ b/cmd/bootstrap/utils/key_generation.go @@ -10,6 +10,12 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// these constants are defined in X9.62 section 4.2 and 4.3 +// see https://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.202.2977&rep=rep1&type=pdf +// they indicate if the conversion to/from a public key (point) in compressed form must involve an inversion of the ordinate coordinate +const X962_NO_INVERSION = uint8(0x02) +const X962_INVERSION = uint8(0x03) + func GenerateMachineAccountKey(seed []byte) (crypto.PrivateKey, error) { keys, err := GenerateKeys(crypto.ECDSAP256, 1, [][]byte{seed}) if err != nil { @@ -18,6 +24,63 @@ func GenerateMachineAccountKey(seed []byte) (crypto.PrivateKey, error) { return keys[0], nil } +// The unstaked nodes have special networking keys, in two aspects: +// - they use crypto.ECDSASecp256k1 keys, not crypto.ECDSAP256 keys, +// - they use only positive keys (in the sense that the elliptic curve point of their public key is positive) +// +// Thanks to various properties of the cryptographic algorithm and libp2p, +// this affords us to not have to maintain a table of flow.NodeID -> NetworkPublicKey +// for those numerous and ephemeral nodes. +// It incurs a one-bit security reduction, which is deemed acceptable. + +// drawUnstakedKey draws a single positive ECDSASecp256k1 key, and returns an error otherwise. +func drawUnstakedKey(seed []byte) (crypto.PrivateKey, error) { + key, err := crypto.GeneratePrivateKey(crypto.ECDSASecp256k1, seed) + if err != nil { + // this should not happen + return nil, err + } else if key.PublicKey().EncodeCompressed()[0] == X962_INVERSION { + // negative key -> unsuitable + return nil, fmt.Errorf("Unsuitable negative key") + } + return key, nil +} + +// GenerateUnstakedNetworkingKey draws ECDSASecp256k1 keys until finding a suitable one. +// though this will return fast, this is not constant-time and will leak ~1 bit of information through its runtime +func GenerateUnstakedNetworkingKey(seed []byte) (key crypto.PrivateKey, err error) { + hkdf := hkdf.New(func() gohash.Hash { return sha256.New() }, seed, nil, []byte("unstaked network")) + round_seed := make([]byte, len(seed)) + max_iterations := 20 // 1/(2^20) failure chance + for i := 0; i < max_iterations; i++ { + if _, err = io.ReadFull(hkdf, round_seed); err != nil { + // the hkdf Reader should not fail + panic(err) + } + if key, err = drawUnstakedKey(round_seed); err == nil { + return + } + } + return +} + +func GenerateUnstakedNetworkingKeys(n int, seeds [][]byte) ([]crypto.PrivateKey, error) { + if n != len(seeds) { + return nil, fmt.Errorf("n needs to match the number of seeds (%v != %v)", n, len(seeds)) + } + + keys := make([]crypto.PrivateKey, n) + + var err error + for i, seed := range seeds { + if keys[i], err = GenerateUnstakedNetworkingKey(seed); err != nil { + return nil, err + } + } + + return keys, nil +} + func GenerateNetworkingKey(seed []byte) (crypto.PrivateKey, error) { keys, err := GenerateKeys(crypto.ECDSAP256, 1, [][]byte{seed}) if err != nil { diff --git a/cmd/bootstrap/utils/key_generation_test.go b/cmd/bootstrap/utils/key_generation_test.go index b77b4da39e9..f6f8fb2a937 100644 --- a/cmd/bootstrap/utils/key_generation_test.go +++ b/cmd/bootstrap/utils/key_generation_test.go @@ -14,6 +14,22 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +func TestGenerateUnstakedNetworkingKey(t *testing.T) { + + key, err := GenerateUnstakedNetworkingKey(unittest.SeedFixture(crypto.KeyGenSeedMinLenECDSASecp256k1)) + require.NoError(t, err) + assert.Equal(t, crypto.ECDSASecp256k1, key.Algorithm()) + assert.Equal(t, X962_NO_INVERSION, key.PublicKey().EncodeCompressed()[0]) + + keys, err := GenerateUnstakedNetworkingKeys(20, unittest.SeedFixtures(20, crypto.KeyGenSeedMinLenECDSASecp256k1)) + require.NoError(t, err) + for _, key := range keys { + assert.Equal(t, crypto.ECDSASecp256k1, key.Algorithm()) + assert.Equal(t, X962_NO_INVERSION, key.PublicKey().EncodeCompressed()[0]) + } + +} + func TestGenerateKeys(t *testing.T) { _, err := GenerateKeys(crypto.BLSBLS12381, 0, unittest.SeedFixtures(2, crypto.KeyGenSeedMinLenBLSBLS12381)) require.EqualError(t, err, "n needs to match the number of seeds (0 != 2)") From 56bf8db5523fd0a8904ad96aa4b92f09e384771c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Fri, 20 Aug 2021 19:30:06 -0400 Subject: [PATCH 095/291] [bootstrap] Key Generation for Unstaked Access Nodes This introduces a generator for the unstaked Access Nodes, which by convention only have positive secp256k1 keys. --- cmd/bootstrap/utils/key_generation.go | 7 ++++++- cmd/bootstrap/utils/key_generation_test.go | 4 ++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/cmd/bootstrap/utils/key_generation.go b/cmd/bootstrap/utils/key_generation.go index 7c157727dd8..b14e693be24 100644 --- a/cmd/bootstrap/utils/key_generation.go +++ b/cmd/bootstrap/utils/key_generation.go @@ -1,7 +1,12 @@ package utils import ( + "crypto/sha256" "fmt" + gohash "hash" + "io" + + "golang.org/x/crypto/hkdf" sdkcrypto "github.com/onflow/flow-go-sdk/crypto" @@ -39,7 +44,7 @@ func drawUnstakedKey(seed []byte) (crypto.PrivateKey, error) { if err != nil { // this should not happen return nil, err - } else if key.PublicKey().EncodeCompressed()[0] == X962_INVERSION { + } else if key.PublicKey().EncodeCompressed()[0] == 0x03 { // negative key -> unsuitable return nil, fmt.Errorf("Unsuitable negative key") } diff --git a/cmd/bootstrap/utils/key_generation_test.go b/cmd/bootstrap/utils/key_generation_test.go index f6f8fb2a937..2c64a27cf02 100644 --- a/cmd/bootstrap/utils/key_generation_test.go +++ b/cmd/bootstrap/utils/key_generation_test.go @@ -19,13 +19,13 @@ func TestGenerateUnstakedNetworkingKey(t *testing.T) { key, err := GenerateUnstakedNetworkingKey(unittest.SeedFixture(crypto.KeyGenSeedMinLenECDSASecp256k1)) require.NoError(t, err) assert.Equal(t, crypto.ECDSASecp256k1, key.Algorithm()) - assert.Equal(t, X962_NO_INVERSION, key.PublicKey().EncodeCompressed()[0]) + assert.Equal(t, uint8(0x02), key.PublicKey().EncodeCompressed()[0]) keys, err := GenerateUnstakedNetworkingKeys(20, unittest.SeedFixtures(20, crypto.KeyGenSeedMinLenECDSASecp256k1)) require.NoError(t, err) for _, key := range keys { assert.Equal(t, crypto.ECDSASecp256k1, key.Algorithm()) - assert.Equal(t, X962_NO_INVERSION, key.PublicKey().EncodeCompressed()[0]) + assert.Equal(t, uint8(0x02), key.PublicKey().EncodeCompressed()[0]) } } From bef943c6e3bed7ab7c5b614a222da6dd33d941af Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 11:02:50 -0700 Subject: [PATCH 096/291] using the unstaked networking key for the consesus follower --- .../unstaked_access_node_builder.go | 6 ------ .../tests/access/unstaked_node_test.go | 19 ++++++++++++++++--- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 6d25c040015..f923db4461d 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -70,12 +70,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() - //builder.EnqueueMetricsServerInit() - // - //builder.RegisterBadgerMetrics() - // - //builder.EnqueueTracer() - builder.PreInit(builder.initUnstakedLocal()) return builder diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index daddce23936..6bdac10eb30 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -2,6 +2,7 @@ package access import ( "context" + "crypto/rand" "fmt" "testing" "time" @@ -10,6 +11,8 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" @@ -69,7 +72,7 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { flow.RoleAccess, testnet.WithID(suite.stakedID), testnet.SupportsUnstakedNodes(), - testnet.WithLogLevel(zerolog.InfoLevel), + testnet.WithLogLevel(zerolog.TraceLevel), ) collectionConfigs := []func(*testnet.NodeConfig){ @@ -100,8 +103,7 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { stakedConfig, } - // consensus follower - unstakedKey, err := unittest.NetworkingKey() + unstakedKey, err := UnstakedNetworkingKey() require.NoError(suite.T(), err) // TODO: derive node id from the key suite.unstakedID = unittest.IdentifierFixture() @@ -110,9 +112,20 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } + // consensus follower conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) suite.follower.AddOnBlockFinalizedConsumer(suite.OnBlockFinalizedConsumer) } + +// TODO: Move this to unittest and resolve the circular dependency issue +func UnstakedNetworkingKey() (crypto.PrivateKey, error) { + seed := make([]byte, crypto.KeyGenSeedMinLenECDSASecp256k1) + n, err := rand.Read(seed) + if err != nil || n != crypto.KeyGenSeedMinLenECDSASecp256k1 { + return nil, err + } + return utils.GenerateUnstakedNetworkingKey(unittest.SeedFixture(n)) +} From ae3f74369b0e06c2651d8d49b9cb7cbd047a696a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:06:10 -0700 Subject: [PATCH 097/291] chance consensus follower to use build function --- follower/consensus_follower.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 33a52064bf6..f35ab03689e 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -96,7 +96,7 @@ func buildAccessNode(accessNodeOptions []access.Option) *access.UnstakedAccessNo nodeBuilder := access.NewUnstakedAccessNodeBuilder(anb) nodeBuilder.Initialize() - nodeBuilder.BuildConsensusFollower() + nodeBuilder.Build() return nodeBuilder } From 136066a8093631734743faafceb7eb950dbbf580 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:27:30 -0700 Subject: [PATCH 098/291] derive node id from key --- integration/testnet/network.go | 19 ++++++++++++------- .../tests/access/unstaked_node_test.go | 4 +--- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 9b14f56bdc4..2a4c4f34f92 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -37,6 +37,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" + "github.com/onflow/flow-go/network/p2p" clusterstate "github.com/onflow/flow-go/state/cluster" "github.com/onflow/flow-go/state/protocol/inmem" "github.com/onflow/flow-go/utils/io" @@ -219,16 +220,20 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier + nodeID flow.Identifier networkingPrivKey crypto.PrivateKey - stakedNodeID flow.Identifier + stakedNodeID flow.Identifier } -func NewConsensusFollowerConfig(networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier, nodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier) ConsensusFollowerConfig { + pid, err := p2p.ExtractPeerID(networkingPrivKey.PublicKey()) + assert.NoError(t, err) + nodeID, err := p2p.NewUnstakedNetworkIDTranslator().GetFlowID(pid) + assert.NoError(t, err) return ConsensusFollowerConfig{ networkingPrivKey: networkingPrivKey, - stakedNodeID: stakedNodeID, - nodeID: nodeID, // TODO: remove this and derive it from the key instead + stakedNodeID: stakedNodeID, + nodeID: nodeID, } } @@ -515,8 +520,8 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, port := uint(portU64) bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ - Host: "localhost", - Port: port, + Host: "localhost", + Port: port, NetworkPublicKey: stakedANContainer.NetworkPubKey(), } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 6bdac10eb30..c4ff35d242b 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -105,11 +105,9 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { unstakedKey, err := UnstakedNetworkingKey() require.NoError(suite.T(), err) - // TODO: derive node id from the key - suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), + testnet.NewConsensusFollowerConfig(suite.T(), unstakedKey, suite.stakedID), } // consensus follower From 05fe476fd008c013ff45e41f353ee814de3691ba Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:30:05 -0700 Subject: [PATCH 099/291] remove comment --- integration/testnet/network.go | 1 - 1 file changed, 1 deletion(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 2a4c4f34f92..a27410ddef7 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -741,7 +741,6 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( - // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address From 46b0d35de1028d70dd3898909ecfa9b6e76296e2 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:42:09 -0700 Subject: [PATCH 100/291] remove topology --- .../node_builder/staked_access_node_builder.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 1d001cd5bac..cb7e42ba7a8 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -118,17 +118,9 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront - top, err := topology.NewTopicBasedTopology( - builder.NodeID, - builder.Logger, - builder.State, - ) - if err != nil { - return nil, fmt.Errorf("could not create topology: %w", err) - } - topologyCache := topology.NewCache(builder.Logger, top) + top := topology.EmptyListTopology{} - network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, topologyCache) + network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, top) builder.MustNot(err) builder.Network = network From 6cdbab36d0f98a76f5ecf3ba4eddb5c168044d16 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:47:51 -0700 Subject: [PATCH 101/291] fix node ID --- integration/testnet/network.go | 24 +++++++++---------- .../tests/access/unstaked_node_test.go | 2 ++ 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index a27410ddef7..26489f23982 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -220,9 +220,9 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier - networkingPrivKey crypto.PrivateKey - stakedNodeID flow.Identifier + NodeID flow.Identifier + NetworkingPrivKey crypto.PrivateKey + StakedNodeID flow.Identifier } func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier) ConsensusFollowerConfig { @@ -231,9 +231,9 @@ func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKe nodeID, err := p2p.NewUnstakedNetworkIDTranslator().GetFlowID(pid) assert.NoError(t, err) return ConsensusFollowerConfig{ - networkingPrivKey: networkingPrivKey, - stakedNodeID: stakedNodeID, - nodeID: nodeID, + NetworkingPrivKey: networkingPrivKey, + StakedNodeID: stakedNodeID, + NodeID: nodeID, } } @@ -507,12 +507,12 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, var stakedANContainer *ContainerConfig // find the upstream Access node container for this follower engine for _, cont := range containers { - if cont.NodeID == followerConf.stakedNodeID { + if cont.NodeID == followerConf.StakedNodeID { stakedANContainer = &cont break } } - require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) + require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.NodeID.String()) portStr := net.AccessPorts[AccessNodeExternalNetworkPort] portU64, err := strconv.ParseUint(portStr, 10, 32) @@ -527,11 +527,11 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower, err := consensus_follower.NewConsensusFollower(followerConf.networkingPrivKey, bindAddr, + follower, err := consensus_follower.NewConsensusFollower(followerConf.NetworkingPrivKey, bindAddr, []consensus_follower.BootstrapNodeInfo{bootstrapNodeInfo}, opts...) // TODO: convert key to node ID? or just store with the network key as map key - net.ConsensusFollowers[followerConf.nodeID] = follower + net.ConsensusFollowers[followerConf.NodeID] = follower } // AddNode creates a node container with the given config and adds it to the @@ -741,11 +741,11 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( - conf.nodeID, + conf.NodeID, flow.RoleAccess, // use Access role "", // no address 0, // no stake - conf.networkingPrivKey, + conf.NetworkingPrivKey, dummyStakingKey, ) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index c4ff35d242b..b48db55734f 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -110,6 +110,8 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.NewConsensusFollowerConfig(suite.T(), unstakedKey, suite.stakedID), } + suite.unstakedID = followerConfigs[0].NodeID + // consensus follower conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) From 44f8ae943d2650f2f9b928bf7258e8a8cb639594 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 11:53:34 -0700 Subject: [PATCH 102/291] removing unneeded flags --- integration/tests/access/unstaked_node_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index c4ff35d242b..2b305eed06e 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -79,15 +79,13 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), testnet.WithAdditionalFlag("--block-rate-delay=100ms"), testnet.WithLogLevel(zerolog.WarnLevel), - // TODO replace these with actual values - testnet.WithAdditionalFlag("--access-address=null"), } consensusConfigs := []func(config *testnet.NodeConfig){ testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), testnet.WithAdditionalFlag("--block-rate-delay=100ms"), - testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 1)), - testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 1)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 0)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 0)), testnet.WithLogLevel(zerolog.WarnLevel), } From 84bd40661297e5b1466c4631ea0b9a810281c966 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 13:09:35 -0700 Subject: [PATCH 103/291] initializing middleware before initiliazing the sync follower for the unstaked node --- .../unstaked_access_node_builder.go | 134 ++++++++++-------- 1 file changed, 73 insertions(+), 61 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index f923db4461d..6d93741a152 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -5,6 +5,7 @@ import ( "errors" "github.com/libp2p/go-libp2p-core/peer" + "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -24,67 +25,69 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } -func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { +func (anb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config - networkingKey := fnb.AccessNodeConfig.NetworkKey + networkingKey := anb.AccessNodeConfig.NetworkKey pubKey, err := p2p.LibP2PPublicKeyFromFlow(networkingKey.PublicKey()) - fnb.MustNot(err) + anb.MustNot(err) peerID, err := peer.IDFromPublicKey(pubKey) - fnb.MustNot(err) - fnb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) - fnb.MustNot(err) - fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig - fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node + anb.MustNot(err) + anb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) + anb.MustNot(err) + anb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig + anb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } -func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) +func (anb *UnstakedAccessNodeBuilder) InitIDProviders() { + anb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, anb.ProtocolEvents) if err != nil { return err } - fnb.IdentityProvider = idCache + anb.IdentityProvider = idCache - fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) + anb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) return nil }) } -func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { +func (anb *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) - builder.Cancel = cancel + anb.Cancel = cancel - builder.validateParams() + anb.validateParams() // if a network key has been passed in the init node info here - if builder.AccessNodeConfig.NetworkKey != nil { - builder.initNodeInfo() + if anb.AccessNodeConfig.NetworkKey != nil { + anb.initNodeInfo() } - builder.InitIDProviders() + anb.InitIDProviders() + + anb.enqueueMiddleware(ctx) - builder.enqueueUnstakedNetworkInit(ctx) + anb.enqueueUnstakedNetworkInit(ctx) - builder.enqueueConnectWithStakedAN() + anb.enqueueConnectWithStakedAN() - builder.PreInit(builder.initUnstakedLocal()) + anb.PreInit(anb.initUnstakedLocal()) - return builder + return anb } -func (builder *UnstakedAccessNodeBuilder) validateParams() { - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { - builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") +func (anb *UnstakedAccessNodeBuilder) validateParams() { + if len(anb.bootstrapNodeAddresses) != len(anb.bootstrapNodePublicKeys) { + anb.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") } } // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files -func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.NodeBuilder, node *cmd.NodeConfig) { +func (anb *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.NodeBuilder, node *cmd.NodeConfig) { return func(_ cmd.NodeBuilder, node *cmd.NodeConfig) { // for an unstaked node, set the identity here explicitly since it will not be found in the protocol state self := &flow.Identity{ @@ -92,21 +95,48 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N NetworkPubKey: node.NetworkKey.PublicKey(), StakingPubKey: nil, // no staking key needed for the unstaked node Role: flow.RoleAccess, // unstaked node can only run as an access node - Address: builder.BindAddr, + Address: anb.BindAddr, } me, err := local.New(self, nil) - builder.MustNot(err).Msg("could not initialize local") + anb.MustNot(err).Msg("could not initialize local") node.Me = me } } +// enqueueMiddleware enqueues the creation of the network middleware +// this needs to be done before sync engine participants module +func (anb *UnstakedAccessNodeBuilder) enqueueMiddleware(ctx context.Context) { + anb. + Module("network middleware", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) error { + + // NodeID for the unstaked node on the unstaked network + unstakedNodeID := node.NodeID + + // Networking key + unstakedNetworkKey := node.NetworkKey + + // Network Metrics + // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics + unstakedNetworkMetrics := metrics.NewNoopCollector() + + libP2PFactory, err := anb.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + anb.MustNot(err) + + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) + + anb.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + + return nil + }) +} + // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. - Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - middleware, ok := node.Middleware.(*p2p.Middleware) + Module("sync engine participants provider", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) error { + middleware, ok := anb.Middleware.(*p2p.Middleware) if !ok { return errors.New("middleware was of unexpected type") } @@ -119,44 +149,26 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { } // enqueueUnstakedNetworkInit enqueues the unstaked network component initialized for the unstaked node -func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { - - builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { +func (anb *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { - // NodeID for the unstaked node on the unstaked network - unstakedNodeID := node.NodeID - - // Networking key - unstakedNetworkKey := node.NetworkKey + anb.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { // Network Metrics // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) - builder.MustNot(err) - - msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) - - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) - builder.MustNot(err) - - builder.Network = network - builder.Middleware = middleware + network, err := anb.initNetwork(anb.Me, unstakedNetworkMetrics, anb.Middleware, nil) + anb.MustNot(err) - // for an unstaked node, the staked network and middleware is set to the same as the unstaked network and middlware - builder.Network = network - builder.Middleware = middleware + anb.Network = network - builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + anb.Logger.Info().Msgf("network will run on address: %s", anb.BindAddr) - idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) - builder.ProtocolEvents.AddConsumer(idEvents) + idEvents := gadgets.NewIdentityDeltas(anb.Middleware.UpdateNodeAddresses) + anb.ProtocolEvents.AddConsumer(idEvents) - return builder.Network, err + return anb.Network, err }) } @@ -166,8 +178,8 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // (https://github.com/libp2p/go-libp2p-pubsub/issues/442). This means that an unstaked AN could end up not being // discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need // of an explicit connect to the staked AN before the node attempts to subscribe to topics. -func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentities, builder.LibP2PNode, builder.Logger), nil +func (anb *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { + anb.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return newUpstreamConnector(anb.bootstrapIdentities, anb.LibP2PNode, anb.Logger), nil }) } From f8343bd27d91ee393e4270047f8a083b256fa766 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 00:07:25 -0700 Subject: [PATCH 104/291] initial --- integration/go.sum | 2 - module/id/id_provider.go | 148 ++++++++++++++++++++++++++++++++++++ network/p2p/idTranslator.go | 33 ++++++++ network/p2p/libp2pNode.go | 10 ++- 4 files changed, 189 insertions(+), 4 deletions(-) create mode 100644 module/id/id_provider.go create mode 100644 network/p2p/idTranslator.go diff --git a/integration/go.sum b/integration/go.sum index db321600a5e..51afd2feff8 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -792,13 +792,11 @@ github.com/libp2p/go-libp2p-peerstore v0.2.0/go.mod h1:N2l3eVIeAitSg3Pi2ipSrJYnq github.com/libp2p/go-libp2p-peerstore v0.2.1/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= -github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.7/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.8 h1:nJghUlUkFVvyk7ccsM67oFA6kqUkwyCM1G4WPVMCWYA= github.com/libp2p/go-libp2p-peerstore v0.2.8/go.mod h1:gGiPlXdz7mIHd2vfAsHzBNAMqSDkt2UBFwgcITgw1lA= github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6na5f0/k= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= -github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= github.com/libp2p/go-libp2p-pubsub v0.4.1 h1:j4umIg5nyus+sqNfU+FWvb9aeYFQH/A+nDFhWj+8yy8= github.com/libp2p/go-libp2p-pubsub v0.4.1/go.mod h1:izkeMLvz6Ht8yAISXjx60XUQZMq9ZMe5h2ih4dLIBIQ= github.com/libp2p/go-libp2p-quic-transport v0.10.0/go.mod h1:RfJbZ8IqXIhxBRm5hqUEJqjiiY8xmEuq3HUDS993MkA= diff --git a/module/id/id_provider.go b/module/id/id_provider.go new file mode 100644 index 00000000000..430780b6dc8 --- /dev/null +++ b/module/id/id_provider.go @@ -0,0 +1,148 @@ +package identity + +import ( + "sync" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" +) + +type IdentifierProvider interface { + Identifiers() flow.IdentifierList +} + +// TODO: rename to ProtocolStateIDProvider +type ProtocolStateIdentifierProvider struct { + events.Noop + identities flow.IdentityList // TODO: actually we *can* just use identifiers after all + state protocol.State + mu sync.RWMutex + filter flow.IdentityFilter + peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + flowIDs map[peer.ID]flow.Identifier +} + +func WithFilter(filter flow.IdentityFilter) ProtocolStateIdentifierProviderOption { + return func(provider *ProtocolStateIdentifierProvider) { + provider.filter = filter + } +} + +type ProtocolStateIdentifierProviderOption func(*ProtocolStateIdentifierProvider) + +// TODO: this one also implements IDTranslator!!! +func NewProtocolStateIdentifierProvider( + state protocol.State, + eventDistributer *events.Distributor, + opts ...ProtocolStateIdentifierProviderOption, +) (*ProtocolStateIdentifierProvider, error) { + provider := &ProtocolStateIdentifierProvider{ + state: state, + filter: filter.Any, + } + + for _, opt := range opts { + opt(provider) + } + + head, err := state.Final().Head() + if err != nil { + return nil, err // TODO: format the error + } + + provider.update(head.ID()) + eventDistributer.AddConsumer(provider) + + return provider, nil +} + +func (p *ProtocolStateIdentifierProvider) EpochTransition(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) update(blockID flow.Identifier) { + identities, err := p.state.AtBlockID(blockID).Identities(p.filter) + if err != nil { + // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + } + + nIds := identities.Count() + + peerIDs := make(map[flow.Identifier]peer.ID, nIds) + flowIDs := make(map[peer.ID]flow.Identifier, nIds) + + for _, identity := range identities { + pid, err := p2p.IdentityToPeerID(identity) + if err != nil { + // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + } + + flowIDs[pid] = identity.NodeID + peerIDs[identity.NodeID] = pid + } + + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities + p.flowIDs = flowIDs + p.peerIDs = peerIDs +} + +func (p *ProtocolStateIdentifierProvider) Identifiers() flow.IdentifierList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.NodeIDs() +} + +func (p *ProtocolStateIdentifierProvider) GetPeerID(flowID flow.Identifier) (pid peer.ID, found bool) { + p.mu.RLock() + defer p.mu.RUnlock() + + pid, found = p.peerIDs[flowID] + return +} + +func (p *ProtocolStateIdentifierProvider) GetFlowID(peerID peer.ID) (fid flow.Identifier, found bool) { + p.mu.RLock() + defer p.mu.RUnlock() + + fid, found = p.flowIDs[peerID] + return +} + +type PeerstoreIdentifierProvider struct { + store peerstore.Peerstore + idTranslator p2p.IDTranslator +} + +func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator p2p.IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} +} + +func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { + var result flow.IdentifierList + + pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? + for _, pid := range pids { + flowID, success := p.idTranslator.GetFlowID(pid) + if success { + result = append(result, flowID) + } + } + + return result +} diff --git a/network/p2p/idTranslator.go b/network/p2p/idTranslator.go new file mode 100644 index 00000000000..f9f567f1fec --- /dev/null +++ b/network/p2p/idTranslator.go @@ -0,0 +1,33 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type IDTranslator interface { + GetPeerID(flow.Identifier) (peer.ID, bool) + GetFlowID(peer.ID) (flow.Identifier, bool) +} + +type UnstakedNetworkPeerIDProvider struct { + // basically, just do the conversion from flow ID to peer ID using whatever scheme we talked about. + // whether this be, with caching or not +} + +func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { + pk, err := PublicKey(id.NetworkPubKey) + if err != nil { + // TODO: format the error + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + // TODO: format the error + return + } + + return +} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 164c3fd18e8..b1a127f2d5e 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -319,16 +319,22 @@ func (n *Node) RemovePeer(ctx context.Context, identity flow.Identity) error { } // CreateStream returns an existing stream connected to identity, if it exists or adds one to identity as a peer and creates a new stream with it. -func (n *Node) CreateStream(ctx context.Context, identity flow.Identity) (libp2pnet.Stream, error) { +func (n *Node) CreateStream(ctx context.Context, nodeID flow.Identifier) (libp2pnet.Stream, error) { // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) - stream, err := n.tryCreateNewStream(ctx, identity, maxConnectAttempt) + stream, err := n.tryCreateNewStream(ctx, nodeID, maxConnectAttempt) if err != nil { + n.host.Peerstore().Addrs() return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (node_id: %s, address: %s): %w", identity.NodeID.String(), identity.Address, err)) } return stream, nil } +func (n *Node) GetPeerIPPort(nodeID flow.Identifier) (string, string, error) { + // TODO: first get peer ID + x := n.host.Peerstore().Addrs() +} + // tryCreateNewStream makes at most maxAttempts to create a stream with the identity. // This was put in as a fix for #2416. PubSub and 1-1 communication compete with each other when trying to connect to // remote nodes and once in a while NewStream returns an error 'both yamux endpoints are clients' From a7ecbceb839de8b40755b3d8e41c80d7c632ae8d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 14:04:49 -0700 Subject: [PATCH 105/291] Update idTranslator.go --- .../node_builder/access_node_builder.go | 182 +++++++----------- .../staked_access_node_builder.go | 36 +++- .../unstaked_access_node_builder.go | 87 +++++---- cmd/collection/main.go | 9 +- cmd/consensus/main.go | 9 +- cmd/execution/main.go | 9 +- cmd/node_builder.go | 9 + cmd/scaffold.go | 38 +++- cmd/verification/main.go | 10 +- engine/common/synchronization/engine.go | 64 +++--- go.sum | 3 - model/flow/identifierList.go | 27 +++ module/id/filtered_provider.go | 18 ++ module/id/id_provider.go | 141 +------------- network/middleware.go | 11 +- network/p2p/connGater.go | 8 +- network/p2p/idTranslator.go | 33 ---- network/p2p/id_translator.go | 12 ++ network/p2p/libp2pConnector.go | 35 ++-- network/p2p/libp2pNode.go | 159 +++++---------- network/p2p/middleware.go | 178 +++++++---------- network/p2p/network.go | 98 +++++----- network/p2p/nodeIDRefresher.go | 82 -------- network/p2p/peerManager.go | 42 ++-- network/p2p/peerstore_provider.go | 32 +++ network/p2p/protocol_state_provider.go | 130 +++++++++++++ network/p2p/unstaked_translator.go | 43 +++++ network/topology.go | 8 +- network/topology/cache.go | 16 +- network/topology/fixedListTopology.go | 10 +- network/topology/randomizedTopology.go | 71 ++++--- network/topology/topicBasedTopology.go | 85 ++++---- network/topology/topology_utils.go | 4 +- utils/grpc/grpc.go | 9 +- 34 files changed, 832 insertions(+), 876 deletions(-) create mode 100644 module/id/filtered_provider.go delete mode 100644 network/p2p/idTranslator.go create mode 100644 network/p2p/id_translator.go delete mode 100644 network/p2p/nodeIDRefresher.go create mode 100644 network/p2p/peerstore_provider.go create mode 100644 network/p2p/protocol_state_provider.go create mode 100644 network/p2p/unstaked_translator.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 17791fff398..403cf14baea 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2,18 +2,17 @@ package node_builder import ( "context" - "encoding/json" "fmt" "strings" "time" - dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow/protobuf/go/flow/access" "github.com/spf13/pflag" "google.golang.org/grpc" "google.golang.org/grpc/credentials" "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/cmd/build" "github.com/onflow/flow-go/consensus" "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/committees" @@ -36,6 +35,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" @@ -43,7 +43,6 @@ import ( "github.com/onflow/flow-go/network" jsoncodec "github.com/onflow/flow-go/network/codec/json" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/validator" "github.com/onflow/flow-go/state/protocol" badgerState "github.com/onflow/flow-go/state/protocol/badger" "github.com/onflow/flow-go/state/protocol/blocktimer" @@ -86,9 +85,7 @@ type AccessNodeBuilder interface { // while for a node running as a library, the config fields are expected to be initialized by the caller. type AccessNodeConfig struct { staked bool - bootstrapNodeAddresses []string - bootstrapNodePublicKeys []string - bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes + stakedAccessNodeIDHex string unstakedNetworkBindAddr string collectionGRPCPort uint executionGRPCPort uint @@ -135,8 +132,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { apiRatelimits: nil, apiBurstlimits: nil, staked: true, - bootstrapNodeAddresses: []string{}, - bootstrapNodePublicKeys: []string{}, + stakedAccessNodeIDHex: "", unstakedNetworkBindAddr: cmd.NotSet, } } @@ -149,25 +145,25 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedLibP2PNode *p2p.Node - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + UnstakedNetwork *p2p.Network + unstakedMiddleware *p2p.Middleware + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower + SyncEngineParticipantsProvider id.IdentifierProvider // engines IngestEng *ingestion.Engine @@ -320,7 +316,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - node.State, + builder.SyncEngineParticipantsProvider, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) @@ -490,9 +486,9 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { type Option func(*AccessNodeConfig) -func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { +func WithUpstreamAccessNodeID(upstreamAccessNodeID flow.Identifier) Option { return func(config *AccessNodeConfig) { - config.bootstrapIdentites = bootstrapNodes + config.stakedAccessNodeIDHex = upstreamAccessNodeID.String() } } @@ -570,42 +566,50 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.StringToIntVar(&builder.apiRatelimits, "api-rate-limits", defaultConfig.apiRatelimits, "per second rate limits for Access API methods e.g. Ping=300,GetTransaction=500 etc.") flags.StringToIntVar(&builder.apiBurstlimits, "api-burst-limits", defaultConfig.apiBurstlimits, "burst limits for Access API methods e.g. Ping=100,GetTransaction=100 etc.") flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") - flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") - flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") + flags.StringVar(&builder.stakedAccessNodeIDHex, "staked-access-node-id", defaultConfig.stakedAccessNodeIDHex, "the node ID of the upstream staked access node if this is an unstaked access node") flags.StringVar(&builder.unstakedNetworkBindAddr, "unstaked-bind-addr", defaultConfig.unstakedNetworkBindAddr, "address to bind on for the unstaked network") }) } // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. // The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance -func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, +func (builder *FlowAccessNodeBuilder) initLibP2PFactory( + ctx context.Context, nodeID flow.Identifier, - networkKey crypto.PrivateKey) (p2p.LibP2PFactoryFunc, error) { + networkMetrics module.NetworkMetrics, + networkKey crypto.PrivateKey, +) (p2p.LibP2PFactoryFunc, error) { - // The staked nodes act as the DHT servers - dhtOptions := []dht.Option{p2p.AsServer(builder.IsStaked())} + // setup the Ping provider to return the software version and the sealed block height + pingProvider := p2p.PingInfoProviderImpl{ + SoftwareVersionFun: func() string { + return build.Semver() + }, + SealedBlockHeightFun: func() (uint64, error) { + head, err := builder.State.Sealed().Head() + if err != nil { + return 0, err + } + return head.Height, nil + }, + } - // if this is an unstaked access node, then seed the DHT with the boostrap identities - if !builder.IsStaked() { - bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentites) - builder.MustNot(err) - dhtOptions = append(dhtOptions, bootstrapPeersOpt) + libP2PNodeFactory, err := p2p.DefaultLibP2PNodeFactory( + ctx, + builder.Logger, + nodeID, + builder.unstakedNetworkBindAddr, + networkKey, + builder.RootBlock.ID().String(), + p2p.DefaultMaxPubSubMsgSize, + networkMetrics, + pingProvider, + ) + if err != nil { + return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) } - return func() (*p2p.Node, error) { - libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.unstakedNetworkBindAddr, networkKey). - SetRootBlockID(builder.RootBlock.ID().String()). - // unlike the staked network where currently all the node addresses are known upfront, - // for the unstaked network the nodes need to discover each other using DHT Discovery. - SetPubsubOptions(p2p.WithDHTDiscovery(dhtOptions...)). - SetLogger(builder.Logger). - Build(ctx) - if err != nil { - return nil, err - } - builder.UnstakedLibP2PNode = libp2pNode - return builder.UnstakedLibP2PNode, nil - }, nil + return libP2PNodeFactory, nil } // initMiddleware creates the network.Middleware implementation with the libp2p factory function, metrics, peer update @@ -613,16 +617,21 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, + peerUpdateInterval time.Duration, + unicastMessageTimeout time.Duration, + connectionGating bool, + managerPeerConnections bool, validators ...network.MessageValidator) *p2p.Middleware { builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, factoryFunc, nodeID, networkMetrics, builder.RootBlock.ID().String(), - time.Hour, // TODO: this is pretty meaningless since there is no peermanager in play. - p2p.DefaultUnicastTimeout, - false, // no connection gating for the unstaked network - false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) + peerUpdateInterval, + unicastMessageTimeout, + connectionGating, + managerPeerConnections, + builder.IDTranslator, validators...) return builder.unstakedMiddleware } @@ -633,7 +642,6 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, middleware *p2p.Middleware, - participants flow.IdentityList, topology network.Topology) (*p2p.Network, error) { codec := jsoncodec.NewCodec() @@ -641,64 +649,20 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, subscriptionManager := p2p.NewChannelSubscriptionManager(middleware) // creates network instance - net, err := p2p.NewNetwork(builder.Logger, + net, err := p2p.NewNetwork( + builder.Logger, codec, - participants, nodeID, builder.unstakedMiddleware, p2p.DefaultCacheSize, topology, subscriptionManager, - networkMetrics) + networkMetrics, + p2p.WithIdentifierProvider(builder.NetworkingIdentifierProvider), + ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) } return net, nil } - -func unstakedNetworkMsgValidators(selfID flow.Identifier) []network.MessageValidator { - return []network.MessageValidator{ - // filter out messages sent by this node itself - validator.ValidateNotSender(selfID), - } -} - -// BootstrapIdentities converts the bootstrap node addresses and keys to a Flow Identity list where -// each Flow Identity is initialized with the passed address, the networking key -// and the Node ID set to ZeroID, role set to Access, 0 stake and no staking key. -func BootstrapIdentities(addresses []string, keys []string) (flow.IdentityList, error) { - - if len(addresses) != len(keys) { - return nil, fmt.Errorf("number of addresses and keys provided for the boostrap nodes don't match") - } - - ids := make([]*flow.Identity, len(addresses)) - for i, address := range addresses { - - key := keys[i] - // json unmarshaller needs a quotes before and after the string - // the pflags.StringSliceVar does not retain quotes for the command line arg even if escaped with \" - // hence this additional check to ensure the key is indeed quoted - if !strings.HasPrefix(key, "\"") { - key = fmt.Sprintf("\"%s\"", key) - } - // networking public key - var networkKey encodable.NetworkPubKey - err := json.Unmarshal([]byte(key), &networkKey) - if err != nil { - return nil, err - } - - // create the identity of the peer by setting only the relevant fields - id := &flow.Identity{ - NodeID: flow.ZeroID, // the NodeID is the hash of the staking key and for the unstaked network it does not apply - Address: address, - Role: flow.RoleAccess, // the upstream node has to be an access node - NetworkPubKey: networkKey, - } - - ids = append(ids, id) - } - return ids, nil -} diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 451aa2f83ad..d022c886700 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -7,7 +7,9 @@ import ( "github.com/onflow/flow-go/cmd" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/topology" ) @@ -24,11 +26,39 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui } } +func (fnb *StakedAccessNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + // translator + // networking provider + fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ) + + // TODO: need special providers here + // for network, needs one that recognizes both protocl state and peerstore + // same for translator + + return nil + }) +} + func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) builder.Cancel = cancel + builder.InitIDProviders() + // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node builder.EnqueueNetworkInit(ctx) @@ -93,14 +123,10 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - // TODO: this list should be the unstaked addresses of all the staked AN that participate in the unstaked network - participants := flow.IdentityList{} - // topology returns empty list since peers are not known upfront top := topology.EmptyListTopology{} - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, top) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, top) builder.MustNot(err) builder.UnstakedNetwork = network diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 2b914d65db4..81c9f4600f7 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -2,12 +2,16 @@ package node_builder import ( "context" + "strings" + "time" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network/topology" ) type UnstakedAccessNodeBuilder struct { @@ -20,6 +24,13 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } +func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + return nil + }) +} + func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) @@ -27,12 +38,10 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() - builder.deriveBootstrapPeerIdentities() + builder.InitIDProviders() builder.enqueueUnstakedNetworkInit(ctx) - builder.enqueueConnectWithStakedAN() - builder.EnqueueMetricsServerInit() builder.RegisterBadgerMetrics() @@ -46,24 +55,17 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { func (builder *UnstakedAccessNodeBuilder) validateParams() { - // for an unstaked access node, the unstaked network bind address must be provided - if builder.unstakedNetworkBindAddr == cmd.NotSet { - builder.Logger.Fatal().Msg("unstaked bind address not set") + // for an unstaked access node, the staked access node ID must be provided + if strings.TrimSpace(builder.stakedAccessNodeIDHex) == "" { + builder.Logger.Fatal().Msg("staked access node ID not specified") } - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { - builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") + // and also the unstaked bind address + if builder.unstakedNetworkBindAddr == cmd.NotSet { + builder.Logger.Fatal().Msg("unstaked bind address not set") } } -// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstreap peers from the parameters. -// These are the identity of the staked and unstaked AN also acting as the DHT bootstrap server -func (builder *UnstakedAccessNodeBuilder) deriveBootstrapPeerIdentities() { - ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) - builder.MustNot(err) - builder.bootstrapIdentites = ids -} - // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files @@ -84,6 +86,16 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N } } +func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { + anb. + Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + anb.SyncEngineParticipantsProvider = node.Network.GetIdentifierProvider() + return nil + }) + anb.FlowAccessNodeBuilder.Build() + return anb +} + // enqueueUnstakedNetworkInit enqueues the unstaked network component initialized for the unstaked node func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { @@ -99,18 +111,35 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + // intialize the LibP2P factory with an empty metrics NoopCollector for now till we have defined the new unstaked + // network metrics + libP2PFactory, err := builder.FlowAccessNodeBuilder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkMetrics, unstakedNetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) + // use the default validators for the staked access node unstaked networks + msgValidators := p2p.DefaultValidators(builder.Logger, unstakedNodeID) - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + // don't need any peer updates since this will be taken care by the DHT discovery mechanism + peerUpdateInterval := time.Hour - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - participants := flow.IdentityList{} + middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, peerUpdateInterval, + node.UnicastMessageTimeout, + false, // no connection gating for the unstaked network + false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) + msgValidators...) - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, nil) + upstreamANIdentifier, err := flow.HexStringToIdentifier(builder.stakedAccessNodeIDHex) + builder.MustNot(err) + + // topology only consist of the upsteam staked AN + top := topology.NewFixedListTopology(upstreamANIdentifier) + + network, err := builder.initNetwork( + builder.Me, + unstakedNetworkMetrics, + middleware, + top, + ) builder.MustNot(err) builder.UnstakedNetwork = network @@ -125,15 +154,3 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context return builder.UnstakedNetwork, err }) } - -// enqueueConnectWithStakedAN enqueues the upstream connector component which connects the libp2p host of the unstaked -// AN with the staked AN. -// Currently, there is an issue with LibP2P stopping advertisements of subscribed topics if no peers are connected -// (https://github.com/libp2p/go-libp2p-pubsub/issues/442). This means that an unstaked AN could end up not being -// discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need -// of an explicit connect to the staked AN before the node attempts to subscribe to topics. -func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.UnstakedLibP2PNode, builder.Logger), nil - }) -} diff --git a/cmd/collection/main.go b/cmd/collection/main.go index 1e055d03a9c..ef364a5421a 100644 --- a/cmd/collection/main.go +++ b/cmd/collection/main.go @@ -35,6 +35,7 @@ import ( builder "github.com/onflow/flow-go/module/builder/collection" "github.com/onflow/flow-go/module/epochs" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/ingress" "github.com/onflow/flow-go/module/mempool" epochpool "github.com/onflow/flow-go/module/mempool/epochs" @@ -264,7 +265,13 @@ func main() { followerEng, mainChainSyncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) 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 524fd11ce4d..30b5d32f285 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -50,6 +50,7 @@ import ( dkgmodule "github.com/onflow/flow-go/module/dkg" "github.com/onflow/flow-go/module/epochs" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -617,7 +618,13 @@ func main() { comp, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/execution/main.go b/cmd/execution/main.go index 0f2987ba47e..d38d2e7a305 100644 --- a/cmd/execution/main.go +++ b/cmd/execution/main.go @@ -42,6 +42,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" chainsync "github.com/onflow/flow-go/module/synchronization" @@ -456,7 +457,13 @@ func main() { followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 9f1b45d13d2..f9e082556d5 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" @@ -43,6 +44,9 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() + // InitIDProviders initializes the IdentityProvider and IDTranslator + InitIDProviders() + // EnqueueNetworkInit enqueues the default network component with the given context EnqueueNetworkInit(ctx context.Context) @@ -133,6 +137,11 @@ type NodeConfig struct { StakingKey crypto.PrivateKey NetworkKey crypto.PrivateKey + // TODO: initialize these in scaffold and unstaked node + IdentityProvider id.IdentityProvider + IDTranslator p2p.IDTranslator + NetworkingIdentifierProvider id.IdentifierProvider + // root state information RootBlock *flow.Block RootQC *flow.QuorumCertificate diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 5381a9ca678..dac6918b91f 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -22,6 +22,7 @@ import ( "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" @@ -176,15 +177,16 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.BaseConfig.UnicastMessageTimeout, true, true, + fnb.IDTranslator, fnb.MsgValidators...) - participants, err := fnb.State.Final().Identities(p2p.NetworkingSetFilter) - if err != nil { - return nil, fmt.Errorf("could not get network identities: %w", err) - } - subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) - top, err := topology.NewTopicBasedTopology(fnb.NodeID, fnb.Logger, fnb.State) + top, err := topology.NewTopicBasedTopology( + fnb.NodeID, + fnb.IdentityProvider, + fnb.Logger, + fnb.State, + ) if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) } @@ -193,21 +195,21 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { // creates network instance net, err := p2p.NewNetwork(fnb.Logger, codec, - participants, fnb.Me, fnb.Middleware, p2p.DefaultCacheSize, topologyCache, subscriptionManager, - fnb.Metrics.Network) + fnb.Metrics.Network, + p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) } fnb.Network = net - idRefresher := p2p.NewNodeIDRefresher(fnb.Logger, fnb.State, net.SetIDs) - idEvents := gadgets.NewIdentityDeltas(idRefresher.OnIdentityTableChanged) + idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) fnb.ProtocolEvents.AddConsumer(idEvents) return net, err @@ -416,6 +418,20 @@ func (fnb *FlowNodeBuilder) initStorage() { } } +func (fnb *FlowNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + fnb.IDTranslator = idCache + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NetworkingSetFilter, idCache) + return nil + }) +} + func (fnb *FlowNodeBuilder) initState() { fnb.ProtocolEvents = events.NewDistributor() @@ -715,6 +731,8 @@ func (fnb *FlowNodeBuilder) Initialize() NodeBuilder { fnb.ParseAndPrintFlags() + fnb.InitIDProviders() + fnb.EnqueueNetworkInit(ctx) if fnb.metricsEnabled { diff --git a/cmd/verification/main.go b/cmd/verification/main.go index 4860da30780..bbd2af01438 100644 --- a/cmd/verification/main.go +++ b/cmd/verification/main.go @@ -24,10 +24,12 @@ import ( "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/encoding" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" @@ -352,7 +354,13 @@ func main() { followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/engine/common/synchronization/engine.go b/engine/common/synchronization/engine.go index d067cc714f9..1a348b44bc5 100644 --- a/engine/common/synchronization/engine.go +++ b/engine/common/synchronization/engine.go @@ -15,14 +15,13 @@ import ( "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/events" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module" + identifier "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" synccore "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" ) @@ -43,11 +42,11 @@ type Engine struct { blocks storage.Blocks comp network.Engine // compliance layer engine - pollInterval time.Duration - scanInterval time.Duration - core module.SyncCore - state protocol.State - finalizedHeader *FinalizedHeaderCache + pollInterval time.Duration + scanInterval time.Duration + core module.SyncCore + participantsProvider identifier.IdentifierProvider + finalizedHeader *FinalizedHeaderCache requestHandler *RequestHandlerEngine // component responsible for handling requests @@ -66,7 +65,7 @@ func New( comp network.Engine, core module.SyncCore, finalizedHeader *FinalizedHeaderCache, - state protocol.State, + participantsProvider identifier.IdentifierProvider, opts ...OptionFunc, ) (*Engine, error) { @@ -81,18 +80,18 @@ func New( // initialize the propagation engine with its dependencies e := &Engine{ - unit: engine.NewUnit(), - lm: lifecycle.NewLifecycleManager(), - log: log.With().Str("engine", "synchronization").Logger(), - metrics: metrics, - me: me, - blocks: blocks, - comp: comp, - core: core, - pollInterval: opt.pollInterval, - scanInterval: opt.scanInterval, - finalizedHeader: finalizedHeader, - state: state, + unit: engine.NewUnit(), + lm: lifecycle.NewLifecycleManager(), + log: log.With().Str("engine", "synchronization").Logger(), + metrics: metrics, + me: me, + blocks: blocks, + comp: comp, + core: core, + pollInterval: opt.pollInterval, + scanInterval: opt.scanInterval, + finalizedHeader: finalizedHeader, + participantsProvider: participantsProvider, } err := e.setupResponseMessageHandler() @@ -332,7 +331,7 @@ CheckLoop: e.pollHeight() case <-scan.C: head := e.finalizedHeader.Get() - participants := e.getParticipants(head.ID()) + participants := e.participantsProvider.Identifiers() ranges, batches := e.core.ScanPending(head) e.sendRequests(participants, ranges, batches) } @@ -342,30 +341,17 @@ CheckLoop: scan.Stop() } -// getParticipants gets all of the consensus nodes from the state at the given block ID. -func (e *Engine) getParticipants(blockID flow.Identifier) flow.IdentityList { - participants, err := e.state.AtBlockID(blockID).Identities(filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(e.me.NodeID())), - )) - if err != nil { - e.log.Fatal().Err(err).Msgf("could not get consensus participants at block ID %v", blockID) - } - - return participants -} - // pollHeight will send a synchronization request to three random nodes. func (e *Engine) pollHeight() { head := e.finalizedHeader.Get() - participants := e.getParticipants(head.ID()) + participants := e.participantsProvider.Identifiers() // send the request for synchronization req := &messages.SyncRequest{ Nonce: rand.Uint64(), Height: head.Height, } - err := e.con.Multicast(req, synccore.DefaultPollNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultPollNodes, participants...) if err != nil { e.log.Warn().Err(err).Msg("sending sync request to poll heights failed") return @@ -374,7 +360,7 @@ func (e *Engine) pollHeight() { } // sendRequests sends a request for each range and batch using consensus participants from last finalized snapshot. -func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Range, batches []flow.Batch) { +func (e *Engine) sendRequests(participants flow.IdentifierList, ranges []flow.Range, batches []flow.Batch) { var errs *multierror.Error for _, ran := range ranges { @@ -383,7 +369,7 @@ func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Rang FromHeight: ran.From, ToHeight: ran.To, } - err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants...) if err != nil { errs = multierror.Append(errs, fmt.Errorf("could not submit range request: %w", err)) continue @@ -402,7 +388,7 @@ func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Rang Nonce: rand.Uint64(), BlockIDs: batch.BlockIDs, } - err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants...) if err != nil { errs = multierror.Append(errs, fmt.Errorf("could not submit batch request: %w", err)) continue diff --git a/go.sum b/go.sum index dfce96e5f69..049679e4eb2 100644 --- a/go.sum +++ b/go.sum @@ -950,7 +950,6 @@ github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:v github.com/olekukonko/tablewriter v0.0.1/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.2-0.20190409134802-7e037d187b0c/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onflow/cadence v0.15.0/go.mod h1:KMzDF6cIv6nb5PJW9aITaqazbmJX8MMeibFcpPP385M= -github.com/onflow/cadence v0.15.0/go.mod h1:KMzDF6cIv6nb5PJW9aITaqazbmJX8MMeibFcpPP385M= github.com/onflow/cadence v0.17.0/go.mod h1:iR/tZpP+1YhM8iRnOBPiBIs7on5dE3hk2ZfunCRQswE= github.com/onflow/cadence v0.18.0/go.mod h1:iR/tZpP+1YhM8iRnOBPiBIs7on5dE3hk2ZfunCRQswE= github.com/onflow/cadence v0.18.1-0.20210730161646-b891a21c51fd h1:+J8msFNYPhdjgPDnQm3uH2q4TQFHAQXAxWTNOJ5VOec= @@ -963,8 +962,6 @@ github.com/onflow/flow-core-contracts/lib/go/templates v0.7.7/go.mod h1:oWNy8Wz5 github.com/onflow/flow-emulator v0.20.3 h1:qsxKp8oty1glaqEyUfRWtsY0qRgTZfejwGiFix2MYzI= github.com/onflow/flow-emulator v0.20.3/go.mod h1:xNdVsrMJiAaYJ59Dwo+xvj0ENdvk/bI14zkGN4V0ozs= github.com/onflow/flow-ft/lib/go/contracts v0.5.0 h1:Cg4gHGVblxcejfNNG5Mfj98Wf4zbY76O0Y28QB0766A= -github.com/onflow/flow-ft/lib/go/contracts v0.5.0 h1:Cg4gHGVblxcejfNNG5Mfj98Wf4zbY76O0Y28QB0766A= -github.com/onflow/flow-ft/lib/go/contracts v0.5.0/go.mod h1:1zoTjp1KzNnOPkyqKmWKerUyf0gciw+e6tAEt0Ks3JE= github.com/onflow/flow-ft/lib/go/contracts v0.5.0/go.mod h1:1zoTjp1KzNnOPkyqKmWKerUyf0gciw+e6tAEt0Ks3JE= github.com/onflow/flow-go v0.18.0/go.mod h1:cQpvFoqth9PR7tarWDa36R/dDOqUK5QYfeYzCdXPLII= github.com/onflow/flow-go-sdk v0.20.0-alpha.1/go.mod h1:52QZyLwU3p3UZ2FXOy+sRl4JPdtvJoae1spIUBOFxA8= diff --git a/model/flow/identifierList.go b/model/flow/identifierList.go index cb1d4780b49..224340b7b02 100644 --- a/model/flow/identifierList.go +++ b/model/flow/identifierList.go @@ -2,6 +2,7 @@ package flow import ( "bytes" + "math/rand" "github.com/rs/zerolog/log" ) @@ -70,6 +71,32 @@ func (il IdentifierList) Contains(target Identifier) bool { return false } +// Union returns a new identifier list containing the union of `il` and `other`. +// There are no duplicates in the output. +func (il IdentifierList) Union(other IdentifierList) IdentifierList { + // stores the output, the union of the two lists + union := make(IdentifierList, 0, len(il)+len(other)) + // efficient lookup to avoid duplicates + lookup := make(map[Identifier]struct{}) + + // add all identifiers, omitted duplicates + for _, identifier := range append(il.Copy(), other...) { + if _, exists := lookup[identifier]; exists { + continue + } + union = append(union, identifier) + lookup[identifier] = struct{}{} + } + + return union +} + +// DeterministicSample returns deterministic random sample from the `IdentifierList` using the given seed +func (il IdentifierList) DeterministicSample(size uint, seed int64) IdentifierList { + rand.Seed(seed) + return il.Sample(size) +} + // Sample returns random sample of length 'size' of the ids // [Fisher-Yates shuffle](https://en.wikipedia.org/wiki/Fisher–Yates_shuffle). func (il IdentifierList) Sample(size uint) IdentifierList { diff --git a/module/id/filtered_provider.go b/module/id/filtered_provider.go new file mode 100644 index 00000000000..d935125232f --- /dev/null +++ b/module/id/filtered_provider.go @@ -0,0 +1,18 @@ +package id + +import ( + "github.com/onflow/flow-go/model/flow" +) + +type FilteredIdentifierProvider struct { + filter flow.IdentityFilter + identityProvider IdentityProvider +} + +func NewFilteredIdentifierProvider(filter flow.IdentityFilter, identityProvider IdentityProvider) *FilteredIdentifierProvider { + return &FilteredIdentifierProvider{filter, identityProvider} +} + +func (p *FilteredIdentifierProvider) Identifiers() flow.IdentifierList { + return p.identityProvider.Identities(p.filter).NodeIDs() +} diff --git a/module/id/id_provider.go b/module/id/id_provider.go index 430780b6dc8..a663a8f9215 100644 --- a/module/id/id_provider.go +++ b/module/id/id_provider.go @@ -1,148 +1,13 @@ -package identity +package id import ( - "sync" - - "github.com/libp2p/go-libp2p-core/peer" - "github.com/libp2p/go-libp2p-core/peerstore" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/state/protocol/events" ) type IdentifierProvider interface { Identifiers() flow.IdentifierList } -// TODO: rename to ProtocolStateIDProvider -type ProtocolStateIdentifierProvider struct { - events.Noop - identities flow.IdentityList // TODO: actually we *can* just use identifiers after all - state protocol.State - mu sync.RWMutex - filter flow.IdentityFilter - peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! - flowIDs map[peer.ID]flow.Identifier -} - -func WithFilter(filter flow.IdentityFilter) ProtocolStateIdentifierProviderOption { - return func(provider *ProtocolStateIdentifierProvider) { - provider.filter = filter - } -} - -type ProtocolStateIdentifierProviderOption func(*ProtocolStateIdentifierProvider) - -// TODO: this one also implements IDTranslator!!! -func NewProtocolStateIdentifierProvider( - state protocol.State, - eventDistributer *events.Distributor, - opts ...ProtocolStateIdentifierProviderOption, -) (*ProtocolStateIdentifierProvider, error) { - provider := &ProtocolStateIdentifierProvider{ - state: state, - filter: filter.Any, - } - - for _, opt := range opts { - opt(provider) - } - - head, err := state.Final().Head() - if err != nil { - return nil, err // TODO: format the error - } - - provider.update(head.ID()) - eventDistributer.AddConsumer(provider) - - return provider, nil -} - -func (p *ProtocolStateIdentifierProvider) EpochTransition(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) update(blockID flow.Identifier) { - identities, err := p.state.AtBlockID(blockID).Identities(p.filter) - if err != nil { - // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. - } - - nIds := identities.Count() - - peerIDs := make(map[flow.Identifier]peer.ID, nIds) - flowIDs := make(map[peer.ID]flow.Identifier, nIds) - - for _, identity := range identities { - pid, err := p2p.IdentityToPeerID(identity) - if err != nil { - // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping - } - - flowIDs[pid] = identity.NodeID - peerIDs[identity.NodeID] = pid - } - - p.mu.Lock() - defer p.mu.Unlock() - p.identities = identities - p.flowIDs = flowIDs - p.peerIDs = peerIDs -} - -func (p *ProtocolStateIdentifierProvider) Identifiers() flow.IdentifierList { - p.mu.RLock() - defer p.mu.RUnlock() - return p.identities.NodeIDs() -} - -func (p *ProtocolStateIdentifierProvider) GetPeerID(flowID flow.Identifier) (pid peer.ID, found bool) { - p.mu.RLock() - defer p.mu.RUnlock() - - pid, found = p.peerIDs[flowID] - return -} - -func (p *ProtocolStateIdentifierProvider) GetFlowID(peerID peer.ID) (fid flow.Identifier, found bool) { - p.mu.RLock() - defer p.mu.RUnlock() - - fid, found = p.flowIDs[peerID] - return -} - -type PeerstoreIdentifierProvider struct { - store peerstore.Peerstore - idTranslator p2p.IDTranslator -} - -func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator p2p.IDTranslator) *PeerstoreIdentifierProvider { - return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} -} - -func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { - var result flow.IdentifierList - - pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? - for _, pid := range pids { - flowID, success := p.idTranslator.GetFlowID(pid) - if success { - result = append(result, flowID) - } - } - - return result +type IdentityProvider interface { + Identities(flow.IdentityFilter) flow.IdentityList } diff --git a/network/middleware.go b/network/middleware.go index ed935995e6c..82928584d43 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -6,6 +6,7 @@ import ( "time" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/message" ) @@ -51,17 +52,17 @@ type Middleware interface { // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. - UpdateAllowList() error + UpdateAllowList() } // Overlay represents the interface that middleware uses to interact with the // overlay network layer. type Overlay interface { - // Topology returns an identity list of nodes which this node should be directly connected to as peers - Topology() (flow.IdentityList, error) - // Identity returns a map of all identifier to flow identity - Identity() (map[flow.Identifier]flow.Identity, error) + // Topology returns an identifier list of nodes which this node should be directly connected to as peers + Topology() (flow.IdentifierList, error) Receive(nodeID flow.Identifier, msg *message.Message) error + SetDefaultIdentifierProvider(id.IdentifierProvider) + GetIdentifierProvider() id.IdentifierProvider } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/connGater.go b/network/p2p/connGater.go index 10a944a5f04..caa05bb1f1a 100644 --- a/network/p2p/connGater.go +++ b/network/p2p/connGater.go @@ -29,14 +29,14 @@ func NewConnGater(log zerolog.Logger) *ConnGater { } // update updates the peer ID map -func (c *ConnGater) update(peerInfos []peer.AddrInfo) { +func (c *ConnGater) update(pids []peer.ID) { // create a new peer.ID map - peerIDs := make(map[peer.ID]struct{}, len(peerInfos)) + peerIDs := make(map[peer.ID]struct{}, len(pids)) // for each peer.AddrInfo, create an entry in the map for the peer.ID - for _, p := range peerInfos { - peerIDs[p.ID] = struct{}{} + for _, pid := range pids { + peerIDs[pid] = struct{}{} } // cache the new map diff --git a/network/p2p/idTranslator.go b/network/p2p/idTranslator.go deleted file mode 100644 index f9f567f1fec..00000000000 --- a/network/p2p/idTranslator.go +++ /dev/null @@ -1,33 +0,0 @@ -package p2p - -import ( - "github.com/libp2p/go-libp2p-core/peer" - - "github.com/onflow/flow-go/model/flow" -) - -type IDTranslator interface { - GetPeerID(flow.Identifier) (peer.ID, bool) - GetFlowID(peer.ID) (flow.Identifier, bool) -} - -type UnstakedNetworkPeerIDProvider struct { - // basically, just do the conversion from flow ID to peer ID using whatever scheme we talked about. - // whether this be, with caching or not -} - -func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { - pk, err := PublicKey(id.NetworkPubKey) - if err != nil { - // TODO: format the error - return - } - - pid, err = peer.IDFromPublicKey(pk) - if err != nil { - // TODO: format the error - return - } - - return -} diff --git a/network/p2p/id_translator.go b/network/p2p/id_translator.go new file mode 100644 index 00000000000..8edb2d500d5 --- /dev/null +++ b/network/p2p/id_translator.go @@ -0,0 +1,12 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type IDTranslator interface { + GetPeerID(flow.Identifier) (peer.ID, error) + GetFlowID(peer.ID) (flow.Identifier, error) +} diff --git a/network/p2p/libp2pConnector.go b/network/p2p/libp2pConnector.go index 71c5b550635..64e68a43170 100644 --- a/network/p2p/libp2pConnector.go +++ b/network/p2p/libp2pConnector.go @@ -65,34 +65,23 @@ func newLibp2pConnector(host host.Host, log zerolog.Logger) (*libp2pConnector, e // UpdatePeers is the implementation of the Connector.UpdatePeers function. It connects to all of the ids and // disconnects from any other connection that the libp2p node might have. -func (l *libp2pConnector) UpdatePeers(ctx context.Context, ids flow.IdentityList) error { - - // derive the peer.AddrInfo from each of the flow.Identity - pInfos, invalidIDs := peerInfosFromIDs(ids) - +func (l *libp2pConnector) UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) { // connect to each of the peer.AddrInfo in pInfos - l.connectToPeers(ctx, pInfos) + l.connectToPeers(ctx, peerIDs) // disconnect from any other peers not in pInfos - l.trimAllConnectionsExcept(pInfos) - - // if some ids didn't translate to peer.AddrInfo, return error - if len(invalidIDs) != 0 { - return NewUnconvertableIdentitiesError(invalidIDs) - } - - return nil + l.trimAllConnectionsExcept(peerIDs) } // connectToPeers connects each of the peer in pInfos -func (l *libp2pConnector) connectToPeers(ctx context.Context, pInfos []peer.AddrInfo) { +func (l *libp2pConnector) connectToPeers(ctx context.Context, peerIDs peer.IDSlice) { // create a channel of peer.AddrInfo as expected by the connector - peerCh := make(chan peer.AddrInfo, len(pInfos)) + peerCh := make(chan peer.AddrInfo, len(peerIDs)) // stuff all the peer.AddrInfo it into the channel - for _, peerInfo := range pInfos { - peerCh <- peerInfo + for _, peerID := range peerIDs { + peerCh <- peer.AddrInfo{ID: peerID} } // close the channel to ensure Connect does not block @@ -102,15 +91,15 @@ func (l *libp2pConnector) connectToPeers(ctx context.Context, pInfos []peer.Addr l.backoffConnector.Connect(ctx, peerCh) } -// trimAllConnectionsExcept trims all connections of the node from peers not part of peerInfos. +// trimAllConnectionsExcept trims all connections of the node from peers not part of peerIDs. // A node would have created such extra connections earlier when the identity list may have been different, or // it may have been target of such connections from node which have now been excluded. -func (l *libp2pConnector) trimAllConnectionsExcept(peerInfos []peer.AddrInfo) { +func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs []peer.ID) { // convert the peerInfos to a peer.ID -> bool map - peersToKeep := make(map[peer.ID]bool, len(peerInfos)) - for _, pInfo := range peerInfos { - peersToKeep[pInfo.ID] = true + peersToKeep := make(map[peer.ID]bool, len(peerIDs)) + for _, pid := range peerIDs { + peersToKeep[pid] = true } // get all current node connections diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index b1a127f2d5e..f5c974550af 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -17,7 +17,6 @@ import ( libp2pnet "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/protocol" - dht "github.com/libp2p/go-libp2p-kad-dht" pubsub "github.com/libp2p/go-libp2p-pubsub" swarm "github.com/libp2p/go-libp2p-swarm" tptu "github.com/libp2p/go-libp2p-transport-upgrader" @@ -55,12 +54,23 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I connGater := NewConnGater(log) + // create PubSub options for libp2p to use + psOptions := []pubsub.Option{ + // skip message signing + pubsub.WithMessageSigning(false), + // skip message signature + pubsub.WithStrictSignatureVerification(false), + // set max message size limit for 1-k PubSub messaging + pubsub.WithMaxMessageSize(maxPubSubMsgSize), + // no discovery + } + return func() (*Node, error) { return NewDefaultLibP2PNodeBuilder(me, address, flowKey). SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize)...). + SetPubsubOptions(psOptions...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). Build(ctx) @@ -71,7 +81,7 @@ type NodeBuilder interface { SetRootBlockID(string) NodeBuilder SetConnectionManager(TagLessConnManager) NodeBuilder SetConnectionGater(*ConnGater) NodeBuilder - SetPubsubOptions(...PubsubOption) NodeBuilder + SetPubsubOptions(...pubsub.Option) NodeBuilder SetPingInfoProvider(PingInfoProvider) NodeBuilder SetLogger(zerolog.Logger) NodeBuilder Build(context.Context) (*Node, error) @@ -86,7 +96,7 @@ type DefaultLibP2PNodeBuilder struct { pingInfoProvider PingInfoProvider pubSubMaker func(context.Context, host.Host, ...pubsub.Option) (*pubsub.PubSub, error) hostMaker func(context.Context, ...config.Option) (host.Host, error) - pubSubOpts []PubsubOption + pubSubOpts []pubsub.Option } func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcrypto.PrivateKey) NodeBuilder { @@ -116,7 +126,7 @@ func (builder *DefaultLibP2PNodeBuilder) SetConnectionGater(connGater *ConnGater return builder } -func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...PubsubOption) NodeBuilder { +func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...pubsub.Option) NodeBuilder { builder.pubSubOpts = opts return builder } @@ -164,11 +174,6 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.connMgr = builder.connMngr } - if builder.rootBlockID == "" { - return nil, errors.New("root block ID must be provided") - } - node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) - if builder.pingInfoProvider != nil { opts = append(opts, libp2p.Ping(true)) } @@ -185,17 +190,7 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.pingService = pingService } - var libp2pPSOptions []pubsub.Option - // generate the libp2p Pubsub options from the given context and host - for _, optionGenerator := range builder.pubSubOpts { - option, err := optionGenerator(ctx, libp2pHost) - if err != nil { - return nil, err - } - libp2pPSOptions = append(libp2pPSOptions, option) - } - - ps, err := builder.pubSubMaker(ctx, libp2pHost, libp2pPSOptions...) + ps, err := builder.pubSubMaker(ctx, libp2pHost, builder.pubSubOpts...) if err != nil { return nil, err } @@ -290,13 +285,8 @@ func (n *Node) Stop() (chan struct{}, error) { } // AddPeer adds a peer to this node by adding it to this node's peerstore and connecting to it -func (n *Node) AddPeer(ctx context.Context, identity flow.Identity) error { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return fmt.Errorf("failed to add peer %s: %w", identity.String(), err) - } - - err = n.host.Connect(ctx, pInfo) +func (n *Node) AddPeer(ctx context.Context, peerID peer.ID) error { + err := n.host.Connect(ctx, peer.AddrInfo{ID: peerID}) if err != nil { return err } @@ -304,51 +294,29 @@ func (n *Node) AddPeer(ctx context.Context, identity flow.Identity) error { return nil } -// RemovePeer closes the connection with the identity. -func (n *Node) RemovePeer(ctx context.Context, identity flow.Identity) error { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return fmt.Errorf("failed to remove peer %x: %w", identity, err) - } - - err = n.host.Network().ClosePeer(pInfo.ID) +// RemovePeer closes the connection with the peer. +func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { + err := n.host.Network().ClosePeer(peerID) if err != nil { - return fmt.Errorf("failed to remove peer %s: %w", identity, err) + return fmt.Errorf("failed to remove peer %s: %w", peerID, err) } return nil } -// CreateStream returns an existing stream connected to identity, if it exists or adds one to identity as a peer and creates a new stream with it. -func (n *Node) CreateStream(ctx context.Context, nodeID flow.Identifier) (libp2pnet.Stream, error) { +// CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. +func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) - stream, err := n.tryCreateNewStream(ctx, nodeID, maxConnectAttempt) + stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) if err != nil { - n.host.Peerstore().Addrs() - return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (node_id: %s, address: %s): %w", identity.NodeID.String(), - identity.Address, err)) + return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (peer_id: %s): %w", peerID, err)) } return stream, nil } -func (n *Node) GetPeerIPPort(nodeID flow.Identifier) (string, string, error) { - // TODO: first get peer ID - x := n.host.Peerstore().Addrs() -} - -// tryCreateNewStream makes at most maxAttempts to create a stream with the identity. +// tryCreateNewStream makes at most maxAttempts to create a stream with the peer. // This was put in as a fix for #2416. PubSub and 1-1 communication compete with each other when trying to connect to // remote nodes and once in a while NewStream returns an error 'both yamux endpoints are clients' -func (n *Node) tryCreateNewStream(ctx context.Context, identity flow.Identity, maxAttempts int) (libp2pnet.Stream, error) { - _, _, key, err := networkingInfo(identity) - if err != nil { - return nil, fmt.Errorf("could not get translate identity to networking info %s: %w", identity.NodeID.String(), err) - } - - peerID, err := peer.IDFromPublicKey(key) - if err != nil { - return nil, fmt.Errorf("could not get peer ID: %w", err) - } - +func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttempts int) (libp2pnet.Stream, error) { // protect the underlying connection from being inadvertently pruned by the peer manager while the stream and // connection creation is being attempted n.connMgr.ProtectPeer(peerID) @@ -381,7 +349,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, identity flow.Identity, m time.Sleep(time.Duration(r) * time.Millisecond) } - err = n.AddPeer(ctx, identity) + err := n.AddPeer(ctx, peerID) if err != nil { // if the connection was rejected due to invalid node id, skip the re-attempt @@ -503,23 +471,18 @@ func (n *Node) Publish(ctx context.Context, topic flownet.Topic, data []byte) er } // Ping pings a remote node and returns the time it took to ping the remote node if successful or the error -func (n *Node) Ping(ctx context.Context, identity flow.Identity) (message.PingResponse, time.Duration, error) { - +func (n *Node) Ping(ctx context.Context, peerID peer.ID) (message.PingResponse, time.Duration, error) { pingError := func(err error) error { - return fmt.Errorf("failed to ping %s (%s): %w", identity.NodeID.String(), identity.Address, err) + return fmt.Errorf("failed to ping peer %s: %w", peerID, err) } - // convert the target node address to libp2p peer info - targetInfo, err := PeerAddressInfo(identity) - if err != nil { - return message.PingResponse{}, -1, pingError(err) - } + targetInfo := peer.AddrInfo{ID: peerID} n.connMgr.ProtectPeer(targetInfo.ID) defer n.connMgr.UnprotectPeer(targetInfo.ID) // connect to the target node - err = n.host.Connect(ctx, targetInfo) + err := n.host.Connect(ctx, targetInfo) if err != nil { return message.PingResponse{}, -1, pingError(err) } @@ -534,22 +497,20 @@ func (n *Node) Ping(ctx context.Context, identity flow.Identity) (message.PingRe } // UpdateAllowList allows the peer allow list to be updated. -func (n *Node) UpdateAllowList(identities flow.IdentityList) error { - // if the node was so far not under allowList +func (n *Node) UpdateAllowList(peers []peer.ID) { if n.connGater == nil { - return fmt.Errorf("could not add an allow list, this node was started without allow listing") - + n.logger.Debug().Hex("node_id", logging.ID(n.id)).Msg("skipping update allow list, connection gating is not enabled") + return } // generates peer address information for all identities - allowlist := make([]peer.AddrInfo, 0, len(identities)) - for _, identity := range identities { - addressInfo, err := PeerAddressInfo(*identity) + allowlist := make([]peer.AddrInfo, len(identities)) + var err error + for i, identity := range identities { + allowlist[i], err = PeerAddressInfo(*identity) if err != nil { - n.logger.Err(err).Str("identity", identity.String()).Msg("could not generate address info") - continue + return fmt.Errorf("could not generate address info: %w", err) } - allowlist = append(allowlist, addressInfo) } n.connGater.update(allowlist) @@ -572,13 +533,9 @@ func (n *Node) SetPingStreamHandler(handler libp2pnet.StreamHandler) { } // IsConnected returns true is address is a direct peer of this node else false -func (n *Node) IsConnected(identity flow.Identity) (bool, error) { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return false, err - } +func (n *Node) IsConnected(peerID peer.ID) (bool, error) { // query libp2p for connectedness status of this peer - isConnected := n.host.Network().Connectedness(pInfo.ID) == libp2pnet.Connected + isConnected := n.host.Network().Connectedness(peerID) == libp2pnet.Connected return isConnected, nil } @@ -649,33 +606,3 @@ func DefaultPubSub(ctx context.Context, host host.Host, psOption ...pubsub.Optio } return pubSub, nil } - -// PubsubOption generates a libp2p pubsub.Option from the given context and host -type PubsubOption func(ctx context.Context, host host.Host) (pubsub.Option, error) - -func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { - pubSubOptionFunc := func(option pubsub.Option) PubsubOption { - return func(_ context.Context, _ host.Host) (pubsub.Option, error) { - return option, nil - } - } - return []PubsubOption{ - // skip message signing - pubSubOptionFunc(pubsub.WithMessageSigning(true)), - // skip message signature - pubSubOptionFunc(pubsub.WithStrictSignatureVerification(true)), - // set max message size limit for 1-k PubSub messaging - pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), - // no discovery - } -} - -func WithDHTDiscovery(option ...dht.Option) PubsubOption { - return func(ctx context.Context, host host.Host) (pubsub.Option, error) { - dhtDiscovery, err := NewDHT(ctx, host, option...) - if err != nil { - return nil, err - } - return pubsub.WithDiscovery(dhtDiscovery), nil - } -} diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 49319fdc627..f24ad9c54a3 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -11,9 +11,9 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -78,6 +78,7 @@ type Middleware struct { unicastMessageTimeout time.Duration connectionGating bool managePeerConnections bool + idTranslator IDTranslator } // NewMiddleware creates a new middleware instance @@ -89,7 +90,8 @@ type Middleware struct { // connectionGating if set to True, restricts this node to only talk to other nodes which are part of the identity list // managePeerConnections if set to True, enables the default PeerManager which continuously updates the node's peer connections // validators are the set of the different message validators that each inbound messages is passed through -func NewMiddleware(log zerolog.Logger, +func NewMiddleware( + log zerolog.Logger, libP2PNodeFactory LibP2PFactoryFunc, flowID flow.Identifier, metrics module.NetworkMetrics, @@ -98,13 +100,9 @@ func NewMiddleware(log zerolog.Logger, unicastMessageTimeout time.Duration, connectionGating bool, managePeerConnections bool, - validators ...network.MessageValidator) *Middleware { - - if len(validators) == 0 { - // add default validators to filter out unwanted messages received by this node - validators = DefaultValidators(log, flowID) - } - + idTranslator IDTranslator, + validators ...network.MessageValidator, +) *Middleware { ctx, cancel := context.WithCancel(context.Background()) if unicastMessageTimeout <= 0 { @@ -112,7 +110,7 @@ func NewMiddleware(log zerolog.Logger, } // create the node entity and inject dependencies & config - return &Middleware{ + mw := &Middleware{ ctx: ctx, cancel: cancel, log: log, @@ -121,12 +119,19 @@ func NewMiddleware(log zerolog.Logger, libP2PNodeFactory: libP2PNodeFactory, metrics: metrics, rootBlockID: rootBlockID, - validators: validators, + validators: DefaultValidators(log, flowID), peerUpdateInterval: peerUpdateInterval, unicastMessageTimeout: unicastMessageTimeout, connectionGating: connectionGating, managePeerConnections: managePeerConnections, + idTranslator: idTranslator, } + + if len(validators) != 0 { + mw.validators = validators + } + + return mw } func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.MessageValidator { @@ -136,6 +141,35 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes } } +func (m *Middleware) topologyPeers() (peer.IDSlice, error) { + identifiers, err := m.ov.Topology() + if err != nil { + // TODO: format error + return nil, err + } + + return m.peerIDs(identifiers), nil +} + +func (m *Middleware) allPeers() peer.IDSlice { + return m.peerIDs(m.ov.GetIdentifierProvider().Identifiers()) +} + +func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { + result := make([]peer.ID, len(flowIDs)) + + for _, fid := range flowIDs { + pid, err := m.idTranslator.GetPeerID(fid) + if err != nil { + // TODO: log here + } + + result = append(result, pid) + } + + return result +} + // Me returns the flow identifier of the this middleware func (m *Middleware) Me() flow.Identifier { return m.me @@ -151,6 +185,8 @@ func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov libP2PNode, err := m.libP2PNodeFactory() + ov.SetDefaultIdentifierProvider(NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator)) + if err != nil { return fmt.Errorf("could not create libp2p node: %w", err) } @@ -158,17 +194,7 @@ func (m *Middleware) Start(ov network.Overlay) error { m.libP2PNode.SetFlowProtocolStreamHandler(m.handleIncomingStream) if m.connectionGating { - - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return fmt.Errorf("could not get identities: %w", err) - } - - err = m.libP2PNode.UpdateAllowList(identityList(idsMap)) - if err != nil { - return fmt.Errorf("could not update approved peer list: %w", err) - } + m.libP2PNode.UpdateAllowList(m.allPeers()) } if m.managePeerConnections { @@ -177,7 +203,7 @@ func (m *Middleware) Start(ov network.Overlay) error { return fmt.Errorf("failed to create libp2pConnector: %w", err) } - m.peerManager = NewPeerManager(m.log, m.ov.Topology, libp2pConnector, WithInterval(m.peerUpdateInterval)) + m.peerManager = NewPeerManager(m.log, m.topologyPeers, libp2pConnector, WithInterval(m.peerUpdateInterval)) select { case <-m.peerManager.Ready(): m.log.Debug().Msg("peer manager successfully started") @@ -222,10 +248,10 @@ func (m *Middleware) Stop() { // Dispatch should be used whenever guaranteed delivery to a specific target is required. Otherwise, Publish is // a more efficient candidate. func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) error { - // translates identifier to identity - targetIdentity, err := m.identity(targetID) + // translates identifier to peer id + peerID, err := m.idTranslator.GetPeerID(targetID) if err != nil { - return fmt.Errorf("could not find identity for target id: %w", err) + return fmt.Errorf("could not find peer id for target id: %w", err) } maxMsgSize := unicastMaxMsgSize(msg) @@ -245,9 +271,9 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) // (streams don't need to be reused and are fairly inexpensive to be created for each send. // A stream creation does NOT incur an RTT as stream negotiation happens as part of the first message // sent out the the receiver - stream, err := m.libP2PNode.CreateStream(ctx, targetIdentity) + stream, err := m.libP2PNode.CreateStream(ctx, peerID) if err != nil { - return fmt.Errorf("failed to create stream for %s :%w", targetID.String(), err) + return fmt.Errorf("failed to create stream for %s :%w", targetID, err) } // create a gogo protobuf writer @@ -256,19 +282,19 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) err = writer.WriteMsg(msg) if err != nil { - return fmt.Errorf("failed to send message to %s: %w", targetID.String(), err) + return fmt.Errorf("failed to send message to %s: %w", targetID, err) } // flush the stream err = bufw.Flush() if err != nil { - return fmt.Errorf("failed to flush stream for %s: %w", targetIdentity.String(), err) + return fmt.Errorf("failed to flush stream for %s: %w", targetID, err) } // close the stream immediately err = stream.Close() if err != nil { - return fmt.Errorf("failed to close the stream for %s: %w", targetIdentity.String(), err) + return fmt.Errorf("failed to close the stream for %s: %w", targetID, err) } // OneToOne communication metrics are reported with topic OneToOne @@ -277,36 +303,6 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) return nil } -// identity returns corresponding identity of an identifier based on overlay identity list. -func (m *Middleware) identity(identifier flow.Identifier) (flow.Identity, error) { - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return flow.Identity{}, fmt.Errorf("could not get identities: %w", err) - } - - // retrieve the flow.Identity for the give flow.ID - flowIdentity, found := idsMap[identifier] - if !found { - return flow.Identity{}, fmt.Errorf("could not get node identity for %s: %w", identifier.String(), err) - } - - return flowIdentity, nil -} - -// identityList translates an identity map into an identity list. -func identityList(identityMap map[flow.Identifier]flow.Identity) flow.IdentityList { - var identities flow.IdentityList - for _, identity := range identityMap { - // casts identity into a local variable to - // avoid shallow copy of the loop variable - id := identity - identities = append(identities, &id) - - } - return identities -} - // handleIncomingStream handles an incoming stream from a remote peer // it is a callback that gets called for each incoming stream by libp2p with a new stream object func (m *Middleware) handleIncomingStream(s libp2pnetwork.Stream) { @@ -361,35 +357,6 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { return nil } -// processAuthenticatedMessage processes a message and a source (indicated by its PublicKey) and eventually passes it to the overlay -// In particular, it checks the claim of protocol authorship situated in the message against `originKey` -// The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` -// this requirement is fulfilled by e.g. the output of readConnection and readSubscription -func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { - identities, err := m.ov.Identity() - if err != nil { - m.log.Error().Err(err).Msg("failed to retrieve identities list while delivering a message") - return - } - - // check the origin of the message corresponds to the one claimed in the OriginID - originID := flow.HashToID(msg.OriginID) - - originIdentity, found := identities[originID] - if !found { - m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) - return - } else if originIdentity.NetworkPubKey == nil { - m.log.Warn().Msgf("received message with claimed originID %x, wich has no network identifiers at this node, and was dropped", originID) - return - } else if !originIdentity.NetworkPubKey.Equals(originKey) { - m.log.Warn().Msgf("received message claiming to be from nodeID %v with key %x was actually signed by %x and dropped", originID, originIdentity.NetworkPubKey, originKey) - return - } - - m.processMessage(msg) -} - // processMessage processes a message and eventually passes it to the overlay func (m *Middleware) processMessage(msg *message.Message) { @@ -443,40 +410,33 @@ func (m *Middleware) Publish(msg *message.Message, channel network.Channel) erro // Ping pings the target node and returns the ping RTT or an error func (m *Middleware) Ping(targetID flow.Identifier) (message.PingResponse, time.Duration, error) { - targetIdentity, err := m.identity(targetID) + peerID, err := m.idTranslator.GetPeerID(targetID) if err != nil { - return message.PingResponse{}, -1, fmt.Errorf("could not find identity for target id: %w", err) + return message.PingResponse{}, -1, fmt.Errorf("could not find peer id for target id: %w", err) } - return m.libP2PNode.Ping(m.ctx, targetIdentity) + return m.libP2PNode.Ping(m.ctx, peerID) } -// UpdateAllowList fetches the most recent identity of the nodes from overlay +// UpdateAllowList fetches the most recent identifiers of the nodes from overlay // and updates the underlying libp2p node. -func (m *Middleware) UpdateAllowList() error { - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return fmt.Errorf("could not get identities: %w", err) - } - +func (m *Middleware) UpdateAllowList() { // update libp2pNode's approve lists if this middleware also does connection gating if m.connectionGating { - err = m.libP2PNode.UpdateAllowList(identityList(idsMap)) - if err != nil { - return fmt.Errorf("failed to update approved peer list: %w", err) - } + m.libP2PNode.UpdateAllowList(m.allPeers()) } // update peer connections if this middleware also does peer management m.peerManagerUpdate() - - return nil } // IsConnected returns true if this node is connected to the node with id nodeID. -func (m *Middleware) IsConnected(identity flow.Identity) (bool, error) { - return m.libP2PNode.IsConnected(identity) +func (m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { + peerID, err := m.idTranslator.GetPeerID(nodeID) + if err != nil { + return false, fmt.Errorf("could not find peer id for target id: %w", err) + } + return m.libP2PNode.IsConnected(peerID) } // unicastMaxMsgSize returns the max permissible size for a unicast message diff --git a/network/p2p/network.go b/network/p2p/network.go index c4151e6ea79..0584143e5c5 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -12,7 +12,9 @@ import ( "github.com/onflow/flow-go/crypto/hash" channels "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" @@ -22,6 +24,14 @@ import ( const DefaultCacheSize = 10e6 +// NetworkingSetFilter is an identity filter that, when applied to the identity +// table at a given snapshot, returns all nodes that we should communicate with +// over the networking layer. +// +// NOTE: The protocol state includes nodes from the previous/next epoch that should +// be included in network communication. We omit any nodes that have been ejected. +var NetworkingSetFilter = filter.Not(filter.Ejected) + type ReadyDoneAwareNetwork interface { module.Network module.ReadyDoneAware @@ -31,20 +41,28 @@ type ReadyDoneAwareNetwork interface { // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { sync.RWMutex - logger zerolog.Logger - codec network.Codec - ids flow.IdentityList - me module.Local - mw network.Middleware - top network.Topology // used to determine fanout connections - metrics module.NetworkMetrics - rcache *RcvCache // used to deduplicate incoming messages - queue network.MessageQueue - ctx context.Context - cancel context.CancelFunc - subMngr network.SubscriptionManager // used to keep track of subscribed channels - lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle - ReadyDoneAwareNetwork + idProvider id.IdentifierProvider + defaultIdProvider id.IdentifierProvider + logger zerolog.Logger + codec network.Codec + me module.Local + mw network.Middleware + top network.Topology // used to determine fanout connections + metrics module.NetworkMetrics + rcache *RcvCache // used to deduplicate incoming messages + queue network.MessageQueue + ctx context.Context + cancel context.CancelFunc + subMngr network.SubscriptionManager // used to keep track of subscribed channels + lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle +} + +type NetworkOption func(*Network) + +func WithIdentifierProvider(provider id.IdentifierProvider) NetworkOption { + return func(net *Network) { + net.idProvider = provider + } } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -54,13 +72,13 @@ type Network struct { func NewNetwork( log zerolog.Logger, codec network.Codec, - ids flow.IdentityList, me module.Local, mw network.Middleware, csize int, top network.Topology, sm network.SubscriptionManager, metrics module.NetworkMetrics, + opts ...NetworkOption, ) (*Network, error) { rcache, err := newRcvCache(csize) @@ -80,7 +98,6 @@ func NewNetwork( lifecycleManager: lifecycle.NewLifecycleManager(), } o.ctx, o.cancel = context.WithCancel(context.Background()) - o.ids = ids // setup the message queue // create priority queue @@ -89,6 +106,10 @@ func NewNetwork( // create workers to read from the queue and call queueSubmitFunc queue.CreateQueueWorkers(o.ctx, queue.DefaultNumWorkers, o.queue, o.queueSubmitFunc) + for _, opt := range opts { + opt(o) + } + return o, nil } @@ -154,25 +175,27 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -// Identity returns a map of all flow.Identifier to flow identity by querying the flow state -func (n *Network) Identity() (map[flow.Identifier]flow.Identity, error) { +func (n *Network) GetIdentifierProvider() id.IdentifierProvider { + if n.idProvider != nil { + return n.idProvider + } n.RLock() defer n.RUnlock() - identifierToID := make(map[flow.Identifier]flow.Identity) - for _, id := range n.ids { - identifierToID[id.NodeID] = *id + if n.defaultIdProvider == nil { + n.logger.Fatal().Msg("TODO") + // TODO } - return identifierToID, nil + return n.defaultIdProvider } -// Topology returns the identities of a uniform subset of nodes in protocol state using the topology provided earlier. +// Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. -func (n *Network) Topology() (flow.IdentityList, error) { +func (n *Network) Topology() (flow.IdentifierList, error) { n.Lock() defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.ids, subscribedChannels) + top, err := n.top.GenerateFanout(n.GetIdentifierProvider().Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } @@ -187,25 +210,6 @@ func (n *Network) Receive(nodeID flow.Identifier, msg *message.Message) error { return nil } -// SetIDs updates the identity list cached by the network layer -func (n *Network) SetIDs(ids flow.IdentityList) error { - - // remove self from id - ids = ids.Filter(n.me.NotMeFilter()) - - n.Lock() - n.ids = ids - n.Unlock() - - // update the allow list - err := n.mw.UpdateAllowList() - if err != nil { - return fmt.Errorf("failed to update middleware allow list: %w", err) - } - - return nil -} - func (n *Network) processNetworkMessage(senderID flow.Identifier, message *message.Message) error { // checks the cache for deduplication and adds the message if not already present if n.rcache.add(message.EventID, network.Channel(message.ChannelID)) { @@ -298,6 +302,12 @@ func (n *Network) genNetworkMessage(channel network.Channel, event interface{}, return msg, nil } +func (n *Network) SetDefaultIdentifierProvider(provider id.IdentifierProvider) { + n.Lock() + n.defaultIdProvider = provider + n.Unlock() +} + // unicast sends the message in a reliable way to the given recipient. // It uses 1-1 direct messaging over the underlying network to deliver the message. // It returns an error if unicasting fails. diff --git a/network/p2p/nodeIDRefresher.go b/network/p2p/nodeIDRefresher.go deleted file mode 100644 index 219951ebbfc..00000000000 --- a/network/p2p/nodeIDRefresher.go +++ /dev/null @@ -1,82 +0,0 @@ -package p2p - -import ( - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/state/protocol" -) - -// NodeIDRefresher derives the latest list of flow identities with which the -// network should be communicating based on identity table changes in the -// protocol state. -type NodeIDRefresher struct { - logger zerolog.Logger - state protocol.State - callBack func(flow.IdentityList) error // callBack to call when the id list has changed -} - -func NewNodeIDRefresher(logger zerolog.Logger, state protocol.State, callBack func(list flow.IdentityList) error) *NodeIDRefresher { - return &NodeIDRefresher{ - logger: logger.With().Str("component", "network-refresher").Logger(), - state: state, - callBack: callBack, - } -} - -func (listener *NodeIDRefresher) getLogger(final protocol.Snapshot) zerolog.Logger { - - log := listener.logger - - // retrieve some contextual information for logging - head, err := final.Head() - if err != nil { - log.Error().Err(err).Msg("failed to get finalized header") - return log - } - log = log.With().Uint64("final_height", head.Height).Logger() - - phase, err := listener.state.Final().Phase() - if err != nil { - log.Error().Err(err).Msg("failed to get epoch phase") - return log - } - log = log.With().Str("epoch_phase", phase.String()).Logger() - - return log -} - -// OnIdentityTableChanged updates the networking layer's list of nodes to connect -// to when the identity table changes in the protocol state. -func (listener *NodeIDRefresher) OnIdentityTableChanged() { - - final := listener.state.Final() - log := listener.getLogger(final) - - log.Info().Msg("updating network ids upon identity table change") - - // get the new set of IDs - newIDs, err := final.Identities(NetworkingSetFilter) - if err != nil { - log.Err(err).Msg("failed to determine new identity table after identity table change") - return - } - - // call the registered callback - err = listener.callBack(newIDs) - if err != nil { - log.Err(err).Msg("failed to update network ids on identity table change") - return - } - - log.Info().Msg("successfully updated network ids upon identity table change") -} - -// NetworkingSetFilter is an identity filter that, when applied to the identity -// table at a given snapshot, returns all nodes that we should communicate with -// over the networking layer. -// -// NOTE: The protocol state includes nodes from the previous/next epoch that should -// be included in network communication. We omit any nodes that have been ejected. -var NetworkingSetFilter = filter.Not(filter.Ejected) diff --git a/network/p2p/peerManager.go b/network/p2p/peerManager.go index 71049031237..3145f73081f 100644 --- a/network/p2p/peerManager.go +++ b/network/p2p/peerManager.go @@ -5,20 +5,20 @@ import ( "fmt" "time" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" ) // Connector connects to peer and disconnects from peer using the underlying networking library type Connector interface { - // UpdatePeers connects to the given flow.Identities and returns a map of identifiers which failed. It also + // UpdatePeers connects to the given peer.IDs and returns a map of peers which failed. It also // disconnects from any other peers with which it may have previously established connection. // UpdatePeers implementation should be idempotent such that multiple calls to connect to the same peer should not // return an error or create multiple connections - UpdatePeers(ctx context.Context, ids flow.IdentityList) error + UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) } // DefaultPeerUpdateInterval is default duration for which the peer manager waits in between attempts to update peer connections @@ -28,10 +28,10 @@ var DefaultPeerUpdateInterval = 10 * time.Minute type PeerManager struct { unit *engine.Unit logger zerolog.Logger - idsProvider func() (flow.IdentityList, error) // callback to retrieve list of peers to connect to - peerRequestQ chan struct{} // a channel to queue a peer update request - connector Connector // connector to connect or disconnect from peers - peerUpdateInterval time.Duration // interval the peer manager runs on + peersProvider func() (peer.IDSlice, error) // callback to retrieve list of peers to connect to + peerRequestQ chan struct{} // a channel to queue a peer update request + connector Connector // connector to connect or disconnect from peers + peerUpdateInterval time.Duration // interval the peer manager runs on } // Option represents an option for the peer manager. @@ -43,16 +43,16 @@ func WithInterval(period time.Duration) Option { } } -// NewPeerManager creates a new peer manager which calls the idsProvider callback to get a list of peers to connect to +// NewPeerManager creates a new peer manager which calls the peersProvider callback to get a list of peers to connect to // and it uses the connector to actually connect or disconnect from peers. -func NewPeerManager(logger zerolog.Logger, idsProvider func() (flow.IdentityList, error), +func NewPeerManager(logger zerolog.Logger, peersProvider func() (peer.IDSlice, error), connector Connector, options ...Option) *PeerManager { pm := &PeerManager{ - unit: engine.NewUnit(), - logger: logger, - idsProvider: idsProvider, - connector: connector, - peerRequestQ: make(chan struct{}, 1), + unit: engine.NewUnit(), + logger: logger, + peersProvider: peersProvider, + connector: connector, + peerRequestQ: make(chan struct{}, 1), } // apply options for _, o := range options { @@ -101,25 +101,21 @@ func (pm *PeerManager) RequestPeerUpdate() { } } -// updatePeers updates the peers by connecting to all the nodes provided by the idsProvider callback and disconnecting from +// updatePeers updates the peers by connecting to all the nodes provided by the peersProvider callback and disconnecting from // previous nodes that are no longer in the new list of nodes. func (pm *PeerManager) updatePeers() { - // get all the ids to connect to - ids, err := pm.idsProvider() + // get all the peer ids to connect to + peers, err := pm.peersProvider() if err != nil { pm.logger.Error().Err(err).Msg("failed to update peers") return } pm.logger.Trace(). - Str("peers", fmt.Sprintf("%v", ids.NodeIDs())). + Str("peers", fmt.Sprintf("%v", peers)). Msg("connecting to peers") // ask the connector to connect to all peers in the list - err = pm.connector.UpdatePeers(pm.unit.Ctx(), ids) - if err != nil { - // one of more identities in the identity table could not be connected to - pm.logger.Error().Err(err).Msg("failed to connect to one or more peers") - } + pm.connector.UpdatePeers(pm.unit.Ctx(), peers) } diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go new file mode 100644 index 00000000000..cea16d466a9 --- /dev/null +++ b/network/p2p/peerstore_provider.go @@ -0,0 +1,32 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/host" + + "github.com/onflow/flow-go/model/flow" +) + +type PeerstoreIdentifierProvider struct { + host host.Host + idTranslator IDTranslator +} + +func NewPeerstoreIdentifierProvider(host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{host: host, idTranslator: idTranslator} +} + +func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { + var result flow.IdentifierList + + pids := p.host.Peerstore().PeersWithAddrs() // TODO: should we just call Peers here? + for _, pid := range pids { + flowID, err := p.idTranslator.GetFlowID(pid) + if err != nil { + // TODO: log error + } else { + result = append(result, flowID) + } + } + + return result +} diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go new file mode 100644 index 00000000000..d47542c3b18 --- /dev/null +++ b/network/p2p/protocol_state_provider.go @@ -0,0 +1,130 @@ +package p2p + +import ( + "fmt" + "sync" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" +) + +type ProtocolStateIDCache struct { + events.Noop + identities flow.IdentityList + state protocol.State + mu sync.RWMutex + peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + flowIDs map[peer.ID]flow.Identifier +} + +func NewProtocolStateIDCache( + state protocol.State, + eventDistributer *events.Distributor, +) (*ProtocolStateIDCache, error) { + provider := &ProtocolStateIDCache{ + state: state, + } + + head, err := state.Final().Head() + if err != nil { + return nil, err // TODO: format the error + } + + provider.update(head.ID()) + eventDistributer.AddConsumer(provider) + + return provider, nil +} + +func (p *ProtocolStateIDCache) EpochTransition(_ uint64, header *flow.Header) { + // TODO: maybe we actually want to log the epoch information from the arguments here (epoch phase, etc) + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { + // TODO: log status here + identities, err := p.state.AtBlockID(blockID).Identities(filter.Any) + if err != nil { + // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + } + + nIds := identities.Count() + + peerIDs := make(map[flow.Identifier]peer.ID, nIds) + flowIDs := make(map[peer.ID]flow.Identifier, nIds) + + for _, identity := range identities { + pid, err := ExtractPeerID(identity.NetworkPubKey) + if err != nil { + // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + } + + flowIDs[pid] = identity.NodeID + peerIDs[identity.NodeID] = pid + } + + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities + p.flowIDs = flowIDs + p.peerIDs = peerIDs +} + +func (p *ProtocolStateIDCache) Identities(filter flow.IdentityFilter) flow.IdentityList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.Filter(filter) +} + +func (p *ProtocolStateIDCache) GetPeerID(flowID flow.Identifier) (pid peer.ID, err error) { + p.mu.RLock() + defer p.mu.RUnlock() + + pid, found := p.peerIDs[flowID] + if !found { + err = fmt.Errorf("flow ID %v was not found in cached identity list", flowID) + } + + return +} + +func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, err error) { + p.mu.RLock() + defer p.mu.RUnlock() + + fid, found := p.flowIDs[peerID] + if !found { + err = fmt.Errorf("peer ID %v was not found in cached identity list", peerID) + } + + return +} + +func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { + pk, err := PublicKey(networkPubKey) + if err != nil { + // TODO: format the error + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + // TODO: format the error + return + } + + return +} diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go new file mode 100644 index 00000000000..5e320ccb236 --- /dev/null +++ b/network/p2p/unstaked_translator.go @@ -0,0 +1,43 @@ +package p2p + +import ( + crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/multiformats/go-multihash" + + "github.com/onflow/flow-go/model/flow" +) + +type UnstakedNetworkIDTranslator struct{} + +func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { + return &UnstakedNetworkIDTranslator{} +} + +func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + data := append([]byte{0x02}, flowID[:]...) + mh, err := multihash.Sum(data, multihash.IDENTITY, -1) + if err != nil { + // TODO: return error + } + + return peer.ID(mh), nil +} + +func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { + pk, err := peerID.ExtractPublicKey() + if err != nil { + // return error + } + + if pk.Type() != crypto_pb.KeyType_ECDSA { + // fail + } + + data, err := pk.Raw() + if err != nil || data[0] != 0x02 { // TODO: check if this is the right byte to check + // fail + } + + return flow.HashToID(data[1:]), nil +} diff --git a/network/topology.go b/network/topology.go index 1d746c70896..1d0faf16212 100644 --- a/network/topology.go +++ b/network/topology.go @@ -6,9 +6,9 @@ import ( // Topology provides a subset of nodes which a given node should directly connect to for 1-k messaging. type Topology interface { - // GenerateFanout receives IdentityList of entire network, and list of channels the node is subscribing to. - // It constructs and returns the fanout IdentityList of node. - // A node directly communicates with its fanout IdentityList on epidemic dissemination + // GenerateFanout receives IdentifierList of entire network, and list of channels the node is subscribing to. + // It constructs and returns the fanout IdentifierList of node. + // A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. @@ -19,5 +19,5 @@ type Topology interface { // // GenerateFanout is not concurrency safe. It is responsibility of caller to lock for it. // with the channels argument, it allows the returned topology to be cached, which is necessary for randomized topology. - GenerateFanout(ids flow.IdentityList, channels ChannelList) (flow.IdentityList, error) + GenerateFanout(ids flow.IdentifierList, channels ChannelList) (flow.IdentifierList, error) } diff --git a/network/topology/cache.go b/network/topology/cache.go index c305140abed..a738e9e9453 100644 --- a/network/topology/cache.go +++ b/network/topology/cache.go @@ -19,10 +19,10 @@ import ( // in a concurrency safe way, i.e., the caller should lock for it. type Cache struct { log zerolog.Logger - top network.Topology // instance of underlying topology. - cachedFanout flow.IdentityList // most recently generated fanout list by invoking underlying topology. - idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. - chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. + top network.Topology // instance of underlying topology. + cachedFanout flow.IdentifierList // most recently generated fanout list by invoking underlying topology. + idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. + chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. } //NewCache creates and returns a topology Cache given an instance of topology implementation. @@ -36,19 +36,19 @@ func NewCache(log zerolog.Logger, top network.Topology) *Cache { } } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList // of this instance. // It caches the most recently generated fanout list, so as long as the input list is the same, it returns // the same output. It invalidates and updates its internal cache the first time input list changes. -// A node directly communicates with its fanout IdentityList on epidemic dissemination +// A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. // // Note that this implementation of GenerateFanout preserves same output as long as input is the same. This // should not be assumed as a 1-1 mapping between input and output. -func (c *Cache) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { - inputIdsFP := ids.Fingerprint() +func (c *Cache) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { + inputIdsFP := flow.MerkleRoot(ids...) inputChansFP := channels.ID() log := c.log.With(). diff --git a/network/topology/fixedListTopology.go b/network/topology/fixedListTopology.go index 1b171ce8919..6091da61d38 100644 --- a/network/topology/fixedListTopology.go +++ b/network/topology/fixedListTopology.go @@ -2,7 +2,7 @@ package topology import ( "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" "github.com/onflow/flow-go/network" ) @@ -17,14 +17,14 @@ func NewFixedListTopology(nodeID flow.Identifier) FixedListTopology { } } -func (r FixedListTopology) GenerateFanout(ids flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { - return ids.Filter(filter.HasNodeID(r.fixedNodeID)), nil +func (r FixedListTopology) GenerateFanout(ids flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { + return ids.Filter(idFilter.Is(r.fixedNodeID)), nil } // EmptyListTopology always returns an empty list as the fanout type EmptyListTopology struct { } -func (r EmptyListTopology) GenerateFanout(_ flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { - return flow.IdentityList{}, nil +func (r EmptyListTopology) GenerateFanout(_ flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { + return flow.IdentifierList{}, nil } diff --git a/network/topology/randomizedTopology.go b/network/topology/randomizedTopology.go index c1bc959ed9e..55be3545025 100644 --- a/network/topology/randomizedTopology.go +++ b/network/topology/randomizedTopology.go @@ -9,6 +9,8 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -17,15 +19,16 @@ import ( // By random topology we mean a node is connected to any other co-channel nodes with some // edge probability. type RandomizedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - chance uint64 // used to translate connectedness probability into a number in [0, 100] - rng random.Rand // used as a stateful random number generator to sample edges - logger zerolog.Logger + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + chance uint64 // used to translate connectedness probability into a number in [0, 100] + rng random.Rand // used as a stateful random number generator to sample edges + logger zerolog.Logger + idProvider id.IdentityProvider } // NewRandomizedTopology returns an instance of the RandomizedTopology. -func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { +func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { // edge probability should be a positive value between 0 and 1. However, // we like it to be strictly greater than zero. Also, at the current scale of // Flow, we need it to be greater than 0.01 to support probabilistic connectedness. @@ -44,33 +47,34 @@ func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgePr } t := &RandomizedTopology{ - myNodeID: nodeID, - state: state, - chance: uint64(100 * edgeProb), - rng: rng, - logger: logger.With().Str("component:", "randomized-topology").Logger(), + myNodeID: nodeID, + state: state, + chance: uint64(100 * edgeProb), + rng: rng, + logger: logger.With().Str("component:", "randomized-topology").Logger(), + idProvider: idProvider, } return t, nil } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList -// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. This should be done with a very high probability // in randomized topology. -func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { +func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. r.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } - var myFanout flow.IdentityList + var myFanout flow.IdentifierList // generates a randomized subgraph per channel for _, myChannel := range myUniqueChannels { @@ -90,12 +94,12 @@ func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels netwo return myFanout, nil } -// subsetChannel returns a random subset of the identity list that is passed. -// Returned identities should all subscribed to the specified `channel`. -// Note: this method should not include identity of its executor. -func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +// subsetChannel returns a random subset of the identifier list that is passed. +// Returned identifiers should all subscribed to the specified `channel`. +// Note: this method should not include identifier of its executor. +func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { // excludes node itself - sampleSpace := ids.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) + sampleSpace := ids.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) // samples a random graph based on whether channel is cluster-based or not. if _, ok := engine.ClusterChannel(channel); ok { @@ -109,12 +113,12 @@ func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network // Independent invocations of this method over different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityList, error) { +func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.IdentifierList, error) { if len(ids) == 0 { - return nil, fmt.Errorf("empty identity list") + return nil, fmt.Errorf("empty identifier list") } - fanout := flow.IdentityList{} + fanout := flow.IdentifierList{} for _, id := range ids { // tosses a biased coin and adds id to fanout accordingly. // biased coin follows the edge probability distribution. @@ -127,7 +131,7 @@ func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityLi } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.IdentityList, error) { +func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow.IdentifierList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(r.myNodeID, r.state) if err != nil { @@ -135,10 +139,10 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.I } // excludes node itself from cluster - clusterPeers = clusterPeers.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) + clusterPeers = clusterPeers.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) + nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } @@ -148,7 +152,7 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.I } // clusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. -func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -160,7 +164,16 @@ func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, chan } // samples fanout among interacting roles - return r.sampleFanout(ids.Filter(filter.HasRole(roles...))) + return r.sampleFanout( + r.idProvider. + Identities( + filter.And( + filter.HasNodeID(ids...), + filter.HasRole(roles...), + ), + ). + NodeIDs(), + ) } // tossBiasedBit returns true with probability equal `r.chance/100`, and returns false otherwise. diff --git a/network/topology/topicBasedTopology.go b/network/topology/topicBasedTopology.go index 9d14e34049b..fdc55f43b1e 100644 --- a/network/topology/topicBasedTopology.go +++ b/network/topology/topicBasedTopology.go @@ -8,6 +8,8 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -15,14 +17,15 @@ import ( // TopicBasedTopology is a deterministic topology mapping that creates a connected graph component among the nodes // involved in each topic. type TopicBasedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - logger zerolog.Logger - seed int64 + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + logger zerolog.Logger + seed int64 + identityProvider id.IdentityProvider } // NewTopicBasedTopology returns an instance of the TopicBasedTopology. -func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { +func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { seed, err := intSeedFromID(nodeID) if err != nil { return nil, fmt.Errorf("could not generate seed from id:%w", err) @@ -38,19 +41,19 @@ func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state return t, nil } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList -// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. -func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { +func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. t.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } // finds all interacting roles with this node @@ -64,7 +67,7 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels netwo } // builds a connected component per role this node interact with, - var myFanout flow.IdentityList + var myFanout flow.IdentifierList for _, role := range myInteractingRoles { if role == flow.RoleCollection { // we do not build connected component for collection nodes based on their role @@ -98,32 +101,36 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels netwo return myFanout, nil } -// subsetChannel returns a random subset of the identity list that is passed. `shouldHave` represents set of -// identities that should be included in the returned subset. -// Returned identities should all subscribed to the specified `channel`. -// Note: this method should not include identity of its executor. -func (t *TopicBasedTopology) subsetChannel(ids flow.IdentityList, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +// subsetChannel returns a random subset of the identifier list that is passed. `shouldHave` represents set of +// identifiers that should be included in the returned subset. +// Returned identifiers should all subscribed to the specified `channel`. +// Note: this method should not include identifier of its executor. +func (t *TopicBasedTopology) subsetChannel(ids flow.IdentifierList, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return t.clusterChannelHandler(ids, shouldHave) } return t.nonClusterChannelHandler(ids, shouldHave, channel) } -// subsetRole returns a random subset of the identity list that is passed. `shouldHave` represents set of -// identities that should be included in the returned subset. -// Returned identities should all be of one of the specified `roles`. -// Note: this method should not include identity of its executor. -func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.IdentityList, roles flow.RoleList) (flow.IdentityList, error) { +// subsetRole returns a random subset of the identifier list that is passed. `shouldHave` represents set of +// identifiers that should be included in the returned subset. +// Returned identifiers should all be of one of the specified `roles`. +// Note: this method should not include the identifier of its executor. +func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow.IdentifierList, roles flow.RoleList) (flow.IdentifierList, error) { // excludes irrelevant roles and the node itself from both should have and ids set - shouldHave = shouldHave.Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )) - - ids = ids.Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )) + shouldHave = t.identityProvider. + Identities(filter.HasNodeID(shouldHave...)). + Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )).NodeIDs() + + ids = t.identityProvider. + Identities(filter.HasNodeID(ids...)). + Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )).NodeIDs() sample, err := t.sampleConnectedGraph(ids, shouldHave) if err != nil { @@ -138,11 +145,11 @@ func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.Id // different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHave flow.IdentityList) (flow.IdentityList, error) { +func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { if len(all) == 0 { t.logger.Debug().Msg("skips sampling connected graph with zero nodes") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } if len(shouldHave) == 0 { @@ -153,9 +160,9 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa } // checks `shouldHave` be a subset of `all` - nonMembers := shouldHave.Filter(filter.Not(filter.In(all))) + nonMembers := shouldHave.Filter(idFilter.Not(idFilter.In(all...))) if len(nonMembers) != 0 { - return nil, fmt.Errorf("should have identities is not a subset of all: %v", nonMembers) + return nil, fmt.Errorf("should have identifiers is not a subset of all: %v", nonMembers) } // total sample size @@ -170,7 +177,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa subsetSize := totalSize - len(shouldHave) // others are all excluding should have ones - others := all.Filter(filter.Not(filter.In(shouldHave))) + others := all.Filter(idFilter.Not(idFilter.In(shouldHave...))) others = others.DeterministicSample(uint(subsetSize), t.seed) return others.Union(shouldHave), nil @@ -178,7 +185,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityList) (flow.IdentityList, error) { +func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(t.myNodeID, t.state) if err != nil { @@ -186,18 +193,18 @@ func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityL } // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) + nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } // samples a connected graph topology from the cluster peers - return t.subsetRole(clusterPeers, shouldHave.Filter(filter.HasNodeID(clusterPeers.NodeIDs()...)), flow.RoleList{flow.RoleCollection}) + return t.subsetRole(clusterPeers, shouldHave.Filter(idFilter.In(clusterPeers...)), flow.RoleList{flow.RoleCollection}) } // nonClusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. // The returned sample contains `shouldHave` ones that also subscribed to `channel`. -func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -209,5 +216,5 @@ func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.Identi } // samples a connected graph topology - return t.subsetRole(ids.Filter(filter.HasRole(roles...)), shouldHave, roles) + return t.subsetRole(ids, shouldHave, roles) } diff --git a/network/topology/topology_utils.go b/network/topology/topology_utils.go index bf1d2b43986..3668d8557a6 100644 --- a/network/topology/topology_utils.go +++ b/network/topology/topology_utils.go @@ -51,7 +51,7 @@ func byteSeedFromID(id flow.Identifier) ([]byte, error) { } // clusterPeers returns the list of other nodes within the same cluster as specified identifier. -func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, error) { +func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList, error) { currentEpoch := state.Final().Epochs().Current() clusterList, err := currentEpoch.Clustering() if err != nil { @@ -63,5 +63,5 @@ func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, return nil, fmt.Errorf("failed to find the cluster for node ID %s", id.String()) } - return myCluster, nil + return myCluster.NodeIDs(), nil } diff --git a/utils/grpc/grpc.go b/utils/grpc/grpc.go index 2fe22c59bde..b22144a7d40 100644 --- a/utils/grpc/grpc.go +++ b/utils/grpc/grpc.go @@ -7,7 +7,6 @@ import ( "fmt" lcrypto "github.com/libp2p/go-libp2p-core/crypto" - "github.com/libp2p/go-libp2p-core/peer" libp2ptls "github.com/libp2p/go-libp2p-tls" "github.com/onflow/flow-go/crypto" @@ -102,13 +101,9 @@ func DefaultClientTLSConfig(publicKey crypto.PublicKey) (*tls.Config, error) { func verifyPeerCertificateFunc(expectedPublicKey crypto.PublicKey) (func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error, error) { // convert the Flow.crypto key to LibP2P key for easy comparision using LibP2P TLS utils - expectedLibP2PKey, err := p2p.LibP2PPublicKeyFromFlow(expectedPublicKey) + remotePeerLibP2PID, err := p2p.ExtractPeerID(expectedPublicKey) if err != nil { - return nil, fmt.Errorf("failed to generate a libp2p key from a Flow key: %w", err) - } - remotePeerLibP2PID, err := peer.IDFromPublicKey(expectedLibP2PKey) - if err != nil { - return nil, fmt.Errorf("failed to derive the libp2p Peer ID from the libp2p public key: %w", err) + return nil, fmt.Errorf("failed to derive the libp2p Peer ID from the Flow key: %w", err) } // We're using InsecureSkipVerify, so the verifiedChains parameter will always be empty. From 847e0f24904c33b094637789ca0924888ca430c8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 13:52:26 -0700 Subject: [PATCH 106/291] add dht peer lookup --- .../node_builder/access_node_builder.go | 134 +++++++++++------- cmd/scaffold.go | 2 + network/p2p/dht.go | 6 +- network/p2p/libp2pNode.go | 115 +++++++++++---- 4 files changed, 176 insertions(+), 81 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 403cf14baea..cfb6e5f71a4 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2,17 +2,18 @@ package node_builder import ( "context" + "encoding/json" "fmt" "strings" "time" + dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow/protobuf/go/flow/access" "github.com/spf13/pflag" "google.golang.org/grpc" "google.golang.org/grpc/credentials" "github.com/onflow/flow-go/cmd" - "github.com/onflow/flow-go/cmd/build" "github.com/onflow/flow-go/consensus" "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/committees" @@ -43,6 +44,7 @@ import ( "github.com/onflow/flow-go/network" jsoncodec "github.com/onflow/flow-go/network/codec/json" "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network/validator" "github.com/onflow/flow-go/state/protocol" badgerState "github.com/onflow/flow-go/state/protocol/badger" "github.com/onflow/flow-go/state/protocol/blocktimer" @@ -85,7 +87,9 @@ type AccessNodeBuilder interface { // while for a node running as a library, the config fields are expected to be initialized by the caller. type AccessNodeConfig struct { staked bool - stakedAccessNodeIDHex string + bootstrapNodeAddresses []string + bootstrapNodePublicKeys []string + bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes unstakedNetworkBindAddr string collectionGRPCPort uint executionGRPCPort uint @@ -132,7 +136,8 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { apiRatelimits: nil, apiBurstlimits: nil, staked: true, - stakedAccessNodeIDHex: "", + bootstrapNodeAddresses: []string{}, + bootstrapNodePublicKeys: []string{}, unstakedNetworkBindAddr: cmd.NotSet, } } @@ -438,7 +443,6 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, - node.Storage.Results, node.RootChainID, anb.TransactionMetrics, anb.collectionGRPCPort, @@ -467,7 +471,7 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { return nil, fmt.Errorf("could not create requester engine: %w", err) } - anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Results, node.Storage.Receipts, anb.TransactionMetrics, + anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, anb.TransactionMetrics, anb.CollectionsToMarkFinalized, anb.CollectionsToMarkExecuted, anb.BlocksToMarkExecuted, anb.RpcEng) anb.RequestEng.WithHandle(anb.IngestEng.OnCollection) anb.FinalizationDistributor.AddConsumer(anb.IngestEng) @@ -486,9 +490,9 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { type Option func(*AccessNodeConfig) -func WithUpstreamAccessNodeID(upstreamAccessNodeID flow.Identifier) Option { +func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { return func(config *AccessNodeConfig) { - config.stakedAccessNodeIDHex = upstreamAccessNodeID.String() + config.bootstrapIdentites = bootstrapNodes } } @@ -566,50 +570,42 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.StringToIntVar(&builder.apiRatelimits, "api-rate-limits", defaultConfig.apiRatelimits, "per second rate limits for Access API methods e.g. Ping=300,GetTransaction=500 etc.") flags.StringToIntVar(&builder.apiBurstlimits, "api-burst-limits", defaultConfig.apiBurstlimits, "burst limits for Access API methods e.g. Ping=100,GetTransaction=100 etc.") flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") - flags.StringVar(&builder.stakedAccessNodeIDHex, "staked-access-node-id", defaultConfig.stakedAccessNodeIDHex, "the node ID of the upstream staked access node if this is an unstaked access node") + flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") + flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") flags.StringVar(&builder.unstakedNetworkBindAddr, "unstaked-bind-addr", defaultConfig.unstakedNetworkBindAddr, "address to bind on for the unstaked network") }) } // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. // The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance -func (builder *FlowAccessNodeBuilder) initLibP2PFactory( - ctx context.Context, +func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, nodeID flow.Identifier, - networkMetrics module.NetworkMetrics, - networkKey crypto.PrivateKey, -) (p2p.LibP2PFactoryFunc, error) { + networkKey crypto.PrivateKey) (p2p.LibP2PFactoryFunc, error) { - // setup the Ping provider to return the software version and the sealed block height - pingProvider := p2p.PingInfoProviderImpl{ - SoftwareVersionFun: func() string { - return build.Semver() - }, - SealedBlockHeightFun: func() (uint64, error) { - head, err := builder.State.Sealed().Head() - if err != nil { - return 0, err - } - return head.Height, nil - }, - } + // The staked nodes act as the DHT servers + dhtOptions := []dht.Option{p2p.AsServer(builder.IsStaked())} - libP2PNodeFactory, err := p2p.DefaultLibP2PNodeFactory( - ctx, - builder.Logger, - nodeID, - builder.unstakedNetworkBindAddr, - networkKey, - builder.RootBlock.ID().String(), - p2p.DefaultMaxPubSubMsgSize, - networkMetrics, - pingProvider, - ) - if err != nil { - return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) + // if this is an unstaked access node, then seed the DHT with the boostrap identities + if !builder.IsStaked() { + bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentites) + builder.MustNot(err) + dhtOptions = append(dhtOptions, bootstrapPeersOpt) } - return libP2PNodeFactory, nil + return func() (*p2p.Node, error) { + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.unstakedNetworkBindAddr, networkKey). + SetRootBlockID(builder.RootBlock.ID().String()). + // unlike the staked network where currently all the node addresses are known upfront, + // for the unstaked network the nodes need to discover each other using DHT Discovery. + SetDHTOptions(dhtOptions...). + SetLogger(builder.Logger). + Build(ctx) + if err != nil { + return nil, err + } + builder.UnstakedLibP2PNode = libp2pNode + return builder.UnstakedLibP2PNode, nil + }, nil } // initMiddleware creates the network.Middleware implementation with the libp2p factory function, metrics, peer update @@ -617,20 +613,16 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory( func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - peerUpdateInterval time.Duration, - unicastMessageTimeout time.Duration, - connectionGating bool, - managerPeerConnections bool, validators ...network.MessageValidator) *p2p.Middleware { builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, factoryFunc, nodeID, networkMetrics, builder.RootBlock.ID().String(), - peerUpdateInterval, - unicastMessageTimeout, - connectionGating, - managerPeerConnections, + time.Hour, // TODO: this is pretty meaningless since there is no peermanager in play. + p2p.DefaultUnicastTimeout, + false, // no connection gating for the unstaked network + false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) builder.IDTranslator, validators...) return builder.unstakedMiddleware @@ -666,3 +658,49 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, return net, nil } + +func unstakedNetworkMsgValidators(selfID flow.Identifier) []network.MessageValidator { + return []network.MessageValidator{ + // filter out messages sent by this node itself + validator.ValidateNotSender(selfID), + } +} + +// BootstrapIdentities converts the bootstrap node addresses and keys to a Flow Identity list where +// each Flow Identity is initialized with the passed address, the networking key +// and the Node ID set to ZeroID, role set to Access, 0 stake and no staking key. +func BootstrapIdentities(addresses []string, keys []string) (flow.IdentityList, error) { + + if len(addresses) != len(keys) { + return nil, fmt.Errorf("number of addresses and keys provided for the boostrap nodes don't match") + } + + ids := make([]*flow.Identity, len(addresses)) + for i, address := range addresses { + + key := keys[i] + // json unmarshaller needs a quotes before and after the string + // the pflags.StringSliceVar does not retain quotes for the command line arg even if escaped with \" + // hence this additional check to ensure the key is indeed quoted + if !strings.HasPrefix(key, "\"") { + key = fmt.Sprintf("\"%s\"", key) + } + // networking public key + var networkKey encodable.NetworkPubKey + err := json.Unmarshal([]byte(key), &networkKey) + if err != nil { + return nil, err + } + + // create the identity of the peer by setting only the relevant fields + id := &flow.Identity{ + NodeID: flow.ZeroID, // the NodeID is the hash of the staking key and for the unstaked network it does not apply + Address: address, + Role: flow.RoleAccess, // the upstream node has to be an access node + NetworkPubKey: networkKey, + } + + ids = append(ids, id) + } + return ids, nil +} diff --git a/cmd/scaffold.go b/cmd/scaffold.go index dac6918b91f..45727f1c7e1 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -211,6 +211,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) fnb.ProtocolEvents.AddConsumer(idEvents) + // TODO: add a consumer which implements setting the permanent peer addresses, + // and expiring the old permanent ones return net, err }) diff --git a/network/p2p/dht.go b/network/p2p/dht.go index 8cd702c5a9a..b21211a9adb 100644 --- a/network/p2p/dht.go +++ b/network/p2p/dht.go @@ -5,7 +5,6 @@ import ( "github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p-core/peer" - discovery "github.com/libp2p/go-libp2p-discovery" dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow-go/model/flow" @@ -13,7 +12,7 @@ import ( // This produces a new IPFS DHT // on the name, see https://github.com/libp2p/go-libp2p-kad-dht/issues/337 -func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*discovery.RoutingDiscovery, error) { +func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*dht.IpfsDHT, error) { defaultOptions := defaultDHTOptions() allOptions := append(defaultOptions, options...) @@ -27,8 +26,7 @@ func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*discov return nil, err } - routingDiscovery := discovery.NewRoutingDiscovery(kdht) - return routingDiscovery, nil + return kdht, nil } // DHT defaults to ModeAuto which will automatically switch the DHT between Server and Client modes based on diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index f5c974550af..561d2b57c81 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -17,6 +17,8 @@ import ( libp2pnet "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/protocol" + discovery "github.com/libp2p/go-libp2p-discovery" + dht "github.com/libp2p/go-libp2p-kad-dht" pubsub "github.com/libp2p/go-libp2p-pubsub" swarm "github.com/libp2p/go-libp2p-swarm" tptu "github.com/libp2p/go-libp2p-transport-upgrader" @@ -54,23 +56,12 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I connGater := NewConnGater(log) - // create PubSub options for libp2p to use - psOptions := []pubsub.Option{ - // skip message signing - pubsub.WithMessageSigning(false), - // skip message signature - pubsub.WithStrictSignatureVerification(false), - // set max message size limit for 1-k PubSub messaging - pubsub.WithMaxMessageSize(maxPubSubMsgSize), - // no discovery - } - return func() (*Node, error) { return NewDefaultLibP2PNodeBuilder(me, address, flowKey). SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(psOptions...). + SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize)...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). Build(ctx) @@ -81,8 +72,9 @@ type NodeBuilder interface { SetRootBlockID(string) NodeBuilder SetConnectionManager(TagLessConnManager) NodeBuilder SetConnectionGater(*ConnGater) NodeBuilder - SetPubsubOptions(...pubsub.Option) NodeBuilder + SetPubsubOptions(...PubsubOption) NodeBuilder SetPingInfoProvider(PingInfoProvider) NodeBuilder + SetDHTOptions(...dht.Option) NodeBuilder SetLogger(zerolog.Logger) NodeBuilder Build(context.Context) (*Node, error) } @@ -96,7 +88,8 @@ type DefaultLibP2PNodeBuilder struct { pingInfoProvider PingInfoProvider pubSubMaker func(context.Context, host.Host, ...pubsub.Option) (*pubsub.PubSub, error) hostMaker func(context.Context, ...config.Option) (host.Host, error) - pubSubOpts []pubsub.Option + pubSubOpts []PubsubOption + dhtOpts []dht.Option } func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcrypto.PrivateKey) NodeBuilder { @@ -111,6 +104,11 @@ func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcr } } +func (builder *DefaultLibP2PNodeBuilder) SetDHTOptions(opts ...dht.Option) NodeBuilder { + builder.dhtOpts = opts + return builder +} + func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId string) NodeBuilder { builder.rootBlockID = rootBlockId return builder @@ -126,7 +124,7 @@ func (builder *DefaultLibP2PNodeBuilder) SetConnectionGater(connGater *ConnGater return builder } -func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...pubsub.Option) NodeBuilder { +func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...PubsubOption) NodeBuilder { builder.pubSubOpts = opts return builder } @@ -174,6 +172,11 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.connMgr = builder.connMngr } + if builder.rootBlockID == "" { + return nil, errors.New("root block ID must be provided") + } + node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) + if builder.pingInfoProvider != nil { opts = append(opts, libp2p.Ping(true)) } @@ -184,13 +187,32 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro } node.host = libp2pHost + if len(builder.dhtOpts) != 0 { + kdht, err := NewDHT(ctx, node.host, builder.dhtOpts...) + if err != nil { + return nil, err + } + node.dht = kdht + builder.pubSubOpts = append(builder.pubSubOpts, WithDHTDiscovery(kdht)) + } + if builder.pingInfoProvider != nil { pingLibP2PProtocolID := generatePingProtcolID(builder.rootBlockID) pingService := NewPingService(libp2pHost, pingLibP2PProtocolID, builder.pingInfoProvider, node.logger) node.pingService = pingService } - ps, err := builder.pubSubMaker(ctx, libp2pHost, builder.pubSubOpts...) + var libp2pPSOptions []pubsub.Option + // generate the libp2p Pubsub options from the given context and host + for _, optionGenerator := range builder.pubSubOpts { + option, err := optionGenerator(ctx, libp2pHost) + if err != nil { + return nil, err + } + libp2pPSOptions = append(libp2pPSOptions, option) + } + + ps, err := builder.pubSubMaker(ctx, libp2pHost, libp2pPSOptions...) if err != nil { return nil, err } @@ -222,6 +244,7 @@ type Node struct { flowLibP2PProtocolID protocol.ID // the unique protocol ID pingService *PingService connMgr TagLessConnManager + dht *dht.IpfsDHT } // Stop stops the libp2p node. @@ -334,7 +357,22 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp } // remove the peer from the peer store if present - n.host.Peerstore().ClearAddrs(peerID) + // TODO: why were we doing this? + // n.host.Peerstore().ClearAddrs(peerID) + + if len(n.host.Peerstore().Addrs(peerID)) == 0 { + if n.dht != nil { + // TODO: adjust timeout + timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) + // try to find the peer + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + } + } + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() @@ -503,18 +541,7 @@ func (n *Node) UpdateAllowList(peers []peer.ID) { return } - // generates peer address information for all identities - allowlist := make([]peer.AddrInfo, len(identities)) - var err error - for i, identity := range identities { - allowlist[i], err = PeerAddressInfo(*identity) - if err != nil { - return fmt.Errorf("could not generate address info: %w", err) - } - } - - n.connGater.update(allowlist) - return nil + n.connGater.update(peers) } // Host returns pointer to host object of node. @@ -606,3 +633,33 @@ func DefaultPubSub(ctx context.Context, host host.Host, psOption ...pubsub.Optio } return pubSub, nil } + +// PubsubOption generates a libp2p pubsub.Option from the given context and host +type PubsubOption func(ctx context.Context, host host.Host) (pubsub.Option, error) + +func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { + pubSubOptionFunc := func(option pubsub.Option) PubsubOption { + return func(_ context.Context, _ host.Host) (pubsub.Option, error) { + return option, nil + } + } + return []PubsubOption{ + // skip message signing + pubSubOptionFunc(pubsub.WithMessageSigning(true)), + // skip message signature + pubSubOptionFunc(pubsub.WithStrictSignatureVerification(true)), + // set max message size limit for 1-k PubSub messaging + pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), + // no discovery + } +} + +func WithDHTDiscovery(option ...dht.Option) PubsubOption { + return func(ctx context.Context, host host.Host) (pubsub.Option, error) { + dhtDiscovery, err := NewDHT(ctx, host, option...) + if err != nil { + return nil, err + } + return pubsub.WithDiscovery(dhtDiscovery), nil + } + } From 9138a4f4c461108336588831ac1a29e85cf6a64f Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 15:42:48 -0700 Subject: [PATCH 107/291] Add dht lookup and default peerstore addresses --- .../unstaked_access_node_builder.go | 77 +++++++++++-------- cmd/scaffold.go | 7 +- network/middleware.go | 5 +- network/p2p/libp2pNode.go | 16 ++-- network/p2p/middleware.go | 18 ++++- network/p2p/network.go | 15 +++- 6 files changed, 89 insertions(+), 49 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 81c9f4600f7..88ce203fe65 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -2,8 +2,6 @@ package node_builder import ( "context" - "strings" - "time" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" @@ -11,7 +9,6 @@ import ( "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/topology" ) type UnstakedAccessNodeBuilder struct { @@ -27,6 +24,14 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + return nil }) } @@ -40,8 +45,12 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() + builder.deriveBootstrapPeerIdentities() + builder.enqueueUnstakedNetworkInit(ctx) + builder.enqueueConnectWithStakedAN() + builder.EnqueueMetricsServerInit() builder.RegisterBadgerMetrics() @@ -55,15 +64,22 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { func (builder *UnstakedAccessNodeBuilder) validateParams() { - // for an unstaked access node, the staked access node ID must be provided - if strings.TrimSpace(builder.stakedAccessNodeIDHex) == "" { - builder.Logger.Fatal().Msg("staked access node ID not specified") - } - - // and also the unstaked bind address + // for an unstaked access node, the unstaked network bind address must be provided if builder.unstakedNetworkBindAddr == cmd.NotSet { builder.Logger.Fatal().Msg("unstaked bind address not set") } + + if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { + builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") + } +} + +// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstreap peers from the parameters. +// These are the identity of the staked and unstaked AN also acting as the DHT bootstrap server +func (builder *UnstakedAccessNodeBuilder) deriveBootstrapPeerIdentities() { + ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) + builder.MustNot(err) + builder.bootstrapIdentites = ids } // initUnstakedLocal initializes the unstaked node ID, network key and network address @@ -111,35 +127,18 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - // intialize the LibP2P factory with an empty metrics NoopCollector for now till we have defined the new unstaked - // network metrics - libP2PFactory, err := builder.FlowAccessNodeBuilder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkMetrics, unstakedNetworkKey) + libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) builder.MustNot(err) - // use the default validators for the staked access node unstaked networks - msgValidators := p2p.DefaultValidators(builder.Logger, unstakedNodeID) + msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) - // don't need any peer updates since this will be taken care by the DHT discovery mechanism - peerUpdateInterval := time.Hour + middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, peerUpdateInterval, - node.UnicastMessageTimeout, - false, // no connection gating for the unstaked network - false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) - msgValidators...) + // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront + participants := flow.IdentityList{} - upstreamANIdentifier, err := flow.HexStringToIdentifier(builder.stakedAccessNodeIDHex) - builder.MustNot(err) - - // topology only consist of the upsteam staked AN - top := topology.NewFixedListTopology(upstreamANIdentifier) - - network, err := builder.initNetwork( - builder.Me, - unstakedNetworkMetrics, - middleware, - top, - ) + // topology is nil since its automatically managed by libp2p + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, nil) builder.MustNot(err) builder.UnstakedNetwork = network @@ -154,3 +153,15 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context return builder.UnstakedNetwork, err }) } + +// enqueueConnectWithStakedAN enqueues the upstream connector component which connects the libp2p host of the unstaked +// AN with the staked AN. +// Currently, there is an issue with LibP2P stopping advertisements of subscribed topics if no peers are connected +// (https://github.com/libp2p/go-libp2p-pubsub/issues/442). This means that an unstaked AN could end up not being +// discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need +// of an explicit connect to the staked AN before the node attempts to subscribe to topics. +func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { + builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return newUpstreamConnector(builder.bootstrapIdentites, builder.UnstakedLibP2PNode, builder.Logger), nil + }) +} diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 45727f1c7e1..a5061652daf 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -209,10 +209,11 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Network = net - idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) + idEvents := gadgets.NewIdentityDeltas(func() { + fnb.Middleware.UpdateNodeAddresses() + fnb.Middleware.UpdateAllowList() + }) fnb.ProtocolEvents.AddConsumer(idEvents) - // TODO: add a consumer which implements setting the permanent peer addresses, - // and expiring the old permanent ones return net, err }) diff --git a/network/middleware.go b/network/middleware.go index 82928584d43..8bd89167549 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -53,6 +53,8 @@ type Middleware interface { // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. UpdateAllowList() + + UpdateNodeAddresses() } // Overlay represents the interface that middleware uses to interact with the @@ -62,7 +64,8 @@ type Overlay interface { Topology() (flow.IdentifierList, error) Receive(nodeID flow.Identifier, msg *message.Message) error SetDefaultIdentifierProvider(id.IdentifierProvider) - GetIdentifierProvider() id.IdentifierProvider + Identifiers() flow.IdentifierList + Identities() flow.IdentityList } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 561d2b57c81..b6a687cad03 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -361,6 +361,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp // n.host.Peerstore().ClearAddrs(peerID) if len(n.host.Peerstore().Addrs(peerID)) == 0 { + // TODO: add bunch of logging here if n.dht != nil { // TODO: adjust timeout timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) @@ -654,12 +655,9 @@ func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { } } -func WithDHTDiscovery(option ...dht.Option) PubsubOption { - return func(ctx context.Context, host host.Host) (pubsub.Option, error) { - dhtDiscovery, err := NewDHT(ctx, host, option...) - if err != nil { - return nil, err - } - return pubsub.WithDiscovery(dhtDiscovery), nil - } - } +func WithDHTDiscovery(kdht *dht.IpfsDHT) PubsubOption { + return func(ctx context.Context, host host.Host) (pubsub.Option, error) { + routingDiscovery := discovery.NewRoutingDiscovery(kdht) + return pubsub.WithDiscovery(routingDiscovery), nil + } +} diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index f24ad9c54a3..ab9b8ed54a2 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -12,11 +12,13 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" @@ -79,6 +81,7 @@ type Middleware struct { connectionGating bool managePeerConnections bool idTranslator IDTranslator + idProvider id.IdentityProvider } // NewMiddleware creates a new middleware instance @@ -101,6 +104,7 @@ func NewMiddleware( connectionGating bool, managePeerConnections bool, idTranslator IDTranslator, + idProvider id.IdentityProvider, validators ...network.MessageValidator, ) *Middleware { ctx, cancel := context.WithCancel(context.Background()) @@ -125,6 +129,7 @@ func NewMiddleware( connectionGating: connectionGating, managePeerConnections: managePeerConnections, idTranslator: idTranslator, + idProvider: idProvider, } if len(validators) != 0 { @@ -152,7 +157,7 @@ func (m *Middleware) topologyPeers() (peer.IDSlice, error) { } func (m *Middleware) allPeers() peer.IDSlice { - return m.peerIDs(m.ov.GetIdentifierProvider().Identifiers()) + return m.peerIDs(m.ov.Identifiers()) } func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { @@ -180,6 +185,15 @@ func (m *Middleware) GetIPPort() (string, string, error) { return m.libP2PNode.GetIPPort() } +func (m *Middleware) UpdateNodeAddresses() { + ids := m.ov.Identities() + infos, _ := peerInfosFromIDs(ids) + + for _, info := range infos { + m.libP2PNode.host.Peerstore().SetAddrs(info.ID, info.Addrs, peerstore.PermanentAddrTTL) + } +} + // Start will start the middleware. func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov @@ -193,6 +207,8 @@ func (m *Middleware) Start(ov network.Overlay) error { m.libP2PNode = libP2PNode m.libP2PNode.SetFlowProtocolStreamHandler(m.handleIncomingStream) + m.UpdateNodeAddresses() + if m.connectionGating { m.libP2PNode.UpdateAllowList(m.allPeers()) } diff --git a/network/p2p/network.go b/network/p2p/network.go index 0584143e5c5..b87957774c4 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -43,6 +43,7 @@ type Network struct { sync.RWMutex idProvider id.IdentifierProvider defaultIdProvider id.IdentifierProvider + identityProvider id.IdentityProvider logger zerolog.Logger codec network.Codec me module.Local @@ -78,6 +79,7 @@ func NewNetwork( top network.Topology, sm network.SubscriptionManager, metrics module.NetworkMetrics, + identityProvider id.IdentityProvider, opts ...NetworkOption, ) (*Network, error) { @@ -96,6 +98,7 @@ func NewNetwork( metrics: metrics, subMngr: sm, lifecycleManager: lifecycle.NewLifecycleManager(), + identityProvider: identityProvider, } o.ctx, o.cancel = context.WithCancel(context.Background()) @@ -175,7 +178,7 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -func (n *Network) GetIdentifierProvider() id.IdentifierProvider { +func (n *Network) getIdentifierProvider() id.IdentifierProvider { if n.idProvider != nil { return n.idProvider } @@ -188,6 +191,14 @@ func (n *Network) GetIdentifierProvider() id.IdentifierProvider { return n.defaultIdProvider } +func (n *Network) Identifiers() flow.IdentifierList { + return n.getIdentifierProvider().Identifiers() +} + +func (n *Network) Identities() flow.IdentityList { + return n.identityProvider.Identities(NetworkingSetFilter) +} + // Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. func (n *Network) Topology() (flow.IdentifierList, error) { @@ -195,7 +206,7 @@ func (n *Network) Topology() (flow.IdentifierList, error) { defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.GetIdentifierProvider().Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } From 931b74f80a585aeff97ce108571d3348fd0c8b5c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 15:48:37 -0700 Subject: [PATCH 108/291] undo topology changes --- .../node_builder/access_node_builder.go | 13 ++- .../staked_access_node_builder.go | 1 + .../unstaked_access_node_builder.go | 8 +- cmd/access/node_builder/upstream_connector.go | 7 +- cmd/scaffold.go | 10 ++- network/middleware.go | 7 +- network/p2p/middleware.go | 38 ++++++--- network/p2p/network.go | 70 ++++----------- network/topology.go | 8 +- network/topology/cache.go | 16 ++-- network/topology/fixedListTopology.go | 10 +-- network/topology/randomizedTopology.go | 71 +++++++--------- network/topology/topicBasedTopology.go | 85 +++++++++---------- network/topology/topology_utils.go | 4 +- 14 files changed, 158 insertions(+), 190 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index cfb6e5f71a4..64c8480d79c 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -150,6 +150,7 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components + UnstakedLibP2PNode *p2p.Node UnstakedNetwork *p2p.Network unstakedMiddleware *p2p.Middleware FollowerState protocol.MutableState @@ -443,6 +444,7 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, + node.Storage.Results, node.RootChainID, anb.TransactionMetrics, anb.collectionGRPCPort, @@ -471,7 +473,7 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { return nil, fmt.Errorf("could not create requester engine: %w", err) } - anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, anb.TransactionMetrics, + anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Results, node.Storage.Receipts, anb.TransactionMetrics, anb.CollectionsToMarkFinalized, anb.CollectionsToMarkExecuted, anb.BlocksToMarkExecuted, anb.RpcEng) anb.RequestEng.WithHandle(anb.IngestEng.OnCollection) anb.FinalizationDistributor.AddConsumer(anb.IngestEng) @@ -614,7 +616,8 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, validators ...network.MessageValidator) *p2p.Middleware { - builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, + builder.unstakedMiddleware = p2p.NewMiddleware( + builder.Logger, factoryFunc, nodeID, networkMetrics, @@ -624,7 +627,9 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, false, // no connection gating for the unstaked network false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) builder.IDTranslator, - validators...) + p2p.WithMessageValidators(validators...), + // use default identifier provider + ) return builder.unstakedMiddleware } @@ -650,7 +655,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, topology, subscriptionManager, networkMetrics, - p2p.WithIdentifierProvider(builder.NetworkingIdentifierProvider), + builder.IdentityProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index d022c886700..a8e5a9731c1 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -11,6 +11,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" ) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 88ce203fe65..d2f9d8e02d0 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -105,7 +105,8 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - anb.SyncEngineParticipantsProvider = node.Network.GetIdentifierProvider() + // use the default identifier provider + anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() return nil }) anb.FlowAccessNodeBuilder.Build() @@ -134,11 +135,8 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - participants := flow.IdentityList{} - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, nil) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) builder.MustNot(err) builder.UnstakedNetwork = network diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index 0a4c3405dbd..aabfa446426 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -105,8 +105,13 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f return default: } + peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) + if err != nil { + // TODO: return formatted error + } + // try and connect to the bootstrap server - err := connector.unstakedNode.AddPeer(ctx, bootstrapPeer) + err = connector.unstakedNode.AddPeer(ctx, peerID) resultChan <- result{ id: bootstrapPeer, err: err, diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a5061652daf..4625f035e01 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -168,7 +168,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) } - fnb.Middleware = p2p.NewMiddleware(fnb.Logger.Level(zerolog.ErrorLevel), + fnb.Middleware = p2p.NewMiddleware( + fnb.Logger.Level(zerolog.ErrorLevel), libP2PNodeFactory, fnb.Me.NodeID(), fnb.Metrics.Network, @@ -178,12 +179,13 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { true, true, fnb.IDTranslator, - fnb.MsgValidators...) + p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + p2p.WithMessageValidators(fnb.MsgValidators...), + ) subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) top, err := topology.NewTopicBasedTopology( fnb.NodeID, - fnb.IdentityProvider, fnb.Logger, fnb.State, ) @@ -201,7 +203,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { topologyCache, subscriptionManager, fnb.Metrics.Network, - p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + fnb.IdentityProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) diff --git a/network/middleware.go b/network/middleware.go index 8bd89167549..b6d538da53a 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -6,7 +6,6 @@ import ( "time" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/message" ) @@ -60,11 +59,9 @@ type Middleware interface { // Overlay represents the interface that middleware uses to interact with the // overlay network layer. type Overlay interface { - // Topology returns an identifier list of nodes which this node should be directly connected to as peers - Topology() (flow.IdentifierList, error) + // Topology returns an identity list of nodes which this node should be directly connected to as peers + Topology() (flow.IdentityList, error) Receive(nodeID flow.Identifier, msg *message.Message) error - SetDefaultIdentifierProvider(id.IdentifierProvider) - Identifiers() flow.IdentifierList Identities() flow.IdentityList } diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index ab9b8ed54a2..e43d7674c08 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -81,7 +81,21 @@ type Middleware struct { connectionGating bool managePeerConnections bool idTranslator IDTranslator - idProvider id.IdentityProvider + idProvider id.IdentifierProvider +} + +type MiddlewareOption func(*Middleware) + +func WithIdentifierProvider(provider id.IdentifierProvider) MiddlewareOption { + return func(mw *Middleware) { + mw.idProvider = provider + } +} + +func WithMessageValidators(validators ...network.MessageValidator) MiddlewareOption { + return func(mw *Middleware) { + mw.validators = validators + } } // NewMiddleware creates a new middleware instance @@ -104,8 +118,7 @@ func NewMiddleware( connectionGating bool, managePeerConnections bool, idTranslator IDTranslator, - idProvider id.IdentityProvider, - validators ...network.MessageValidator, + opts ...MiddlewareOption, ) *Middleware { ctx, cancel := context.WithCancel(context.Background()) @@ -129,11 +142,10 @@ func NewMiddleware( connectionGating: connectionGating, managePeerConnections: managePeerConnections, idTranslator: idTranslator, - idProvider: idProvider, } - if len(validators) != 0 { - mw.validators = validators + for _, opt := range opts { + opt(mw) } return mw @@ -147,17 +159,17 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes } func (m *Middleware) topologyPeers() (peer.IDSlice, error) { - identifiers, err := m.ov.Topology() + identities, err := m.ov.Topology() if err != nil { // TODO: format error return nil, err } - return m.peerIDs(identifiers), nil + return m.peerIDs(identities.NodeIDs()), nil } func (m *Middleware) allPeers() peer.IDSlice { - return m.peerIDs(m.ov.Identifiers()) + return m.peerIDs(m.idProvider.Identifiers()) } func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { @@ -199,7 +211,9 @@ func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov libP2PNode, err := m.libP2PNodeFactory() - ov.SetDefaultIdentifierProvider(NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator)) + if m.idProvider == nil { + m.idProvider = NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator) + } if err != nil { return fmt.Errorf("could not create libp2p node: %w", err) @@ -446,6 +460,10 @@ func (m *Middleware) UpdateAllowList() { m.peerManagerUpdate() } +func (m *Middleware) IdentifierProvider() id.IdentifierProvider { + return m.idProvider +} + // IsConnected returns true if this node is connected to the node with id nodeID. func (m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { peerID, err := m.idTranslator.GetPeerID(nodeID) diff --git a/network/p2p/network.go b/network/p2p/network.go index b87957774c4..82235bd4938 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -41,29 +41,19 @@ type ReadyDoneAwareNetwork interface { // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { sync.RWMutex - idProvider id.IdentifierProvider - defaultIdProvider id.IdentifierProvider - identityProvider id.IdentityProvider - logger zerolog.Logger - codec network.Codec - me module.Local - mw network.Middleware - top network.Topology // used to determine fanout connections - metrics module.NetworkMetrics - rcache *RcvCache // used to deduplicate incoming messages - queue network.MessageQueue - ctx context.Context - cancel context.CancelFunc - subMngr network.SubscriptionManager // used to keep track of subscribed channels - lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle -} - -type NetworkOption func(*Network) - -func WithIdentifierProvider(provider id.IdentifierProvider) NetworkOption { - return func(net *Network) { - net.idProvider = provider - } + identityProvider id.IdentityProvider + logger zerolog.Logger + codec network.Codec + me module.Local + mw network.Middleware + top network.Topology // used to determine fanout connections + metrics module.NetworkMetrics + rcache *RcvCache // used to deduplicate incoming messages + queue network.MessageQueue + ctx context.Context + cancel context.CancelFunc + subMngr network.SubscriptionManager // used to keep track of subscribed channels + lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -80,7 +70,6 @@ func NewNetwork( sm network.SubscriptionManager, metrics module.NetworkMetrics, identityProvider id.IdentityProvider, - opts ...NetworkOption, ) (*Network, error) { rcache, err := newRcvCache(csize) @@ -109,10 +98,6 @@ func NewNetwork( // create workers to read from the queue and call queueSubmitFunc queue.CreateQueueWorkers(o.ctx, queue.DefaultNumWorkers, o.queue, o.queueSubmitFunc) - for _, opt := range opts { - opt(o) - } - return o, nil } @@ -178,35 +163,18 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -func (n *Network) getIdentifierProvider() id.IdentifierProvider { - if n.idProvider != nil { - return n.idProvider - } - n.RLock() - defer n.RUnlock() - if n.defaultIdProvider == nil { - n.logger.Fatal().Msg("TODO") - // TODO - } - return n.defaultIdProvider -} - -func (n *Network) Identifiers() flow.IdentifierList { - return n.getIdentifierProvider().Identifiers() -} - func (n *Network) Identities() flow.IdentityList { return n.identityProvider.Identities(NetworkingSetFilter) } -// Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. +// Topology returns the identitiess of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. -func (n *Network) Topology() (flow.IdentifierList, error) { +func (n *Network) Topology() (flow.IdentityList, error) { n.Lock() defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.Identities(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } @@ -313,12 +281,6 @@ func (n *Network) genNetworkMessage(channel network.Channel, event interface{}, return msg, nil } -func (n *Network) SetDefaultIdentifierProvider(provider id.IdentifierProvider) { - n.Lock() - n.defaultIdProvider = provider - n.Unlock() -} - // unicast sends the message in a reliable way to the given recipient. // It uses 1-1 direct messaging over the underlying network to deliver the message. // It returns an error if unicasting fails. diff --git a/network/topology.go b/network/topology.go index 1d0faf16212..1d746c70896 100644 --- a/network/topology.go +++ b/network/topology.go @@ -6,9 +6,9 @@ import ( // Topology provides a subset of nodes which a given node should directly connect to for 1-k messaging. type Topology interface { - // GenerateFanout receives IdentifierList of entire network, and list of channels the node is subscribing to. - // It constructs and returns the fanout IdentifierList of node. - // A node directly communicates with its fanout IdentifierList on epidemic dissemination + // GenerateFanout receives IdentityList of entire network, and list of channels the node is subscribing to. + // It constructs and returns the fanout IdentityList of node. + // A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. @@ -19,5 +19,5 @@ type Topology interface { // // GenerateFanout is not concurrency safe. It is responsibility of caller to lock for it. // with the channels argument, it allows the returned topology to be cached, which is necessary for randomized topology. - GenerateFanout(ids flow.IdentifierList, channels ChannelList) (flow.IdentifierList, error) + GenerateFanout(ids flow.IdentityList, channels ChannelList) (flow.IdentityList, error) } diff --git a/network/topology/cache.go b/network/topology/cache.go index a738e9e9453..c305140abed 100644 --- a/network/topology/cache.go +++ b/network/topology/cache.go @@ -19,10 +19,10 @@ import ( // in a concurrency safe way, i.e., the caller should lock for it. type Cache struct { log zerolog.Logger - top network.Topology // instance of underlying topology. - cachedFanout flow.IdentifierList // most recently generated fanout list by invoking underlying topology. - idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. - chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. + top network.Topology // instance of underlying topology. + cachedFanout flow.IdentityList // most recently generated fanout list by invoking underlying topology. + idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. + chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. } //NewCache creates and returns a topology Cache given an instance of topology implementation. @@ -36,19 +36,19 @@ func NewCache(log zerolog.Logger, top network.Topology) *Cache { } } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList // of this instance. // It caches the most recently generated fanout list, so as long as the input list is the same, it returns // the same output. It invalidates and updates its internal cache the first time input list changes. -// A node directly communicates with its fanout IdentifierList on epidemic dissemination +// A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. // // Note that this implementation of GenerateFanout preserves same output as long as input is the same. This // should not be assumed as a 1-1 mapping between input and output. -func (c *Cache) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { - inputIdsFP := flow.MerkleRoot(ids...) +func (c *Cache) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { + inputIdsFP := ids.Fingerprint() inputChansFP := channels.ID() log := c.log.With(). diff --git a/network/topology/fixedListTopology.go b/network/topology/fixedListTopology.go index 6091da61d38..1b171ce8919 100644 --- a/network/topology/fixedListTopology.go +++ b/network/topology/fixedListTopology.go @@ -2,7 +2,7 @@ package topology import ( "github.com/onflow/flow-go/model/flow" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/network" ) @@ -17,14 +17,14 @@ func NewFixedListTopology(nodeID flow.Identifier) FixedListTopology { } } -func (r FixedListTopology) GenerateFanout(ids flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { - return ids.Filter(idFilter.Is(r.fixedNodeID)), nil +func (r FixedListTopology) GenerateFanout(ids flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { + return ids.Filter(filter.HasNodeID(r.fixedNodeID)), nil } // EmptyListTopology always returns an empty list as the fanout type EmptyListTopology struct { } -func (r EmptyListTopology) GenerateFanout(_ flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { - return flow.IdentifierList{}, nil +func (r EmptyListTopology) GenerateFanout(_ flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { + return flow.IdentityList{}, nil } diff --git a/network/topology/randomizedTopology.go b/network/topology/randomizedTopology.go index 55be3545025..c1bc959ed9e 100644 --- a/network/topology/randomizedTopology.go +++ b/network/topology/randomizedTopology.go @@ -9,8 +9,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -19,16 +17,15 @@ import ( // By random topology we mean a node is connected to any other co-channel nodes with some // edge probability. type RandomizedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - chance uint64 // used to translate connectedness probability into a number in [0, 100] - rng random.Rand // used as a stateful random number generator to sample edges - logger zerolog.Logger - idProvider id.IdentityProvider + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + chance uint64 // used to translate connectedness probability into a number in [0, 100] + rng random.Rand // used as a stateful random number generator to sample edges + logger zerolog.Logger } // NewRandomizedTopology returns an instance of the RandomizedTopology. -func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { +func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { // edge probability should be a positive value between 0 and 1. However, // we like it to be strictly greater than zero. Also, at the current scale of // Flow, we need it to be greater than 0.01 to support probabilistic connectedness. @@ -47,34 +44,33 @@ func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvide } t := &RandomizedTopology{ - myNodeID: nodeID, - state: state, - chance: uint64(100 * edgeProb), - rng: rng, - logger: logger.With().Str("component:", "randomized-topology").Logger(), - idProvider: idProvider, + myNodeID: nodeID, + state: state, + chance: uint64(100 * edgeProb), + rng: rng, + logger: logger.With().Str("component:", "randomized-topology").Logger(), } return t, nil } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList -// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. This should be done with a very high probability // in randomized topology. -func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { +func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. r.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } - var myFanout flow.IdentifierList + var myFanout flow.IdentityList // generates a randomized subgraph per channel for _, myChannel := range myUniqueChannels { @@ -94,12 +90,12 @@ func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels net return myFanout, nil } -// subsetChannel returns a random subset of the identifier list that is passed. -// Returned identifiers should all subscribed to the specified `channel`. -// Note: this method should not include identifier of its executor. -func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +// subsetChannel returns a random subset of the identity list that is passed. +// Returned identities should all subscribed to the specified `channel`. +// Note: this method should not include identity of its executor. +func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { // excludes node itself - sampleSpace := ids.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) + sampleSpace := ids.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) // samples a random graph based on whether channel is cluster-based or not. if _, ok := engine.ClusterChannel(channel); ok { @@ -113,12 +109,12 @@ func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel netwo // Independent invocations of this method over different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.IdentifierList, error) { +func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityList, error) { if len(ids) == 0 { - return nil, fmt.Errorf("empty identifier list") + return nil, fmt.Errorf("empty identity list") } - fanout := flow.IdentifierList{} + fanout := flow.IdentityList{} for _, id := range ids { // tosses a biased coin and adds id to fanout accordingly. // biased coin follows the edge probability distribution. @@ -131,7 +127,7 @@ func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.Identifi } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow.IdentifierList, error) { +func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.IdentityList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(r.myNodeID, r.state) if err != nil { @@ -139,10 +135,10 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow } // excludes node itself from cluster - clusterPeers = clusterPeers.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) + clusterPeers = clusterPeers.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) + nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } @@ -152,7 +148,7 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow } // clusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. -func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -164,16 +160,7 @@ func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, ch } // samples fanout among interacting roles - return r.sampleFanout( - r.idProvider. - Identities( - filter.And( - filter.HasNodeID(ids...), - filter.HasRole(roles...), - ), - ). - NodeIDs(), - ) + return r.sampleFanout(ids.Filter(filter.HasRole(roles...))) } // tossBiasedBit returns true with probability equal `r.chance/100`, and returns false otherwise. diff --git a/network/topology/topicBasedTopology.go b/network/topology/topicBasedTopology.go index fdc55f43b1e..9d14e34049b 100644 --- a/network/topology/topicBasedTopology.go +++ b/network/topology/topicBasedTopology.go @@ -8,8 +8,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -17,15 +15,14 @@ import ( // TopicBasedTopology is a deterministic topology mapping that creates a connected graph component among the nodes // involved in each topic. type TopicBasedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - logger zerolog.Logger - seed int64 - identityProvider id.IdentityProvider + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + logger zerolog.Logger + seed int64 } // NewTopicBasedTopology returns an instance of the TopicBasedTopology. -func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { +func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { seed, err := intSeedFromID(nodeID) if err != nil { return nil, fmt.Errorf("could not generate seed from id:%w", err) @@ -41,19 +38,19 @@ func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvide return t, nil } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList -// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. -func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. t.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } // finds all interacting roles with this node @@ -67,7 +64,7 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels net } // builds a connected component per role this node interact with, - var myFanout flow.IdentifierList + var myFanout flow.IdentityList for _, role := range myInteractingRoles { if role == flow.RoleCollection { // we do not build connected component for collection nodes based on their role @@ -101,36 +98,32 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels net return myFanout, nil } -// subsetChannel returns a random subset of the identifier list that is passed. `shouldHave` represents set of -// identifiers that should be included in the returned subset. -// Returned identifiers should all subscribed to the specified `channel`. -// Note: this method should not include identifier of its executor. -func (t *TopicBasedTopology) subsetChannel(ids flow.IdentifierList, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +// subsetChannel returns a random subset of the identity list that is passed. `shouldHave` represents set of +// identities that should be included in the returned subset. +// Returned identities should all subscribed to the specified `channel`. +// Note: this method should not include identity of its executor. +func (t *TopicBasedTopology) subsetChannel(ids flow.IdentityList, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return t.clusterChannelHandler(ids, shouldHave) } return t.nonClusterChannelHandler(ids, shouldHave, channel) } -// subsetRole returns a random subset of the identifier list that is passed. `shouldHave` represents set of -// identifiers that should be included in the returned subset. -// Returned identifiers should all be of one of the specified `roles`. -// Note: this method should not include the identifier of its executor. -func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow.IdentifierList, roles flow.RoleList) (flow.IdentifierList, error) { +// subsetRole returns a random subset of the identity list that is passed. `shouldHave` represents set of +// identities that should be included in the returned subset. +// Returned identities should all be of one of the specified `roles`. +// Note: this method should not include identity of its executor. +func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.IdentityList, roles flow.RoleList) (flow.IdentityList, error) { // excludes irrelevant roles and the node itself from both should have and ids set - shouldHave = t.identityProvider. - Identities(filter.HasNodeID(shouldHave...)). - Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )).NodeIDs() - - ids = t.identityProvider. - Identities(filter.HasNodeID(ids...)). - Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )).NodeIDs() + shouldHave = shouldHave.Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )) + + ids = ids.Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )) sample, err := t.sampleConnectedGraph(ids, shouldHave) if err != nil { @@ -145,11 +138,11 @@ func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow. // different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHave flow.IdentityList) (flow.IdentityList, error) { if len(all) == 0 { t.logger.Debug().Msg("skips sampling connected graph with zero nodes") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } if len(shouldHave) == 0 { @@ -160,9 +153,9 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should } // checks `shouldHave` be a subset of `all` - nonMembers := shouldHave.Filter(idFilter.Not(idFilter.In(all...))) + nonMembers := shouldHave.Filter(filter.Not(filter.In(all))) if len(nonMembers) != 0 { - return nil, fmt.Errorf("should have identifiers is not a subset of all: %v", nonMembers) + return nil, fmt.Errorf("should have identities is not a subset of all: %v", nonMembers) } // total sample size @@ -177,7 +170,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should subsetSize := totalSize - len(shouldHave) // others are all excluding should have ones - others := all.Filter(idFilter.Not(idFilter.In(shouldHave...))) + others := all.Filter(filter.Not(filter.In(shouldHave))) others = others.DeterministicSample(uint(subsetSize), t.seed) return others.Union(shouldHave), nil @@ -185,7 +178,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityList) (flow.IdentityList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(t.myNodeID, t.state) if err != nil { @@ -193,18 +186,18 @@ func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.Identifie } // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) + nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } // samples a connected graph topology from the cluster peers - return t.subsetRole(clusterPeers, shouldHave.Filter(idFilter.In(clusterPeers...)), flow.RoleList{flow.RoleCollection}) + return t.subsetRole(clusterPeers, shouldHave.Filter(filter.HasNodeID(clusterPeers.NodeIDs()...)), flow.RoleList{flow.RoleCollection}) } // nonClusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. // The returned sample contains `shouldHave` ones that also subscribed to `channel`. -func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -216,5 +209,5 @@ func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.Identi } // samples a connected graph topology - return t.subsetRole(ids, shouldHave, roles) + return t.subsetRole(ids.Filter(filter.HasRole(roles...)), shouldHave, roles) } diff --git a/network/topology/topology_utils.go b/network/topology/topology_utils.go index 3668d8557a6..bf1d2b43986 100644 --- a/network/topology/topology_utils.go +++ b/network/topology/topology_utils.go @@ -51,7 +51,7 @@ func byteSeedFromID(id flow.Identifier) ([]byte, error) { } // clusterPeers returns the list of other nodes within the same cluster as specified identifier. -func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList, error) { +func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, error) { currentEpoch := state.Final().Epochs().Current() clusterList, err := currentEpoch.Clustering() if err != nil { @@ -63,5 +63,5 @@ func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList return nil, fmt.Errorf("failed to find the cluster for node ID %s", id.String()) } - return myCluster.NodeIDs(), nil + return myCluster, nil } From 9d23c0b5789daf124f0a12b0d8c6e9e2d9c76337 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 16:50:41 -0700 Subject: [PATCH 109/291] add custom id translator --- .../staked_access_node_builder.go | 11 ++++-- network/p2p/hierarchical_translator.go | 37 +++++++++++++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) create mode 100644 network/p2p/hierarchical_translator.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a8e5a9731c1..b7bb142d921 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -36,18 +36,20 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache // translator - // networking provider fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NetworkingSetFilter, ), idCache, ) + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) // TODO: need special providers here - // for network, needs one that recognizes both protocl state and peerstore - // same for translator + // NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // doesn't participate in unstaked network. + // If it does, then we can just use the default one (peerstoreProvider) return nil }) @@ -62,7 +64,8 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - builder.EnqueueNetworkInit(ctx) + // TODO: we should remove this call since we are no longer instantiating two networks + // builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network if builder.ParticipatesInUnstakedNetwork() { diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go new file mode 100644 index 00000000000..28dbdb03c31 --- /dev/null +++ b/network/p2p/hierarchical_translator.go @@ -0,0 +1,37 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type HierarchicalIDTranslator struct { + translators []IDTranslator +} + +func NewHierarchicalIDTranslator(translators ...IDTranslator) *HierarchicalIDTranslator { + return &HierarchicalIDTranslator{translators} +} + +func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + for _, translator := range t.translators { + pid, err := translator.GetPeerID(flowID) + if err == nil { + return pid, nil + } + } + return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v", flowID) +} + +func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { + for _, translator := range t.translators { + fid, err := translator.GetFlowID(peerID) + if err == nil { + return fid, nil + } + } + return flow.ZeroID, fmt.Errorf("could not find corresponding flow ID for peer ID %v", peerID) +} From e82d2d2320b1b3573b68d57f76bfb6cc181ecdef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 18 Aug 2021 11:10:53 -0400 Subject: [PATCH 110/291] validate ID translation --- network/p2p/unstaked_translator.go | 12 +-- network/p2p/unstaked_translator_test.go | 98 +++++++++++++++++++++++++ 2 files changed, 105 insertions(+), 5 deletions(-) create mode 100644 network/p2p/unstaked_translator_test.go diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 5e320ccb236..a3211c1fc68 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -1,6 +1,8 @@ package p2p import ( + "fmt" + crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/libp2p/go-libp2p-core/peer" "github.com/multiformats/go-multihash" @@ -27,16 +29,16 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { pk, err := peerID.ExtractPublicKey() if err != nil { - // return error + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) } - if pk.Type() != crypto_pb.KeyType_ECDSA { - // fail + if pk.Type() != crypto_pb.KeyType_Secp256k1 { + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) } data, err := pk.Raw() - if err != nil || data[0] != 0x02 { // TODO: check if this is the right byte to check - // fail + if err != nil || data[0] != 0x02 { + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) } return flow.HashToID(data[1:]), nil diff --git a/network/p2p/unstaked_translator_test.go b/network/p2p/unstaked_translator_test.go new file mode 100644 index 00000000000..46a3b1dd58c --- /dev/null +++ b/network/p2p/unstaked_translator_test.go @@ -0,0 +1,98 @@ +package p2p + +import ( + "crypto/rand" + "math" + "testing" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/stretchr/testify/require" + + fcrypto "github.com/onflow/flow-go/crypto" +) + +// This test shows we can't use ECDSA P-256 keys for libp2p and expect PeerID <=> PublicKey bijections +func TestIDTranslationP256(t *testing.T) { + loops := 50 + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSAP256) + + // check that we can not extract the public key back + // This makes sense: the x509 serialization of ECDSA P-256 keys in uncompressed form is 64 + 2 bytes, + // and libp2p uses multihash.IDENTITY only on serializations of less than 42 bytes + _, err := pID.ExtractPublicKey() + require.NotNil(t, err) + + } +} + +// This test shows we can use ECDSA Secp256k1 keys for libp2p and expect PeerID <=> PublicKey bijections +func TestIDTranslationSecp256k1(t *testing.T) { + loops := 50 + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) + + // check that we can extract the public key back + // This makes sense: the compressed serialization of ECDSA Secp256k1 keys is 33 + 2 bytes, + // and libp2p uses multihash.IDENTITY on serializations of less than 42 bytes + _, err := pID.ExtractPublicKey() + require.NoError(t, err) + + } +} + +func TestUnstakedTranslationRoundTrip(t *testing.T) { + loops := 50 + unstakedTranslator := NewUnstakedNetworkIDTranslator() + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) + + pk, err := pID.ExtractPublicKey() + require.NoError(t, err) + + // for a secp256k1 key, this is compressed representation preceded by 00 bits + // indicating the multihash.IDENTITY + pkBytes, err := pk.Raw() + require.NoError(t, err) + + // key is positive, roundtrip should be possible + if pkBytes[0] == 0x02 { + flowID, err := unstakedTranslator.GetFlowID(pID) + require.NoError(t, err) + retrievedPeerID, err := unstakedTranslator.GetPeerID(flowID) + require.NoError(t, err) + require.Equal(t, pID, retrievedPeerID) + } + + } +} + +func createPeerIDFromAlgo(t *testing.T, sa fcrypto.SigningAlgorithm) peer.ID { + seed := createSeed(t) + + // this matches GenerateNetworkingKeys, and is intended to validate the choices in cmd/bootstrap + key, err := fcrypto.GeneratePrivateKey(sa, seed) + require.NoError(t, err) + + // get the public key + pubKey := key.PublicKey() + + // extract the corresponding libp2p public Key + libp2pPubKey, err := LibP2PPublicKeyFromFlow(pubKey) + require.NoError(t, err) + + // obtain the PeerID based on libp2p's own rules + pID, err := peer.IDFromPublicKey(libp2pPubKey) + require.NoError(t, err) + + return pID +} + +func createSeed(t *testing.T) []byte { + seedLen := int(math.Max(fcrypto.KeyGenSeedMinLenECDSAP256, fcrypto.KeyGenSeedMinLenECDSASecp256k1)) + seed := make([]byte, seedLen) + n, err := rand.Read(seed) + require.NoError(t, err) + require.Equal(t, n, seedLen) + return seed +} From c9f2e35747b13e584448ea70e43f74cb102c0f89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 18 Aug 2021 13:14:42 -0400 Subject: [PATCH 111/291] [network] make sure TestUnstakedTranslationRoundTrip runs 50 times --- network/p2p/unstaked_translator_test.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/network/p2p/unstaked_translator_test.go b/network/p2p/unstaked_translator_test.go index 46a3b1dd58c..9850cfa775a 100644 --- a/network/p2p/unstaked_translator_test.go +++ b/network/p2p/unstaked_translator_test.go @@ -11,6 +11,9 @@ import ( fcrypto "github.com/onflow/flow-go/crypto" ) +// For these test, refer to https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md for libp2p +// PeerID specifications and how they relate to keys. + // This test shows we can't use ECDSA P-256 keys for libp2p and expect PeerID <=> PublicKey bijections func TestIDTranslationP256(t *testing.T) { loops := 50 @@ -42,21 +45,25 @@ func TestIDTranslationSecp256k1(t *testing.T) { } func TestUnstakedTranslationRoundTrip(t *testing.T) { - loops := 50 + max_iterations := 50 unstakedTranslator := NewUnstakedNetworkIDTranslator() - for i := 0; i < loops; i++ { + + tested_vectors := 0 + + for ok := true; ok; ok = tested_vectors < max_iterations { pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) pk, err := pID.ExtractPublicKey() require.NoError(t, err) - // for a secp256k1 key, this is compressed representation preceded by 00 bits - // indicating the multihash.IDENTITY + // for a secp256k1 key, this is just the compressed representation pkBytes, err := pk.Raw() require.NoError(t, err) // key is positive, roundtrip should be possible if pkBytes[0] == 0x02 { + tested_vectors++ + flowID, err := unstakedTranslator.GetFlowID(pID) require.NoError(t, err) retrievedPeerID, err := unstakedTranslator.GetPeerID(flowID) From 4e7615924383d5b9c205c3baaf2f79870cbf4960 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 11:47:34 -0700 Subject: [PATCH 112/291] TODOs update mocks, fix a test --- .../staked_access_node_builder.go | 5 +- .../unstaked_access_node_builder.go | 2 +- cmd/access/node_builder/upstream_connector.go | 6 +- cmd/node_builder.go | 2 +- cmd/scaffold.go | 2 +- consensus/integration/nodes_test.go | 22 +++- module/id/fixed_provider.go | 17 +++ network/mocknetwork/connector.go | 18 +-- network/mocknetwork/middleware.go | 16 +-- network/mocknetwork/overlay.go | 19 +-- network/p2p/connGater.go | 2 +- network/p2p/dht_test.go | 2 +- network/p2p/libp2pConnector.go | 2 +- network/p2p/libp2pNode.go | 44 ++++--- network/p2p/libp2pNode_test.go | 124 ++++++++++-------- network/p2p/middleware.go | 8 +- network/p2p/peerstore_provider.go | 19 ++- network/p2p/protocol_state_provider.go | 35 +++-- network/p2p/unstaked_translator.go | 8 +- 19 files changed, 204 insertions(+), 149 deletions(-) create mode 100644 module/id/fixed_provider.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index b7bb142d921..233cb56819d 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -29,7 +29,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } @@ -46,8 +46,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { ) fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: need special providers here - // NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // TODO: NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN // doesn't participate in unstaked network. // If it does, then we can just use the default one (peerstoreProvider) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index d2f9d8e02d0..f6fd01e63d2 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -25,7 +25,7 @@ func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index aabfa446426..00d4f7a7f3d 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -105,9 +105,13 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f return default: } + peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) if err != nil { - // TODO: return formatted error + resultChan <- result{ + id: flow.Identity{}, + err: err, + } } // try and connect to the bootstrap server diff --git a/cmd/node_builder.go b/cmd/node_builder.go index f9e082556d5..ab027fceda8 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -137,7 +137,7 @@ type NodeConfig struct { StakingKey crypto.PrivateKey NetworkKey crypto.PrivateKey - // TODO: initialize these in scaffold and unstaked node + // ID providers IdentityProvider id.IdentityProvider IDTranslator p2p.IDTranslator NetworkingIdentifierProvider id.IdentifierProvider diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 4625f035e01..e12dd98d456 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -425,7 +425,7 @@ func (fnb *FlowNodeBuilder) initStorage() { func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index e25254c252e..9320579bce8 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -24,6 +24,7 @@ import ( "github.com/onflow/flow-go/module/buffer" builder "github.com/onflow/flow-go/module/builder/consensus" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -127,7 +128,7 @@ func createNode( setupsDB := storage.NewEpochSetups(metrics, db) commitsDB := storage.NewEpochCommits(metrics, db) statusesDB := storage.NewEpochStatuses(metrics, db) - consumer := events.NewNoop() + consumer := events.NewDistributor() state, err := bprotocol.Bootstrap(metrics, db, headersDB, sealsDB, resultsDB, blocksDB, setupsDB, commitsDB, statusesDB, rootSnapshot) require.NoError(t, err) @@ -230,8 +231,25 @@ func createNode( finalizedHeader, err := synceng.NewFinalizedHeaderCache(log, state, pubsub.NewFinalizationDistributor()) require.NoError(t, err) + identities, err := state.Final().Identities(filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(me.NodeID())), + )) + require.NoError(t, err) + idProvider := id.NewFixedIdentifierProvider(identities.NodeIDs()) + // initialize the synchronization engine - sync, err := synceng.New(log, metrics, net, me, blocksDB, comp, syncCore, finalizedHeader, state) + sync, err := synceng.New( + log, + metrics, + net, + me, + blocksDB, + comp, + syncCore, + finalizedHeader, + idProvider, + ) require.NoError(t, err) pending := []*flow.Header{} diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go new file mode 100644 index 00000000000..dbdf10b9403 --- /dev/null +++ b/module/id/fixed_provider.go @@ -0,0 +1,17 @@ +package id + +import ( + "github.com/onflow/flow-go/model/flow" +) + +type FixedIdentifierProvider struct { + identifiers flow.IdentifierList +} + +func NewFixedIdentifierProvider(identifiers flow.IdentifierList) *FixedIdentifierProvider { + return &FixedIdentifierProvider{identifiers} +} + +func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { + return p.identifiers +} diff --git a/network/mocknetwork/connector.go b/network/mocknetwork/connector.go index 02f12add1b5..3f32fad44f6 100644 --- a/network/mocknetwork/connector.go +++ b/network/mocknetwork/connector.go @@ -5,8 +5,9 @@ package mocknetwork import ( context "context" - flow "github.com/onflow/flow-go/model/flow" mock "github.com/stretchr/testify/mock" + + peer "github.com/libp2p/go-libp2p-core/peer" ) // Connector is an autogenerated mock type for the Connector type @@ -14,16 +15,7 @@ type Connector struct { mock.Mock } -// UpdatePeers provides a mock function with given fields: ctx, ids -func (_m *Connector) UpdatePeers(ctx context.Context, ids flow.IdentityList) error { - ret := _m.Called(ctx, ids) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, flow.IdentityList) error); ok { - r0 = rf(ctx, ids) - } else { - r0 = ret.Error(0) - } - - return r0 +// UpdatePeers provides a mock function with given fields: ctx, peerIDs +func (_m *Connector) UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) { + _m.Called(ctx, peerIDs) } diff --git a/network/mocknetwork/middleware.go b/network/mocknetwork/middleware.go index 232990ff4cf..8d0a57067f6 100644 --- a/network/mocknetwork/middleware.go +++ b/network/mocknetwork/middleware.go @@ -122,15 +122,11 @@ func (_m *Middleware) Unsubscribe(channel network.Channel) error { } // UpdateAllowList provides a mock function with given fields: -func (_m *Middleware) UpdateAllowList() error { - ret := _m.Called() - - var r0 error - if rf, ok := ret.Get(0).(func() error); ok { - r0 = rf() - } else { - r0 = ret.Error(0) - } +func (_m *Middleware) UpdateAllowList() { + _m.Called() +} - return r0 +// UpdateNodeAddresses provides a mock function with given fields: +func (_m *Middleware) UpdateNodeAddresses() { + _m.Called() } diff --git a/network/mocknetwork/overlay.go b/network/mocknetwork/overlay.go index c096adf32b1..7c98d0b8fe5 100644 --- a/network/mocknetwork/overlay.go +++ b/network/mocknetwork/overlay.go @@ -14,27 +14,20 @@ type Overlay struct { mock.Mock } -// Identity provides a mock function with given fields: -func (_m *Overlay) Identity() (map[flow.Identifier]flow.Identity, error) { +// Identities provides a mock function with given fields: +func (_m *Overlay) Identities() flow.IdentityList { ret := _m.Called() - var r0 map[flow.Identifier]flow.Identity - if rf, ok := ret.Get(0).(func() map[flow.Identifier]flow.Identity); ok { + var r0 flow.IdentityList + if rf, ok := ret.Get(0).(func() flow.IdentityList); ok { r0 = rf() } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(map[flow.Identifier]flow.Identity) + r0 = ret.Get(0).(flow.IdentityList) } } - var r1 error - if rf, ok := ret.Get(1).(func() error); ok { - r1 = rf() - } else { - r1 = ret.Error(1) - } - - return r0, r1 + return r0 } // Receive provides a mock function with given fields: nodeID, msg diff --git a/network/p2p/connGater.go b/network/p2p/connGater.go index caa05bb1f1a..64ebe02eed0 100644 --- a/network/p2p/connGater.go +++ b/network/p2p/connGater.go @@ -29,7 +29,7 @@ func NewConnGater(log zerolog.Logger) *ConnGater { } // update updates the peer ID map -func (c *ConnGater) update(pids []peer.ID) { +func (c *ConnGater) update(pids peer.IDSlice) { // create a new peer.ID map peerIDs := make(map[peer.ID]struct{}, len(pids)) diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index 8d2134f0591..dc300338ac4 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -180,7 +180,7 @@ func (suite *DHTTestSuite) CreateNodes(count int, dhtServer bool) (nodes []*Node n, err := NewDefaultLibP2PNodeBuilder(flow.Identifier{}, "0.0.0.0:0", key). SetRootBlockID(rootBlockID). SetConnectionManager(connManager). - SetPubsubOptions(WithDHTDiscovery(AsServer(dhtServer))). + SetDHTOptions(AsServer(dhtServer)). SetPingInfoProvider(pingInfoProvider). SetLogger(logger). Build(suite.ctx) diff --git a/network/p2p/libp2pConnector.go b/network/p2p/libp2pConnector.go index 64e68a43170..d9e16853aec 100644 --- a/network/p2p/libp2pConnector.go +++ b/network/p2p/libp2pConnector.go @@ -94,7 +94,7 @@ func (l *libp2pConnector) connectToPeers(ctx context.Context, peerIDs peer.IDSli // trimAllConnectionsExcept trims all connections of the node from peers not part of peerIDs. // A node would have created such extra connections earlier when the identity list may have been different, or // it may have been target of such connections from node which have now been excluded. -func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs []peer.ID) { +func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs peer.IDSlice) { // convert the peerInfos to a peer.ID -> bool map peersToKeep := make(map[peer.ID]bool, len(peerIDs)) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index b6a687cad03..cae93c5daa7 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -42,6 +42,10 @@ const ( // maximum number of attempts to be made to connect to a remote node for 1-1 direct communication maxConnectAttempt = 3 + + // timeout for FindPeer queries to the DHT + // TODO: is this a sensible value? + findPeerQueryTimeout = 15 * time.Second ) // LibP2PFactoryFunc is a factory function type for generating libp2p Node instances. @@ -308,8 +312,8 @@ func (n *Node) Stop() (chan struct{}, error) { } // AddPeer adds a peer to this node by adding it to this node's peerstore and connecting to it -func (n *Node) AddPeer(ctx context.Context, peerID peer.ID) error { - err := n.host.Connect(ctx, peer.AddrInfo{ID: peerID}) +func (n *Node) AddPeer(ctx context.Context, peerInfo peer.AddrInfo) error { + err := n.host.Connect(ctx, peerInfo) if err != nil { return err } @@ -328,6 +332,21 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { // CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { + if len(n.host.Peerstore().Addrs(peerID)) == 0 { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") + if n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + // try to find the peer using the dht + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + } + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + } + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) if err != nil { @@ -356,25 +375,10 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp default: } + // TODO: why were we doing this? Is it okay to remove? // remove the peer from the peer store if present - // TODO: why were we doing this? // n.host.Peerstore().ClearAddrs(peerID) - if len(n.host.Peerstore().Addrs(peerID)) == 0 { - // TODO: add bunch of logging here - if n.dht != nil { - // TODO: adjust timeout - timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) - // try to find the peer - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) - } - } - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) - } - // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() if swm, ok := network.(*swarm.Swarm); ok { @@ -388,7 +392,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp time.Sleep(time.Duration(r) * time.Millisecond) } - err := n.AddPeer(ctx, peerID) + err := n.AddPeer(ctx, peer.AddrInfo{ID: peerID}) if err != nil { // if the connection was rejected due to invalid node id, skip the re-attempt @@ -536,7 +540,7 @@ func (n *Node) Ping(ctx context.Context, peerID peer.ID) (message.PingResponse, } // UpdateAllowList allows the peer allow list to be updated. -func (n *Node) UpdateAllowList(peers []peer.ID) { +func (n *Node) UpdateAllowList(peers peer.IDSlice) { if n.connGater == nil { n.logger.Debug().Hex("node_id", logging.ID(n.id)).Msg("skipping update allow list, connection gating is not enabled") return diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index f7f5540a51c..2497d3f3f76 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -15,6 +15,7 @@ import ( golog "github.com/ipfs/go-log" addrutil "github.com/libp2p/go-addr-util" "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" swarm "github.com/libp2p/go-libp2p-swarm" "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr/net" @@ -140,22 +141,13 @@ func (suite *LibP2PNodeTestSuite) TestAddPeers() { // add the remaining nodes to the first node as its set of peers for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, *identity)) + peerInfo, err := PeerAddressInfo(*identity) + require.NoError(suite.T(), err) + require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, peerInfo)) } // Checks if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Peerstore().Peers(), count) - - // Checks whether the first node is connected to the rest - for _, peer := range nodes[0].host.Peerstore().Peers() { - // A node is also a peer to itself but not marked as connected, hence skip checking that. - if nodes[0].host.ID().String() == peer.String() { - continue - } - assert.Eventuallyf(suite.T(), func() bool { - return network.Connected == nodes[0].host.Network().Connectedness(peer) - }, 2*time.Second, tickForAssertEventually, fmt.Sprintf(" first node is not connected to %s", peer.String())) - } + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) } // TestAddPeers checks if nodes can be added as peers to a given node @@ -165,39 +157,27 @@ func (suite *LibP2PNodeTestSuite) TestRemovePeers() { // create nodes nodes, identities := suite.NodesFixture(count, nil, false) + peerInfos, errs := peerInfosFromIDs(identities) + assert.Len(suite.T(), errs, 0) defer StopNodes(suite.T(), nodes) // add nodes two and three to the first node as its peers - for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, *identity)) + for _, pInfo := range peerInfos[1:] { + require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, pInfo)) } // check if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Peerstore().Peers(), count) - - // check whether the first node is connected to the rest - for _, peer := range nodes[0].host.Peerstore().Peers() { - // A node is also a peer to itself but not marked as connected, hence skip checking that. - if nodes[0].host.ID().String() == peer.String() { - continue - } - assert.Eventually(suite.T(), func() bool { - return network.Connected == nodes[0].host.Network().Connectedness(peer) - }, 2*time.Second, tickForAssertEventually) - } + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) // disconnect from each peer and assert that the connection no longer exists - for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].RemovePeer(suite.ctx, *identity)) - pInfo, err := PeerAddressInfo(*identity) - assert.NoError(suite.T(), err) + for _, pInfo := range peerInfos[1:] { + require.NoError(suite.T(), nodes[0].RemovePeer(suite.ctx, pInfo.ID)) assert.Equal(suite.T(), network.NotConnected, nodes[0].host.Network().Connectedness(pInfo.ID)) } } // TestCreateStreams checks if a new streams is created each time when CreateStream is called and an existing stream is not reused func (suite *LibP2PNodeTestSuite) TestCreateStream() { - count := 2 // Creates nodes @@ -213,7 +193,10 @@ func (suite *LibP2PNodeTestSuite) TestCreateStream() { // Now attempt to create another 100 outbound stream to the same destination by calling CreateStream var streams []network.Stream for i := 0; i < 100; i++ { - anotherStream, err := nodes[0].CreateStream(context.Background(), *id2) + pInfo, err := PeerAddressInfo(*id2) + require.NoError(suite.T(), err) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo)) + anotherStream, err := nodes[0].CreateStream(context.Background(), pInfo.ID) // Assert that a stream was returned without error require.NoError(suite.T(), err) require.NotNil(suite.T(), anotherStream) @@ -264,9 +247,14 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { id1 := *identities[0] id2 := *identities[1] + pInfo1, err := PeerAddressInfo(id1) + require.NoError(suite.T(), err) + pInfo2, err := PeerAddressInfo(id2) + require.NoError(suite.T(), err) // Create stream from node 1 to node 2 - s1, err := nodes[0].CreateStream(context.Background(), id2) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo2)) + s1, err := nodes[0].CreateStream(context.Background(), pInfo2.ID) assert.NoError(suite.T(), err) rw := bufio.NewReadWriter(bufio.NewReader(s1), bufio.NewWriter(s1)) @@ -287,7 +275,8 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } // Create stream from node 2 to node 1 - s2, err := nodes[1].CreateStream(context.Background(), id1) + require.NoError(suite.T(), nodes[1].AddPeer(context.Background(), pInfo1)) + s2, err := nodes[1].CreateStream(context.Background(), pInfo1.ID) assert.NoError(suite.T(), err) rw = bufio.NewReadWriter(bufio.NewReader(s2), bufio.NewWriter(s2)) @@ -322,16 +311,19 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() require.NoError(suite.T(), listener.Close()) }() + silentNodeInfo, err := PeerAddressInfo(silentNodeId) + require.NoError(suite.T(), err) + // setup the context to expire after the default timeout ctx, cancel := context.WithTimeout(context.Background(), DefaultUnicastTimeout) defer cancel() // attempt to create a stream from node 1 to node 2 and assert that it fails after timeout grace := 1 * time.Second - var err error unittest.AssertReturnsBefore(suite.T(), func() { - _, err = nodes[0].CreateStream(ctx, silentNodeId) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), silentNodeInfo)) + _, err = nodes[0].CreateStream(ctx, silentNodeInfo.ID) }, DefaultUnicastTimeout+grace) assert.Error(suite.T(), err) @@ -344,17 +336,22 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { goodNodes, goodNodeIds := suite.NodesFixture(2, nil, false) defer StopNodes(suite.T(), goodNodes) require.Len(suite.T(), goodNodeIds, 2) + goodNodeInfo1, err := PeerAddressInfo(*goodNodeIds[1]) + require.NoError(suite.T(), err) // create a silent node which never replies listener, silentNodeId := silentNodeFixture(suite.T()) defer func() { require.NoError(suite.T(), listener.Close()) }() + silentNodeInfo, err := PeerAddressInfo(silentNodeId) + require.NoError(suite.T(), err) // creates a stream to unresponsive node and makes sure that the stream creation is blocked blockedCallCh := unittest.RequireNeverReturnBefore(suite.T(), func() { - _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeId) // this call will block + require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), silentNodeInfo)) + _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeInfo.ID) // this call will block }, 1*time.Second, "CreateStream attempt to the unresponsive peer did not block") @@ -362,7 +359,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // requires same peer can still connect to the other regular peer without being blocked unittest.RequireReturnsBefore(suite.T(), func() { - _, err := goodNodes[0].CreateStream(suite.ctx, *goodNodeIds[1]) + require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), goodNodeInfo1)) + _, err := goodNodes[0].CreateStream(suite.ctx, goodNodeInfo1.ID) require.NoError(suite.T(), err) }, 1*time.Second, "creating stream to a responsive node failed while concurrently blocked on unresponsive node") @@ -381,6 +379,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { nodes, identities := suite.NodesFixture(2, nil, false) defer StopNodes(suite.T(), nodes) require.Len(suite.T(), identities, 2) + nodeInfo1, err := PeerAddressInfo(*identities[1]) + require.NoError(suite.T(), err) wg := sync.WaitGroup{} @@ -389,7 +389,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { createStream := func() { <-gate - _, err := nodes[0].CreateStream(suite.ctx, *identities[1]) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + _, err := nodes[0].CreateStream(suite.ctx, nodeInfo1.ID) assert.NoError(suite.T(), err) // assert that stream was successfully created wg.Done() } @@ -449,10 +450,13 @@ func (suite *LibP2PNodeTestSuite) TestStreamClosing() { // Creates nodes nodes, identities := suite.NodesFixture(2, handler, false) defer StopNodes(suite.T(), nodes) + nodeInfo1, err := PeerAddressInfo(*identities[1]) + require.NoError(suite.T(), err) for i := 0; i < count; i++ { // Create stream from node 1 to node 2 (reuse if one already exists) - s, err := nodes[0].CreateStream(context.Background(), *identities[1]) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + s, err := nodes[0].CreateStream(context.Background(), nodeInfo1.ID) assert.NoError(suite.T(), err) w := bufio.NewWriter(s) @@ -509,7 +513,9 @@ func (suite *LibP2PNodeTestSuite) TestPing() { func testPing(t *testing.T, source *Node, target flow.Identity, expectedVersion string, expectedHeight uint64) { pctx, cancel := context.WithCancel(context.Background()) defer cancel() - resp, rtt, err := source.Ping(pctx, target) + pid, err := ExtractPeerID(target.NetworkPubKey) + assert.NoError(t, err) + resp, rtt, err := source.Ping(pctx, pid) assert.NoError(t, err) assert.NotZero(t, rtt) assert.Equal(t, expectedVersion, resp.Version) @@ -525,10 +531,14 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { node1 := nodes[0] node1Id := *identities[0] defer StopNode(suite.T(), node1) + node1Info, err := PeerAddressInfo(node1Id) + assert.NoError(suite.T(), err) node2 := nodes[1] node2Id := *identities[1] defer StopNode(suite.T(), node2) + node2Info, err := PeerAddressInfo(node2Id) + assert.NoError(suite.T(), err) requireError := func(err error) { require.Error(suite.T(), err) @@ -537,38 +547,40 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { suite.Run("outbound connection to a not-allowed node is rejected", func() { // node1 and node2 both have no allowListed peers - _, err := node1.CreateStream(suite.ctx, node2Id) + requireError(node1.AddPeer(context.Background(), node2Info)) + _, err := node1.CreateStream(suite.ctx, node2Info.ID) requireError(err) - _, err = node2.CreateStream(suite.ctx, node1Id) + requireError(node2.AddPeer(context.Background(), node1Info)) + _, err = node2.CreateStream(suite.ctx, node1Info.ID) requireError(err) }) suite.Run("inbound connection from an allowed node is rejected", func() { // node1 allowlists node2 but node2 does not allowlists node1 - err := node1.UpdateAllowList(flow.IdentityList{&node2Id}) - require.NoError(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node2Info.ID}) // node1 attempts to connect to node2 // node2 should reject the inbound connection - _, err = node1.CreateStream(suite.ctx, node2Id) + require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.Error(suite.T(), err) }) suite.Run("outbound connection to an approved node is allowed", func() { // node1 allowlists node2 - err := node1.UpdateAllowList(flow.IdentityList{&node2Id}) - require.NoError(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node2Info.ID}) // node2 allowlists node1 - err = node2.UpdateAllowList(flow.IdentityList{&node1Id}) - require.NoError(suite.T(), err) + node2.UpdateAllowList(peer.IDSlice{node1Info.ID}) // node1 should be allowed to connect to node2 - _, err = node1.CreateStream(suite.ctx, node2Id) + require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.NoError(suite.T(), err) // node2 should be allowed to connect to node1 - _, err = node2.CreateStream(suite.ctx, node1Id) + require.NoError(suite.T(), node2.AddPeer(context.Background(), node1Info)) + _, err = node2.CreateStream(suite.ctx, node1Info.ID) require.NoError(suite.T(), err) }) } @@ -579,12 +591,12 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGatingBootstrap() { node1 := node[0] node1Id := identity[0] defer StopNode(suite.T(), node1) + node1Info, err := PeerAddressInfo(*node1Id) + assert.NoError(suite.T(), err) suite.Run("updating allowlist of node w/o ConnGater does not crash", func() { - // node1 allowlists node1 - err := node1.UpdateAllowList(flow.IdentityList{node1Id}) - require.Error(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node1Info.ID}) }) } diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index e43d7674c08..8f55761a386 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -161,7 +161,6 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes func (m *Middleware) topologyPeers() (peer.IDSlice, error) { identities, err := m.ov.Topology() if err != nil { - // TODO: format error return nil, err } @@ -178,7 +177,10 @@ func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { for _, fid := range flowIDs { pid, err := m.idTranslator.GetPeerID(fid) if err != nil { - // TODO: log here + // We probably don't need to fail the entire function here, since the other + // translations may still succeed + m.log.Err(err).Str("flowID", fid.String()).Msg("failed to translate to peer ID") + continue } result = append(result, pid) @@ -212,7 +214,7 @@ func (m *Middleware) Start(ov network.Overlay) error { libP2PNode, err := m.libP2PNodeFactory() if m.idProvider == nil { - m.idProvider = NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator) + m.idProvider = NewPeerstoreIdentifierProvider(m.log, libP2PNode.host, m.idTranslator) } if err != nil { diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go index cea16d466a9..c31b2cf6917 100644 --- a/network/p2p/peerstore_provider.go +++ b/network/p2p/peerstore_provider.go @@ -2,6 +2,7 @@ package p2p import ( "github.com/libp2p/go-libp2p-core/host" + "github.com/rs/zerolog" "github.com/onflow/flow-go/model/flow" ) @@ -9,23 +10,29 @@ import ( type PeerstoreIdentifierProvider struct { host host.Host idTranslator IDTranslator + logger zerolog.Logger } -func NewPeerstoreIdentifierProvider(host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { - return &PeerstoreIdentifierProvider{host: host, idTranslator: idTranslator} +func NewPeerstoreIdentifierProvider(logger zerolog.Logger, host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{ + logger: logger.With().Str("component", "peerstore-id-provider").Logger(), + host: host, + idTranslator: idTranslator, + } } func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { var result flow.IdentifierList - pids := p.host.Peerstore().PeersWithAddrs() // TODO: should we just call Peers here? + pids := p.host.Peerstore().PeersWithAddrs() for _, pid := range pids { flowID, err := p.idTranslator.GetFlowID(pid) if err != nil { - // TODO: log error - } else { - result = append(result, flowID) + p.logger.Err(err).Str("peerID", pid.Pretty()).Msg("failed to translate to Flow ID") + continue } + + result = append(result, flowID) } return result diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index d47542c3b18..9290a860e06 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -5,6 +5,7 @@ import ( "sync" "github.com/libp2p/go-libp2p-core/peer" + "github.com/rs/zerolog" "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" @@ -18,21 +19,24 @@ type ProtocolStateIDCache struct { identities flow.IdentityList state protocol.State mu sync.RWMutex - peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + peerIDs map[flow.Identifier]peer.ID flowIDs map[peer.ID]flow.Identifier + logger zerolog.Logger } func NewProtocolStateIDCache( + logger zerolog.Logger, state protocol.State, eventDistributer *events.Distributor, ) (*ProtocolStateIDCache, error) { provider := &ProtocolStateIDCache{ - state: state, + state: state, + logger: logger.With().Str("component", "protocol-state-id-cache").Logger(), } head, err := state.Final().Head() if err != nil { - return nil, err // TODO: format the error + return nil, fmt.Errorf("failed to get latest state header: %w", err) } provider.update(head.ID()) @@ -41,24 +45,28 @@ func NewProtocolStateIDCache( return provider, nil } -func (p *ProtocolStateIDCache) EpochTransition(_ uint64, header *flow.Header) { - // TODO: maybe we actually want to log the epoch information from the arguments here (epoch phase, etc) +func (p *ProtocolStateIDCache) EpochTransition(newEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("newEpochCounter", newEpochCounter).Msg("epoch transition") p.update(header.ID()) } -func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { +func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(currentEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("currentEpochCounter", currentEpochCounter).Msg("epoch setup phase started") p.update(header.ID()) } -func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { +func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(currentEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("currentEpochCounter", currentEpochCounter).Msg("epoch committed phase started") p.update(header.ID()) } func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { - // TODO: log status here + p.logger.Info().Str("blockID", blockID.String()).Msg("updating cached identities") + identities, err := p.state.AtBlockID(blockID).Identities(filter.Any) if err != nil { - // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + // We don't want to continue with an expired identity list. + p.logger.Fatal().Err(err).Msg("failed to fetch new identities") } nIds := identities.Count() @@ -67,9 +75,12 @@ func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { flowIDs := make(map[peer.ID]flow.Identifier, nIds) for _, identity := range identities { + p.logger.Debug().Interface("identity", identity).Msg("extracting peer ID from network key") + pid, err := ExtractPeerID(identity.NetworkPubKey) if err != nil { - // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + p.logger.Err(err).Interface("identity", identity).Msg("failed to extract peer ID from network key") + continue } flowIDs[pid] = identity.NodeID @@ -116,13 +127,13 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { pk, err := PublicKey(networkPubKey) if err != nil { - // TODO: format the error + err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) return } pid, err = peer.IDFromPublicKey(pk) if err != nil { - // TODO: format the error + err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) return } diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index a3211c1fc68..62f5bde9668 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -20,7 +20,7 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID data := append([]byte{0x02}, flowID[:]...) mh, err := multihash.Sum(data, multihash.IDENTITY, -1) if err != nil { - // TODO: return error + return "", fmt.Errorf("failed to compute multihash: %w", err) } return peer.ID(mh), nil @@ -29,16 +29,16 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { pk, err := peerID.ExtractPublicKey() if err != nil { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) } if pk.Type() != crypto_pb.KeyType_Secp256k1 { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) } data, err := pk.Raw() if err != nil || data[0] != 0x02 { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) } return flow.HashToID(data[1:]), nil From 35844881143813044da80a0edcc533d4db6e219c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 13:01:56 -0700 Subject: [PATCH 113/291] Update peerManager_test.go Update sporking_test.go Update protocol_state_provider.go --- network/p2p/peerManager_test.go | 42 +++++++++++++++++++++----- network/p2p/protocol_state_provider.go | 2 +- network/p2p/sporking_test.go | 30 +++++++++++------- 3 files changed, 54 insertions(+), 20 deletions(-) diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 65cfa184c71..571a4a8108b 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -7,9 +7,11 @@ import ( "time" "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/model/flow" @@ -41,8 +43,14 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { currentIDs := unittest.IdentityListFixture(10) // setup a ID provider callback to return currentIDs - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } // track IDs that should be disconnected @@ -105,8 +113,14 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } connector := new(mocknetwork.Connector) @@ -137,8 +151,14 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } // chooses peer interval rate deliberately long to capture on demand peer update @@ -180,8 +200,14 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } connector := new(mocknetwork.Connector) diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 9290a860e06..6f223a4d0fe 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -125,7 +125,7 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e } func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { - pk, err := PublicKey(networkPubKey) + pk, err := LibP2PPublicKeyFromFlow(networkPubKey) if err != nil { err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) return diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 935ac173a54..429f522bb0f 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -6,6 +6,7 @@ import ( "time" golog "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" @@ -13,7 +14,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/utils/unittest" ) @@ -50,9 +50,11 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { // create and start node 2 on localhost and random port node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootBlockID, nil, false, defaultAddress) + peerInfo2, err := PeerAddressInfo(id2) + require.NoError(suite.T(), err) // create stream from node 1 to node 2 - testOneToOneMessagingSucceeds(suite.T(), node1, id2) + testOneToOneMessagingSucceeds(suite.T(), node1, peerInfo2) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain @@ -71,7 +73,7 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { // attempt to create a stream from node 1 (old chain) to node 2 (new chain) // this time it should fail since node 2 is using a different public key // (and therefore has a different libp2p node id) - testOneToOneMessagingFails(suite.T(), node1, id2) + testOneToOneMessagingFails(suite.T(), node1, peerInfo2) } // TestOneToOneCrosstalkPrevention tests that a node from the old chain cannot talk directly to a node in the new chain @@ -85,13 +87,15 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { node1key := generateNetworkingKey(suite.T()) node1, id1 := NodeFixture(suite.T(), suite.logger, node1key, rootID1, nil, false, defaultAddress) defer StopNode(suite.T(), node1) + peerInfo1, err := PeerAddressInfo(id1) + require.NoError(suite.T(), err) // create and start node 2 on localhost and random port node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootID1, nil, false, defaultAddress) // create stream from node 2 to node 1 - testOneToOneMessagingSucceeds(suite.T(), node2, id1) + testOneToOneMessagingSucceeds(suite.T(), node2, peerInfo1) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain @@ -110,7 +114,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { // attempt to create a stream from node 2 (new chain) to node 1 (old chain) // this time it should fail since node 2 is listening on a different protocol - testOneToOneMessagingFails(suite.T(), node2, id1) + testOneToOneMessagingFails(suite.T(), node2, peerInfo1) } // TestOneToKCrosstalkPrevention tests that a node from the old chain cannot talk to a node in the new chain via PubSub @@ -128,7 +132,9 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { // create and start node 2 on localhost and random port with the same root block ID node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootIDBeforeSpork, nil, false, defaultAddress) + pInfo2, err := PeerAddressInfo(id2) defer StopNode(suite.T(), node2) + require.NoError(suite.T(), err) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -137,13 +143,13 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { topicBeforeSpork := engine.TopicFromChannel(engine.TestNetwork, rootIDBeforeSpork) // both nodes are initially on the same spork and subscribed to the same topic - _, err := node1.Subscribe(ctx, topicBeforeSpork) + _, err = node1.Subscribe(ctx, topicBeforeSpork) require.NoError(suite.T(), err) sub2, err := node2.Subscribe(ctx, topicBeforeSpork) require.NoError(suite.T(), err) // add node 2 as a peer of node 1 - err = node1.AddPeer(ctx, id2) + err = node1.AddPeer(ctx, pInfo2) require.NoError(suite.T(), err) // let the two nodes form the mesh @@ -170,17 +176,19 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { testOneToKMessagingFails(ctx, suite.T(), node1, sub2, topicAfterSpork) } -func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, targetId flow.Identity) { +func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { + require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from node 1 to node 2 - s, err := sourceNode.CreateStream(context.Background(), targetId) + s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation succeeded require.NoError(t, err) assert.NotNil(t, s) } -func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, targetId flow.Identity) { +func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { + require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from source node to destination address - _, err := sourceNode.CreateStream(context.Background(), targetId) + _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation failed assert.Error(t, err) // assert that it failed with the expected error From a2e7db0b039a9cc60194ea059c828e9710973cbb Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 19:29:35 -0700 Subject: [PATCH 114/291] Add new DHT test --- module/mocks/network.go | 63 ++++++------ network/p2p/dht.go | 2 +- network/p2p/dht_test.go | 46 +++++++++ network/p2p/libp2pNode.go | 23 +++-- storage/mocks/storage.go | 207 +++++++++++++++++++------------------- 5 files changed, 195 insertions(+), 146 deletions(-) diff --git a/module/mocks/network.go b/module/mocks/network.go index 38f65a552ce..b680ae44c89 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -5,38 +5,39 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" network "github.com/onflow/flow-go/network" - reflect "reflect" ) -// MockNetwork is a mock of Network interface +// MockNetwork is a mock of Network interface. type MockNetwork struct { ctrl *gomock.Controller recorder *MockNetworkMockRecorder } -// MockNetworkMockRecorder is the mock recorder for MockNetwork +// MockNetworkMockRecorder is the mock recorder for MockNetwork. type MockNetworkMockRecorder struct { mock *MockNetwork } -// NewMockNetwork creates a new mock instance +// NewMockNetwork creates a new mock instance. func NewMockNetwork(ctrl *gomock.Controller) *MockNetwork { mock := &MockNetwork{ctrl: ctrl} mock.recorder = &MockNetworkMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockNetwork) EXPECT() *MockNetworkMockRecorder { return m.recorder } -// Register mocks base method +// Register mocks base method. func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (network.Conduit, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Register", arg0, arg1) @@ -45,36 +46,36 @@ func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (netwo return ret0, ret1 } -// Register indicates an expected call of Register +// Register indicates an expected call of Register. func (mr *MockNetworkMockRecorder) Register(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Register", reflect.TypeOf((*MockNetwork)(nil).Register), arg0, arg1) } -// MockLocal is a mock of Local interface +// MockLocal is a mock of Local interface. type MockLocal struct { ctrl *gomock.Controller recorder *MockLocalMockRecorder } -// MockLocalMockRecorder is the mock recorder for MockLocal +// MockLocalMockRecorder is the mock recorder for MockLocal. type MockLocalMockRecorder struct { mock *MockLocal } -// NewMockLocal creates a new mock instance +// NewMockLocal creates a new mock instance. func NewMockLocal(ctrl *gomock.Controller) *MockLocal { mock := &MockLocal{ctrl: ctrl} mock.recorder = &MockLocalMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockLocal) EXPECT() *MockLocalMockRecorder { return m.recorder } -// Address mocks base method +// Address mocks base method. func (m *MockLocal) Address() string { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Address") @@ -82,13 +83,13 @@ func (m *MockLocal) Address() string { return ret0 } -// Address indicates an expected call of Address +// Address indicates an expected call of Address. func (mr *MockLocalMockRecorder) Address() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Address", reflect.TypeOf((*MockLocal)(nil).Address)) } -// NodeID mocks base method +// NodeID mocks base method. func (m *MockLocal) NodeID() flow.Identifier { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NodeID") @@ -96,13 +97,13 @@ func (m *MockLocal) NodeID() flow.Identifier { return ret0 } -// NodeID indicates an expected call of NodeID +// NodeID indicates an expected call of NodeID. func (mr *MockLocalMockRecorder) NodeID() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NodeID", reflect.TypeOf((*MockLocal)(nil).NodeID)) } -// NotMeFilter mocks base method +// NotMeFilter mocks base method. func (m *MockLocal) NotMeFilter() flow.IdentityFilter { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NotMeFilter") @@ -110,13 +111,13 @@ func (m *MockLocal) NotMeFilter() flow.IdentityFilter { return ret0 } -// NotMeFilter indicates an expected call of NotMeFilter +// NotMeFilter indicates an expected call of NotMeFilter. func (mr *MockLocalMockRecorder) NotMeFilter() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NotMeFilter", reflect.TypeOf((*MockLocal)(nil).NotMeFilter)) } -// Sign mocks base method +// Sign mocks base method. func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Sign", arg0, arg1) @@ -125,13 +126,13 @@ func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error return ret0, ret1 } -// Sign indicates an expected call of Sign +// Sign indicates an expected call of Sign. func (mr *MockLocalMockRecorder) Sign(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sign", reflect.TypeOf((*MockLocal)(nil).Sign), arg0, arg1) } -// SignFunc mocks base method +// SignFunc mocks base method. func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.PrivateKey, []byte, hash.Hasher) (crypto.Signature, error)) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SignFunc", arg0, arg1, arg2) @@ -140,66 +141,66 @@ func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.Pri return ret0, ret1 } -// SignFunc indicates an expected call of SignFunc +// SignFunc indicates an expected call of SignFunc. func (mr *MockLocalMockRecorder) SignFunc(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignFunc", reflect.TypeOf((*MockLocal)(nil).SignFunc), arg0, arg1, arg2) } -// MockRequester is a mock of Requester interface +// MockRequester is a mock of Requester interface. type MockRequester struct { ctrl *gomock.Controller recorder *MockRequesterMockRecorder } -// MockRequesterMockRecorder is the mock recorder for MockRequester +// MockRequesterMockRecorder is the mock recorder for MockRequester. type MockRequesterMockRecorder struct { mock *MockRequester } -// NewMockRequester creates a new mock instance +// NewMockRequester creates a new mock instance. func NewMockRequester(ctrl *gomock.Controller) *MockRequester { mock := &MockRequester{ctrl: ctrl} mock.recorder = &MockRequesterMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockRequester) EXPECT() *MockRequesterMockRecorder { return m.recorder } -// EntityByID mocks base method +// EntityByID mocks base method. func (m *MockRequester) EntityByID(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "EntityByID", arg0, arg1) } -// EntityByID indicates an expected call of EntityByID +// EntityByID indicates an expected call of EntityByID. func (mr *MockRequesterMockRecorder) EntityByID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EntityByID", reflect.TypeOf((*MockRequester)(nil).EntityByID), arg0, arg1) } -// Force mocks base method +// Force mocks base method. func (m *MockRequester) Force() { m.ctrl.T.Helper() m.ctrl.Call(m, "Force") } -// Force indicates an expected call of Force +// Force indicates an expected call of Force. func (mr *MockRequesterMockRecorder) Force() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Force", reflect.TypeOf((*MockRequester)(nil).Force)) } -// Query mocks base method +// Query mocks base method. func (m *MockRequester) Query(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Query", arg0, arg1) } -// Query indicates an expected call of Query +// Query indicates an expected call of Query. func (mr *MockRequesterMockRecorder) Query(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockRequester)(nil).Query), arg0, arg1) diff --git a/network/p2p/dht.go b/network/p2p/dht.go index b21211a9adb..a5b82657cb9 100644 --- a/network/p2p/dht.go +++ b/network/p2p/dht.go @@ -39,7 +39,7 @@ func AsServer(enable bool) dht.Option { if enable { return dht.Mode(dht.ModeServer) } - return dht.Mode(dht.ModeAuto) + return dht.Mode(dht.ModeClient) } func WithBootstrapPeers(bootstrapNodes flow.IdentityList) (dht.Option, error) { diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index dc300338ac4..cc01eedaafb 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -43,6 +43,52 @@ func (suite *DHTTestSuite) TearDownTest() { suite.cancel() } +func (suite *DHTTestSuite) TestFindPeerWithDHT() { + count := 10 + golog.SetAllLoggers(golog.LevelFatal) // change this to Debug if libp2p logs are needed + + dhtServerNodes := suite.CreateNodes(2, true) + require.Len(suite.T(), dhtServerNodes, 2) + + dhtClientNodes := suite.CreateNodes(count-2, false) + + nodes := append(dhtServerNodes, dhtClientNodes...) + defer suite.StopNodes(nodes) + + getDhtServerAddr := func(i uint) peer.AddrInfo { + return peer.AddrInfo{ID: dhtServerNodes[i].host.ID(), Addrs: dhtServerNodes[i].host.Addrs()} + } + + for i, clientNode := range dhtClientNodes { + err := clientNode.host.Connect(suite.ctx, getDhtServerAddr(uint(i%2))) + require.NoError(suite.T(), err) + } + + require.Eventually(suite.T(), func() bool { + for i, clientNode := range dhtClientNodes { + if clientNode.dht.RoutingTable().Find(getDhtServerAddr(uint(i%2)).ID) == "" { + return false + } + } + return true + }, time.Second*5, tickForAssertEventually, "nodes failed to connect") + + err := dhtServerNodes[0].host.Connect(suite.ctx, getDhtServerAddr(1)) + require.NoError(suite.T(), err) + + require.Eventually(suite.T(), func() bool { + return dhtServerNodes[0].dht.RoutingTable().Find(getDhtServerAddr(1).ID) != "" + }, time.Second*5, tickForAssertEventually, "dht servers failed to connect") + + for i := 0; i < len(dhtClientNodes); i += 2 { + for j := 1; j < len(dhtClientNodes); j += 2 { + dhtClientNodes[i].host.Peerstore().ClearAddrs(dhtClientNodes[j].host.ID()) + _, err = dhtClientNodes[i].CreateStream(suite.ctx, dhtClientNodes[j].host.ID()) + require.NoError(suite.T(), err) + } + } +} + // TestPubSub checks if nodes can subscribe to a topic and send and receive a message on that topic. The DHT discovery // mechanism is used for nodes to find each other. func (suite *DHTTestSuite) TestPubSubWithDHTDiscovery() { diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index cae93c5daa7..40a682795f5 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -334,18 +334,19 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { if len(n.host.Peerstore().Addrs(peerID)) == 0 { n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - if n.dht != nil { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - // try to find the peer using the dht - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) - } - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + if n.dht == nil { + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } + + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + // try to find the peer using the dht + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) } - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 2967a3e5cce..7a829429fc4 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -5,37 +5,38 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" - reflect "reflect" ) -// MockBlocks is a mock of Blocks interface +// MockBlocks is a mock of Blocks interface. type MockBlocks struct { ctrl *gomock.Controller recorder *MockBlocksMockRecorder } -// MockBlocksMockRecorder is the mock recorder for MockBlocks +// MockBlocksMockRecorder is the mock recorder for MockBlocks. type MockBlocksMockRecorder struct { mock *MockBlocks } -// NewMockBlocks creates a new mock instance +// NewMockBlocks creates a new mock instance. func NewMockBlocks(ctrl *gomock.Controller) *MockBlocks { mock := &MockBlocks{ctrl: ctrl} mock.recorder = &MockBlocksMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockBlocks) EXPECT() *MockBlocksMockRecorder { return m.recorder } -// ByCollectionID mocks base method +// ByCollectionID mocks base method. func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByCollectionID", arg0) @@ -44,13 +45,13 @@ func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByCollectionID indicates an expected call of ByCollectionID +// ByCollectionID indicates an expected call of ByCollectionID. func (mr *MockBlocksMockRecorder) ByCollectionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByCollectionID", reflect.TypeOf((*MockBlocks)(nil).ByCollectionID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -59,13 +60,13 @@ func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockBlocksMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockBlocks)(nil).ByHeight), arg0) } -// ByID mocks base method +// ByID mocks base method. func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -74,13 +75,13 @@ func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockBlocksMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockBlocks)(nil).ByID), arg0) } -// GetLastFullBlockHeight mocks base method +// GetLastFullBlockHeight mocks base method. func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetLastFullBlockHeight") @@ -89,13 +90,13 @@ func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { return ret0, ret1 } -// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight +// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight. func (mr *MockBlocksMockRecorder) GetLastFullBlockHeight() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).GetLastFullBlockHeight)) } -// IndexBlockForCollections mocks base method +// IndexBlockForCollections mocks base method. func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexBlockForCollections", arg0, arg1) @@ -103,13 +104,13 @@ func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow. return ret0 } -// IndexBlockForCollections indicates an expected call of IndexBlockForCollections +// IndexBlockForCollections indicates an expected call of IndexBlockForCollections. func (mr *MockBlocksMockRecorder) IndexBlockForCollections(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexBlockForCollections", reflect.TypeOf((*MockBlocks)(nil).IndexBlockForCollections), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockBlocks) Store(arg0 *flow.Block) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -117,13 +118,13 @@ func (m *MockBlocks) Store(arg0 *flow.Block) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockBlocksMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockBlocks)(nil).Store), arg0) } -// StoreTx mocks base method +// StoreTx mocks base method. func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreTx", arg0) @@ -131,13 +132,13 @@ func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { return ret0 } -// StoreTx indicates an expected call of StoreTx +// StoreTx indicates an expected call of StoreTx. func (mr *MockBlocksMockRecorder) StoreTx(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreTx", reflect.TypeOf((*MockBlocks)(nil).StoreTx), arg0) } -// UpdateLastFullBlockHeight mocks base method +// UpdateLastFullBlockHeight mocks base method. func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "UpdateLastFullBlockHeight", arg0) @@ -145,36 +146,36 @@ func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { return ret0 } -// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight +// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight. func (mr *MockBlocksMockRecorder) UpdateLastFullBlockHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).UpdateLastFullBlockHeight), arg0) } -// MockHeaders is a mock of Headers interface +// MockHeaders is a mock of Headers interface. type MockHeaders struct { ctrl *gomock.Controller recorder *MockHeadersMockRecorder } -// MockHeadersMockRecorder is the mock recorder for MockHeaders +// MockHeadersMockRecorder is the mock recorder for MockHeaders. type MockHeadersMockRecorder struct { mock *MockHeaders } -// NewMockHeaders creates a new mock instance +// NewMockHeaders creates a new mock instance. func NewMockHeaders(ctrl *gomock.Controller) *MockHeaders { mock := &MockHeaders{ctrl: ctrl} mock.recorder = &MockHeadersMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockHeaders) EXPECT() *MockHeadersMockRecorder { return m.recorder } -// BatchIndexByChunkID mocks base method +// BatchIndexByChunkID mocks base method. func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchIndexByChunkID", arg0, arg1, arg2) @@ -182,13 +183,13 @@ func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 stora return ret0 } -// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID +// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID. func (mr *MockHeadersMockRecorder) BatchIndexByChunkID(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchIndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).BatchIndexByChunkID), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -197,13 +198,13 @@ func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockHeadersMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockHeaders)(nil).ByBlockID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -212,13 +213,13 @@ func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockHeadersMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockHeaders)(nil).ByHeight), arg0) } -// ByParentID mocks base method +// ByParentID mocks base method. func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByParentID", arg0) @@ -227,13 +228,13 @@ func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { return ret0, ret1 } -// ByParentID indicates an expected call of ByParentID +// ByParentID indicates an expected call of ByParentID. func (mr *MockHeadersMockRecorder) ByParentID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByParentID", reflect.TypeOf((*MockHeaders)(nil).ByParentID), arg0) } -// IDByChunkID mocks base method +// IDByChunkID mocks base method. func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IDByChunkID", arg0) @@ -242,13 +243,13 @@ func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) return ret0, ret1 } -// IDByChunkID indicates an expected call of IDByChunkID +// IDByChunkID indicates an expected call of IDByChunkID. func (mr *MockHeadersMockRecorder) IDByChunkID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDByChunkID", reflect.TypeOf((*MockHeaders)(nil).IDByChunkID), arg0) } -// IndexByChunkID mocks base method +// IndexByChunkID mocks base method. func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexByChunkID", arg0, arg1) @@ -256,13 +257,13 @@ func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { return ret0 } -// IndexByChunkID indicates an expected call of IndexByChunkID +// IndexByChunkID indicates an expected call of IndexByChunkID. func (mr *MockHeadersMockRecorder) IndexByChunkID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).IndexByChunkID), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockHeaders) Store(arg0 *flow.Header) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -270,36 +271,36 @@ func (m *MockHeaders) Store(arg0 *flow.Header) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockHeadersMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockHeaders)(nil).Store), arg0) } -// MockPayloads is a mock of Payloads interface +// MockPayloads is a mock of Payloads interface. type MockPayloads struct { ctrl *gomock.Controller recorder *MockPayloadsMockRecorder } -// MockPayloadsMockRecorder is the mock recorder for MockPayloads +// MockPayloadsMockRecorder is the mock recorder for MockPayloads. type MockPayloadsMockRecorder struct { mock *MockPayloads } -// NewMockPayloads creates a new mock instance +// NewMockPayloads creates a new mock instance. func NewMockPayloads(ctrl *gomock.Controller) *MockPayloads { mock := &MockPayloads{ctrl: ctrl} mock.recorder = &MockPayloadsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockPayloads) EXPECT() *MockPayloadsMockRecorder { return m.recorder } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -308,13 +309,13 @@ func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockPayloadsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockPayloads)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -322,36 +323,36 @@ func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockPayloadsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockPayloads)(nil).Store), arg0, arg1) } -// MockCollections is a mock of Collections interface +// MockCollections is a mock of Collections interface. type MockCollections struct { ctrl *gomock.Controller recorder *MockCollectionsMockRecorder } -// MockCollectionsMockRecorder is the mock recorder for MockCollections +// MockCollectionsMockRecorder is the mock recorder for MockCollections. type MockCollectionsMockRecorder struct { mock *MockCollections } -// NewMockCollections creates a new mock instance +// NewMockCollections creates a new mock instance. func NewMockCollections(ctrl *gomock.Controller) *MockCollections { mock := &MockCollections{ctrl: ctrl} mock.recorder = &MockCollectionsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCollections) EXPECT() *MockCollectionsMockRecorder { return m.recorder } -// ByID mocks base method +// ByID mocks base method. func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -360,13 +361,13 @@ func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockCollectionsMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockCollections)(nil).ByID), arg0) } -// LightByID mocks base method +// LightByID mocks base method. func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByID", arg0) @@ -375,13 +376,13 @@ func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection return ret0, ret1 } -// LightByID indicates an expected call of LightByID +// LightByID indicates an expected call of LightByID. func (mr *MockCollectionsMockRecorder) LightByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByID", reflect.TypeOf((*MockCollections)(nil).LightByID), arg0) } -// LightByTransactionID mocks base method +// LightByTransactionID mocks base method. func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByTransactionID", arg0) @@ -390,13 +391,13 @@ func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.Ligh return ret0, ret1 } -// LightByTransactionID indicates an expected call of LightByTransactionID +// LightByTransactionID indicates an expected call of LightByTransactionID. func (mr *MockCollectionsMockRecorder) LightByTransactionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByTransactionID", reflect.TypeOf((*MockCollections)(nil).LightByTransactionID), arg0) } -// Remove mocks base method +// Remove mocks base method. func (m *MockCollections) Remove(arg0 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Remove", arg0) @@ -404,13 +405,13 @@ func (m *MockCollections) Remove(arg0 flow.Identifier) error { return ret0 } -// Remove indicates an expected call of Remove +// Remove indicates an expected call of Remove. func (mr *MockCollectionsMockRecorder) Remove(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockCollections)(nil).Remove), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCollections) Store(arg0 *flow.Collection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -418,13 +419,13 @@ func (m *MockCollections) Store(arg0 *flow.Collection) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCollectionsMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCollections)(nil).Store), arg0) } -// StoreLight mocks base method +// StoreLight mocks base method. func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLight", arg0) @@ -432,13 +433,13 @@ func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { return ret0 } -// StoreLight indicates an expected call of StoreLight +// StoreLight indicates an expected call of StoreLight. func (mr *MockCollectionsMockRecorder) StoreLight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLight", reflect.TypeOf((*MockCollections)(nil).StoreLight), arg0) } -// StoreLightAndIndexByTransaction mocks base method +// StoreLightAndIndexByTransaction mocks base method. func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLightAndIndexByTransaction", arg0) @@ -446,36 +447,36 @@ func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollec return ret0 } -// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction +// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction. func (mr *MockCollectionsMockRecorder) StoreLightAndIndexByTransaction(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLightAndIndexByTransaction", reflect.TypeOf((*MockCollections)(nil).StoreLightAndIndexByTransaction), arg0) } -// MockCommits is a mock of Commits interface +// MockCommits is a mock of Commits interface. type MockCommits struct { ctrl *gomock.Controller recorder *MockCommitsMockRecorder } -// MockCommitsMockRecorder is the mock recorder for MockCommits +// MockCommitsMockRecorder is the mock recorder for MockCommits. type MockCommitsMockRecorder struct { mock *MockCommits } -// NewMockCommits creates a new mock instance +// NewMockCommits creates a new mock instance. func NewMockCommits(ctrl *gomock.Controller) *MockCommits { mock := &MockCommits{ctrl: ctrl} mock.recorder = &MockCommitsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -483,13 +484,13 @@ func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockCommitsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockCommits)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -498,13 +499,13 @@ func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, err return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockCommitsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockCommits)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -512,36 +513,36 @@ func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) err return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCommitsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCommits)(nil).Store), arg0, arg1) } -// MockEvents is a mock of Events interface +// MockEvents is a mock of Events interface. type MockEvents struct { ctrl *gomock.Controller recorder *MockEventsMockRecorder } -// MockEventsMockRecorder is the mock recorder for MockEvents +// MockEventsMockRecorder is the mock recorder for MockEvents. type MockEventsMockRecorder struct { mock *MockEvents } -// NewMockEvents creates a new mock instance +// NewMockEvents creates a new mock instance. func NewMockEvents(ctrl *gomock.Controller) *MockEvents { mock := &MockEvents{ctrl: ctrl} mock.recorder = &MockEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockEvents) EXPECT() *MockEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -549,13 +550,13 @@ func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, ar return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -564,13 +565,13 @@ func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockEvents)(nil).ByBlockID), arg0) } -// ByBlockIDEventType mocks base method +// ByBlockIDEventType mocks base method. func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventType) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDEventType", arg0, arg1) @@ -579,13 +580,13 @@ func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventTyp return ret0, ret1 } -// ByBlockIDEventType indicates an expected call of ByBlockIDEventType +// ByBlockIDEventType indicates an expected call of ByBlockIDEventType. func (mr *MockEventsMockRecorder) ByBlockIDEventType(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDEventType", reflect.TypeOf((*MockEvents)(nil).ByBlockIDEventType), arg0, arg1) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -594,36 +595,36 @@ func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow. return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockEventsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockEvents)(nil).ByBlockIDTransactionID), arg0, arg1) } -// MockServiceEvents is a mock of ServiceEvents interface +// MockServiceEvents is a mock of ServiceEvents interface. type MockServiceEvents struct { ctrl *gomock.Controller recorder *MockServiceEventsMockRecorder } -// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents +// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents. type MockServiceEventsMockRecorder struct { mock *MockServiceEvents } -// NewMockServiceEvents creates a new mock instance +// NewMockServiceEvents creates a new mock instance. func NewMockServiceEvents(ctrl *gomock.Controller) *MockServiceEvents { mock := &MockServiceEvents{ctrl: ctrl} mock.recorder = &MockServiceEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -631,13 +632,13 @@ func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockServiceEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockServiceEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -646,36 +647,36 @@ func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockServiceEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockServiceEvents)(nil).ByBlockID), arg0) } -// MockTransactionResults is a mock of TransactionResults interface +// MockTransactionResults is a mock of TransactionResults interface. type MockTransactionResults struct { ctrl *gomock.Controller recorder *MockTransactionResultsMockRecorder } -// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults +// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults. type MockTransactionResultsMockRecorder struct { mock *MockTransactionResults } -// NewMockTransactionResults creates a new mock instance +// NewMockTransactionResults creates a new mock instance. func NewMockTransactionResults(ctrl *gomock.Controller) *MockTransactionResults { mock := &MockTransactionResults{ctrl: ctrl} mock.recorder = &MockTransactionResultsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -683,13 +684,13 @@ func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.Tr return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockTransactionResultsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockTransactionResults)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) (*flow.TransactionResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -698,7 +699,7 @@ func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifi return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockTransactionResultsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockTransactionResults)(nil).ByBlockIDTransactionID), arg0, arg1) From f3c404a00d0c05556214cf5540b552d72dae828d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 19:44:20 -0700 Subject: [PATCH 115/291] fix test --- engine/common/synchronization/engine_test.go | 17 ++++++++++++++--- engine/testutil/nodes.go | 11 ++++++++++- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 093fa9cc2a0..92d488c787c 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -18,12 +18,15 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" synccore "github.com/onflow/flow-go/module/synchronization" netint "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/mocknetwork" + "github.com/onflow/flow-go/network/p2p" protocolint "github.com/onflow/flow-go/state/protocol" + protocolEvents "github.com/onflow/flow-go/state/protocol/events" protocol "github.com/onflow/flow-go/state/protocol/mock" storerr "github.com/onflow/flow-go/storage" storage "github.com/onflow/flow-go/storage/mock" @@ -161,7 +164,15 @@ func (ss *SyncSuite) SetupTest() { finalizedHeader, err := NewFinalizedHeaderCache(log, ss.state, pubsub.NewFinalizationDistributor()) require.NoError(ss.T(), err, "could not create finalized snapshot cache") - e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, ss.state) + idCache, err := p2p.NewProtocolStateIDCache(log, ss.state, protocolEvents.NewDistributor()) + e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(ss.me.NodeID())), + ), + idCache, + )) require.NoError(ss.T(), err, "should pass engine initialization") ss.e = e @@ -415,7 +426,7 @@ func (ss *SyncSuite) TestSendRequests() { ss.core.On("BatchRequested", batches[0]) // exclude my node ID - ss.e.sendRequests(ss.participants[1:], ranges, batches) + ss.e.sendRequests(ss.participants[1:].NodeIDs(), ranges, batches) ss.con.AssertExpectations(ss.T()) } @@ -472,6 +483,6 @@ func (ss *SyncSuite) TestOnFinalizedBlock() { err := ss.e.finalizedHeader.updateHeader() require.NoError(ss.T(), err) actualHeader := ss.e.finalizedHeader.Get() - require.ElementsMatch(ss.T(), ss.e.getParticipants(actualHeader.ID()), ss.participants[1:]) + require.ElementsMatch(ss.T(), ss.e.participantsProvider.Identifiers(), ss.participants[1:].NodeIDs()) require.Equal(ss.T(), actualHeader, &finalizedBlock) } diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index eedd08a0f8a..3688e2c6727 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -53,6 +53,7 @@ import ( "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" @@ -64,6 +65,7 @@ import ( chainsync "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/module/validation" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/stub" "github.com/onflow/flow-go/state/protocol" badgerstate "github.com/onflow/flow-go/state/protocol/badger" @@ -625,6 +627,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit finalizedHeader, err := synchronization.NewFinalizedHeaderCache(node.Log, node.State, finalizationDistributor) require.NoError(t, err) + idCache, err := p2p.NewProtocolStateIDCache(node.Log, node.State, events.NewDistributor()) syncEngine, err := synchronization.New( node.Log, node.Metrics, @@ -634,7 +637,13 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ), synchronization.WithPollInterval(time.Duration(0)), ) require.NoError(t, err) From 2f6b4d2d9c59ac998598521dc9bb7e51f5a02045 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 20:03:22 -0700 Subject: [PATCH 116/291] Update testUtil.go --- network/test/testUtil.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 8f46326bcca..868f957027f 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -142,7 +142,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultPeerUpdateInterval, p2p.DefaultUnicastTimeout, true, - true) + true, + ) } return mws } From c391f1b19fece0cc237076d3f4ad695f885e231b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Fri, 20 Aug 2021 12:51:30 -0400 Subject: [PATCH 117/291] [network] Create a simple FixedTableIdentityTranslator, fix compilation --- network/test/testUtil.go | 47 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 868f957027f..72686913292 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -89,6 +89,49 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics } } +type FixedTableIdentityTranslator struct { + flow2p2p map[flow.Identifier]peer.ID + p2p2flow map[peer.ID]flow.Identifier +} + +func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + nodeID, ok := t.p2p2flow[p] + if !ok { + return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) + } + return nodeID, nil +} + +func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + peerID, ok := t.flow2p2p[n] + if !ok { + return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) + } + return peerID, nil +} + +func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableIdentityTranslator { + flow2p2p := make(map[flow.Identifier]peer.ID) + p2p2flow := make(map[peer.ID]flow.Identifier) + + for identity := range identities { + nodeID := identity.ID() + networkKey := identity.NetworkPublicKey + peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + if err != nil { + panic("could not interpret a network public key from Flow, test identities setup problem") + } + peerID, err := peer.IDFromPublicKey(peerPK) + if err != nil { + panic("could not generate a PeerID from public Key, test identities setup problem") + } + + flow2p2p[nodeID] = peerID + p2p2flow[peerID] = nodeID + } + return &FixedTableIdentityTranslator{flow2p2p, p2p2flow} +} + // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. @@ -133,6 +176,9 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } + // create a fixed id translator for the identities + tableTranslator := NewFixedTableIdentityTranslator(identities) + // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, factory, @@ -143,6 +189,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultUnicastTimeout, true, true, + tableTranslator, ) } return mws From 1dc197050bb39542746af5e4d36cb14941b7c2d3 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 10:13:31 -0700 Subject: [PATCH 118/291] Update libp2pNode_test.go Update middleware_test.go Update testUtil.go fixed identity provider Update testUtil.go [network] Fix the compilation of a bunch of remaining tests --- cmd/access/node_builder/upstream_connector.go | 5 +- module/id/fixed_provider.go | 12 +++++ network/p2p/dht_test.go | 2 + network/p2p/libp2pNode.go | 15 ++++-- network/p2p/libp2pNode_test.go | 54 ++++++++++--------- network/p2p/sporking_test.go | 5 +- network/test/middleware_test.go | 3 +- network/test/testUtil.go | 20 ++++--- 8 files changed, 74 insertions(+), 42 deletions(-) diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index 00d4f7a7f3d..fdc608cfadb 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -106,7 +106,8 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f default: } - peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) + peerAddrInfo, err := p2p.PeerAddressInfo(bootstrapPeer) + if err != nil { resultChan <- result{ id: flow.Identity{}, @@ -115,7 +116,7 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f } // try and connect to the bootstrap server - err = connector.unstakedNode.AddPeer(ctx, peerID) + err = connector.unstakedNode.AddPeer(ctx, peerAddrInfo) resultChan <- result{ id: bootstrapPeer, err: err, diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index dbdf10b9403..106047510b6 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -15,3 +15,15 @@ func NewFixedIdentifierProvider(identifiers flow.IdentifierList) *FixedIdentifie func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { return p.identifiers } + +type FixedIdentityProvider struct { + identities flow.IdentityList +} + +func NewFixedIdentityProvider(identities flow.IdentityList) *FixedIdentityProvider { + return &FixedIdentityProvider{identities} +} + +func (p *FixedIdentityProvider) Identities(flow.IdentityFilter) flow.IdentityList { + return p.identities +} diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index cc01eedaafb..6643a7652b0 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -29,6 +29,8 @@ type DHTTestSuite struct { cancel context.CancelFunc // used to cancel the context } +// TODO: test that dht findPeer times out + // TestDHTTestSuite test the libp2p pubsub with DHT for discovery func TestDHTTestSuite(t *testing.T) { suite.Run(t, new(DHTTestSuite)) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 40a682795f5..0fcb148ccd2 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -45,7 +45,7 @@ const ( // timeout for FindPeer queries to the DHT // TODO: is this a sensible value? - findPeerQueryTimeout = 15 * time.Second + findPeerQueryTimeout = 10 * time.Second ) // LibP2PFactoryFunc is a factory function type for generating libp2p Node instances. @@ -339,10 +339,15 @@ func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stre } n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - // try to find the peer using the dht - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + if err != nil { return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) } diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index 2497d3f3f76..f6ad627c76c 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -16,6 +16,7 @@ import ( addrutil "github.com/libp2p/go-addr-util" "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" swarm "github.com/libp2p/go-libp2p-swarm" "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr/net" @@ -146,11 +147,11 @@ func (suite *LibP2PNodeTestSuite) TestAddPeers() { require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, peerInfo)) } - // Checks if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) + // Checks if both of the other nodes have been added as peers to the first node + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count-1) } -// TestAddPeers checks if nodes can be added as peers to a given node +// TestRemovePeers checks if nodes can be removed as peers from a given node func (suite *LibP2PNodeTestSuite) TestRemovePeers() { count := 3 @@ -166,8 +167,8 @@ func (suite *LibP2PNodeTestSuite) TestRemovePeers() { require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, pInfo)) } - // check if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) + // check if all other nodes have been added as peers to the first node + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count-1) // disconnect from each peer and assert that the connection no longer exists for _, pInfo := range peerInfos[1:] { @@ -195,7 +196,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStream() { for i := 0; i < 100; i++ { pInfo, err := PeerAddressInfo(*id2) require.NoError(suite.T(), err) - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo)) + nodes[0].host.Peerstore().AddAddrs(pInfo.ID, pInfo.Addrs, peerstore.AddressTTL) anotherStream, err := nodes[0].CreateStream(context.Background(), pInfo.ID) // Assert that a stream was returned without error require.NoError(suite.T(), err) @@ -253,7 +254,7 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { require.NoError(suite.T(), err) // Create stream from node 1 to node 2 - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo2)) + nodes[0].host.Peerstore().AddAddrs(pInfo2.ID, pInfo2.Addrs, peerstore.AddressTTL) s1, err := nodes[0].CreateStream(context.Background(), pInfo2.ID) assert.NoError(suite.T(), err) rw := bufio.NewReadWriter(bufio.NewReader(s1), bufio.NewWriter(s1)) @@ -275,7 +276,7 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } // Create stream from node 2 to node 1 - require.NoError(suite.T(), nodes[1].AddPeer(context.Background(), pInfo1)) + nodes[1].host.Peerstore().AddAddrs(pInfo1.ID, pInfo1.Addrs, peerstore.AddressTTL) s2, err := nodes[1].CreateStream(context.Background(), pInfo1.ID) assert.NoError(suite.T(), err) rw = bufio.NewReadWriter(bufio.NewReader(s2), bufio.NewWriter(s2)) @@ -296,8 +297,8 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } } -// TestCreateStreamTimeoutWithUnresponsiveNode tests that the CreateStream call does not block longer than the default -// unicast timeout interval +// TestCreateStreamTimeoutWithUnresponsiveNode tests that the CreateStream call does not block longer than the +// timeout interval func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() { // creates a regular node @@ -314,18 +315,20 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() silentNodeInfo, err := PeerAddressInfo(silentNodeId) require.NoError(suite.T(), err) + timeout := 1 * time.Second + // setup the context to expire after the default timeout - ctx, cancel := context.WithTimeout(context.Background(), DefaultUnicastTimeout) + ctx, cancel := context.WithTimeout(context.Background(), timeout) defer cancel() // attempt to create a stream from node 1 to node 2 and assert that it fails after timeout - grace := 1 * time.Second + grace := 100 * time.Millisecond unittest.AssertReturnsBefore(suite.T(), func() { - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), silentNodeInfo)) + nodes[0].host.Peerstore().AddAddrs(silentNodeInfo.ID, silentNodeInfo.Addrs, peerstore.AddressTTL) _, err = nodes[0].CreateStream(ctx, silentNodeInfo.ID) }, - DefaultUnicastTimeout+grace) + timeout+grace) assert.Error(suite.T(), err) } @@ -350,7 +353,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // creates a stream to unresponsive node and makes sure that the stream creation is blocked blockedCallCh := unittest.RequireNeverReturnBefore(suite.T(), func() { - require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), silentNodeInfo)) + goodNodes[0].host.Peerstore().AddAddrs(silentNodeInfo.ID, silentNodeInfo.Addrs, peerstore.AddressTTL) _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeInfo.ID) // this call will block }, 1*time.Second, @@ -359,7 +362,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // requires same peer can still connect to the other regular peer without being blocked unittest.RequireReturnsBefore(suite.T(), func() { - require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), goodNodeInfo1)) + goodNodes[0].host.Peerstore().AddAddrs(goodNodeInfo1.ID, goodNodeInfo1.Addrs, peerstore.AddressTTL) _, err := goodNodes[0].CreateStream(suite.ctx, goodNodeInfo1.ID) require.NoError(suite.T(), err) }, @@ -389,7 +392,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { createStream := func() { <-gate - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + nodes[0].host.Peerstore().AddAddrs(nodeInfo1.ID, nodeInfo1.Addrs, peerstore.AddressTTL) _, err := nodes[0].CreateStream(suite.ctx, nodeInfo1.ID) assert.NoError(suite.T(), err) // assert that stream was successfully created wg.Done() @@ -455,7 +458,7 @@ func (suite *LibP2PNodeTestSuite) TestStreamClosing() { for i := 0; i < count; i++ { // Create stream from node 1 to node 2 (reuse if one already exists) - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + nodes[0].host.Peerstore().AddAddrs(nodeInfo1.ID, nodeInfo1.Addrs, peerstore.AddressTTL) s, err := nodes[0].CreateStream(context.Background(), nodeInfo1.ID) assert.NoError(suite.T(), err) w := bufio.NewWriter(s) @@ -513,9 +516,10 @@ func (suite *LibP2PNodeTestSuite) TestPing() { func testPing(t *testing.T, source *Node, target flow.Identity, expectedVersion string, expectedHeight uint64) { pctx, cancel := context.WithCancel(context.Background()) defer cancel() - pid, err := ExtractPeerID(target.NetworkPubKey) + pInfo, err := PeerAddressInfo(target) assert.NoError(t, err) - resp, rtt, err := source.Ping(pctx, pid) + source.host.Peerstore().AddAddrs(pInfo.ID, pInfo.Addrs, peerstore.AddressTTL) + resp, rtt, err := source.Ping(pctx, pInfo.ID) assert.NoError(t, err) assert.NotZero(t, rtt) assert.Equal(t, expectedVersion, resp.Version) @@ -547,10 +551,10 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { suite.Run("outbound connection to a not-allowed node is rejected", func() { // node1 and node2 both have no allowListed peers - requireError(node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err := node1.CreateStream(suite.ctx, node2Info.ID) requireError(err) - requireError(node2.AddPeer(context.Background(), node1Info)) + node2.host.Peerstore().AddAddrs(node1Info.ID, node1Info.Addrs, peerstore.AddressTTL) _, err = node2.CreateStream(suite.ctx, node1Info.ID) requireError(err) }) @@ -562,7 +566,7 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { // node1 attempts to connect to node2 // node2 should reject the inbound connection - require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.Error(suite.T(), err) }) @@ -575,11 +579,11 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { node2.UpdateAllowList(peer.IDSlice{node1Info.ID}) // node1 should be allowed to connect to node2 - require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.NoError(suite.T(), err) // node2 should be allowed to connect to node1 - require.NoError(suite.T(), node2.AddPeer(context.Background(), node1Info)) + node2.host.Peerstore().AddAddrs(node1Info.ID, node1Info.Addrs, peerstore.AddressTTL) _, err = node2.CreateStream(suite.ctx, node1Info.ID) require.NoError(suite.T(), err) }) diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 429f522bb0f..b3462e05ccb 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -7,6 +7,7 @@ import ( golog "github.com/ipfs/go-log" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" @@ -177,8 +178,8 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { } func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { - require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from node 1 to node 2 + sourceNode.host.Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation succeeded require.NoError(t, err) @@ -186,8 +187,8 @@ func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer } func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { - require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from source node to destination address + sourceNode.host.Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation failed assert.Error(t, err) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 1a9d5b3fa16..3ccdc66a058 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -110,8 +110,7 @@ func (m *MiddlewareTestSuite) SetupTest() { } for i, mw := range m.mws { assert.NoError(m.T(), mw.Start(m.ov[i])) - err := mw.UpdateAllowList() - require.NoError(m.T(), err) + mw.UpdateAllowList() } } diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 72686913292..5d3b824c1a6 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -114,10 +114,10 @@ func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableId flow2p2p := make(map[flow.Identifier]peer.ID) p2p2flow := make(map[peer.ID]flow.Identifier) - for identity := range identities { + for _, identity := range identities { nodeID := identity.ID() - networkKey := identity.NetworkPublicKey - peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + networkKey := identity.NetworkPubKey + peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) if err != nil { panic("could not interpret a network public key from Flow, test identities setup problem") } @@ -232,7 +232,17 @@ func GenerateNetworks(t *testing.T, me.On("Address").Return(ids[i].Address) // create the network - net, err := p2p.NewNetwork(log, cbor.NewCodec(), ids, me, mws[i], csize, tops[i], sms[i], metrics) + net, err := p2p.NewNetwork( + log, + cbor.NewCodec(), + me, + mws[i], + csize, + tops[i], + sms[i], + metrics, + id.NewFixedIdentityProvider(ids), + ) require.NoError(t, err) nets = append(nets, net) @@ -242,8 +252,6 @@ func GenerateNetworks(t *testing.T, if !dryRunMode { for _, net := range nets { <-net.Ready() - err := net.SetIDs(ids) - require.NoError(t, err) } } return nets From 478342bff033372e694aa6702f3a38f510e3b159 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 14:00:53 -0400 Subject: [PATCH 119/291] Last testUtil fixes Update peerManager_test.go fix epoch transition test Update unstaked_translator.go fixed middleware_test Update execution_test.go --- engine/common/synchronization/engine_test.go | 6 + engine/execution/execution_test.go | 68 ++++++++-- network/p2p/hierarchical_translator_test.go | 15 +++ network/p2p/middleware.go | 26 +++- network/p2p/peerManager_test.go | 128 +++++++++++-------- network/p2p/unstaked_translator.go | 15 ++- network/test/epochtransition_test.go | 57 +++------ network/test/middleware_test.go | 8 +- network/test/testUtil.go | 16 +-- 9 files changed, 215 insertions(+), 124 deletions(-) create mode 100644 network/p2p/hierarchical_translator_test.go diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 92d488c787c..4e79938fab7 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -61,6 +61,12 @@ func (ss *SyncSuite) SetupTest() { // generate own ID ss.participants = unittest.IdentityListFixture(3, unittest.WithRole(flow.RoleConsensus)) + keys, err := unittest.NetworkingKeys(len(ss.participants)) + require.NoError(ss.T(), err) + + for i, p := range ss.participants { + p.NetworkPubKey = keys[i].PublicKey() + } ss.myID = ss.participants[0].NodeID // generate a header for the final state diff --git a/engine/execution/execution_test.go b/engine/execution/execution_test.go index c7201df0e15..76d2ae13137 100644 --- a/engine/execution/execution_test.go +++ b/engine/execution/execution_test.go @@ -37,10 +37,24 @@ func TestExecutionFlow(t *testing.T) { chainID := flow.Testnet - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exeID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - verID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) + keys, errs := unittest.NetworkingKeys(4) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exeID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) + verID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[3].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID) @@ -286,12 +300,23 @@ func TestExecutionStateSyncMultipleExecutionNodes(t *testing.T) { chainID := flow.Emulator - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exe1ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - // exe2ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) + keys, errs := unittest.NetworkingKeys(4) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exe1ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exe1ID) + identities[3].NetworkPubKey = keys[3].PublicKey() collectionNode := testutil.GenericNodeFromParticipants(t, hub, colID, identities, chainID) defer collectionNode.Done() @@ -424,11 +449,28 @@ func TestBroadcastToMultipleVerificationNodes(t *testing.T) { chainID := flow.Emulator - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exeID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - ver1ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - ver2ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) + keys, errs := unittest.NetworkingKeys(5) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exeID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) + ver1ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[3].PublicKey()), + ) + ver2ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[4].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, ver1ID, ver2ID) diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go new file mode 100644 index 00000000000..e111ecd0e85 --- /dev/null +++ b/network/p2p/hierarchical_translator_test.go @@ -0,0 +1,15 @@ +package p2p + +import ( + "testing" + + "github.com/stretchr/testify/suite" +) + +type HierarchicalTranslatorTestSuite struct { + suite.Suite +} + +func TestHierarchicalTranslator(t *testing.T) { + suite.Run(t, new(HierarchicalTranslatorTestSuite)) +} diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 8f55761a386..32b3ee16ef5 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -249,7 +249,6 @@ func (m *Middleware) Start(ov network.Overlay) error { // Stop will end the execution of the middleware and wait for it to end. func (m *Middleware) Stop() { - mgr, found := m.peerMgr() if found { // stops peer manager @@ -389,6 +388,31 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { return nil } +// processAuthenticatedMessage processes a message and a source (indicated by its PublicKey) and eventually passes it to the overlay +// In particular, it checks the claim of protocol authorship situated in the message against `originKey` +// The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` +// this requirement is fulfilled by e.g. the output of readConnection and readSubscription +func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { + identities := m.ov.Identities() + + // check the origin of the message corresponds to the one claimed in the OriginID + originID := flow.HashToID(msg.OriginID) + + originIdentity, found := identities.ByNodeID(originID) + if !found { + m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) + return + } else if originIdentity.NetworkPubKey == nil { + m.log.Warn().Msgf("received message with claimed originID %x, wich has no network identifiers at this node, and was dropped", originID) + return + } else if !originIdentity.NetworkPubKey.Equals(originKey) { + m.log.Warn().Msgf("received message claiming to be from nodeID %v with key %x was actually signed by %x and dropped", originID, originIdentity.NetworkPubKey, originKey) + return + } + + m.processMessage(msg) +} + // processMessage processes a message and eventually passes it to the overlay func (m *Middleware) processMessage(msg *message.Message) { diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 571a4a8108b..5c6632fbc76 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -1,6 +1,7 @@ package p2p import ( + "math/rand" "os" "sync" "testing" @@ -15,8 +16,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/utils/unittest" ) @@ -35,33 +34,41 @@ func (suite *PeerManagerTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) } +func (suite *PeerManagerTestSuite) generateIdentities(n int) (flow.IdentityList, peer.IDSlice) { + // create some test ids + currentIDs := unittest.IdentityListFixture(n) + + // setup a ID provider callback to return currentIDs + pids := peer.IDSlice{} + for _, id := range currentIDs { + key, err := generateFlowNetworkingKey(id.NodeID) + require.NoError(suite.T(), err) + id.NetworkPubKey = key.PublicKey() + pid, err := ExtractPeerID(key.PublicKey()) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + + return currentIDs, pids +} + // TestUpdatePeers tests that updatePeers calls the connector with the expected list of ids to connect and disconnect // from. The tests are cumulative and ordered. func (suite *PeerManagerTestSuite) TestUpdatePeers() { - // create some test ids - currentIDs := unittest.IdentityListFixture(10) + _, pids := suite.generateIdentities(10) - // setup a ID provider callback to return currentIDs + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } - // track IDs that should be disconnected - var extraIDs flow.IdentityList - // create the connector mock to check ids requested for connect and disconnect connector := new(mocknetwork.Connector) - connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("flow.IdentityList")). + connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("peer.IDSlice")). Run(func(args mock.Arguments) { - idArg := args[1].(flow.IdentityList) - assertListsEqual(suite.T(), currentIDs, idArg) + idArg := args[1].(peer.IDSlice) + assertListsEqual(suite.T(), pids, idArg) }). Return(nil) @@ -77,8 +84,8 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // a subsequent call to updatePeers should request a connector.UpdatePeers to existing ids and new ids suite.Run("updatePeers connects to old and new peers", func() { // create a new id - newIDs := unittest.IdentityListFixture(1) - currentIDs = append(currentIDs, newIDs...) + _, newPIDs := suite.generateIdentities(1) + pids = append(pids, newPIDs...) pm.updatePeers() connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 2) @@ -87,8 +94,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // when ids are only excluded, connector.UpdatePeers should be called suite.Run("updatePeers disconnects from extra peers", func() { // delete an id - extraIDs = currentIDs.Sample(1) - currentIDs = currentIDs.Filter(filter.Not(filter.In(extraIDs))) + pids = removeRandomElement(pids) pm.updatePeers() connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 3) @@ -97,12 +103,12 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // addition and deletion of ids should result in a call to connector.UpdatePeers suite.Run("updatePeers connects to new peers and disconnects from extra peers", func() { // remove a couple of ids - extraIDs = currentIDs.Sample(2) - currentIDs = currentIDs.Filter(filter.Not(filter.In(extraIDs))) + pids = removeRandomElement(pids) + pids = removeRandomElement(pids) // add a couple of new ids - newIDs := unittest.IdentityListFixture(2) - currentIDs = append(currentIDs, newIDs...) + _, newPIDs := suite.generateIdentities(2) + pids = append(pids, newPIDs...) pm.updatePeers() @@ -110,16 +116,19 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { }) } +func removeRandomElement(pids peer.IDSlice) peer.IDSlice { + i := rand.Intn(len(pids)) + pids[i] = pids[len(pids)-1] + return pids[:len(pids)-1] +} + // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -150,14 +159,11 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -199,14 +205,11 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -244,9 +247,32 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { }, 10*time.Second, 100*time.Millisecond) } -// assertListsEqual asserts that two identity list are equal ignoring the order -func assertListsEqual(t *testing.T, list1, list2 flow.IdentityList) { - list1 = list1.Sort(order.ByNodeIDAsc) - list2 = list2.Sort(order.ByNodeIDAsc) - assert.Equal(t, list1, list2) +// assertListsEqual asserts that two peer ID slices are equal ignoring the order +func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { + listsEqual := func() bool { + if len(list1) != len(list2) { + return false + } + + map1 := make(map[peer.ID]int) + map2 := make(map[peer.ID]int) + + for _, e1 := range list1 { + map1[e1]++ + } + for _, e2 := range list2 { + map2[e2]++ + } + + for key, val := range map1 { + if map2[key] != val { + return false + } + } + return true + } + + if !listsEqual() { + assert.Failf(t, "peer ID slices not equal", "list1: %v, list2: %v", list1, list2) + } } diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 62f5bde9668..400d77fd04c 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -3,9 +3,9 @@ package p2p import ( "fmt" + lcrypto "github.com/libp2p/go-libp2p-core/crypto" crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/libp2p/go-libp2p-core/peer" - "github.com/multiformats/go-multihash" "github.com/onflow/flow-go/model/flow" ) @@ -18,12 +18,19 @@ func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { data := append([]byte{0x02}, flowID[:]...) - mh, err := multihash.Sum(data, multihash.IDENTITY, -1) + + um := lcrypto.PubKeyUnmarshallers[crypto_pb.KeyType_Secp256k1] + key, err := um(data) + if err != nil { + return "", fmt.Errorf("failed to convert flow ID to libp2p public key: %w", err) + } + + pid, err := peer.IDFromPublicKey(key) if err != nil { - return "", fmt.Errorf("failed to compute multihash: %w", err) + return "", fmt.Errorf("failed to get peer ID from libp2p public key: %w", err) } - return peer.ID(mh), nil + return pid, nil } func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { diff --git a/network/test/epochtransition_test.go b/network/test/epochtransition_test.go index af1c8ee7001..26de777dfa7 100644 --- a/network/test/epochtransition_test.go +++ b/network/test/epochtransition_test.go @@ -43,11 +43,10 @@ type MutableIdentityTableSuite struct { // testNode encapsulates the node state which includes its identity, middleware, network, // mesh engine and the id refresher type testNode struct { - id *flow.Identity - mw *p2p.Middleware - net *p2p.Network - engine *MeshEngine - idRefresher *p2p.NodeIDRefresher + id *flow.Identity + mw *p2p.Middleware + net *p2p.Network + engine *MeshEngine } // testNodeList encapsulates a list of test node and @@ -106,16 +105,6 @@ func (t *testNodeList) engines() []*MeshEngine { return engs } -func (t *testNodeList) idRefreshers() []*p2p.NodeIDRefresher { - t.RLock() - defer t.RUnlock() - idRefreshers := make([]*p2p.NodeIDRefresher, len(t.nodes)) - for i, node := range t.nodes { - idRefreshers[i] = node.idRefresher - } - return idRefreshers -} - func (t *testNodeList) networks() []*p2p.Network { t.RLock() defer t.RUnlock() @@ -131,6 +120,13 @@ func TestEpochTransitionTestSuite(t *testing.T) { t.Skip(fmt.Sprintf("test is flaky: %v", &MutableIdentityTableSuite{})) } +func (suite *MutableIdentityTableSuite) signalIdentityChanged() { + for _, n := range suite.testNodes.nodes { + n.mw.UpdateNodeAddresses() + n.mw.UpdateAllowList() + } +} + func (suite *MutableIdentityTableSuite) SetupTest() { suite.testNodes = newTestNodeList() suite.removedTestNodes = newTestNodeList() @@ -180,17 +176,13 @@ func (suite *MutableIdentityTableSuite) addNodes(count int) { // create the engines for the new nodes engines := GenerateEngines(suite.T(), nets) - // create the node refreshers - idRefereshers := suite.generateNodeIDRefreshers(nets) - // create the test engines for i := 0; i < count; i++ { node := testNode{ - id: ids[i], - mw: mws[i], - net: nets[i], - engine: engines[i], - idRefresher: idRefereshers[i], + id: ids[i], + mw: mws[i], + net: nets[i], + engine: engines[i], } suite.testNodes.append(node) } @@ -302,13 +294,6 @@ func (suite *MutableIdentityTableSuite) TestNodesAddedAndRemoved() { suite.assertNetworkPrimitives(remainingIDs, remainingEngs, removedIDs, removedEngines) } -// signalIdentityChanged update IDs for all the current set of nodes (simulating an epoch) -func (suite *MutableIdentityTableSuite) signalIdentityChanged() { - for _, r := range suite.testNodes.idRefreshers() { - r.OnIdentityTableChanged() - } -} - // assertConnected checks that the middleware of a node is directly connected // to at least half of the other nodes. func (suite *MutableIdentityTableSuite) assertConnected(mw *p2p.Middleware, ids flow.IdentityList) { @@ -317,7 +302,7 @@ func (suite *MutableIdentityTableSuite) assertConnected(mw *p2p.Middleware, ids require.Eventuallyf(t, func() bool { connections := 0 for _, id := range ids { - connected, err := mw.IsConnected(*id) + connected, err := mw.IsConnected(id.NodeID) require.NoError(t, err) if connected { connections++ @@ -337,7 +322,7 @@ func (suite *MutableIdentityTableSuite) assertDisconnected(mw *p2p.Middleware, i t := suite.T() require.Eventuallyf(t, func() bool { for _, id := range ids { - connected, err := mw.IsConnected(*id) + connected, err := mw.IsConnected(id.NodeID) require.NoError(t, err) if connected { return false @@ -452,11 +437,3 @@ func (suite *MutableIdentityTableSuite) sendMessage(fromID flow.Identifier, return send(event, fromEngine.con, toIDs.NodeIDs()...) } - -func (suite *MutableIdentityTableSuite) generateNodeIDRefreshers(nets []*p2p.Network) []*p2p.NodeIDRefresher { - refreshers := make([]*p2p.NodeIDRefresher, len(nets)) - for i, net := range nets { - refreshers[i] = p2p.NewNodeIDRefresher(suite.logger, suite.state, net.SetIDs) - } - return refreshers -} diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 3ccdc66a058..7f97b9ad9a1 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -65,7 +65,7 @@ type MiddlewareTestSuite struct { } // TestMiddlewareTestSuit runs all the test methods in this test suit -func TestMiddlewareTestSuit(t *testing.T) { +func TestMiddlewareTestSuite(t *testing.T) { suite.Run(t, new(MiddlewareTestSuite)) } @@ -101,11 +101,7 @@ func (m *MiddlewareTestSuite) SetupTest() { overlay := &mocknetwork.Overlay{} m.ov = append(m.ov, overlay) - identifierToID := make(map[flow.Identifier]flow.Identity) - for _, id := range m.ids { - identifierToID[id.NodeID] = *id - } - overlay.On("Identity").Maybe().Return(identifierToID, nil) + overlay.On("Identities").Maybe().Return(flow.IdentityList(m.ids), nil) overlay.On("Topology").Maybe().Return(flow.IdentityList(m.ids), nil) } for i, mw := range m.mws { diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 5d3b824c1a6..329bc1b2e68 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,6 +20,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" @@ -110,21 +111,18 @@ func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, er return peerID, nil } -func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableIdentityTranslator { +func NewFixedTableIdentityTranslator(t *testing.T, identities flow.IdentityList) *FixedTableIdentityTranslator { flow2p2p := make(map[flow.Identifier]peer.ID) p2p2flow := make(map[peer.ID]flow.Identifier) for _, identity := range identities { - nodeID := identity.ID() + nodeID := identity.NodeID networkKey := identity.NetworkPubKey peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) - if err != nil { - panic("could not interpret a network public key from Flow, test identities setup problem") - } + require.NoError(t, err) + peerID, err := peer.IDFromPublicKey(peerPK) - if err != nil { - panic("could not generate a PeerID from public Key, test identities setup problem") - } + require.NoError(t, err) flow2p2p[nodeID] = peerID p2p2flow[peerID] = nodeID @@ -177,7 +175,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id } // create a fixed id translator for the identities - tableTranslator := NewFixedTableIdentityTranslator(identities) + tableTranslator := NewFixedTableIdentityTranslator(t, identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, From e19d348bd2e4384ec9dca6c2d6433f57f5d0783c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 16:05:51 -0700 Subject: [PATCH 120/291] add test for hierchical translator --- network/p2p/fixed_translator.go | 53 +++++++++++++++++++++ network/p2p/hierarchical_translator.go | 5 +- network/p2p/hierarchical_translator_test.go | 43 +++++++++++++++++ network/test/testUtil.go | 43 +---------------- 4 files changed, 102 insertions(+), 42 deletions(-) create mode 100644 network/p2p/fixed_translator.go diff --git a/network/p2p/fixed_translator.go b/network/p2p/fixed_translator.go new file mode 100644 index 00000000000..ecfff75fbb1 --- /dev/null +++ b/network/p2p/fixed_translator.go @@ -0,0 +1,53 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type FixedTableIdentityTranslator struct { + flow2p2p map[flow.Identifier]peer.ID + p2p2flow map[peer.ID]flow.Identifier +} + +func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + nodeID, ok := t.p2p2flow[p] + if !ok { + return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) + } + return nodeID, nil +} + +func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + peerID, ok := t.flow2p2p[n] + if !ok { + return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) + } + return peerID, nil +} + +func NewFixedTableIdentityTranslator(identities flow.IdentityList) (*FixedTableIdentityTranslator, error) { + flow2p2p := make(map[flow.Identifier]peer.ID) + p2p2flow := make(map[peer.ID]flow.Identifier) + + for _, identity := range identities { + nodeID := identity.NodeID + networkKey := identity.NetworkPubKey + peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + if err != nil { + return nil, err + } + + peerID, err := peer.IDFromPublicKey(peerPK) + if err != nil { + return nil, err + } + + flow2p2p[nodeID] = peerID + p2p2flow[peerID] = nodeID + } + return &FixedTableIdentityTranslator{flow2p2p, p2p2flow}, nil +} diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go index 28dbdb03c31..4e4d0e77cd5 100644 --- a/network/p2p/hierarchical_translator.go +++ b/network/p2p/hierarchical_translator.go @@ -3,6 +3,7 @@ package p2p import ( "fmt" + "github.com/hashicorp/go-multierror" "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/model/flow" @@ -17,13 +18,15 @@ func NewHierarchicalIDTranslator(translators ...IDTranslator) *HierarchicalIDTra } func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + var errs *multierror.Error for _, translator := range t.translators { pid, err := translator.GetPeerID(flowID) if err == nil { return pid, nil } + errs = multierror.Append(errs, err) } - return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v", flowID) + return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v: %w", flowID, errs) } func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go index e111ecd0e85..6866547f378 100644 --- a/network/p2p/hierarchical_translator_test.go +++ b/network/p2p/hierarchical_translator_test.go @@ -3,13 +3,56 @@ package p2p import ( "testing" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "gotest.tools/assert" ) type HierarchicalTranslatorTestSuite struct { suite.Suite + translator *HierarchicalIDTranslator + ids flow.IdentityList +} + +func (suite *HierarchicalTranslatorTestSuite) SetupTest() { + suite.ids = unittest.IdentityListFixture(2, unittest.WithKeys) + t1, err := NewFixedTableIdentityTranslator(suite.ids[:1]) + require.NoError(suite.T(), err) + t2, err := NewFixedTableIdentityTranslator(suite.ids[1:]) + require.NoError(suite.T(), err) + + suite.translator = NewHierarchicalIDTranslator(t1, t2) } func TestHierarchicalTranslator(t *testing.T) { suite.Run(t, new(HierarchicalTranslatorTestSuite)) } + +func (suite *HierarchicalTranslatorTestSuite) TestFirstTranslatorSuccess() { + pid, err := suite.translator.GetPeerID(suite.ids[0].NodeID) + require.NoError(suite.T(), err) + fid, err := suite.translator.GetFlowID(pid) + assert.Equal(suite.T(), fid, suite.ids[0].NodeID) +} + +func (suite *HierarchicalTranslatorTestSuite) TestSecondTranslatorSuccess() { + pid, err := suite.translator.GetPeerID(suite.ids[1].NodeID) + require.NoError(suite.T(), err) + fid, err := suite.translator.GetFlowID(pid) + assert.Equal(suite.T(), fid, suite.ids[1].NodeID) +} + +func (suite *HierarchicalTranslatorTestSuite) TestTranslationFailure() { + fid := unittest.IdentifierFixture() + _, err := suite.translator.GetPeerID(fid) + require.Error(suite.T(), err) + + key, err := LibP2PPrivKeyFromFlow(generateNetworkingKey(suite.T())) + require.NoError(suite.T(), err) + pid, err := peer.IDFromPrivateKey(key) + _, err = suite.translator.GetFlowID(pid) + require.Error(suite.T(), err) +} diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 329bc1b2e68..d101779b899 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -90,46 +90,6 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics } } -type FixedTableIdentityTranslator struct { - flow2p2p map[flow.Identifier]peer.ID - p2p2flow map[peer.ID]flow.Identifier -} - -func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { - nodeID, ok := t.p2p2flow[p] - if !ok { - return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) - } - return nodeID, nil -} - -func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { - peerID, ok := t.flow2p2p[n] - if !ok { - return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) - } - return peerID, nil -} - -func NewFixedTableIdentityTranslator(t *testing.T, identities flow.IdentityList) *FixedTableIdentityTranslator { - flow2p2p := make(map[flow.Identifier]peer.ID) - p2p2flow := make(map[peer.ID]flow.Identifier) - - for _, identity := range identities { - nodeID := identity.NodeID - networkKey := identity.NetworkPubKey - peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) - require.NoError(t, err) - - peerID, err := peer.IDFromPublicKey(peerPK) - require.NoError(t, err) - - flow2p2p[nodeID] = peerID - p2p2flow[peerID] = nodeID - } - return &FixedTableIdentityTranslator{flow2p2p, p2p2flow} -} - // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. @@ -175,7 +135,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id } // create a fixed id translator for the identities - tableTranslator := NewFixedTableIdentityTranslator(t, identities) + tableTranslator, err := p2p.NewFixedTableIdentityTranslator(identities) + require.NoError(t, err) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, From a35e6eae0c2700100b0dd0e5b37e7db8b58234fa Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 16:09:17 -0700 Subject: [PATCH 121/291] Update peerManager_test.go Update execution_test.go update middleware --- cmd/scaffold.go | 1 + engine/execution/execution_test.go | 34 ++++++++---------- network/p2p/middleware.go | 57 +++++++++++++++++++----------- network/p2p/peerManager_test.go | 27 ++++++-------- 4 files changed, 63 insertions(+), 56 deletions(-) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index e12dd98d456..1445dd1630d 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -736,6 +736,7 @@ func (fnb *FlowNodeBuilder) Initialize() NodeBuilder { fnb.ParseAndPrintFlags() + // ID providers must be initialized before the network fnb.InitIDProviders() fnb.EnqueueNetworkInit(ctx) diff --git a/engine/execution/execution_test.go b/engine/execution/execution_test.go index 76d2ae13137..d46d69a8dd7 100644 --- a/engine/execution/execution_test.go +++ b/engine/execution/execution_test.go @@ -37,23 +37,21 @@ func TestExecutionFlow(t *testing.T) { chainID := flow.Testnet - keys, errs := unittest.NetworkingKeys(4) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exeID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) verID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[3].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID) @@ -300,23 +298,23 @@ func TestExecutionStateSyncMultipleExecutionNodes(t *testing.T) { chainID := flow.Emulator - keys, errs := unittest.NetworkingKeys(4) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exe1ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exe1ID) - identities[3].NetworkPubKey = keys[3].PublicKey() + key, err := unittest.NetworkingKey() + require.NoError(t, err) + identities[3].NetworkPubKey = key.PublicKey() collectionNode := testutil.GenericNodeFromParticipants(t, hub, colID, identities, chainID) defer collectionNode.Done() @@ -449,27 +447,25 @@ func TestBroadcastToMultipleVerificationNodes(t *testing.T) { chainID := flow.Emulator - keys, errs := unittest.NetworkingKeys(5) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exeID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) ver1ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[3].PublicKey()), + unittest.WithKeys, ) ver2ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[4].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, ver1ID, ver2ID) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 32b3ee16ef5..611180b6c9c 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -64,24 +64,25 @@ const ( // our neighbours on the peer-to-peer network. type Middleware struct { sync.Mutex - ctx context.Context - cancel context.CancelFunc - log zerolog.Logger - ov network.Overlay - wg *sync.WaitGroup - libP2PNode *Node - libP2PNodeFactory LibP2PFactoryFunc - me flow.Identifier - metrics module.NetworkMetrics - rootBlockID string - validators []network.MessageValidator - peerManager *PeerManager - peerUpdateInterval time.Duration - unicastMessageTimeout time.Duration - connectionGating bool - managePeerConnections bool - idTranslator IDTranslator - idProvider id.IdentifierProvider + ctx context.Context + cancel context.CancelFunc + log zerolog.Logger + ov network.Overlay + wg *sync.WaitGroup + libP2PNode *Node + libP2PNodeFactory LibP2PFactoryFunc + me flow.Identifier + metrics module.NetworkMetrics + rootBlockID string + validators []network.MessageValidator + peerManager *PeerManager + peerUpdateInterval time.Duration + unicastMessageTimeout time.Duration + connectionGating bool + managePeerConnections bool + idTranslator IDTranslator + idProvider id.IdentifierProvider + previousProtocolStatePeers []peer.AddrInfo } type MiddlewareOption func(*Middleware) @@ -200,12 +201,28 @@ func (m *Middleware) GetIPPort() (string, string, error) { } func (m *Middleware) UpdateNodeAddresses() { + m.log.Info().Msg("Updating protocol state node addresses") + ids := m.ov.Identities() - infos, _ := peerInfosFromIDs(ids) + newInfos, invalid := peerInfosFromIDs(ids) + + for id, err := range invalid { + m.log.Err(err).Str("node_id", id.String()).Msg("failed to extract peer info from identity") + } + + m.Lock() + defer m.Unlock() - for _, info := range infos { + // set old addresses to expire + for _, oldInfo := range m.previousProtocolStatePeers { + m.libP2PNode.host.Peerstore().SetAddrs(oldInfo.ID, oldInfo.Addrs, peerstore.TempAddrTTL) + } + + for _, info := range newInfos { m.libP2PNode.host.Peerstore().SetAddrs(info.ID, info.Addrs, peerstore.PermanentAddrTTL) } + + m.previousProtocolStatePeers = newInfos } // Start will start the middleware. diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 5c6632fbc76..bdaa6aca015 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -15,7 +15,6 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/utils/unittest" ) @@ -34,29 +33,23 @@ func (suite *PeerManagerTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) } -func (suite *PeerManagerTestSuite) generateIdentities(n int) (flow.IdentityList, peer.IDSlice) { - // create some test ids - currentIDs := unittest.IdentityListFixture(n) - - // setup a ID provider callback to return currentIDs +func (suite *PeerManagerTestSuite) generatePeerIDs(n int) peer.IDSlice { pids := peer.IDSlice{} - for _, id := range currentIDs { - key, err := generateFlowNetworkingKey(id.NodeID) - require.NoError(suite.T(), err) - id.NetworkPubKey = key.PublicKey() + for i := 0; i < n; i++ { + key := generateNetworkingKey(suite.T()) pid, err := ExtractPeerID(key.PublicKey()) require.NoError(suite.T(), err) pids = append(pids, pid) } - return currentIDs, pids + return pids } // TestUpdatePeers tests that updatePeers calls the connector with the expected list of ids to connect and disconnect // from. The tests are cumulative and ordered. func (suite *PeerManagerTestSuite) TestUpdatePeers() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -84,7 +77,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // a subsequent call to updatePeers should request a connector.UpdatePeers to existing ids and new ids suite.Run("updatePeers connects to old and new peers", func() { // create a new id - _, newPIDs := suite.generateIdentities(1) + newPIDs := suite.generatePeerIDs(1) pids = append(pids, newPIDs...) pm.updatePeers() @@ -107,7 +100,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { pids = removeRandomElement(pids) // add a couple of new ids - _, newPIDs := suite.generateIdentities(2) + newPIDs := suite.generatePeerIDs(2) pids = append(pids, newPIDs...) pm.updatePeers() @@ -125,7 +118,7 @@ func removeRandomElement(pids peer.IDSlice) peer.IDSlice { // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -160,7 +153,7 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -206,7 +199,7 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { From cbc4be2d5845ca12bdd2104180ce7784a2ed9b48 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 20:00:32 -0700 Subject: [PATCH 122/291] Address comments --- .../staked_access_node_builder.go | 2 +- cmd/scaffold.go | 2 +- network/p2p/libp2pUtils.go | 17 +++++++++++ network/p2p/network.go | 6 ++-- network/p2p/peerManager_test.go | 28 ++----------------- network/p2p/protocol_state_provider.go | 17 ----------- 6 files changed, 24 insertions(+), 48 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 233cb56819d..a2fcec2fa82 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -40,7 +40,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), - p2p.NetworkingSetFilter, + p2p.NotEjectedFilter, ), idCache, ) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 1445dd1630d..a5ddb71fbf7 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -432,7 +432,7 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache fnb.IDTranslator = idCache - fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NetworkingSetFilter, idCache) + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) return nil }) } diff --git a/network/p2p/libp2pUtils.go b/network/p2p/libp2pUtils.go index ad43adbc877..04a2996c2aa 100644 --- a/network/p2p/libp2pUtils.go +++ b/network/p2p/libp2pUtils.go @@ -16,6 +16,7 @@ import ( "github.com/multiformats/go-multiaddr" "github.com/rs/zerolog" + fcrypto "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" ) @@ -231,3 +232,19 @@ func flowStream(conn network.Conn) network.Stream { } return nil } + +func ExtractPeerID(networkPubKey fcrypto.PublicKey) (pid peer.ID, err error) { + pk, err := LibP2PPublicKeyFromFlow(networkPubKey) + if err != nil { + err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) + return + } + + return +} diff --git a/network/p2p/network.go b/network/p2p/network.go index 82235bd4938..1c0d010a16f 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -24,13 +24,13 @@ import ( const DefaultCacheSize = 10e6 -// NetworkingSetFilter is an identity filter that, when applied to the identity +// NotEjectedFilter is an identity filter that, when applied to the identity // table at a given snapshot, returns all nodes that we should communicate with // over the networking layer. // // NOTE: The protocol state includes nodes from the previous/next epoch that should // be included in network communication. We omit any nodes that have been ejected. -var NetworkingSetFilter = filter.Not(filter.Ejected) +var NotEjectedFilter = filter.Not(filter.Ejected) type ReadyDoneAwareNetwork interface { module.Network @@ -164,7 +164,7 @@ func (n *Network) unregister(channel network.Channel) error { } func (n *Network) Identities() flow.IdentityList { - return n.identityProvider.Identities(NetworkingSetFilter) + return n.identityProvider.Identities(NotEjectedFilter) } // Topology returns the identitiess of a uniform subset of nodes in protocol state using the topology provided earlier. diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index bdaa6aca015..ad8471f8be5 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -242,30 +242,6 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { // assertListsEqual asserts that two peer ID slices are equal ignoring the order func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { - listsEqual := func() bool { - if len(list1) != len(list2) { - return false - } - - map1 := make(map[peer.ID]int) - map2 := make(map[peer.ID]int) - - for _, e1 := range list1 { - map1[e1]++ - } - for _, e2 := range list2 { - map2[e2]++ - } - - for key, val := range map1 { - if map2[key] != val { - return false - } - } - return true - } - - if !listsEqual() { - assert.Failf(t, "peer ID slices not equal", "list1: %v, list2: %v", list1, list2) - } + assert.Subset(t, list1, list2) + assert.Subset(t, list2, list1) } diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 6f223a4d0fe..b1a5bdcf834 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -7,7 +7,6 @@ import ( "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/state/protocol" @@ -123,19 +122,3 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e return } - -func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { - pk, err := LibP2PPublicKeyFromFlow(networkPubKey) - if err != nil { - err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) - return - } - - pid, err = peer.IDFromPublicKey(pk) - if err != nil { - err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) - return - } - - return -} From 1f0582f6279c25a05572783055b8588149404b02 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 21:29:16 -0700 Subject: [PATCH 123/291] protocol state provider test --- network/p2p/peerManager_test.go | 8 +- network/p2p/protocol_state_provider_test.go | 138 ++++++++++++++++++++ 2 files changed, 139 insertions(+), 7 deletions(-) create mode 100644 network/p2p/protocol_state_provider_test.go diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index ad8471f8be5..08b3df05193 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -61,7 +61,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("peer.IDSlice")). Run(func(args mock.Arguments) { idArg := args[1].(peer.IDSlice) - assertListsEqual(suite.T(), pids, idArg) + assert.ElementsMatch(suite.T(), pids, idArg) }). Return(nil) @@ -239,9 +239,3 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { return connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 2) }, 10*time.Second, 100*time.Millisecond) } - -// assertListsEqual asserts that two peer ID slices are equal ignoring the order -func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { - assert.Subset(t, list1, list2) - assert.Subset(t, list2, list1) -} diff --git a/network/p2p/protocol_state_provider_test.go b/network/p2p/protocol_state_provider_test.go new file mode 100644 index 00000000000..55ac4074a5e --- /dev/null +++ b/network/p2p/protocol_state_provider_test.go @@ -0,0 +1,138 @@ +package p2p + +import ( + "testing" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" + mockprotocol "github.com/onflow/flow-go/state/protocol/mock" + "github.com/onflow/flow-go/utils/unittest" +) + +type ProtocolStateProviderTestSuite struct { + suite.Suite + provider *ProtocolStateIDCache + distributor *events.Distributor + state protocol.State + snapshot protocol.Snapshot + head *flow.Header + participants flow.IdentityList + epochNum uint64 +} + +func (suite *ProtocolStateProviderTestSuite) SetupTest() { + suite.distributor = events.NewDistributor() + + // set up protocol state mock + state := &mockprotocol.State{} + state.On("Final").Return( + func() protocol.Snapshot { + return suite.snapshot + }, + ) + state.On("AtBlockID", mock.Anything).Return( + func(blockID flow.Identifier) protocol.Snapshot { + if suite.head.ID() == blockID { + return suite.snapshot + } else { + return unittest.StateSnapshotForUnknownBlock() + } + }, + ) + suite.state = state + suite.epochNum = 0 + + suite.triggerUpdate() + + provider, err := NewProtocolStateIDCache(zerolog.Logger{}, state, suite.distributor) + require.NoError(suite.T(), err) + + suite.provider = provider +} + +func (suite *ProtocolStateProviderTestSuite) triggerUpdate() { + suite.participants = unittest.IdentityListFixture(5, unittest.WithAllRoles(), unittest.WithKeys) + + block := unittest.BlockFixture() + suite.head = block.Header + + // set up protocol snapshot mock + snapshot := &mockprotocol.Snapshot{} + snapshot.On("Identities", mock.Anything).Return( + func(filter flow.IdentityFilter) flow.IdentityList { + return suite.participants.Filter(filter) + }, + nil, + ) + snapshot.On("Identity", mock.Anything).Return(func(id flow.Identifier) *flow.Identity { + for _, n := range suite.participants { + if n.ID() == id { + return n + } + } + return nil + }, nil) + snapshot.On("Head").Return( + func() *flow.Header { + return suite.head + }, + nil, + ) + suite.snapshot = snapshot + suite.epochNum += 1 + + suite.distributor.EpochTransition(suite.epochNum, suite.head) +} + +func TestProtocolStateProvider(t *testing.T) { + suite.Run(t, new(ProtocolStateProviderTestSuite)) +} + +func (suite *ProtocolStateProviderTestSuite) checkStateTransition() { + oldParticipants := suite.participants + + suite.triggerUpdate() + + assert.ElementsMatch(suite.T(), suite.participants, suite.provider.Identities(filter.Any)) + for _, participant := range suite.participants { + pid, err := suite.provider.GetPeerID(participant.NodeID) + require.NoError(suite.T(), err) + fid, err := suite.provider.GetFlowID(pid) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), fid, participant.NodeID) + } + for _, participant := range oldParticipants { + _, err := suite.provider.GetPeerID(participant.NodeID) + require.Error(suite.T(), err) + } +} + +func (suite *ProtocolStateProviderTestSuite) TestUpdateState() { + for i := 0; i < 10; i++ { + suite.checkStateTransition() + } +} + +func (suite *ProtocolStateProviderTestSuite) TestIDTranslation() { + for _, participant := range suite.participants { + pid, err := suite.provider.GetPeerID(participant.NodeID) + require.NoError(suite.T(), err) + key, err := LibP2PPublicKeyFromFlow(participant.NetworkPubKey) + require.NoError(suite.T(), err) + expectedPid, err := peer.IDFromPublicKey(key) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), expectedPid, pid) + fid, err := suite.provider.GetFlowID(pid) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), fid, participant.NodeID) + } +} From c8bce7815e770bd5786e5c03c1476d923468c550 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 22:59:39 -0700 Subject: [PATCH 124/291] updating origin ID checks to use idtranslator --- network/p2p/middleware.go | 25 +++++++------- network/p2p/readConnection.go | 20 ++++-------- network/p2p/readSubscription.go | 58 +++------------------------------ 3 files changed, 22 insertions(+), 81 deletions(-) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 611180b6c9c..72866bd977d 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -405,25 +405,22 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { return nil } -// processAuthenticatedMessage processes a message and a source (indicated by its PublicKey) and eventually passes it to the overlay -// In particular, it checks the claim of protocol authorship situated in the message against `originKey` -// The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` +// processAuthenticatedMessage processes a message and a source (indicated by its peer ID) and eventually passes it to the overlay +// In particular, it checks the claim of protocol authorship situated in the message against `peerID` +// The assumption is that the message has been authenticated at the network level (libp2p) to originate from the peer with ID `peerID` // this requirement is fulfilled by e.g. the output of readConnection and readSubscription -func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { - identities := m.ov.Identities() +func (m *Middleware) processAuthenticatedMessage(msg *message.Message, peerID peer.ID) { + flowID, err := m.idTranslator.GetFlowID(peerID) + if err != nil { + m.log.Warn().Err(err).Msgf("received message from unknown peer %v, and was dropped", peerID.String()) + return + } // check the origin of the message corresponds to the one claimed in the OriginID originID := flow.HashToID(msg.OriginID) - originIdentity, found := identities.ByNodeID(originID) - if !found { - m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) - return - } else if originIdentity.NetworkPubKey == nil { - m.log.Warn().Msgf("received message with claimed originID %x, wich has no network identifiers at this node, and was dropped", originID) - return - } else if !originIdentity.NetworkPubKey.Equals(originKey) { - m.log.Warn().Msgf("received message claiming to be from nodeID %v with key %x was actually signed by %x and dropped", originID, originIdentity.NetworkPubKey, originKey) + if flowID != originID { + m.log.Warn().Msgf("received message claiming to be from nodeID %v was actually from %v and dropped", originID, flowID) return } diff --git a/network/p2p/readConnection.go b/network/p2p/readConnection.go index 5d6a0711872..c71b8bb1e25 100644 --- a/network/p2p/readConnection.go +++ b/network/p2p/readConnection.go @@ -7,10 +7,10 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/message" @@ -21,17 +21,17 @@ import ( type readConnection struct { ctx context.Context stream libp2pnetwork.Stream - remoteKey crypto.PublicKey + remoteID peer.ID log zerolog.Logger metrics module.NetworkMetrics maxMsgSize int - callback func(msg *message.Message, pk crypto.PublicKey) + callback func(msg *message.Message, peerID peer.ID) } // newReadConnection creates a new readConnection func newReadConnection(ctx context.Context, stream libp2pnetwork.Stream, - callback func(msg *message.Message, pubKey crypto.PublicKey), + callback func(msg *message.Message, peerID peer.ID), log zerolog.Logger, metrics module.NetworkMetrics, maxMsgSize int) *readConnection { @@ -40,18 +40,10 @@ func newReadConnection(ctx context.Context, maxMsgSize = DefaultMaxUnicastMsgSize } - remoteKey := stream.Conn().RemotePublicKey() - flowKey, err := FlowPublicKeyFromLibP2P(remoteKey) - // this should not happen if the stream was setup properly - if err != nil { - log.Err(err).Msg("failed to extract flow public key of stream libp2p key") - return nil - } - c := readConnection{ ctx: ctx, stream: stream, - remoteKey: flowKey, + remoteID: stream.Conn().RemotePeer(), callback: callback, log: log, metrics: metrics, @@ -113,7 +105,7 @@ func (rc *readConnection) receiveLoop(wg *sync.WaitGroup) { rc.metrics.NetworkMessageReceived(msg.Size(), metrics.ChannelOneToOne, msg.Type) // call the callback - rc.callback(&msg, rc.remoteKey) + rc.callback(&msg, rc.remoteID) } } diff --git a/network/p2p/readSubscription.go b/network/p2p/readSubscription.go index 3b1b66f8e69..db87b097afa 100644 --- a/network/p2p/readSubscription.go +++ b/network/p2p/readSubscription.go @@ -2,17 +2,14 @@ package p2p import ( "context" - "fmt" "strings" "sync" - lcrypto "github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/network/message" _ "github.com/onflow/flow-go/utils/binstat" @@ -25,13 +22,13 @@ type readSubscription struct { log zerolog.Logger sub *pubsub.Subscription metrics module.NetworkMetrics - callback func(msg *message.Message, pubKey crypto.PublicKey) + callback func(msg *message.Message, peerID peer.ID) } // newReadSubscription reads the messages coming in on the subscription func newReadSubscription(ctx context.Context, sub *pubsub.Subscription, - callback func(msg *message.Message, pubKey crypto.PublicKey), + callback func(msg *message.Message, peerID peer.ID), log zerolog.Logger, metrics module.NetworkMetrics) *readSubscription { @@ -82,16 +79,9 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { return } - // if pubsub.WithMessageSigning(true) and pubsub.WithStrictSignatureVerification(true), - // the emitter is authenticated - emitterKey, err := messagePubKey(rawMsg) + pid, err := peer.IDFromBytes(rawMsg.From) if err != nil { - r.log.Err(err).Msg("failed to extract libp2p public key of message") - return - } - flowKey, err := FlowPublicKeyFromLibP2P(emitterKey) - if err != nil { - r.log.Err(err).Msg("failed to extract flow public key of libp2p key") + r.log.Err(err).Msg("failed to validate peer ID of incoming message") return } @@ -109,44 +99,6 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { r.metrics.NetworkMessageReceived(msg.Size(), msg.ChannelID, msg.Type) // call the callback - r.callback(&msg, flowKey) - } -} - -// messagePubKey extracts the public key of the envelope signer from a libp2p message. -// The location of that key depends on the type of the key, see: -// https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md -// This reproduces the exact logic of the private function doing the same decoding in libp2p: -// https://github.com/libp2p/go-libp2p-pubsub/blob/ba28f8ecfc551d4d916beb748d3384951bce3ed0/sign.go#L77 -func messagePubKey(m *pubsub.Message) (lcrypto.PubKey, error) { - var pubk lcrypto.PubKey - - // m.From is the original sender of the message (versus `m.ReceivedFrom` which is the last hop which sent us this message) - pid, err := peer.IDFromBytes(m.From) - if err != nil { - return nil, err - } - - if m.Key == nil { - // no attached key, it must be extractable from the source ID - pubk, err = pid.ExtractPublicKey() - if err != nil { - return nil, fmt.Errorf("cannot extract signing key: %s", err.Error()) - } - if pubk == nil { - return nil, fmt.Errorf("cannot extract signing key") - } - } else { - pubk, err = lcrypto.UnmarshalPublicKey(m.Key) - if err != nil { - return nil, fmt.Errorf("cannot unmarshal signing key: %s", err.Error()) - } - - // verify that the source ID matches the attached key - if !pid.MatchesPublicKey(pubk) { - return nil, fmt.Errorf("bad signing key; source ID %s doesn't match key", pid) - } + r.callback(&msg, pid) } - - return pubk, nil } From b7f5a2ab699f239af6047bfe5ab755b10306d74f Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 14:12:50 -0700 Subject: [PATCH 125/291] new providers remove comments Update hierarchical_translator_test.go --- .../staked_access_node_builder.go | 3 +- engine/common/synchronization/engine_test.go | 1 + engine/testutil/nodes.go | 1 + model/flow/filter/identity.go | 12 +++ module/id/fixed_provider.go | 4 +- module/id/updatable_provider.go | 34 +++++++++ network/p2p/dht_test.go | 2 - network/p2p/hierarchical_translator_test.go | 8 +- network/p2p/identity_provider_translator.go | 51 +++++++++++++ network/p2p/libp2pNode.go | 36 ++++----- network/test/middleware_test.go | 73 ++++++++++++++++--- network/test/testUtil.go | 25 ++++--- 12 files changed, 199 insertions(+), 51 deletions(-) create mode 100644 module/id/updatable_provider.go create mode 100644 network/p2p/identity_provider_translator.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a2fcec2fa82..8e5b5285e3f 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -63,8 +63,7 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - // TODO: we should remove this call since we are no longer instantiating two networks - // builder.EnqueueNetworkInit(ctx) + builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network if builder.ParticipatesInUnstakedNetwork() { diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 4e79938fab7..5dce4ea28a5 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -171,6 +171,7 @@ func (ss *SyncSuite) SetupTest() { require.NoError(ss.T(), err, "could not create finalized snapshot cache") idCache, err := p2p.NewProtocolStateIDCache(log, ss.state, protocolEvents.NewDistributor()) + require.NoError(ss.T(), err, "could not create protocol state identity cache") e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, id.NewFilteredIdentifierProvider( filter.And( diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index 3688e2c6727..ecaad824601 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -628,6 +628,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit require.NoError(t, err) idCache, err := p2p.NewProtocolStateIDCache(node.Log, node.State, events.NewDistributor()) + require.NoError(t, err, "could not create finalized snapshot cache") syncEngine, err := synchronization.New( node.Log, node.Metrics, diff --git a/model/flow/filter/identity.go b/model/flow/filter/identity.go index f55b2e5227b..dc39e4ce475 100644 --- a/model/flow/filter/identity.go +++ b/model/flow/filter/identity.go @@ -3,6 +3,7 @@ package filter import ( + "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" ) @@ -61,6 +62,17 @@ func HasNodeID(nodeIDs ...flow.Identifier) flow.IdentityFilter { } } +func HasNetworkingKey(keys ...crypto.PublicKey) flow.IdentityFilter { + return func(identity *flow.Identity) bool { + for _, key := range keys { + if key.Equals(identity.NetworkPubKey) { + return true + } + } + return false + } +} + // HasStake returns a filter for nodes with non-zero stake. func HasStake(hasStake bool) flow.IdentityFilter { return func(identity *flow.Identity) bool { diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index 106047510b6..39aac7567a1 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -24,6 +24,6 @@ func NewFixedIdentityProvider(identities flow.IdentityList) *FixedIdentityProvid return &FixedIdentityProvider{identities} } -func (p *FixedIdentityProvider) Identities(flow.IdentityFilter) flow.IdentityList { - return p.identities +func (p *FixedIdentityProvider) Identities(filter flow.IdentityFilter) flow.IdentityList { + return p.identities.Filter(filter) } diff --git a/module/id/updatable_provider.go b/module/id/updatable_provider.go new file mode 100644 index 00000000000..2288b9e6834 --- /dev/null +++ b/module/id/updatable_provider.go @@ -0,0 +1,34 @@ +package id + +import ( + "sync" + + "github.com/onflow/flow-go/model/flow" +) + +type UpdatableIDProvider struct { + mu sync.RWMutex + identities flow.IdentityList +} + +func NewUpdatableIDProvider(identities flow.IdentityList) *UpdatableIDProvider { + return &UpdatableIDProvider{identities: identities} +} + +func (p *UpdatableIDProvider) SetIdentities(identities flow.IdentityList) { + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities +} + +func (p *UpdatableIDProvider) Identities(filter flow.IdentityFilter) flow.IdentityList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.Filter(filter) +} + +func (p *UpdatableIDProvider) Identifiers() flow.IdentifierList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.NodeIDs() +} diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index 6643a7652b0..cc01eedaafb 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -29,8 +29,6 @@ type DHTTestSuite struct { cancel context.CancelFunc // used to cancel the context } -// TODO: test that dht findPeer times out - // TestDHTTestSuite test the libp2p pubsub with DHT for discovery func TestDHTTestSuite(t *testing.T) { suite.Run(t, new(DHTTestSuite)) diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go index 6866547f378..90245d82a97 100644 --- a/network/p2p/hierarchical_translator_test.go +++ b/network/p2p/hierarchical_translator_test.go @@ -4,11 +4,12 @@ import ( "testing" "github.com/libp2p/go-libp2p-core/peer" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/utils/unittest" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "gotest.tools/assert" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" ) type HierarchicalTranslatorTestSuite struct { @@ -35,6 +36,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestFirstTranslatorSuccess() { pid, err := suite.translator.GetPeerID(suite.ids[0].NodeID) require.NoError(suite.T(), err) fid, err := suite.translator.GetFlowID(pid) + require.NoError(suite.T(), err) assert.Equal(suite.T(), fid, suite.ids[0].NodeID) } @@ -42,6 +44,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestSecondTranslatorSuccess() { pid, err := suite.translator.GetPeerID(suite.ids[1].NodeID) require.NoError(suite.T(), err) fid, err := suite.translator.GetFlowID(pid) + require.NoError(suite.T(), err) assert.Equal(suite.T(), fid, suite.ids[1].NodeID) } @@ -53,6 +56,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestTranslationFailure() { key, err := LibP2PPrivKeyFromFlow(generateNetworkingKey(suite.T())) require.NoError(suite.T(), err) pid, err := peer.IDFromPrivateKey(key) + require.NoError(suite.T(), err) _, err = suite.translator.GetFlowID(pid) require.Error(suite.T(), err) } diff --git a/network/p2p/identity_provider_translator.go b/network/p2p/identity_provider_translator.go new file mode 100644 index 00000000000..61504b1ef05 --- /dev/null +++ b/network/p2p/identity_provider_translator.go @@ -0,0 +1,51 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module/id" +) + +type IdentityProviderIdentityTranslator struct { + idProvider id.IdentityProvider +} + +func (t *IdentityProviderIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + key, err := p.ExtractPublicKey() + if err != nil { + return flow.ZeroID, err + } + flowKey, err := FlowPublicKeyFromLibP2P(key) + if err != nil { + return flow.ZeroID, err + } + ids := t.idProvider.Identities(filter.HasNetworkingKey(flowKey)) + if len(ids) == 0 { + return flow.ZeroID, fmt.Errorf("could not find identity corresponding to peer id %v", p.Pretty()) + } + return ids[0].NodeID, nil +} + +func (t *IdentityProviderIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + ids := t.idProvider.Identities(filter.HasNodeID(n)) + if len(ids) == 0 { + return "", fmt.Errorf("could not find identity with id %v", n.String()) + } + key, err := LibP2PPublicKeyFromFlow(ids[0].NetworkPubKey) + if err != nil { + return "", err + } + pid, err := peer.IDFromPublicKey(key) + if err != nil { + return "", err + } + return pid, nil +} + +func NewIdentityProviderIdentityTranslator(provider id.IdentityProvider) *IdentityProviderIdentityTranslator { + return &IdentityProviderIdentityTranslator{provider} +} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 0fcb148ccd2..7d0d2a5fbd6 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -334,24 +334,24 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { if len(n.host.Peerstore().Addrs(peerID)) == 0 { n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - if n.dht == nil { - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) - } - - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - - var err error - func() { - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - defer cancel() - // try to find the peer using the dht - _, err = n.dht.FindPeer(timedCtx, peerID) - }() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + if n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + + if err != nil { + n.logger.Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") + } else { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + } } - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) @@ -381,10 +381,6 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp default: } - // TODO: why were we doing this? Is it okay to remove? - // remove the peer from the peer store if present - // n.host.Peerstore().ClearAddrs(peerID) - // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() if swm, ok := network.(*swarm.Swarm); ok { diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 7f97b9ad9a1..3b33df7f312 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -8,8 +8,10 @@ import ( "time" "github.com/ipfs/go-log" + swarm "github.com/libp2p/go-libp2p-swarm" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" mockery "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -17,6 +19,7 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/observable" "github.com/onflow/flow-go/network/codec/cbor" @@ -56,12 +59,14 @@ func (co *tagsObserver) OnComplete() { type MiddlewareTestSuite struct { suite.Suite - size int // used to determine number of middlewares under test - mws []*p2p.Middleware // used to keep track of middlewares under test - ov []*mocknetwork.Overlay - obs chan string // used to keep track of Protect events tagged by pubsub messages - ids []*flow.Identity - metrics *metrics.NoopCollector // no-op performance monitoring simulation + size int // used to determine number of middlewares under test + mws []*p2p.Middleware // used to keep track of middlewares under test + ov []*mocknetwork.Overlay + obs chan string // used to keep track of Protect events tagged by pubsub messages + ids []*flow.Identity + metrics *metrics.NoopCollector // no-op performance monitoring simulation + logger zerolog.Logger + providers []*id.UpdatableIDProvider } // TestMiddlewareTestSuit runs all the test methods in this test suit @@ -73,6 +78,7 @@ func TestMiddlewareTestSuite(t *testing.T) { func (m *MiddlewareTestSuite) SetupTest() { logger := zerolog.New(os.Stderr).Level(zerolog.ErrorLevel) log.SetAllLoggers(log.LevelError) + m.logger = logger m.size = 2 // operates on two middlewares m.metrics = metrics.NewNoopCollector() @@ -85,7 +91,7 @@ func (m *MiddlewareTestSuite) SetupTest() { log: logger, } - m.ids, m.mws, obs = GenerateIDsAndMiddlewares(m.T(), m.size, !DryRun, logger) + m.ids, m.mws, obs, m.providers = GenerateIDsAndMiddlewares(m.T(), m.size, !DryRun, logger) for _, observableConnMgr := range obs { observableConnMgr.Subscribe(&ob) @@ -98,11 +104,7 @@ func (m *MiddlewareTestSuite) SetupTest() { // create the mock overlays for i := 0; i < m.size; i++ { - overlay := &mocknetwork.Overlay{} - m.ov = append(m.ov, overlay) - - overlay.On("Identities").Maybe().Return(flow.IdentityList(m.ids), nil) - overlay.On("Topology").Maybe().Return(flow.IdentityList(m.ids), nil) + m.ov = append(m.ov, m.createOverlay()) } for i, mw := range m.mws { assert.NoError(m.T(), mw.Start(m.ov[i])) @@ -110,6 +112,18 @@ func (m *MiddlewareTestSuite) SetupTest() { } } +func (m *MiddlewareTestSuite) createOverlay() *mocknetwork.Overlay { + overlay := &mocknetwork.Overlay{} + + overlay.On("Identities").Maybe().Return(func() flow.IdentityList { + return flow.IdentityList(m.ids) + }, nil) + overlay.On("Topology").Maybe().Return(func() flow.IdentityList { + return flow.IdentityList(m.ids) + }, nil) + return overlay +} + func (m *MiddlewareTestSuite) TearDownTest() { m.stopMiddlewares() } @@ -462,6 +476,41 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { require.Error(m.Suite.T(), err) } +func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { + ids, mws, _, providers := GenerateIDsAndMiddlewares(m.T(), 1, false, m.logger) + require.Len(m.T(), ids, 1) + require.Len(m.T(), providers, 1) + require.Len(m.T(), mws, 1) + newId := ids[0] + newMw := mws[0] + newProvider := providers[0] + + idList := flow.IdentityList(append(m.ids, newId)) + + newProvider.SetIdentities(idList) + overlay := m.createOverlay() + overlay.On("Receive", + m.ids[0].NodeID, + mock.AnythingOfType("*message.Message"), + ).Return(nil) + assert.NoError(m.T(), newMw.Start(overlay)) + + // needed to enable ID translation + m.providers[0].SetIdentities(idList) + m.mws[0].UpdateAllowList() + + msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") + + err := m.mws[0].SendDirect(msg, newId.NodeID) + require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) + + m.ids = idList + m.mws[0].UpdateNodeAddresses() + + err = m.mws[0].SendDirect(msg, newId.NodeID) + require.NoError(m.T(), err) +} + // TestUnsubscribe tests that an engine can unsubscribe from a topic it was earlier subscribed to and stop receiving // messages. func (m *MiddlewareTestSuite) TestUnsubscribe() { diff --git a/network/test/testUtil.go b/network/test/testUtil.go index d101779b899..9f8c1ef063a 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -21,6 +21,7 @@ import ( message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" + idModule "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" @@ -121,9 +122,10 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) []*p2p.Middleware { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*id.UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) + idProviders := make([]*id.UpdatableIDProvider, len(identities)) for i, id := range identities { // casts libP2PNode instance to a local variable to avoid closure @@ -134,9 +136,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } - // create a fixed id translator for the identities - tableTranslator, err := p2p.NewFixedTableIdentityTranslator(identities) - require.NoError(t, err) + idProviders[i] = idModule.NewUpdatableIDProvider(identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, @@ -148,10 +148,13 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultUnicastTimeout, true, true, - tableTranslator, + p2p.NewIdentityProviderIdentityTranslator(idProviders[i]), + p2p.WithIdentifierProvider( + idProviders[i], + ), ) } - return mws + return mws, idProviders } // GenerateNetworks generates the network for the given middlewares @@ -200,7 +203,7 @@ func GenerateNetworks(t *testing.T, tops[i], sms[i], metrics, - id.NewFixedIdentityProvider(ids), + idModule.NewFixedIdentityProvider(ids), ) require.NoError(t, err) @@ -220,11 +223,11 @@ func GenerateNetworks(t *testing.T, func GenerateIDsAndMiddlewares(t *testing.T, n int, dryRunMode bool, - logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable) { + logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*id.UpdatableIDProvider) { ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) - mws := GenerateMiddlewares(t, logger, ids, libP2PNodes) - return ids, mws, protectObservables + mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) + return ids, mws, protectObservables, providers } func GenerateIDsMiddlewaresNetworks(t *testing.T, @@ -234,7 +237,7 @@ func GenerateIDsMiddlewaresNetworks(t *testing.T, tops []network.Topology, dryRun bool, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []*p2p.Network, []observable.Observable) { - ids, mws, observables := GenerateIDsAndMiddlewares(t, n, dryRun, log, opts...) + ids, mws, observables, _ := GenerateIDsAndMiddlewares(t, n, dryRun, log, opts...) sms := GenerateSubscriptionManagers(t, mws) networks := GenerateNetworks(t, log, ids, mws, csize, tops, sms, dryRun) return ids, mws, networks, observables From 082dfc23f3426ebe3cc2710249531d62be6146e8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Mon, 23 Aug 2021 09:57:56 -0400 Subject: [PATCH 126/291] add basic test for identity provider --- module/id/fixed_provider_test.go | 62 ++++++++++++++++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 module/id/fixed_provider_test.go diff --git a/module/id/fixed_provider_test.go b/module/id/fixed_provider_test.go new file mode 100644 index 00000000000..9fd224106da --- /dev/null +++ b/module/id/fixed_provider_test.go @@ -0,0 +1,62 @@ +package id + +import ( + "math/rand" + "testing" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + + "github.com/onflow/flow-go/utils/unittest" + "github.com/stretchr/testify/assert" +) + +func TestFixedIdentifierProvider(t *testing.T) { + identifiers := make([]flow.Identifier, 10) + for i := 0; i < len(identifiers); i++ { + identifiers[i] = unittest.IdentifierFixture() + } + + fp := NewFixedIdentifierProvider(identifiers) + + in := identifiers[rand.Intn(10)] + out := unittest.IdentifierFixture() + + assert.True(t, contains(fp.Identifiers(), in)) + assert.False(t, contains(fp.Identifiers(), out)) + +} + +func TestFixedIdentitiesProvider(t *testing.T) { + identities := make([]*flow.Identity, 10) + for i := 0; i < len(identities); i++ { + identities[i] = unittest.IdentityFixture() + } + + fp := NewFixedIdentityProvider(identities) + + in := identities[rand.Intn(10)] + out := unittest.IdentityFixture() + + assert.True(t, idContains(fp.Identities(filter.Any), in)) + assert.False(t, idContains(fp.Identities(filter.Any), out)) + +} + +func contains(a []flow.Identifier, b flow.Identifier) bool { + for _, i := range a { + if b == i { + return true + } + } + return false +} + +func idContains(a []*flow.Identity, b *flow.Identity) bool { + for _, i := range a { + if b == i { + return true + } + } + return false +} From 2a69f1350ac77a059fc532458fa3acebb6711bbf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Mon, 23 Aug 2021 10:14:20 -0400 Subject: [PATCH 127/291] basic test for filtered identifier provider make goimports happy --- module/id/filtered_provider_test.go | 46 +++++++++++++++++++++++++++++ module/id/fixed_provider_test.go | 3 +- module/mocks/network.go | 1 + storage/mocks/storage.go | 1 + 4 files changed, 50 insertions(+), 1 deletion(-) create mode 100644 module/id/filtered_provider_test.go diff --git a/module/id/filtered_provider_test.go b/module/id/filtered_provider_test.go new file mode 100644 index 00000000000..52cffeec180 --- /dev/null +++ b/module/id/filtered_provider_test.go @@ -0,0 +1,46 @@ +package id + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + + "github.com/onflow/flow-go/utils/unittest" +) + +func TestFilteredIdentitiesProvider(t *testing.T) { + identities := make([]*flow.Identity, 10) + for i := 0; i < len(identities); i++ { + identities[i] = unittest.IdentityFixture() + } + identifiers := (flow.IdentityList)(identities).NodeIDs() + + oddIdentifiers := make([]flow.Identifier, 5) + for j := 0; j < 5; j++ { + oddIdentifiers[j] = identifiers[2*j+1] + } + + oddIdentities := make([]*flow.Identity, 5) + for j := 0; j < 5; j++ { + oddIdentities[j] = identities[2*j+1] + } + + ip := NewFixedIdentityProvider(identities) + fp := NewFilteredIdentifierProvider(filter.In(oddIdentities), ip) + + in := 0 + out := 0 + for _, id := range identifiers { + if contains(fp.Identifiers(), id) { + in++ + } else { + out++ + } + } + require.Equal(t, 5, in) + require.Equal(t, 5, out) + +} diff --git a/module/id/fixed_provider_test.go b/module/id/fixed_provider_test.go index 9fd224106da..b53e84c7d2d 100644 --- a/module/id/fixed_provider_test.go +++ b/module/id/fixed_provider_test.go @@ -7,8 +7,9 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/utils/unittest" "github.com/stretchr/testify/assert" + + "github.com/onflow/flow-go/utils/unittest" ) func TestFixedIdentifierProvider(t *testing.T) { diff --git a/module/mocks/network.go b/module/mocks/network.go index b680ae44c89..305cee9f346 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -8,6 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" + crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 7a829429fc4..969044a53ee 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -8,6 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" + flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" From 03f35dfe4a61173d77e6ba54b0ee7d9178af5c90 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 12:21:34 -0700 Subject: [PATCH 128/291] add default identifier provider for sync engine --- .../node_builder/access_node_builder.go | 42 +++++++++---------- .../staked_access_node_builder.go | 3 +- .../unstaked_access_node_builder.go | 2 +- cmd/collection/main.go | 9 +--- cmd/consensus/main.go | 9 +--- cmd/execution/main.go | 9 +--- cmd/node_builder.go | 1 + cmd/scaffold.go | 16 +++++-- cmd/verification/main.go | 10 +---- 9 files changed, 39 insertions(+), 62 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 64c8480d79c..d23024ed603 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -36,7 +36,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" @@ -150,26 +149,25 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedLibP2PNode *p2p.Node - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower - SyncEngineParticipantsProvider id.IdentifierProvider + UnstakedLibP2PNode *p2p.Node + UnstakedNetwork *p2p.Network + unstakedMiddleware *p2p.Middleware + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower // engines IngestEng *ingestion.Engine @@ -322,7 +320,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - builder.SyncEngineParticipantsProvider, + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 8e5b5285e3f..7a8023d495a 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -35,8 +35,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { } fnb.IdentityProvider = idCache - // translator - fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( + fnb.SyncEngineIdentifierProvider = id.NewFilteredIdentifierProvider( filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index f6fd01e63d2..a4e0085eaca 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -106,7 +106,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { // use the default identifier provider - anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() + node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() return nil }) anb.FlowAccessNodeBuilder.Build() diff --git a/cmd/collection/main.go b/cmd/collection/main.go index ef364a5421a..14b83348a65 100644 --- a/cmd/collection/main.go +++ b/cmd/collection/main.go @@ -35,7 +35,6 @@ import ( builder "github.com/onflow/flow-go/module/builder/collection" "github.com/onflow/flow-go/module/epochs" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/ingress" "github.com/onflow/flow-go/module/mempool" epochpool "github.com/onflow/flow-go/module/mempool/epochs" @@ -265,13 +264,7 @@ func main() { followerEng, mainChainSyncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) 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 30b5d32f285..5e6d7712d28 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -50,7 +50,6 @@ import ( dkgmodule "github.com/onflow/flow-go/module/dkg" "github.com/onflow/flow-go/module/epochs" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -618,13 +617,7 @@ func main() { comp, syncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/execution/main.go b/cmd/execution/main.go index d38d2e7a305..4c15ca0d95a 100644 --- a/cmd/execution/main.go +++ b/cmd/execution/main.go @@ -42,7 +42,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" chainsync "github.com/onflow/flow-go/module/synchronization" @@ -457,13 +456,7 @@ func main() { followerEng, syncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index ab027fceda8..a65435206b6 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -141,6 +141,7 @@ type NodeConfig struct { IdentityProvider id.IdentityProvider IDTranslator p2p.IDTranslator NetworkingIdentifierProvider id.IdentifierProvider + SyncEngineIdentifierProvider id.IdentifierProvider // root state information RootBlock *flow.Block diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a5ddb71fbf7..123a7694c55 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -21,6 +21,7 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" @@ -425,14 +426,21 @@ func (fnb *FlowNodeBuilder) initStorage() { func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, node.ProtocolEvents) if err != nil { return err } - fnb.IdentityProvider = idCache - fnb.IDTranslator = idCache - fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) + node.IdentityProvider = idCache + node.IDTranslator = idCache + node.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) + node.SyncEngineIdentifierProvider = id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ) return nil }) } diff --git a/cmd/verification/main.go b/cmd/verification/main.go index bbd2af01438..fe4532b1ca0 100644 --- a/cmd/verification/main.go +++ b/cmd/verification/main.go @@ -24,12 +24,10 @@ import ( "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/encoding" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" @@ -354,13 +352,7 @@ func main() { followerEng, syncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) From b22f4db73ad5d136618aeda871e7916407e413c8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 15:18:13 -0700 Subject: [PATCH 129/291] add comments --- .../staked_access_node_builder.go | 2 +- cmd/node_builder.go | 2 +- model/flow/filter/identity.go | 2 + module/id/filtered_provider.go | 2 + module/id/filtered_provider_test.go | 4 ++ module/id/fixed_provider.go | 4 ++ module/id/id_provider.go | 4 ++ module/id/updatable_provider.go | 5 +++ network/middleware.go | 7 +++- network/p2p/dht_test.go | 12 ++++++ network/p2p/fixed_translator.go | 2 + network/p2p/hierarchical_translator.go | 10 ++++- network/p2p/id_translator.go | 5 +++ network/p2p/identity_provider_translator.go | 12 +++--- network/p2p/libp2pNode.go | 37 +++++++++---------- network/p2p/libp2pUtils.go | 1 + network/p2p/peerstore_provider.go | 4 ++ network/p2p/protocol_state_provider.go | 5 +++ network/p2p/protocol_state_provider_test.go | 3 ++ network/p2p/unstaked_translator.go | 8 ++++ network/test/epochtransition_test.go | 1 + network/test/middleware_test.go | 7 ++++ network/test/testUtil.go | 2 +- 23 files changed, 111 insertions(+), 30 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 7a8023d495a..e17ce0e9d73 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -29,7 +29,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, node.ProtocolEvents) if err != nil { return err } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index a65435206b6..6f1300df68a 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -44,7 +44,7 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() - // InitIDProviders initializes the IdentityProvider and IDTranslator + // InitIDProviders initializes the ID providers needed by various components InitIDProviders() // EnqueueNetworkInit enqueues the default network component with the given context diff --git a/model/flow/filter/identity.go b/model/flow/filter/identity.go index dc39e4ce475..0a02d15cf4c 100644 --- a/model/flow/filter/identity.go +++ b/model/flow/filter/identity.go @@ -62,6 +62,8 @@ func HasNodeID(nodeIDs ...flow.Identifier) flow.IdentityFilter { } } +// HasNetworkingKey returns a filter that returns true for any identity with a +// networking public key matching any of the inputs. func HasNetworkingKey(keys ...crypto.PublicKey) flow.IdentityFilter { return func(identity *flow.Identity) bool { for _, key := range keys { diff --git a/module/id/filtered_provider.go b/module/id/filtered_provider.go index d935125232f..5564ebe3990 100644 --- a/module/id/filtered_provider.go +++ b/module/id/filtered_provider.go @@ -4,6 +4,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// FilteredIdentifierProvider implements an IdentifierProvider which provides the identifiers +// resulting from applying a filter to an IdentityProvider. type FilteredIdentifierProvider struct { filter flow.IdentityFilter identityProvider IdentityProvider diff --git a/module/id/filtered_provider_test.go b/module/id/filtered_provider_test.go index 52cffeec180..28f4f24b94f 100644 --- a/module/id/filtered_provider_test.go +++ b/module/id/filtered_provider_test.go @@ -3,6 +3,7 @@ package id import ( "testing" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/flow" @@ -31,6 +32,9 @@ func TestFilteredIdentitiesProvider(t *testing.T) { ip := NewFixedIdentityProvider(identities) fp := NewFilteredIdentifierProvider(filter.In(oddIdentities), ip) + assert.ElementsMatch(t, fp.Identifiers(), + (flow.IdentityList)(oddIdentities).NodeIDs()) + in := 0 out := 0 for _, id := range identifiers { diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index 39aac7567a1..66cbed93685 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -4,6 +4,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// FixedIdentifierProvider implements an IdentifierProvider which provides a fixed list +// of identifiers. type FixedIdentifierProvider struct { identifiers flow.IdentifierList } @@ -16,6 +18,8 @@ func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { return p.identifiers } +// FixedIdentityProvider implements an IdentityProvider which provides a fixed list +// of identities. type FixedIdentityProvider struct { identities flow.IdentityList } diff --git a/module/id/id_provider.go b/module/id/id_provider.go index a663a8f9215..73f1e87e587 100644 --- a/module/id/id_provider.go +++ b/module/id/id_provider.go @@ -4,10 +4,14 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// IdentifierProvider provides an interface to get a list of Identifiers representing +// a specific set of nodes in the network. type IdentifierProvider interface { Identifiers() flow.IdentifierList } +// IdentifierProvider provides an interface to get a list of Identities representing +// the set of staked participants in the Flow protocol. type IdentityProvider interface { Identities(flow.IdentityFilter) flow.IdentityList } diff --git a/module/id/updatable_provider.go b/module/id/updatable_provider.go index 2288b9e6834..d35a3da0566 100644 --- a/module/id/updatable_provider.go +++ b/module/id/updatable_provider.go @@ -6,6 +6,10 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// UpdatableIDProvider implements an IdentityProvider which can be manually updated by setting +// the IdentityList to a new value. +// It also implements an IdentifierProvider which provides the identifiers of the IdentityList. +// This is mainly used to simulate epoch transitions in tests. type UpdatableIDProvider struct { mu sync.RWMutex identities flow.IdentityList @@ -15,6 +19,7 @@ func NewUpdatableIDProvider(identities flow.IdentityList) *UpdatableIDProvider { return &UpdatableIDProvider{identities: identities} } +// SetIdentities updates the IdentityList returned by this provider. func (p *UpdatableIDProvider) SetIdentities(identities flow.IdentityList) { p.mu.Lock() defer p.mu.Unlock() diff --git a/network/middleware.go b/network/middleware.go index b6d538da53a..ea2c157eb51 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -53,6 +53,8 @@ type Middleware interface { // and updates the underlying libp2p node. UpdateAllowList() + // UpdateNodeAddresses fetches and updates the addresses of all the staked participants + // in the Flow protocol. UpdateNodeAddresses() } @@ -61,8 +63,11 @@ type Middleware interface { type Overlay interface { // Topology returns an identity list of nodes which this node should be directly connected to as peers Topology() (flow.IdentityList, error) - Receive(nodeID flow.Identifier, msg *message.Message) error + + // Identities returns a list of all Flow identities on the network Identities() flow.IdentityList + + Receive(nodeID flow.Identifier, msg *message.Message) error } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index cc01eedaafb..4d2cebcb00e 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -43,6 +43,8 @@ func (suite *DHTTestSuite) TearDownTest() { suite.cancel() } +// TestFindPeerWithDHT checks that if a node is configured to participate in the DHT, it is +// able to create new streams with peers even without knowing their address info beforehand. func (suite *DHTTestSuite) TestFindPeerWithDHT() { count := 10 golog.SetAllLoggers(golog.LevelFatal) // change this to Debug if libp2p logs are needed @@ -59,11 +61,13 @@ func (suite *DHTTestSuite) TestFindPeerWithDHT() { return peer.AddrInfo{ID: dhtServerNodes[i].host.ID(), Addrs: dhtServerNodes[i].host.Addrs()} } + // connect even numbered clients to the first DHT server, and odd number clients to the second for i, clientNode := range dhtClientNodes { err := clientNode.host.Connect(suite.ctx, getDhtServerAddr(uint(i%2))) require.NoError(suite.T(), err) } + // wait for clients to connect to DHT servers and update their routing tables require.Eventually(suite.T(), func() bool { for i, clientNode := range dhtClientNodes { if clientNode.dht.RoutingTable().Find(getDhtServerAddr(uint(i%2)).ID) == "" { @@ -73,16 +77,24 @@ func (suite *DHTTestSuite) TestFindPeerWithDHT() { return true }, time.Second*5, tickForAssertEventually, "nodes failed to connect") + // connect the two DHT servers to each other err := dhtServerNodes[0].host.Connect(suite.ctx, getDhtServerAddr(1)) require.NoError(suite.T(), err) + // wait for the first server to connect to the second and update its routing table require.Eventually(suite.T(), func() bool { return dhtServerNodes[0].dht.RoutingTable().Find(getDhtServerAddr(1).ID) != "" }, time.Second*5, tickForAssertEventually, "dht servers failed to connect") + // check that all even numbered clients can create streams with all odd numbered clients for i := 0; i < len(dhtClientNodes); i += 2 { for j := 1; j < len(dhtClientNodes); j += 2 { + // client i should not yet know the address of client j, but we clear any addresses + // here just in case. dhtClientNodes[i].host.Peerstore().ClearAddrs(dhtClientNodes[j].host.ID()) + + // Try to create a stream from client i to client j. This should resort to a DHT + // lookup since client i does not know client j's address. _, err = dhtClientNodes[i].CreateStream(suite.ctx, dhtClientNodes[j].host.ID()) require.NoError(suite.T(), err) } diff --git a/network/p2p/fixed_translator.go b/network/p2p/fixed_translator.go index ecfff75fbb1..89bae14c743 100644 --- a/network/p2p/fixed_translator.go +++ b/network/p2p/fixed_translator.go @@ -8,6 +8,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// FixedTableIdentityTranslator implements an IDTranslator which translates ID's for a +// fixed list of identities. type FixedTableIdentityTranslator struct { flow2p2p map[flow.Identifier]peer.ID p2p2flow map[peer.ID]flow.Identifier diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go index 4e4d0e77cd5..0a3d98174ce 100644 --- a/network/p2p/hierarchical_translator.go +++ b/network/p2p/hierarchical_translator.go @@ -9,6 +9,10 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// HierarchicalIDTranslator implements an IDTranslator which combines the ID translation +// capabilities of multiple IDTranslators. +// When asked to translate an ID, it will iterate through all of the IDTranslators it was +// given and return the first successful translation. type HierarchicalIDTranslator struct { translators []IDTranslator } @@ -26,15 +30,17 @@ func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, e } errs = multierror.Append(errs, err) } - return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v: %w", flowID, errs) + return "", fmt.Errorf("could not translate the given flow ID: %w", errs) } func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { + var errs *multierror.Error for _, translator := range t.translators { fid, err := translator.GetFlowID(peerID) if err == nil { return fid, nil } + errs = multierror.Append(errs, err) } - return flow.ZeroID, fmt.Errorf("could not find corresponding flow ID for peer ID %v", peerID) + return flow.ZeroID, fmt.Errorf("could not translate the given peer ID: %w", errs) } diff --git a/network/p2p/id_translator.go b/network/p2p/id_translator.go index 8edb2d500d5..a07c23f56da 100644 --- a/network/p2p/id_translator.go +++ b/network/p2p/id_translator.go @@ -6,7 +6,12 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// IDTranslator provides an interface for converting from Flow ID's to LibP2P peer ID's +// and vice versa. type IDTranslator interface { + // GetPeerID returns the peer ID for the given Flow ID GetPeerID(flow.Identifier) (peer.ID, error) + + // GetFlowID returns the Flow ID for the given peer ID GetFlowID(peer.ID) (flow.Identifier, error) } diff --git a/network/p2p/identity_provider_translator.go b/network/p2p/identity_provider_translator.go index 61504b1ef05..61aa33ecf0f 100644 --- a/network/p2p/identity_provider_translator.go +++ b/network/p2p/identity_provider_translator.go @@ -10,11 +10,13 @@ import ( "github.com/onflow/flow-go/module/id" ) -type IdentityProviderIdentityTranslator struct { +// IdentityProviderIDTranslator implements an IDTranslator which provides ID +// translation capabilities for an IdentityProvider. +type IdentityProviderIDTranslator struct { idProvider id.IdentityProvider } -func (t *IdentityProviderIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { +func (t *IdentityProviderIDTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { key, err := p.ExtractPublicKey() if err != nil { return flow.ZeroID, err @@ -30,7 +32,7 @@ func (t *IdentityProviderIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifi return ids[0].NodeID, nil } -func (t *IdentityProviderIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { +func (t *IdentityProviderIDTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { ids := t.idProvider.Identities(filter.HasNodeID(n)) if len(ids) == 0 { return "", fmt.Errorf("could not find identity with id %v", n.String()) @@ -46,6 +48,6 @@ func (t *IdentityProviderIdentityTranslator) GetPeerID(n flow.Identifier) (peer. return pid, nil } -func NewIdentityProviderIdentityTranslator(provider id.IdentityProvider) *IdentityProviderIdentityTranslator { - return &IdentityProviderIdentityTranslator{provider} +func NewIdentityProviderIDTranslator(provider id.IdentityProvider) *IdentityProviderIDTranslator { + return &IdentityProviderIDTranslator{provider} } diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 7d0d2a5fbd6..89dc3e00428 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -332,25 +332,24 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { // CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { - if len(n.host.Peerstore().Addrs(peerID)) == 0 { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - - if n.dht != nil { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - - var err error - func() { - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - defer cancel() - // try to find the peer using the dht - _, err = n.dht.FindPeer(timedCtx, peerID) - }() - - if err != nil { - n.logger.Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") - } else { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") - } + // If we do not currently have any addresses for the given peer, stream creation will almost + // certainly fail. If this Node was configure with a DHT, we can try to lookup the address of + // the peer in the DHT as a last resort. + if len(n.host.Peerstore().Addrs(peerID)) == 0 && n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore, searching for peer in dht") + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + + if err != nil { + n.logger.Warn().Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") + } else { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("addresses found") } } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) diff --git a/network/p2p/libp2pUtils.go b/network/p2p/libp2pUtils.go index 04a2996c2aa..e37b4460b1b 100644 --- a/network/p2p/libp2pUtils.go +++ b/network/p2p/libp2pUtils.go @@ -233,6 +233,7 @@ func flowStream(conn network.Conn) network.Stream { return nil } +// ExtractPeerID extracts the LibP2P peer ID associated with the given Flow public key. func ExtractPeerID(networkPubKey fcrypto.PublicKey) (pid peer.ID, err error) { pk, err := LibP2PPublicKeyFromFlow(networkPubKey) if err != nil { diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go index c31b2cf6917..3f665589a10 100644 --- a/network/p2p/peerstore_provider.go +++ b/network/p2p/peerstore_provider.go @@ -7,6 +7,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// PeerstoreIdentifierProvider implements an IdentifierProvider which provides the identifiers +// of the peers present in the given LibP2P host's peerstore. type PeerstoreIdentifierProvider struct { host host.Host idTranslator IDTranslator @@ -24,7 +26,9 @@ func NewPeerstoreIdentifierProvider(logger zerolog.Logger, host host.Host, idTra func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { var result flow.IdentifierList + // get all peers with addresses from the peerstore pids := p.host.Peerstore().PeersWithAddrs() + for _, pid := range pids { flowID, err := p.idTranslator.GetFlowID(pid) if err != nil { diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index b1a5bdcf834..ad678e6babe 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -13,6 +13,8 @@ import ( "github.com/onflow/flow-go/state/protocol/events" ) +// ProtocolStateIDCache implements an IdentityProvider and IDTranslator for the set of staked +// Flow network participants as according to the given `protocol.State`. type ProtocolStateIDCache struct { events.Noop identities flow.IdentityList @@ -59,6 +61,9 @@ func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(currentEpochCounter ui p.update(header.ID()) } +// update updates the cached identities stored in this provider. +// This is called whenever an epoch event occurs, signaling a possible change in +// protocol state identities. func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { p.logger.Info().Str("blockID", blockID.String()).Msg("updating cached identities") diff --git a/network/p2p/protocol_state_provider_test.go b/network/p2p/protocol_state_provider_test.go index 55ac4074a5e..22167d83757 100644 --- a/network/p2p/protocol_state_provider_test.go +++ b/network/p2p/protocol_state_provider_test.go @@ -59,6 +59,7 @@ func (suite *ProtocolStateProviderTestSuite) SetupTest() { suite.provider = provider } +// triggerUpdate simulates an epoch transition func (suite *ProtocolStateProviderTestSuite) triggerUpdate() { suite.participants = unittest.IdentityListFixture(5, unittest.WithAllRoles(), unittest.WithKeys) @@ -97,6 +98,8 @@ func TestProtocolStateProvider(t *testing.T) { suite.Run(t, new(ProtocolStateProviderTestSuite)) } +// checkStateTransition triggers an epoch transition and checks that the updated +// state is reflected by the provider being tested. func (suite *ProtocolStateProviderTestSuite) checkStateTransition() { oldParticipants := suite.participants diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 400d77fd04c..50802126611 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -10,6 +10,14 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// UnstakedNetworkIDTranslator implements an IDTranslator which translates IDs for peers +// on the unstaked network. +// On the unstaked network, a Flow ID is derived from a peer ID by extracting the public +// key from the peer ID, dropping the first byte (parity byte), and using the remaining +// 32 bytes as the Flow ID. +// Network keys for unstaked nodes must be generated using the Secp256k1 curve, and must +// be positive. It is assumed that these requirements are enforced during key generation, +// and any peer ID's which don't follow these conventions are considered invalid. type UnstakedNetworkIDTranslator struct{} func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { diff --git a/network/test/epochtransition_test.go b/network/test/epochtransition_test.go index 26de777dfa7..d63351a54ef 100644 --- a/network/test/epochtransition_test.go +++ b/network/test/epochtransition_test.go @@ -120,6 +120,7 @@ func TestEpochTransitionTestSuite(t *testing.T) { t.Skip(fmt.Sprintf("test is flaky: %v", &MutableIdentityTableSuite{})) } +// signalIdentityChanged update IDs for all the current set of nodes (simulating an epoch) func (suite *MutableIdentityTableSuite) signalIdentityChanged() { for _, n := range suite.testNodes.nodes { n.mw.UpdateNodeAddresses() diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 3b33df7f312..29c9d980e4a 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -476,7 +476,10 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { require.Error(m.Suite.T(), err) } +// TestUpdateNodeAddresses tests that the UpdateNodeAddresses method correctly updates +// the addresses of the staked network participants. func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { + // create a new staked identity ids, mws, _, providers := GenerateIDsAndMiddlewares(m.T(), 1, false, m.logger) require.Len(m.T(), ids, 1) require.Len(m.T(), providers, 1) @@ -501,12 +504,16 @@ func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") + // message should fail to send because no address is known yet + // for the new identity err := m.mws[0].SendDirect(msg, newId.NodeID) require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) + // update the addresses m.ids = idList m.mws[0].UpdateNodeAddresses() + // now the message should send successfully err = m.mws[0].SendDirect(msg, newId.NodeID) require.NoError(m.T(), err) } diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 9f8c1ef063a..9fe6dfe2447 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -148,7 +148,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultUnicastTimeout, true, true, - p2p.NewIdentityProviderIdentityTranslator(idProviders[i]), + p2p.NewIdentityProviderIDTranslator(idProviders[i]), p2p.WithIdentifierProvider( idProviders[i], ), From a7448d22e4dd45419ec5bbcf3a01887035da8efa Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 13:59:55 -0700 Subject: [PATCH 130/291] using a factory method for the SyncEngineParticipantsProvider to wait for middleware to have started for the unstaked an --- .../node_builder/access_node_builder.go | 38 +++++++++---------- .../staked_access_node_builder.go | 18 +++++---- .../unstaked_access_node_builder.go | 3 +- 3 files changed, 31 insertions(+), 28 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 87dd23db409..f8ac32cd083 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -152,24 +152,24 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - LibP2PNode *p2p.Node - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower - SyncEngineParticipantsProvider id.IdentifierProvider + LibP2PNode *p2p.Node + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower + SyncEngineParticipantsProviderFactory func() id.IdentifierProvider // engines IngestEng *ingestion.Engine @@ -322,7 +322,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - builder.SyncEngineParticipantsProvider, + builder.SyncEngineParticipantsProviderFactory(), ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index cb7e42ba7a8..89677b34c31 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -40,14 +40,16 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache // translator - fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - p2p.NotEjectedFilter, - ), - idCache, - ) + fnb.SyncEngineParticipantsProviderFactory = func() id.IdentifierProvider { + return id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NotEjectedFilter, + ), + idCache, + ) + } fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 6d93741a152..ba74ea42755 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -9,6 +9,7 @@ import ( "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" @@ -141,7 +142,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { return errors.New("middleware was of unexpected type") } // use the default identifier provider - anb.SyncEngineParticipantsProvider = middleware.IdentifierProvider() + anb.SyncEngineParticipantsProviderFactory = func() id.IdentifierProvider { return middleware.IdentifierProvider() } return nil }) anb.FlowAccessNodeBuilder.BuildConsensusFollower() From 574b4e041ba11bbfb7f5554ab73eb9d14e523793 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 14:05:36 -0700 Subject: [PATCH 131/291] define new subscription manager --- .../node_builder/access_node_builder.go | 8 ++-- .../unstaked_access_node_builder.go | 8 ++-- network/channel_reassigner/network.go | 25 ---------- network/converter/subscriptionManager.go | 47 +++++++++++++++++++ 4 files changed, 56 insertions(+), 32 deletions(-) delete mode 100644 network/channel_reassigner/network.go create mode 100644 network/converter/subscriptionManager.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 87dd23db409..7f49266d36f 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -660,12 +660,12 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, middleware network.Middleware, - topology network.Topology) (*p2p.Network, error) { + topology network.Topology, + subMngr network.SubscriptionManager, +) (*p2p.Network, error) { codec := cborcodec.NewCodec() - subscriptionManager := p2p.NewChannelSubscriptionManager(middleware) - // creates network instance net, err := p2p.NewNetwork( builder.Logger, @@ -674,7 +674,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, builder.Middleware, p2p.DefaultCacheSize, topology, - subscriptionManager, + subMngr, networkMetrics, builder.IdentityProvider, ) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 56909cf44c3..e0d64bcfb22 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -11,7 +11,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/network/channel_reassigner" + "github.com/onflow/flow-go/network/converter" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol/events/gadgets" ) @@ -148,11 +148,13 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + subscriptionManager := converter.NewSubscriptionManager(p2p.NewChannelSubscriptionManager(middleware), engine.SyncCommittee, engine.UnstakedSyncCommittee) + // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil, subscriptionManager) builder.MustNot(err) - builder.Network = channel_reassigner.NewChannelReassignerNetwork(network, engine.SyncCommittee, engine.UnstakedSyncCommittee) + builder.Network = network builder.Middleware = middleware builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) diff --git a/network/channel_reassigner/network.go b/network/channel_reassigner/network.go deleted file mode 100644 index 3daedbbccd0..00000000000 --- a/network/channel_reassigner/network.go +++ /dev/null @@ -1,25 +0,0 @@ -package channel_reassigner - -import ( - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/network" -) - -type ChannelReassignerNetwork struct { - module.ReadyDoneAwareNetwork - from network.Channel - to network.Channel -} - -func NewChannelReassignerNetwork(net module.ReadyDoneAwareNetwork, from, to network.Channel) *ChannelReassignerNetwork { - return &ChannelReassignerNetwork{net, from, to} -} - -func (n *ChannelReassignerNetwork) Register(channel network.Channel, engine network.Engine) (network.Conduit, error) { - reassignedChannel := channel - if channel == n.from { - reassignedChannel = n.to - } - - return n.ReadyDoneAwareNetwork.Register(reassignedChannel, engine) -} diff --git a/network/converter/subscriptionManager.go b/network/converter/subscriptionManager.go new file mode 100644 index 00000000000..169ecb47036 --- /dev/null +++ b/network/converter/subscriptionManager.go @@ -0,0 +1,47 @@ +package converter + +import "github.com/onflow/flow-go/network" + +type SubscriptionManager struct { + subMngr network.SubscriptionManager + from network.Channel + to network.Channel +} + +func NewSubscriptionManager(subMngr network.SubscriptionManager, from network.Channel, to network.Channel) *SubscriptionManager { + return &SubscriptionManager{subMngr, from, to} +} + +func (sm *SubscriptionManager) convert(channel network.Channel) network.Channel { + if channel == sm.from { + return sm.to + } + return channel +} + +func (sm *SubscriptionManager) reverse(channel network.Channel) network.Channel { + if channel == sm.to { + return sm.from + } + return channel +} + +func (sm *SubscriptionManager) Register(channel network.Channel, engine network.Engine) error { + return sm.subMngr.Register(sm.convert(channel), engine) +} + +func (sm *SubscriptionManager) Unregister(channel network.Channel) error { + return sm.subMngr.Unregister(sm.convert(channel)) +} + +func (sm *SubscriptionManager) GetEngine(channel network.Channel) (network.Engine, error) { + return sm.subMngr.GetEngine(sm.convert(channel)) +} + +func (sm *SubscriptionManager) Channels() network.ChannelList { + var channels network.ChannelList + for _, ch := range sm.subMngr.Channels() { + channels = append(channels, sm.reverse(ch)) + } + return channels +} From 37a648ff043d2d4eeb9610ad6035238f9d7b845b Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 14:06:43 -0700 Subject: [PATCH 132/291] adding comment --- cmd/access/node_builder/access_node_builder.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index f8ac32cd083..791a63c6429 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -169,6 +169,9 @@ type FlowAccessNodeBuilder struct { Finalized *flow.Header Pending []*flow.Header FollowerCore module.HotStuffFollower + // for the untsaked access node, the sync engine participants provider is the libp2p peer store which is not + // available until after the network has started. Hence, a factory function that needs to be called just before + // creating the sync engine SyncEngineParticipantsProviderFactory func() id.IdentifierProvider // engines From 0efaf206c5391e8467e3ca8e4828e0fc2ca7f3ea Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 14:07:30 -0700 Subject: [PATCH 133/291] Update unstaked_access_node_builder.go --- cmd/access/node_builder/unstaked_access_node_builder.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index ba74ea42755..bcc9aaaaa66 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -158,8 +158,10 @@ func (anb *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Con // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() + subscriptionManager := p2p.NewChannelSubscriptionManager(anb.Middleware) + // topology is nil since its automatically managed by libp2p - network, err := anb.initNetwork(anb.Me, unstakedNetworkMetrics, anb.Middleware, nil) + network, err := anb.initNetwork(anb.Me, unstakedNetworkMetrics, anb.Middleware, nil, subscriptionManager) anb.MustNot(err) anb.Network = network From 08bf26b632f456fe38c1a16d14d159a90b17f8a0 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 16:46:41 -0700 Subject: [PATCH 134/291] wip --- .../node_builder/access_node_builder.go | 34 +++++++++---------- cmd/scaffold.go | 8 ----- integration/tests/access/access_api_test.go | 2 +- network/middleware.go | 1 - 4 files changed, 18 insertions(+), 27 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 791a63c6429..a0a320778be 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -152,23 +152,23 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - LibP2PNode *p2p.Node - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + LibP2PNode *p2p.Node + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower // for the untsaked access node, the sync engine participants provider is the libp2p peer store which is not // available until after the network has started. Hence, a factory function that needs to be called just before // creating the sync engine diff --git a/cmd/scaffold.go b/cmd/scaffold.go index e70b7cf3514..a77f215d90d 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -21,7 +21,6 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" @@ -434,13 +433,6 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { node.IdentityProvider = idCache node.IDTranslator = idCache node.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) - node.SyncEngineIdentifierProvider = id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - idCache, - ) return nil }) } diff --git a/integration/tests/access/access_api_test.go b/integration/tests/access/access_api_test.go index 24fd024f720..56e437afc26 100644 --- a/integration/tests/access/access_api_test.go +++ b/integration/tests/access/access_api_test.go @@ -66,7 +66,7 @@ func (suite *AccessSuite) SetupTest() { collConfig := testnet.NewNodeConfig(flow.RoleCollection, testnet.WithID(collID)) nodeConfigs = append(nodeConfigs, collConfig) - conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs, nil) + conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) // start the network diff --git a/network/middleware.go b/network/middleware.go index fdb6f86b488..ea2c157eb51 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -68,7 +68,6 @@ type Overlay interface { Identities() flow.IdentityList Receive(nodeID flow.Identifier, msg *message.Message) error - Identities() flow.IdentityList } // Connection represents an interface to read from & write to a connection. From 589dfc2c76d1af27fac816fee75f8f221465a0b8 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 16:49:49 -0700 Subject: [PATCH 135/291] reverting change to network/p2p/libp2pNode.go --- network/p2p/libp2pNode.go | 37 ++++++++++++++++++------------------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 7d0d2a5fbd6..89dc3e00428 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -332,25 +332,24 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { // CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { - if len(n.host.Peerstore().Addrs(peerID)) == 0 { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - - if n.dht != nil { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - - var err error - func() { - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - defer cancel() - // try to find the peer using the dht - _, err = n.dht.FindPeer(timedCtx, peerID) - }() - - if err != nil { - n.logger.Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") - } else { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") - } + // If we do not currently have any addresses for the given peer, stream creation will almost + // certainly fail. If this Node was configure with a DHT, we can try to lookup the address of + // the peer in the DHT as a last resort. + if len(n.host.Peerstore().Addrs(peerID)) == 0 && n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore, searching for peer in dht") + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + + if err != nil { + n.logger.Warn().Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") + } else { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("addresses found") } } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) From 07b6705ee0cf7a87df16a245dc658bc6f7ae4190 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 17:43:32 -0700 Subject: [PATCH 136/291] added back conerteer network --- .../node_builder/access_node_builder.go | 3 +-- .../unstaked_access_node_builder.go | 8 +++--- network/converter/network.go | 27 +++++++++++++++++++ 3 files changed, 32 insertions(+), 6 deletions(-) create mode 100644 network/converter/network.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index fdfe23dfb93..005d0f374b7 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -661,7 +661,6 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, middleware network.Middleware, topology network.Topology, - subMngr network.SubscriptionManager, ) (*p2p.Network, error) { codec := cborcodec.NewCodec() @@ -674,7 +673,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, builder.Middleware, p2p.DefaultCacheSize, topology, - subMngr, + p2p.NewChannelSubscriptionManager(middleware), networkMetrics, builder.IdentityProvider, ) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index bcc9aaaaa66..5b4efef6cc2 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -7,11 +7,13 @@ import ( "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/converter" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol/events/gadgets" ) @@ -158,13 +160,11 @@ func (anb *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Con // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - subscriptionManager := p2p.NewChannelSubscriptionManager(anb.Middleware) - // topology is nil since its automatically managed by libp2p - network, err := anb.initNetwork(anb.Me, unstakedNetworkMetrics, anb.Middleware, nil, subscriptionManager) + network, err := anb.initNetwork(anb.Me, unstakedNetworkMetrics, anb.Middleware, nil) anb.MustNot(err) - anb.Network = network + anb.Network = converter.NewNetwork(network, engine.SyncCommittee, engine.UnstakedSyncCommittee) anb.Logger.Info().Msgf("network will run on address: %s", anb.BindAddr) diff --git a/network/converter/network.go b/network/converter/network.go new file mode 100644 index 00000000000..703a88be291 --- /dev/null +++ b/network/converter/network.go @@ -0,0 +1,27 @@ +package converter + +import ( + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/network" +) + +type Network struct { + module.ReadyDoneAwareNetwork + from network.Channel + to network.Channel +} + +func NewNetwork(net module.ReadyDoneAwareNetwork, from network.Channel, to network.Channel) *Network { + return &Network{net, from, to} +} + +func (n *Network) convert(channel network.Channel) network.Channel { + if channel == n.from { + return n.to + } + return channel +} + +func (n *Network) Register(channel network.Channel, engine network.Engine) (network.Conduit, error) { + return n.ReadyDoneAwareNetwork.Register(n.convert(channel), engine) +} From 9617911bd9eee740aa122cf46108cd83a771212f Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 18:26:18 -0700 Subject: [PATCH 137/291] define unsatked metrics --- module/metrics/engine.go | 6 ++-- module/metrics/namespaces.go | 4 +-- module/metrics/network.go | 26 +++++++------- module/metrics/ping.go | 4 +-- module/metrics/unstaked/engine.go | 25 +++++++++++++ module/metrics/unstaked/network.go | 57 ++++++++++++++++++++++++++++++ 6 files changed, 102 insertions(+), 20 deletions(-) create mode 100644 module/metrics/unstaked/engine.go create mode 100644 module/metrics/unstaked/network.go diff --git a/module/metrics/engine.go b/module/metrics/engine.go index 16e9cf834c9..b785a57afbb 100644 --- a/module/metrics/engine.go +++ b/module/metrics/engine.go @@ -17,21 +17,21 @@ func NewEngineCollector() *EngineCollector { sent: promauto.NewCounterVec(prometheus.CounterOpts{ Name: "messages_sent_total", - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemEngine, Help: "the number of messages sent by engines", }, []string{EngineLabel, LabelMessage}), received: promauto.NewCounterVec(prometheus.CounterOpts{ Name: "messages_received_total", - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemEngine, Help: "the number of messages received by engines", }, []string{EngineLabel, LabelMessage}), handled: promauto.NewCounterVec(prometheus.CounterOpts{ Name: "messages_handled_total", - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemEngine, Help: "the number of messages handled by engines", }, []string{EngineLabel, LabelMessage}), diff --git a/module/metrics/namespaces.go b/module/metrics/namespaces.go index 2187ab87964..3480bc74e93 100644 --- a/module/metrics/namespaces.go +++ b/module/metrics/namespaces.go @@ -2,7 +2,7 @@ package metrics // Prometheus metric namespaces const ( - namespaceNetwork = "network" + NamespaceNetwork = "network" namespaceStorage = "storage" namespaceAccess = "access" namespaceCollection = "collection" @@ -17,7 +17,7 @@ const ( // subsystemLibp2p = "libp2p" subsystemGossip = "gossip" subsystemEngine = "engine" - subsystemQueue = "queue" + SubsystemQueue = "queue" ) // Storage subsystems represent the various components of the storage layer. diff --git a/module/metrics/network.go b/module/metrics/network.go index 1d7e724768f..6ce11f7320e 100644 --- a/module/metrics/network.go +++ b/module/metrics/network.go @@ -31,7 +31,7 @@ func NewNetworkCollector() *NetworkCollector { nc := &NetworkCollector{ outboundMessageSize: promauto.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemGossip, Name: "outbound_message_size_bytes", Help: "size of the outbound network message", @@ -39,7 +39,7 @@ func NewNetworkCollector() *NetworkCollector { }, []string{LabelChannel, LabelMessage}), inboundMessageSize: promauto.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemGossip, Name: "inbound_message_size_bytes", Help: "size of the inbound network message", @@ -47,44 +47,44 @@ func NewNetworkCollector() *NetworkCollector { }, []string{LabelChannel, LabelMessage}), duplicateMessagesDropped: promauto.NewCounterVec(prometheus.CounterOpts{ - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemGossip, Name: "duplicate_messages_dropped", Help: "number of duplicate messages dropped", }, []string{LabelChannel, LabelMessage}), queueSize: promauto.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: namespaceNetwork, - Subsystem: subsystemQueue, + Namespace: NamespaceNetwork, + Subsystem: SubsystemQueue, Name: "message_queue_size", Help: "the number of elements in the message receive queue", }, []string{LabelPriority}), queueDuration: promauto.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: namespaceNetwork, - Subsystem: subsystemQueue, + Namespace: NamespaceNetwork, + Subsystem: SubsystemQueue, Name: "message_queue_duration_seconds", Help: "duration [seconds; measured with float64 precision] of how long a message spent in the queue before delivered to an engine.", Buckets: []float64{0.01, 0.1, 0.5, 1, 2, 5}, // 10ms, 100ms, 500ms, 1s, 2s, 5s }, []string{LabelPriority}), inboundProcessTime: promauto.NewCounterVec(prometheus.CounterOpts{ - Namespace: namespaceNetwork, - Subsystem: subsystemQueue, + Namespace: NamespaceNetwork, + Subsystem: SubsystemQueue, Name: "engine_message_processing_time_seconds", Help: "duration [seconds; measured with float64 precision] of how long a queue worker blocked for an engine processing message", }, []string{LabelChannel}), outboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: namespaceNetwork, - Subsystem: subsystemQueue, + Namespace: NamespaceNetwork, + Subsystem: SubsystemQueue, Name: "outbound_connection_count", Help: "the number of outbound connections of this node", }), inboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: namespaceNetwork, - Subsystem: subsystemQueue, + Namespace: NamespaceNetwork, + Subsystem: SubsystemQueue, Name: "inbound_connection_count", Help: "the number of inbound connections of this node", }), diff --git a/module/metrics/ping.go b/module/metrics/ping.go index 691d361016e..38a7411df31 100644 --- a/module/metrics/ping.go +++ b/module/metrics/ping.go @@ -18,13 +18,13 @@ func NewPingCollector() *PingCollector { pc := &PingCollector{ reachable: promauto.NewGaugeVec(prometheus.GaugeOpts{ Name: "node_reachable", - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemGossip, Help: "report whether a node is reachable", }, []string{LabelNodeID, LabelNodeAddress, LabelNodeRole, LabelNodeInfo}), sealedHeight: promauto.NewGaugeVec(prometheus.GaugeOpts{ Name: "sealed_height", - Namespace: namespaceNetwork, + Namespace: NamespaceNetwork, Subsystem: subsystemGossip, Help: "the last sealed height of a node", }, []string{LabelNodeID, LabelNodeAddress, LabelNodeRole, LabelNodeInfo, LabelNodeVersion}), diff --git a/module/metrics/unstaked/engine.go b/module/metrics/unstaked/engine.go new file mode 100644 index 00000000000..ed372db0f4a --- /dev/null +++ b/module/metrics/unstaked/engine.go @@ -0,0 +1,25 @@ +package unstaked + +import ( + "github.com/onflow/flow-go/module/metrics" +) + +type EngineCollector struct { + collector metrics.EngineCollector +} + +func NewUnstakedEngineCollector(collector metrics.EngineCollector) *EngineCollector { + return &EngineCollector{collector} +} + +func (ec *EngineCollector) MessageSent(engine string, message string) { + ec.collector.MessageSent("unstaked_"+engine, message) +} + +func (ec *EngineCollector) MessageReceived(engine string, message string) { + ec.collector.MessageReceived("unstaked_"+engine, message) +} + +func (ec *EngineCollector) MessageHandled(engine string, message string) { + ec.collector.MessageHandled("unstaked_"+engine, message) +} diff --git a/module/metrics/unstaked/network.go b/module/metrics/unstaked/network.go new file mode 100644 index 00000000000..81f8c6e6c7d --- /dev/null +++ b/module/metrics/unstaked/network.go @@ -0,0 +1,57 @@ +package unstaked + +import ( + "github.com/onflow/flow-go/module/metrics" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +type NetworkCollector struct { + metrics.NetworkCollector + + outboundConnectionCount prometheus.Gauge + inboundConnectionCount prometheus.Gauge +} + +func NewUnstakedNetworkCollector(collector metrics.NetworkCollector) *NetworkCollector { + return &NetworkCollector{ + collector, + promauto.NewGauge(prometheus.GaugeOpts{ + Namespace: metrics.NamespaceNetwork, + Subsystem: metrics.SubsystemQueue, + Name: "unstaked_outbound_connection_count", + Help: "the number of outbound connections to unstaked nodes", + }), + promauto.NewGauge(prometheus.GaugeOpts{ + Namespace: metrics.NamespaceNetwork, + Subsystem: metrics.SubsystemQueue, + Name: "unstaked_inbound_connection_count", + Help: "the number of inbound connections from unstaked nodes", + }), + } +} + +// NetworkMessageSent tracks the message size of the last message sent out on the wire +// in bytes for the given topic +func (nc *NetworkCollector) NetworkMessageSent(sizeBytes int, topic string, messageType string) { + nc.NetworkCollector.NetworkMessageSent(sizeBytes, "unstaked_"+topic, messageType) +} + +// NetworkMessageReceived tracks the message size of the last message received on the wire +// in bytes for the given topic +func (nc *NetworkCollector) NetworkMessageReceived(sizeBytes int, topic string, messageType string) { + nc.NetworkCollector.NetworkMessageReceived(sizeBytes, "unstaked_"+topic, messageType) +} + +// NetworkDuplicateMessagesDropped tracks the number of messages dropped by the network layer due to duplication +func (nc *NetworkCollector) NetworkDuplicateMessagesDropped(topic, messageType string) { + nc.NetworkCollector.NetworkDuplicateMessagesDropped("unstaked_"+topic, messageType) +} + +func (nc *NetworkCollector) OutboundConnections(connectionCount uint) { + nc.outboundConnectionCount.Set(float64(connectionCount)) +} + +func (nc *NetworkCollector) InboundConnections(connectionCount uint) { + nc.inboundConnectionCount.Set(float64(connectionCount)) +} From 5d38cc4c8335dbfa4c92ce8836b02a339279570f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Tue, 24 Aug 2021 22:56:31 -0400 Subject: [PATCH 138/291] [network] test for peerstore ID provider --- network/test/peerstore_provider_test.go | 114 ++++++++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 network/test/peerstore_provider_test.go diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go new file mode 100644 index 00000000000..26fd1dfcf9b --- /dev/null +++ b/network/test/peerstore_provider_test.go @@ -0,0 +1,114 @@ +package test + +import ( + "context" + "math/rand" + "os" + "strconv" + "testing" + "time" + + "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/multiformats/go-multiaddr" + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/utils/unittest" +) + +type PeerStoreProviderTestSuite struct { + suite.Suite + logger zerolog.Logger + nodes []*p2p.Node + libp2pPeersIDs []peer.ID + peerIDprovider *p2p.PeerstoreIdentifierProvider + translator *p2p.HierarchicalIDTranslator + ids flow.IdentityList +} + +func TestPeerStoreProviderTestSuite(t *testing.T) { + suite.Run(t, new(PeerStoreProviderTestSuite)) +} + +const nodeCount = 2 +const peerCount = 3 +const testNodeIndex = 0 // testNodeIndex < nodeCount + +func (suite *PeerStoreProviderTestSuite) SetupTest() { + suite.logger = zerolog.New(os.Stderr).Level(zerolog.ErrorLevel) + log.SetAllLoggers(log.LevelError) + ctx := context.Background() + + suite.ids, suite.nodes, _ = GenerateIDs(suite.T(), suite.logger, nodeCount, !DryRun) + t, err := p2p.NewFixedTableIdentityTranslator(suite.ids) + require.NoError(suite.T(), err) + + u := p2p.NewUnstakedNetworkIDTranslator() + suite.translator = p2p.NewHierarchicalIDTranslator(u, t) + + // emulate the middleware behavior in populating the testnode's peer store + libp2pPeers := make([]peer.ID, peerCount) + for i := 0; i < peerCount; i++ { + peerAddrInfo := suite.randomPeerInfoWithStubNetwork() + err := suite.nodes[testNodeIndex].AddPeer(ctx, peerAddrInfo) + // conn gater (then connection routine) will complain + require.Error(suite.T(), err) + libp2pPeers[i] = peerAddrInfo.ID + } + suite.libp2pPeersIDs = libp2pPeers + suite.peerIDprovider = p2p.NewPeerstoreIdentifierProvider(suite.logger, suite.nodes[testNodeIndex].Host(), suite.translator) + + // sanity checks + assert.Len(suite.T(), suite.nodes, nodeCount) + assert.Len(suite.T(), suite.libp2pPeersIDs, peerCount) + assert.Len(suite.T(), suite.ids, nodeCount) + + // give the Node the time to establish those connections + time.Sleep(2 * time.Second) +} + +func (suite *PeerStoreProviderTestSuite) TestTranslationPeers() { + + identifiers := suite.peerIDprovider.Identifiers() + + peerIDs := make([]peer.ID, len(identifiers)) + for i := 0; i < len(identifiers); i++ { + + pID, err := suite.translator.GetPeerID(identifiers[i]) + require.NoError(suite.T(), err) + peerIDs[i] = pID + } + // check we can find the libp2p peers + assert.Subset(suite.T(), peerIDs, suite.libp2pPeersIDs, "peer IDs should include those in the peer Store") + +} + +func (suite *PeerStoreProviderTestSuite) randomPeerInfoWithStubNetwork() peer.AddrInfo { + + // we don't care about network information, but those peers need an address + ip := "127.0.0.1" + port := strconv.Itoa(rand.Intn(65535 - 1024)) + + addr := p2p.MultiAddressStr(ip, port) + maddr, err := multiaddr.NewMultiaddr(addr) + require.NoError(suite.T(), err) + + privKey, err := utils.GenerateUnstakedNetworkingKey(unittest.SeedFixture(crypto.KeyGenSeedMinLenECDSASecp256k1)) + require.NoError(suite.T(), err) + + libp2pKey, err := p2p.LibP2PPublicKeyFromFlow(privKey.PublicKey()) + require.NoError(suite.T(), err) + + id, err := peer.IDFromPublicKey(libp2pKey) + require.NoError(suite.T(), err) + + pInfo := peer.AddrInfo{ID: id, Addrs: []multiaddr.Multiaddr{maddr}} + return pInfo +} From c8791cafb0d289a2a313cb0ae365dea4ca8c7c12 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Tue, 24 Aug 2021 23:11:38 -0400 Subject: [PATCH 139/291] [network] Correct bug in ID validation Introduced in c8bce7815e770bd5786e5c03c1476d923468c550 The From field of a message is not necessarily the depository of the authentified signing information of a message: it can be an attached public key that may or may not match (hash to) the libp2p PeerID. This reinstates the pid.MatchesPublicKey(pubk) call that checks that. --- network/p2p/readSubscription.go | 43 ++++++++++++++++++++++++++++++++- 1 file changed, 42 insertions(+), 1 deletion(-) diff --git a/network/p2p/readSubscription.go b/network/p2p/readSubscription.go index db87b097afa..681721286cb 100644 --- a/network/p2p/readSubscription.go +++ b/network/p2p/readSubscription.go @@ -2,9 +2,11 @@ package p2p import ( "context" + "fmt" "strings" "sync" + lcrypto "github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" @@ -79,7 +81,7 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { return } - pid, err := peer.IDFromBytes(rawMsg.From) + pid, err := messageSigningID(rawMsg) if err != nil { r.log.Err(err).Msg("failed to validate peer ID of incoming message") return @@ -102,3 +104,42 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { r.callback(&msg, pid) } } + +// messagePubKey extracts the public key of the envelope signer from a libp2p message. +// The location of that key depends on the type of the key, see: +// https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md +// This reproduces the exact logic of the private function doing the same decoding in libp2p: +// https://github.com/libp2p/go-libp2p-pubsub/blob/ba28f8ecfc551d4d916beb748d3384951bce3ed0/sign.go#L77 +func messageSigningID(m *pubsub.Message) (peer.ID, error) { + var pubk lcrypto.PubKey + + // m.From is the original sender of the message (versus `m.ReceivedFrom` which is the last hop which sent us this message) + pid, err := peer.IDFromBytes(m.From) + if err != nil { + return "", err + } + + if m.Key == nil { + // no attached key, it must be extractable from the source ID + pubk, err = pid.ExtractPublicKey() + if err != nil { + return "", fmt.Errorf("cannot extract signing key: %s", err.Error()) + } + if pubk == nil { + return "", fmt.Errorf("cannot extract signing key") + } + } else { + pubk, err = lcrypto.UnmarshalPublicKey(m.Key) + if err != nil { + return "", fmt.Errorf("cannot unmarshal signing key: %s", err.Error()) + } + + // verify that the source ID matches the attached key + if !pid.MatchesPublicKey(pubk) { + return "", fmt.Errorf("bad signing key; source ID %s doesn't match key", pid) + } + } + + // the pid either contains or matches the signing pubKey + return pid, nil +} From c0e1fdaaaba4207aa00396e6c3818b2ffed4d2fb Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 21:39:31 -0700 Subject: [PATCH 140/291] add new network metrics --- .../staked_access_node_builder.go | 9 +-- module/metrics.go | 12 +++- module/metrics/network.go | 58 +++++++++++++------ module/metrics/unstaked/engine.go | 14 ++--- module/metrics/unstaked/network.go | 57 ------------------ network/p2p/connManager.go | 38 +++++++++--- network/p2p/middleware.go | 2 +- network/p2p/readConnection.go | 6 ++ 8 files changed, 100 insertions(+), 96 deletions(-) delete mode 100644 module/metrics/unstaked/network.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 43aa3843126..51ba4ccfee9 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -16,6 +16,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/module/metrics/unstaked" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" @@ -111,10 +112,10 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { }). Component("unstaked sync request handler", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { syncRequestHandler := synceng.NewRequestHandlerEngine( - node.Logger, // TODO: use different logger for unstaked network? - node.Metrics.Engine, // TODO: use different metrics for unstaked network? + node.Logger.With().Bool("unstaked", true).Logger(), + unstaked.NewUnstakedEngineCollector(node.Metrics.Engine), unstakedNetworkConduit, - node.Me, // TODO: does staked node use same Node ID on unstaked network? + node.Me, node.Storage.Blocks, anb.SyncCore, anb.FinalizedHeader, @@ -189,7 +190,7 @@ func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, myAddr = builder.BaseConfig.BindAddr } - connManager := p2p.NewConnManager(builder.Logger, builder.Metrics.Network) + connManager := p2p.NewConnManager(builder.Logger, builder.IdentityProvider, builder.Metrics.Network) return func() (*p2p.Node, error) { libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). diff --git a/module/metrics.go b/module/metrics.go index fb62d106690..c3d87527822 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -12,13 +12,13 @@ type EntriesFunc func() uint // Network Metrics type NetworkMetrics interface { // NetworkMessageSent size in bytes and count of the network message sent - NetworkMessageSent(sizeBytes int, topic string, messageType string) + NetworkMessageSent(sizeBytes int, topic string, messageType string, targetID string) // NetworkMessageReceived size in bytes and count of the network message received - NetworkMessageReceived(sizeBytes int, topic string, messageType string) + NetworkMessageReceived(sizeBytes int, topic string, messageType string, originID string) // NetworkDuplicateMessagesDropped counts number of messages dropped due to duplicate detection - NetworkDuplicateMessagesDropped(topic string, messageType string) + NetworkDuplicateMessagesDropped(topic string, messageType string, originID string) // Message receive queue metrics // MessageAdded increments the metric tracking the number of messages in the queue with the given priority @@ -38,6 +38,12 @@ type NetworkMetrics interface { // InboundConnections updates the metric tracking the number of inbound connections of this node InboundConnections(connectionCount uint) + + // UnstakedOutboundConnections updates the metric tracking the number of outbound connections to unstaked nodes + UnstakedOutboundConnections(connectionCount uint) + + // UnstakedInboundConnections updates the metric tracking the number of inbound connections from unstaked nodes + UnstakedInboundConnections(connectionCount uint) } type EngineMetrics interface { diff --git a/module/metrics/network.go b/module/metrics/network.go index 6ce11f7320e..077612c35b2 100644 --- a/module/metrics/network.go +++ b/module/metrics/network.go @@ -16,14 +16,16 @@ const ( ) type NetworkCollector struct { - outboundMessageSize *prometheus.HistogramVec - inboundMessageSize *prometheus.HistogramVec - duplicateMessagesDropped *prometheus.CounterVec - queueSize *prometheus.GaugeVec - queueDuration *prometheus.HistogramVec - inboundProcessTime *prometheus.CounterVec - outboundConnectionCount prometheus.Gauge - inboundConnectionCount prometheus.Gauge + outboundMessageSize *prometheus.HistogramVec + inboundMessageSize *prometheus.HistogramVec + duplicateMessagesDropped *prometheus.CounterVec + queueSize *prometheus.GaugeVec + queueDuration *prometheus.HistogramVec + inboundProcessTime *prometheus.CounterVec + outboundConnectionCount prometheus.Gauge + inboundConnectionCount prometheus.Gauge + unstakedOutboundConnectionCount prometheus.Gauge + unstakedInboundConnectionCount prometheus.Gauge } func NewNetworkCollector() *NetworkCollector { @@ -36,7 +38,7 @@ func NewNetworkCollector() *NetworkCollector { Name: "outbound_message_size_bytes", Help: "size of the outbound network message", Buckets: []float64{KiB, 100 * KiB, 500 * KiB, 1 * MiB, 2 * MiB, 4 * MiB}, - }, []string{LabelChannel, LabelMessage}), + }, []string{LabelChannel, LabelMessage, LabelNodeID}), inboundMessageSize: promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: NamespaceNetwork, @@ -44,14 +46,14 @@ func NewNetworkCollector() *NetworkCollector { Name: "inbound_message_size_bytes", Help: "size of the inbound network message", Buckets: []float64{KiB, 100 * KiB, 500 * KiB, 1 * MiB, 2 * MiB, 4 * MiB}, - }, []string{LabelChannel, LabelMessage}), + }, []string{LabelChannel, LabelMessage, LabelNodeID}), duplicateMessagesDropped: promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: NamespaceNetwork, Subsystem: subsystemGossip, Name: "duplicate_messages_dropped", Help: "number of duplicate messages dropped", - }, []string{LabelChannel, LabelMessage}), + }, []string{LabelChannel, LabelMessage, LabelNodeID}), queueSize: promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: NamespaceNetwork, @@ -88,6 +90,20 @@ func NewNetworkCollector() *NetworkCollector { Name: "inbound_connection_count", Help: "the number of inbound connections of this node", }), + + unstakedOutboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ + Namespace: NamespaceNetwork, + Subsystem: SubsystemQueue, + Name: "unstaked_outbound_connection_count", + Help: "the number of outbound connections to unstaked nodes", + }), + + unstakedInboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ + Namespace: NamespaceNetwork, + Subsystem: SubsystemQueue, + Name: "unstaked_inbound_connection_count", + Help: "the number of inbound connections from unstaked nodes", + }), } return nc @@ -95,19 +111,19 @@ func NewNetworkCollector() *NetworkCollector { // NetworkMessageSent tracks the message size of the last message sent out on the wire // in bytes for the given topic -func (nc *NetworkCollector) NetworkMessageSent(sizeBytes int, topic string, messageType string) { - nc.outboundMessageSize.WithLabelValues(topic, messageType).Observe(float64(sizeBytes)) +func (nc *NetworkCollector) NetworkMessageSent(sizeBytes int, topic string, messageType string, targetID string) { + nc.outboundMessageSize.WithLabelValues(topic, messageType, targetID).Observe(float64(sizeBytes)) } // NetworkMessageReceived tracks the message size of the last message received on the wire // in bytes for the given topic -func (nc *NetworkCollector) NetworkMessageReceived(sizeBytes int, topic string, messageType string) { - nc.inboundMessageSize.WithLabelValues(topic, messageType).Observe(float64(sizeBytes)) +func (nc *NetworkCollector) NetworkMessageReceived(sizeBytes int, topic string, messageType string, originID string) { + nc.inboundMessageSize.WithLabelValues(topic, messageType, originID).Observe(float64(sizeBytes)) } // NetworkDuplicateMessagesDropped tracks the number of messages dropped by the network layer due to duplication -func (nc *NetworkCollector) NetworkDuplicateMessagesDropped(topic, messageType string) { - nc.duplicateMessagesDropped.WithLabelValues(topic, messageType).Add(1) +func (nc *NetworkCollector) NetworkDuplicateMessagesDropped(topic, messageType string, originID string) { + nc.duplicateMessagesDropped.WithLabelValues(topic, messageType, originID).Add(1) } func (nc *NetworkCollector) MessageAdded(priority int) { @@ -134,3 +150,11 @@ func (nc *NetworkCollector) OutboundConnections(connectionCount uint) { func (nc *NetworkCollector) InboundConnections(connectionCount uint) { nc.inboundConnectionCount.Set(float64(connectionCount)) } + +func (nc *NetworkCollector) UnstakedOutboundConnections(connectionCount uint) { + nc.unstakedOutboundConnectionCount.Set(float64(connectionCount)) +} + +func (nc *NetworkCollector) UnstakedInboundConnections(connectionCount uint) { + nc.unstakedInboundConnectionCount.Set(float64(connectionCount)) +} diff --git a/module/metrics/unstaked/engine.go b/module/metrics/unstaked/engine.go index ed372db0f4a..d75d6058cd9 100644 --- a/module/metrics/unstaked/engine.go +++ b/module/metrics/unstaked/engine.go @@ -1,25 +1,25 @@ package unstaked import ( - "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/module" ) type EngineCollector struct { - collector metrics.EngineCollector + metrics module.EngineMetrics } -func NewUnstakedEngineCollector(collector metrics.EngineCollector) *EngineCollector { - return &EngineCollector{collector} +func NewUnstakedEngineCollector(metrics module.EngineMetrics) *EngineCollector { + return &EngineCollector{metrics} } func (ec *EngineCollector) MessageSent(engine string, message string) { - ec.collector.MessageSent("unstaked_"+engine, message) + ec.metrics.MessageSent("unstaked_"+engine, message) } func (ec *EngineCollector) MessageReceived(engine string, message string) { - ec.collector.MessageReceived("unstaked_"+engine, message) + ec.metrics.MessageReceived("unstaked_"+engine, message) } func (ec *EngineCollector) MessageHandled(engine string, message string) { - ec.collector.MessageHandled("unstaked_"+engine, message) + ec.metrics.MessageHandled("unstaked_"+engine, message) } diff --git a/module/metrics/unstaked/network.go b/module/metrics/unstaked/network.go deleted file mode 100644 index 81f8c6e6c7d..00000000000 --- a/module/metrics/unstaked/network.go +++ /dev/null @@ -1,57 +0,0 @@ -package unstaked - -import ( - "github.com/onflow/flow-go/module/metrics" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" -) - -type NetworkCollector struct { - metrics.NetworkCollector - - outboundConnectionCount prometheus.Gauge - inboundConnectionCount prometheus.Gauge -} - -func NewUnstakedNetworkCollector(collector metrics.NetworkCollector) *NetworkCollector { - return &NetworkCollector{ - collector, - promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: metrics.NamespaceNetwork, - Subsystem: metrics.SubsystemQueue, - Name: "unstaked_outbound_connection_count", - Help: "the number of outbound connections to unstaked nodes", - }), - promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: metrics.NamespaceNetwork, - Subsystem: metrics.SubsystemQueue, - Name: "unstaked_inbound_connection_count", - Help: "the number of inbound connections from unstaked nodes", - }), - } -} - -// NetworkMessageSent tracks the message size of the last message sent out on the wire -// in bytes for the given topic -func (nc *NetworkCollector) NetworkMessageSent(sizeBytes int, topic string, messageType string) { - nc.NetworkCollector.NetworkMessageSent(sizeBytes, "unstaked_"+topic, messageType) -} - -// NetworkMessageReceived tracks the message size of the last message received on the wire -// in bytes for the given topic -func (nc *NetworkCollector) NetworkMessageReceived(sizeBytes int, topic string, messageType string) { - nc.NetworkCollector.NetworkMessageReceived(sizeBytes, "unstaked_"+topic, messageType) -} - -// NetworkDuplicateMessagesDropped tracks the number of messages dropped by the network layer due to duplication -func (nc *NetworkCollector) NetworkDuplicateMessagesDropped(topic, messageType string) { - nc.NetworkCollector.NetworkDuplicateMessagesDropped("unstaked_"+topic, messageType) -} - -func (nc *NetworkCollector) OutboundConnections(connectionCount uint) { - nc.outboundConnectionCount.Set(float64(connectionCount)) -} - -func (nc *NetworkCollector) InboundConnections(connectionCount uint) { - nc.inboundConnectionCount.Set(float64(connectionCount)) -} diff --git a/network/p2p/connManager.go b/network/p2p/connManager.go index 7d5ee0af2dc..ed5b0f8c176 100644 --- a/network/p2p/connManager.go +++ b/network/p2p/connManager.go @@ -10,6 +10,7 @@ import ( "github.com/rs/zerolog" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" ) // TagLessConnManager is a companion interface to libp2p-core.connmgr.ConnManager which implements a (simplified) tagless variant of the Protect / Unprotect logic @@ -37,14 +38,17 @@ type ConnManager struct { streamSetupInProgressCnt map[peer.ID]int // mutex for the stream setup map streamSetupMapLk sync.RWMutex + + idProvider id.IdentityProvider } -func NewConnManager(log zerolog.Logger, metrics module.NetworkMetrics) *ConnManager { +func NewConnManager(log zerolog.Logger, idProvider id.IdentityProvider, metrics module.NetworkMetrics) *ConnManager { cn := &ConnManager{ log: log, NullConnMgr: connmgr.NullConnMgr{}, metrics: metrics, streamSetupInProgressCnt: make(map[peer.ID]int), + idProvider: idProvider, } n := &network.NotifyBundle{ListenCloseF: cn.ListenCloseNotifee, ListenF: cn.ListenNotifee, @@ -83,19 +87,39 @@ func (c *ConnManager) Disconnected(n network.Network, con network.Conn) { } func (c *ConnManager) updateConnectionMetric(n network.Network) { - var inbound uint = 0 - var outbound uint = 0 + var stakedInbound uint = 0 + var stakedOutbound uint = 0 + var totalInbound uint = 0 + var totalOutbound uint = 0 + + stakedPeers := make(map[peer.ID]struct{}) + for _, id := range c.idProvider.Identities(NotEjectedFilter) { + pid, err := ExtractPeerID(id.NetworkPubKey) + if err != nil { + c.log.Fatal().Err(err).Msg("failed to convert network public key of staked node to peer ID") + } + stakedPeers[pid] = struct{}{} + } for _, conn := range n.Conns() { + _, isStaked := stakedPeers[conn.RemotePeer()] switch conn.Stat().Direction { case network.DirInbound: - inbound++ + totalInbound++ + if isStaked { + stakedInbound++ + } case network.DirOutbound: - outbound++ + totalOutbound++ + if isStaked { + stakedOutbound++ + } } } - c.metrics.InboundConnections(inbound) - c.metrics.OutboundConnections(outbound) + c.metrics.InboundConnections(stakedInbound) + c.metrics.OutboundConnections(stakedOutbound) + c.metrics.UnstakedInboundConnections(totalInbound - stakedInbound) + c.metrics.UnstakedOutboundConnections(totalOutbound - stakedOutbound) } func (c *ConnManager) logConnectionUpdate(n network.Network, con network.Conn, logMsg string) { diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 72866bd977d..482e5a1bdea 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -346,7 +346,7 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) } // OneToOne communication metrics are reported with topic OneToOne - m.metrics.NetworkMessageSent(msg.Size(), metrics.ChannelOneToOne, msg.Type) + m.metrics.NetworkMessageSent(msg.Size(), metrics.ChannelOneToOne, msg.Type, targetID.String()) return nil } diff --git a/network/p2p/readConnection.go b/network/p2p/readConnection.go index c71b8bb1e25..bb0a1eacf86 100644 --- a/network/p2p/readConnection.go +++ b/network/p2p/readConnection.go @@ -101,6 +101,12 @@ func (rc *readConnection) receiveLoop(wg *sync.WaitGroup) { return } + // TODO + channel := metrics.ChannelOneToOne + _, found := m.ov.Identities().ByNodeID(targetID) + if !found { + channel = "unstaked_" + channel + } // log metrics with the channel name as OneToOne rc.metrics.NetworkMessageReceived(msg.Size(), metrics.ChannelOneToOne, msg.Type) From 305e4621671ad50bf3c3f4c7aa2b52b198bc9561 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 22:15:19 -0700 Subject: [PATCH 141/291] update metrics --- module/metrics.go | 6 +++--- module/metrics/labels.go | 3 ++- module/metrics/network.go | 18 +++++++++--------- network/p2p/middleware.go | 14 ++++++++++++-- network/p2p/readConnection.go | 14 ++++++++------ 5 files changed, 34 insertions(+), 21 deletions(-) diff --git a/module/metrics.go b/module/metrics.go index c3d87527822..b44f4e4f6a2 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -12,13 +12,13 @@ type EntriesFunc func() uint // Network Metrics type NetworkMetrics interface { // NetworkMessageSent size in bytes and count of the network message sent - NetworkMessageSent(sizeBytes int, topic string, messageType string, targetID string) + NetworkMessageSent(sizeBytes int, topic string, messageType string) // NetworkMessageReceived size in bytes and count of the network message received - NetworkMessageReceived(sizeBytes int, topic string, messageType string, originID string) + NetworkMessageReceived(sizeBytes int, topic string, messageType string) // NetworkDuplicateMessagesDropped counts number of messages dropped due to duplicate detection - NetworkDuplicateMessagesDropped(topic string, messageType string, originID string) + NetworkDuplicateMessagesDropped(topic string, messageType string) // Message receive queue metrics // MessageAdded increments the metric tracking the number of messages in the queue with the given priority diff --git a/module/metrics/labels.go b/module/metrics/labels.go index bfbb6f09a3c..2c5a3e6e897 100644 --- a/module/metrics/labels.go +++ b/module/metrics/labels.go @@ -16,7 +16,8 @@ const ( ) const ( - ChannelOneToOne = "OneToOne" + ChannelOneToOne = "OneToOne" + ChannelOneToOneUnstaked = "OneToOneUnstaked" ) const ( diff --git a/module/metrics/network.go b/module/metrics/network.go index 077612c35b2..089cb4349ff 100644 --- a/module/metrics/network.go +++ b/module/metrics/network.go @@ -38,7 +38,7 @@ func NewNetworkCollector() *NetworkCollector { Name: "outbound_message_size_bytes", Help: "size of the outbound network message", Buckets: []float64{KiB, 100 * KiB, 500 * KiB, 1 * MiB, 2 * MiB, 4 * MiB}, - }, []string{LabelChannel, LabelMessage, LabelNodeID}), + }, []string{LabelChannel, LabelMessage}), inboundMessageSize: promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: NamespaceNetwork, @@ -46,14 +46,14 @@ func NewNetworkCollector() *NetworkCollector { Name: "inbound_message_size_bytes", Help: "size of the inbound network message", Buckets: []float64{KiB, 100 * KiB, 500 * KiB, 1 * MiB, 2 * MiB, 4 * MiB}, - }, []string{LabelChannel, LabelMessage, LabelNodeID}), + }, []string{LabelChannel, LabelMessage}), duplicateMessagesDropped: promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: NamespaceNetwork, Subsystem: subsystemGossip, Name: "duplicate_messages_dropped", Help: "number of duplicate messages dropped", - }, []string{LabelChannel, LabelMessage, LabelNodeID}), + }, []string{LabelChannel, LabelMessage}), queueSize: promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: NamespaceNetwork, @@ -111,19 +111,19 @@ func NewNetworkCollector() *NetworkCollector { // NetworkMessageSent tracks the message size of the last message sent out on the wire // in bytes for the given topic -func (nc *NetworkCollector) NetworkMessageSent(sizeBytes int, topic string, messageType string, targetID string) { - nc.outboundMessageSize.WithLabelValues(topic, messageType, targetID).Observe(float64(sizeBytes)) +func (nc *NetworkCollector) NetworkMessageSent(sizeBytes int, topic string, messageType string) { + nc.outboundMessageSize.WithLabelValues(topic, messageType).Observe(float64(sizeBytes)) } // NetworkMessageReceived tracks the message size of the last message received on the wire // in bytes for the given topic -func (nc *NetworkCollector) NetworkMessageReceived(sizeBytes int, topic string, messageType string, originID string) { - nc.inboundMessageSize.WithLabelValues(topic, messageType, originID).Observe(float64(sizeBytes)) +func (nc *NetworkCollector) NetworkMessageReceived(sizeBytes int, topic string, messageType string) { + nc.inboundMessageSize.WithLabelValues(topic, messageType).Observe(float64(sizeBytes)) } // NetworkDuplicateMessagesDropped tracks the number of messages dropped by the network layer due to duplication -func (nc *NetworkCollector) NetworkDuplicateMessagesDropped(topic, messageType string, originID string) { - nc.duplicateMessagesDropped.WithLabelValues(topic, messageType, originID).Add(1) +func (nc *NetworkCollector) NetworkDuplicateMessagesDropped(topic, messageType string) { + nc.duplicateMessagesDropped.WithLabelValues(topic, messageType).Add(1) } func (nc *NetworkCollector) MessageAdded(priority int) { diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 482e5a1bdea..8236931a863 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -345,8 +345,12 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) return fmt.Errorf("failed to close the stream for %s: %w", targetID, err) } + channel := metrics.ChannelOneToOne + if _, isStaked := m.ov.Identities().ByNodeID(targetID); !isStaked { + channel = metrics.ChannelOneToOneUnstaked + } // OneToOne communication metrics are reported with topic OneToOne - m.metrics.NetworkMessageSent(msg.Size(), metrics.ChannelOneToOne, msg.Type, targetID.String()) + m.metrics.NetworkMessageSent(msg.Size(), channel, msg.Type) return nil } @@ -360,8 +364,14 @@ func (m *Middleware) handleIncomingStream(s libp2pnetwork.Stream) { log.Info().Msg("incoming stream received") + nodeID, err := m.idTranslator.GetFlowID(s.Conn().RemotePeer()) + if err != nil { + log.Err(err).Str("peer_id", s.Conn().RemotePeer().Pretty()).Msg("could not translate peer ID of incoming stream") + } + _, isStaked := m.ov.Identities().ByNodeID(nodeID) + //create a new readConnection with the context of the middleware - conn := newReadConnection(m.ctx, s, m.processAuthenticatedMessage, log, m.metrics, LargeMsgMaxUnicastMsgSize) + conn := newReadConnection(m.ctx, s, m.processAuthenticatedMessage, log, m.metrics, LargeMsgMaxUnicastMsgSize, isStaked) // kick off the receive loop to continuously receive messages m.wg.Add(1) diff --git a/network/p2p/readConnection.go b/network/p2p/readConnection.go index bb0a1eacf86..75bfd8bff14 100644 --- a/network/p2p/readConnection.go +++ b/network/p2p/readConnection.go @@ -26,6 +26,7 @@ type readConnection struct { metrics module.NetworkMetrics maxMsgSize int callback func(msg *message.Message, peerID peer.ID) + isStaked bool } // newReadConnection creates a new readConnection @@ -34,7 +35,9 @@ func newReadConnection(ctx context.Context, callback func(msg *message.Message, peerID peer.ID), log zerolog.Logger, metrics module.NetworkMetrics, - maxMsgSize int) *readConnection { + maxMsgSize int, + isStaked bool, +) *readConnection { if maxMsgSize <= 0 { maxMsgSize = DefaultMaxUnicastMsgSize @@ -48,6 +51,7 @@ func newReadConnection(ctx context.Context, log: log, metrics: metrics, maxMsgSize: maxMsgSize, + isStaked: isStaked, } return &c } @@ -101,14 +105,12 @@ func (rc *readConnection) receiveLoop(wg *sync.WaitGroup) { return } - // TODO channel := metrics.ChannelOneToOne - _, found := m.ov.Identities().ByNodeID(targetID) - if !found { - channel = "unstaked_" + channel + if !rc.isStaked { + channel = metrics.ChannelOneToOneUnstaked } // log metrics with the channel name as OneToOne - rc.metrics.NetworkMessageReceived(msg.Size(), metrics.ChannelOneToOne, msg.Type) + rc.metrics.NetworkMessageReceived(msg.Size(), channel, msg.Type) // call the callback rc.callback(&msg, rc.remoteID) From 8cfccdea6444630fed21d154af535122f961b25f Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 22:20:50 -0700 Subject: [PATCH 142/291] name namespaces and subsystems back to private --- module/metrics/engine.go | 6 +++--- module/metrics/namespaces.go | 4 ++-- module/metrics/network.go | 34 +++++++++++++++++----------------- module/metrics/ping.go | 4 ++-- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/module/metrics/engine.go b/module/metrics/engine.go index b785a57afbb..16e9cf834c9 100644 --- a/module/metrics/engine.go +++ b/module/metrics/engine.go @@ -17,21 +17,21 @@ func NewEngineCollector() *EngineCollector { sent: promauto.NewCounterVec(prometheus.CounterOpts{ Name: "messages_sent_total", - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemEngine, Help: "the number of messages sent by engines", }, []string{EngineLabel, LabelMessage}), received: promauto.NewCounterVec(prometheus.CounterOpts{ Name: "messages_received_total", - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemEngine, Help: "the number of messages received by engines", }, []string{EngineLabel, LabelMessage}), handled: promauto.NewCounterVec(prometheus.CounterOpts{ Name: "messages_handled_total", - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemEngine, Help: "the number of messages handled by engines", }, []string{EngineLabel, LabelMessage}), diff --git a/module/metrics/namespaces.go b/module/metrics/namespaces.go index 3480bc74e93..2187ab87964 100644 --- a/module/metrics/namespaces.go +++ b/module/metrics/namespaces.go @@ -2,7 +2,7 @@ package metrics // Prometheus metric namespaces const ( - NamespaceNetwork = "network" + namespaceNetwork = "network" namespaceStorage = "storage" namespaceAccess = "access" namespaceCollection = "collection" @@ -17,7 +17,7 @@ const ( // subsystemLibp2p = "libp2p" subsystemGossip = "gossip" subsystemEngine = "engine" - SubsystemQueue = "queue" + subsystemQueue = "queue" ) // Storage subsystems represent the various components of the storage layer. diff --git a/module/metrics/network.go b/module/metrics/network.go index 089cb4349ff..e3001e3f0ec 100644 --- a/module/metrics/network.go +++ b/module/metrics/network.go @@ -33,7 +33,7 @@ func NewNetworkCollector() *NetworkCollector { nc := &NetworkCollector{ outboundMessageSize: promauto.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemGossip, Name: "outbound_message_size_bytes", Help: "size of the outbound network message", @@ -41,7 +41,7 @@ func NewNetworkCollector() *NetworkCollector { }, []string{LabelChannel, LabelMessage}), inboundMessageSize: promauto.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemGossip, Name: "inbound_message_size_bytes", Help: "size of the inbound network message", @@ -49,58 +49,58 @@ func NewNetworkCollector() *NetworkCollector { }, []string{LabelChannel, LabelMessage}), duplicateMessagesDropped: promauto.NewCounterVec(prometheus.CounterOpts{ - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemGossip, Name: "duplicate_messages_dropped", Help: "number of duplicate messages dropped", }, []string{LabelChannel, LabelMessage}), queueSize: promauto.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: NamespaceNetwork, - Subsystem: SubsystemQueue, + Namespace: namespaceNetwork, + Subsystem: subsystemQueue, Name: "message_queue_size", Help: "the number of elements in the message receive queue", }, []string{LabelPriority}), queueDuration: promauto.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: NamespaceNetwork, - Subsystem: SubsystemQueue, + Namespace: namespaceNetwork, + Subsystem: subsystemQueue, Name: "message_queue_duration_seconds", Help: "duration [seconds; measured with float64 precision] of how long a message spent in the queue before delivered to an engine.", Buckets: []float64{0.01, 0.1, 0.5, 1, 2, 5}, // 10ms, 100ms, 500ms, 1s, 2s, 5s }, []string{LabelPriority}), inboundProcessTime: promauto.NewCounterVec(prometheus.CounterOpts{ - Namespace: NamespaceNetwork, - Subsystem: SubsystemQueue, + Namespace: namespaceNetwork, + Subsystem: subsystemQueue, Name: "engine_message_processing_time_seconds", Help: "duration [seconds; measured with float64 precision] of how long a queue worker blocked for an engine processing message", }, []string{LabelChannel}), outboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: NamespaceNetwork, - Subsystem: SubsystemQueue, + Namespace: namespaceNetwork, + Subsystem: subsystemQueue, Name: "outbound_connection_count", Help: "the number of outbound connections of this node", }), inboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: NamespaceNetwork, - Subsystem: SubsystemQueue, + Namespace: namespaceNetwork, + Subsystem: subsystemQueue, Name: "inbound_connection_count", Help: "the number of inbound connections of this node", }), unstakedOutboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: NamespaceNetwork, - Subsystem: SubsystemQueue, + Namespace: namespaceNetwork, + Subsystem: subsystemQueue, Name: "unstaked_outbound_connection_count", Help: "the number of outbound connections to unstaked nodes", }), unstakedInboundConnectionCount: promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: NamespaceNetwork, - Subsystem: SubsystemQueue, + Namespace: namespaceNetwork, + Subsystem: subsystemQueue, Name: "unstaked_inbound_connection_count", Help: "the number of inbound connections from unstaked nodes", }), diff --git a/module/metrics/ping.go b/module/metrics/ping.go index 38a7411df31..691d361016e 100644 --- a/module/metrics/ping.go +++ b/module/metrics/ping.go @@ -18,13 +18,13 @@ func NewPingCollector() *PingCollector { pc := &PingCollector{ reachable: promauto.NewGaugeVec(prometheus.GaugeOpts{ Name: "node_reachable", - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemGossip, Help: "report whether a node is reachable", }, []string{LabelNodeID, LabelNodeAddress, LabelNodeRole, LabelNodeInfo}), sealedHeight: promauto.NewGaugeVec(prometheus.GaugeOpts{ Name: "sealed_height", - Namespace: NamespaceNetwork, + Namespace: namespaceNetwork, Subsystem: subsystemGossip, Help: "the last sealed height of a node", }, []string{LabelNodeID, LabelNodeAddress, LabelNodeRole, LabelNodeInfo, LabelNodeVersion}), From 702b5578cb9cd88c04ab0be8a674eaa5e59cef9a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 22:27:28 -0700 Subject: [PATCH 143/291] add idProvider argument to libp2pnodefactory --- cmd/scaffold.go | 4 +++- module/metrics/noop.go | 2 ++ network/p2p/libp2pNode.go | 5 +++-- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a77f215d90d..ce89713f68a 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -163,7 +163,9 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.RootBlock.ID().String(), p2p.DefaultMaxPubSubMsgSize, fnb.Metrics.Network, - pingProvider) + pingProvider, + node.IdentityProvider, + ) if err != nil { return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) } diff --git a/module/metrics/noop.go b/module/metrics/noop.go index ab55f7db371..03ed1c94ab9 100644 --- a/module/metrics/noop.go +++ b/module/metrics/noop.go @@ -27,6 +27,8 @@ func (nc *NoopCollector) MessageReceived(engine string, message string) func (nc *NoopCollector) MessageHandled(engine string, message string) {} func (nc *NoopCollector) OutboundConnections(_ uint) {} func (nc *NoopCollector) InboundConnections(_ uint) {} +func (nc *NoopCollector) UnstakedOutboundConnections(_ uint) {} +func (nc *NoopCollector) UnstakedInboundConnections(_ uint) {} func (nc *NoopCollector) RanGC(duration time.Duration) {} func (nc *NoopCollector) BadgerLSMSize(sizeBytes int64) {} func (nc *NoopCollector) BadgerVLogSize(sizeBytes int64) {} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 89dc3e00428..9b9efa0c626 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -30,6 +30,7 @@ import ( fcrypto "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/logging" @@ -54,9 +55,9 @@ type LibP2PFactoryFunc func() (*Node, error) // DefaultLibP2PNodeFactory returns a LibP2PFactoryFunc which generates the libp2p host initialized with the // default options for the host, the pubsub and the ping service. func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.Identifier, address string, flowKey fcrypto.PrivateKey, rootBlockID string, - maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider) (LibP2PFactoryFunc, error) { + maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider, idProvider id.IdentityProvider) (LibP2PFactoryFunc, error) { - connManager := NewConnManager(log, metrics) + connManager := NewConnManager(log, idProvider, metrics) connGater := NewConnGater(log) From 3416bf786e9660d761c3bcad312316e447fafabe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 25 Aug 2021 09:36:05 -0400 Subject: [PATCH 144/291] [network] Move UpdatableIDProvider to tests --- network/test/middleware_test.go | 3 +-- network/test/testUtil.go | 9 ++++----- {module/id => network/test}/updatable_provider.go | 2 +- 3 files changed, 6 insertions(+), 8 deletions(-) rename {module/id => network/test}/updatable_provider.go (98%) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 29c9d980e4a..c177ade677f 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -19,7 +19,6 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/observable" "github.com/onflow/flow-go/network/codec/cbor" @@ -66,7 +65,7 @@ type MiddlewareTestSuite struct { ids []*flow.Identity metrics *metrics.NoopCollector // no-op performance monitoring simulation logger zerolog.Logger - providers []*id.UpdatableIDProvider + providers []*UpdatableIDProvider } // TestMiddlewareTestSuit runs all the test methods in this test suit diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 9fe6dfe2447..13d5fd80554 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,7 +20,6 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/id" idModule "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" @@ -122,10 +121,10 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*id.UpdatableIDProvider) { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) - idProviders := make([]*id.UpdatableIDProvider, len(identities)) + idProviders := make([]*UpdatableIDProvider, len(identities)) for i, id := range identities { // casts libP2PNode instance to a local variable to avoid closure @@ -136,7 +135,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } - idProviders[i] = idModule.NewUpdatableIDProvider(identities) + idProviders[i] = NewUpdatableIDProvider(identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, @@ -223,7 +222,7 @@ func GenerateNetworks(t *testing.T, func GenerateIDsAndMiddlewares(t *testing.T, n int, dryRunMode bool, - logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*id.UpdatableIDProvider) { + logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*UpdatableIDProvider) { ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) diff --git a/module/id/updatable_provider.go b/network/test/updatable_provider.go similarity index 98% rename from module/id/updatable_provider.go rename to network/test/updatable_provider.go index d35a3da0566..7c4377a1ae5 100644 --- a/module/id/updatable_provider.go +++ b/network/test/updatable_provider.go @@ -1,4 +1,4 @@ -package id +package test import ( "sync" From 9208f005ead09b5ce8f79e3856f9bda89f4918b9 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 10:06:17 -0700 Subject: [PATCH 145/291] add NotEjectedFilter to sync engine id provider --- cmd/scaffold.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 123a7694c55..a4cc5b9c1a1 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -438,6 +438,7 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NotEjectedFilter, ), idCache, ) From a5b836e1e6f3c4498a180ab553ba31ed6ee4be2e Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 10:29:31 -0700 Subject: [PATCH 146/291] update assert.Subset to assert.ElementsMatch --- network/test/peerstore_provider_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index 26fd1dfcf9b..959b0e34c36 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -86,7 +86,7 @@ func (suite *PeerStoreProviderTestSuite) TestTranslationPeers() { peerIDs[i] = pID } // check we can find the libp2p peers - assert.Subset(suite.T(), peerIDs, suite.libp2pPeersIDs, "peer IDs should include those in the peer Store") + assert.ElementsMatch(suite.T(), peerIDs, append(suite.libp2pPeersIDs, suite.nodes[testNodeIndex].Host().ID()), "peer IDs should include those in the peer Store") } From ba13bc566d36f8e007dc0bb06a6b32f79ea3ce1a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 10:29:49 -0700 Subject: [PATCH 147/291] remove time.Sleep --- network/test/peerstore_provider_test.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index 959b0e34c36..093c55072a0 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -6,7 +6,6 @@ import ( "os" "strconv" "testing" - "time" "github.com/ipfs/go-log" "github.com/libp2p/go-libp2p-core/peer" @@ -70,8 +69,6 @@ func (suite *PeerStoreProviderTestSuite) SetupTest() { assert.Len(suite.T(), suite.libp2pPeersIDs, peerCount) assert.Len(suite.T(), suite.ids, nodeCount) - // give the Node the time to establish those connections - time.Sleep(2 * time.Second) } func (suite *PeerStoreProviderTestSuite) TestTranslationPeers() { From 2f0bd716959e5d7aaf1ad7c35537e80d4939dbe4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 6 Aug 2021 22:18:19 -0700 Subject: [PATCH 148/291] init --- .../node_builder/access_node_builder.go | 19 +- .../staked_access_node_builder.go | 23 ++ cmd/node_builder.go | 33 +-- cmd/scaffold.go | 3 + engine/access/ping/engine.go | 6 +- engine/access/relay/engine.go | 3 +- integration/Makefile | 6 +- integration/testnet/container.go | 13 +- integration/testnet/network.go | 64 +++++- integration/testnet/network_test.go | 2 + .../tests/access/unstaked_node_test.go | 204 ++++++++++++++++++ 11 files changed, 327 insertions(+), 49 deletions(-) create mode 100644 integration/tests/access/unstaked_node_test.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index d23024ed603..75e87a708dd 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -73,8 +73,8 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // ParticipatesInUnstakedNetwork returns True if this is a staked Access node which also participates - // in the unstaked network acting as an upstream for other unstaked access nodes, False otherwise. + // ParticipatesInUnstakedNetwork returns True if this an Access Node which participates in the unstaked network, + // False otherwise ParticipatesInUnstakedNetwork() bool // Build defines all of the Access node's components and modules. @@ -150,8 +150,8 @@ type FlowAccessNodeBuilder struct { // components UnstakedLibP2PNode *p2p.Node - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware + UnstakedNetwork p2p.ReadyDoneAwareNetwork + unstakedMiddleware network.Middleware FollowerState protocol.MutableState SyncCore *synchronization.Core RpcEng *rpc.Engine @@ -525,12 +525,6 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { } func (builder *FlowAccessNodeBuilder) ParticipatesInUnstakedNetwork() bool { - // unstaked access nodes can't be upstream of other unstaked access nodes for now - if !builder.IsStaked() { - return false - } - // if an unstaked network bind address is provided, then this staked access node will act as the upstream for - // unstaked access nodes return builder.unstakedNetworkBindAddr != cmd.NotSet } @@ -613,7 +607,8 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - validators ...network.MessageValidator) *p2p.Middleware { + + validators ...network.MessageValidator) *network.Middleware { builder.unstakedMiddleware = p2p.NewMiddleware( builder.Logger, factoryFunc, @@ -636,7 +631,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, // updated by calling network.SetIDs. func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, - middleware *p2p.Middleware, + middleware network.Middleware, topology network.Topology) (*p2p.Network, error) { codec := jsoncodec.NewCodec() diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index e17ce0e9d73..37428de013d 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -6,6 +6,8 @@ import ( "github.com/onflow/flow-go/cmd" pingeng "github.com/onflow/flow-go/engine/access/ping" + "github.com/onflow/flow-go/engine/access/relay" + splitternetwork "github.com/onflow/flow-go/engine/common/splitter/network" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -79,6 +81,10 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { + anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) + return node.Network, nil + }) anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -96,6 +102,23 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { } return ping, nil }) + if anb.ParticipatesInUnstakedNetwork() { + // create relay engine + anb.Component("relay engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + channels := node.SubscriptionManager.Channels() + if len(channels) == 0 { + return nil, fmt.Errorf("no subscribed channels to relay") + } + + relayEngine, err := relay.New(node.Logger, channels, node.Network, anb.UnstakedNetwork) + + if err != nil { + return nil, fmt.Errorf("could not create relay engine: %w", err) + } + + return relayEngine, nil + }) + } return anb } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 6f1300df68a..b62b9752ee5 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -120,22 +120,23 @@ type BaseConfig struct { type NodeConfig struct { Cancel context.CancelFunc // cancel function for the context that is passed to the networking layer BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware *p2p.Middleware - Network *p2p.Network - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware network.Middleware + Network p2p.ReadyDoneAwareNetwork + SubscriptionManager network.SubscriptionManager + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // ID providers IdentityProvider id.IdentityProvider diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a4cc5b9c1a1..a7e9a52b20b 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -185,11 +185,14 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { ) subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) + top, err := topology.NewTopicBasedTopology( fnb.NodeID, fnb.Logger, fnb.State, ) + node.SubscriptionManager = subscriptionManager + if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) } diff --git a/engine/access/ping/engine.go b/engine/access/ping/engine.go index dfcb3c70f5a..291d54ce232 100644 --- a/engine/access/ping/engine.go +++ b/engine/access/ping/engine.go @@ -9,7 +9,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -22,7 +22,7 @@ type Engine struct { pingEnabled bool pingInterval time.Duration - middleware *p2p.Middleware + middleware network.Middleware nodeInfo map[flow.Identifier]string // additional details about a node such as operator name } @@ -32,7 +32,7 @@ func New( me module.Local, metrics module.PingMetrics, pingEnabled bool, - mw *p2p.Middleware, + mw network.Middleware, nodeInfoFile string, ) (*Engine, error) { diff --git a/engine/access/relay/engine.go b/engine/access/relay/engine.go index b850f3d48f0..6c20ab72544 100644 --- a/engine/access/relay/engine.go +++ b/engine/access/relay/engine.go @@ -99,10 +99,11 @@ func (e *Engine) process(channel network.Channel, originID flow.Identifier, even conduit, ok := e.conduits[channel] if !ok { - e.log.Trace().Interface("event", event).Str("channel", channel.String()).Str("originID", originID.String()).Msg("unknown channel") return fmt.Errorf("received message on unknown channel %s", channel) } + e.log.Trace().Interface("event", event).Str("channel", channel.String()).Str("originID", originID.String()).Msg("relaying message") + // We use a dummy target ID here so that events are broadcast to the entire network if err := conduit.Publish(event, flow.ZeroID); err != nil { return fmt.Errorf("could not relay message: %w", err) diff --git a/integration/Makefile b/integration/Makefile index 231543fd848..81276cd00c6 100644 --- a/integration/Makefile +++ b/integration/Makefile @@ -12,7 +12,11 @@ integration-test: common-tests execution-tests verification-tests collection-tes # NOTE: Currently skipping collection and consensus tests due to them not passing properly on CI .PHONY: ci-integration-test -ci-integration-test: common-tests execution-tests epoch-tests verification-tests # collection-tests # consensus-tests +ci-integration-test: common-tests execution-tests epoch-tests verification-tests access-tests # collection-tests # consensus-tests + +.PHONY: access-tests +access-tests: + GO111MODULE=on go test -tags relic -count=1 ./tests/access .PHONY: collection-tests collection-tests: diff --git a/integration/testnet/container.go b/integration/testnet/container.go index da269214da0..666c54d349a 100644 --- a/integration/testnet/container.go +++ b/integration/testnet/container.go @@ -38,12 +38,13 @@ func init() { // ContainerConfig represents configuration for a node container in the network. type ContainerConfig struct { bootstrap.NodeInfo - ContainerName string - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - Unstaked bool + ContainerName string + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + Unstaked bool + ParticipatesInUnstakedNetwork bool } // ImageName returns the Docker image name for the given config. diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 04d5c4e88a4..def109f29ff 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -4,7 +4,6 @@ import ( "context" "encoding/hex" "fmt" - "github.com/onflow/flow-go/cmd/bootstrap/utils" "io/ioutil" "math/rand" "os" @@ -15,6 +14,8 @@ import ( "testing" "time" + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/docker/docker/api/types/container" dockerclient "github.com/docker/docker/client" "github.com/rs/zerolog" @@ -65,12 +66,20 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" + // UnstakedNetworkPort is the name used for the access node unstaked libp2p network port. + UnstakedNetworkPort = "access-unstaked-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" // ExeNodeMetricsPort ExeNodeMetricsPort = "exe-metrics-port" + // default staked network port + DefaultStakedFlowPort = 2137 + + // default unstaked network port + DefaultUnstakedFlowPort = 7312 + DefaultViewsInStakingAuction uint64 = 5 DefaultViewsInDKGPhase uint64 = 50 DefaultViewsInEpoch uint64 = 180 @@ -288,7 +297,8 @@ type NodeConfig struct { Debug bool // Unstaked - only applicable to Access Node. Access nodes can be staked or unstaked. // Unstaked nodes are not part of the identity table - Unstaked bool // only applicable to Access node + Unstaked bool // only applicable to Access node + ParticipatesInUnstakedNetwork bool } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -364,6 +374,12 @@ func AsGhost() func(config *NodeConfig) { } } +func AsUnstakedNetworkParticipant() func(config *NodeConfig) { + return func(config *NodeConfig) { + config.ParticipatesInUnstakedNetwork = true + } +} + // WithAdditionalFlag adds additional flags to the command func WithAdditionalFlag(flag string) func(config *NodeConfig) { return func(config *NodeConfig) { @@ -562,6 +578,15 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort + if nodeConf.ParticipatesInUnstakedNetwork { + hostUnstakedPort := testingdock.RandomPort(t) + containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) + nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) + nodeContainer.addFlag("unstaked-bind-addr", fmt.Sprintf("%s:%d", nodeContainer.Name(), DefaultUnstakedFlowPort)) + nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + net.AccessPorts[UnstakedNetworkPort] = hostUnstakedPort + } + case flow.RoleConsensus: // use 1 here instead of the default 5, because the integration // tests only start 1 verification node @@ -581,6 +606,20 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.addFlag("rpc-addr", fmt.Sprintf("%s:9000", nodeContainer.Name())) nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort + + if nodeConf.ParticipatesInUnstakedNetwork { + hostUnstakedPort := testingdock.RandomPort(t) + containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) + + // TODO: Currently, it is not possible to create a staked ghost AN which + // participates on the unstaked network, because the ghost node only joins + // a single network during startup. The ghost node needs to support the + // "unstaked-bind-addr" flag which can be used to specify a bind address + // for the unstaked network. + + nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) + nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + } } if nodeConf.Debug { @@ -817,7 +856,11 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { // define the node's name _ and address : name := fmt.Sprintf("%s_%d", conf.Role.String(), roleCounter[conf.Role]+1) - addr := fmt.Sprintf("%s:%d", name, 2137) + flowPort := DefaultStakedFlowPort + if conf.Unstaked { + flowPort = DefaultUnstakedFlowPort + } + addr := fmt.Sprintf("%s:%d", name, flowPort) roleCounter[conf.Role]++ info := bootstrap.NewPrivateNodeInfo( @@ -830,13 +873,14 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - Unstaked: conf.Unstaked, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + Unstaked: conf.Unstaked, + ParticipatesInUnstakedNetwork: conf.ParticipatesInUnstakedNetwork, } confs = append(confs, containerConf) diff --git a/integration/testnet/network_test.go b/integration/testnet/network_test.go index 1a3be04cbdb..fa979f50c00 100644 --- a/integration/testnet/network_test.go +++ b/integration/testnet/network_test.go @@ -33,6 +33,7 @@ func TestNetworkSetupBasic(t *testing.T) { testnet.NewNodeConfig(flow.RoleConsensus), testnet.NewNodeConfig(flow.RoleExecution), testnet.NewNodeConfig(flow.RoleVerification), + testnet.NewNodeConfig(flow.RoleAccess), } conf := testnet.NewNetworkConfig("meta_test_basic", nodes) @@ -50,6 +51,7 @@ func TestNetworkSetupBasic(t *testing.T) { {image: defaultRegistry + "/consensus:latest", name: "consensus_3", address: "consensus_3:2137"}, {image: defaultRegistry + "/execution:latest", name: "execution_1", address: "execution_1:2137"}, {image: defaultRegistry + "/verification:latest", name: "verification_1", address: "verification_1:2137"}, + {image: defaultRegistry + "/access:latest", name: "access_1", address: "access_1:2137"}, } assert.Subset(t, realData, expectedData) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go new file mode 100644 index 00000000000..194d151b1a8 --- /dev/null +++ b/integration/tests/access/unstaked_node_test.go @@ -0,0 +1,204 @@ +package access + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/engine" + ghostclient "github.com/onflow/flow-go/engine/ghost/client" + "github.com/onflow/flow-go/integration/testnet" + "github.com/onflow/flow-go/integration/tests/common" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/messages" + "github.com/onflow/flow-go/utils/unittest" +) + +type UnstakedAccessSuite struct { + suite.Suite + + // root context for the current test + ctx context.Context + cancel context.CancelFunc + + net *testnet.FlowNetwork + unstakedGhost *ghostclient.GhostClient + conGhost *ghostclient.GhostClient + stakedID flow.Identifier + unstakedID flow.Identifier + conID flow.Identifier +} + +func TestUnstakedAccessSuite(t *testing.T) { + suite.Run(t, new(UnstakedAccessSuite)) +} + +func (suite *UnstakedAccessSuite) TearDownTest() { + // avoid nil pointer errors for skipped tests + if suite.cancel != nil { + defer suite.cancel() + } + if suite.net != nil { + suite.net.Remove() + } +} + +func (suite *UnstakedAccessSuite) SetupTest() { + nodeConfigs := []testnet.NodeConfig{} + + // staked access node + suite.stakedID = unittest.IdentifierFixture() + stakedConfig := testnet.NewNodeConfig( + flow.RoleAccess, + testnet.WithID(suite.stakedID), + testnet.AsUnstakedNetworkParticipant(), + testnet.WithLogLevel(zerolog.InfoLevel) + ) + nodeConfigs = append(nodeConfigs, stakedConfig) + + // consensus node (ghost) + suite.conID = unittest.IdentifierFixture() + conConfig := testnet.NewNodeConfig( + flow.RoleConsensus, + testnet.WithID(suite.conID), + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, conConfig) + + // execution node (unused) + exeConfig := testnet.NewNodeConfig( + flow.RoleExecution, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, exeConfig) + + // verification node (unused) + verConfig := testnet.NewNodeConfig( + flow.RoleVerification, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, verConfig) + + // collection node (unused) + collConfig := testnet.NewNodeConfig( + flow.RoleCollection, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel) + ) + nodeConfigs = append(nodeConfigs, collConfig) + + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs) + suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + + suite.setupConsensusFollower() + + // start the network + suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.net.Start(suite.ctx) + + conGhost := suite.net.ContainerByID(suite.conID) + client, err = common.GetGhostClient(conGhost) + require.NoError(suite.T(), err, "could not get ghost client") + suite.conGhost = client +} + +func (suite *UnstakedAccessSuite) setupConsensusFollower() { + // create a temporary directory to store all bootstrapping files, these + // will be shared between all nodes + bootstrapDir, err := ioutil.TempDir(TmpRoot, "flow-integration-bootstrap") + require.Nil(t, err) + + // get a temporary directory in the host. On macOS the default tmp + // directory is NOT accessible to Docker by default, so we use /tmp + // instead. + tmpdir, err := ioutil.TempDir(TmpRoot, "flow-integration-node") + if err != nil { + return fmt.Errorf("could not get tmp dir: %w", err) + } + + // create a directory for the node database + dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) + err = os.Mkdir(flowDBDir, 0700) + require.NoError(t, err) + + // create a directory for the bootstrap files + // we create a node-specific bootstrap directory to enable testing nodes + // bootstrapping from different root state snapshots and epochs + followerBootstrapDir := filepath.Join(tmpdir, DefaultBootstrapDir) + err = os.Mkdir(nodeBootstrapDir, 0700) + require.NoError(t, err) + + // copy bootstrap files to node-specific bootstrap directory + err = io.CopyDirectory(bootstrapDir, followerBootstrapDir) + require.NoError(t, err) + + // consensus follower + suite.unstakedID = unittest.IdentifierFixture() + bindPort := testingdock.RandomPort(suite.T()) + bindAddr := fmt.Sprintf(":%v", bindPort) // TODO: verify this + opts := []consensus_follower.Option{ + consensus_follower.WithDataDir(dataDir), + consensus_follower.WithBootstrapDir(bootstrapDir), + } // TODO + upstreamANPort := suite.net.ContainerByID(suite.stakedID).Ports[testnet.UnstakedNetworkPort] // TODO + consensus_follower.NewConsensusFollower( + suite.unstakedID, + suite.stakedID, + bindAddr, + opts..., + ) +} + +func (suite *UnstakedAccessSuite) TestReceiveBlocks() { + // 1. Send new block from consensus node to staked AN + // 2. Check that unstaked AN (ghost) receives it + // 3. Check that staked AN also processed the block. This can be done by calling the + // Access API on the staked AN. + + block := unittest.BlockFixture() + + proposal := &messages.BlockProposal{ + Header: block.Header, + Payload: block.Payload, + } + + // Send block proposal fron consensus node to staked AN + suite.conGhost.Send(suite.ctx, engine.PushBlocks, proposal, suite.stakedID) + + m := make(chan interface{}) + go func() { + _, msg, err := suite.unstakedReader.Next() + suite.Require().Nil(err, "could not read next message") + suite.T().Logf("unstaked ghost recv: %T", msg) + + m <- msg + }() + + // Check that the unstaked AN receives the message + select { + case msg := <-m: + suite.Assert().Equal(msg, proposal) + case <-time.After(5 * time.Second): + suite.T().Fatal("timed out waiting for next message") + } + + // TODO: Since the staked AN follower engine will perform validation on received blocks, + // the following check may not work unless we send a "valid" block. In particular we will + // probably at least need to generate a block with ParentID equal to the root block ID + // (suite.net.Root().ID()) + + // chain := suite.net.Root().Header.ChainID.Chain() + + // stakedContainer := suite.net.ContainerByID(suite.stakedID) + // stakedClient, err := testnet.NewClient(stakedContainer.Addr(testnet.AccessNodeAPIPort), chain) + // require.NoError(suite.T(), err) + + // suite.Assert().Equal(stakedClient.GetLatestBlockID(suite.ctx), block.ID()) + +} From 29cd93e44ae81570d136d2b20b9b90bbcc5e36f3 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 15:14:03 -0700 Subject: [PATCH 149/291] s --- integration/testnet/network.go | 100 +++++++++-- .../tests/access/unstaked_node_test.go | 158 ++++-------------- 2 files changed, 120 insertions(+), 138 deletions(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index def109f29ff..cfbfeed0a52 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -32,6 +32,7 @@ import ( "github.com/onflow/cadence" "github.com/onflow/flow-go/cmd/bootstrap/run" + consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" dkgmod "github.com/onflow/flow-go/model/dkg" @@ -91,17 +92,18 @@ func init() { // FlowNetwork represents a test network of Flow nodes running in Docker containers. type FlowNetwork struct { - t *testing.T - suite *testingdock.Suite - config NetworkConfig - cli *dockerclient.Client - network *testingdock.Network - Containers map[string]*Container - AccessPorts map[string]string - root *flow.Block - result *flow.ExecutionResult - seal *flow.Seal - bootstrapDir string + t *testing.T + suite *testingdock.Suite + config NetworkConfig + cli *dockerclient.Client + network *testingdock.Network + Containers map[string]*Container + ConsensusFollowers map[flow.Identifier]consensus_follower.ConsensusFollower + AccessPorts map[string]string + root *flow.Block + result *flow.ExecutionResult + seal *flow.Seal + bootstrapDir string } // Identities returns a list of identities, one for each node in the network. @@ -133,6 +135,9 @@ func (net *FlowNetwork) Start(ctx context.Context) { // makes it easier to see logs for a specific test case fmt.Println(">>>> starting network: ", net.config.Name) net.suite.Start(ctx) + for _, cf := range net.ConsensusFollowers { + go cf.Run(ctx) + } } // Remove stops the network, removes all the containers and cleans up all resources. @@ -204,6 +209,14 @@ func (net *FlowNetwork) ContainerByID(id flow.Identifier) *Container { return nil } +// ConsensusFollowerByID returns the ConsensusFollower with the given node ID, if it exists. +// Otherwise fails the test. +func (net *FlowNetwork) ConsensusFollowerByID(id flow.Identifier) consensus_follower.ConsensusFollower { + follower, ok := net.ConsensusFollowers[id] + require.True(net.t, ok) + return follower +} + // ContainerByName returns the container with the given name, if it exists. // Otherwise fails the test. func (net *FlowNetwork) ContainerByName(name string) *Container { @@ -212,9 +225,22 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { return container } +type ConsensusFollowerConfig struct { + nodeID flow.Identifier + upstreamNodeID flow.Identifier +} + +func NewConsensusFollowerConfig(nodeID flow.Identifier, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { + return ConsensusFollowerConfig{ + nodeID: nodeID, + upstreamNodeID: upstreamNodeID, + } +} + // NetworkConfig is the config for the network. type NetworkConfig struct { Nodes []NodeConfig + ConsensusFollowers []ConsensusFollowerConfig Name string NClusters uint ViewsInDKGPhase uint64 @@ -224,9 +250,10 @@ type NetworkConfig struct { type NetworkConfigOpt func(*NetworkConfig) -func NewNetworkConfig(name string, nodes []NodeConfig, opts ...NetworkConfigOpt) NetworkConfig { +func NewNetworkConfig(name string, nodes []NodeConfig, followers []ConsensusFollowerConfig, opts ...NetworkConfigOpt) NetworkConfig { c := NetworkConfig{ Nodes: nodes, + ConsensusFollowers: followers, Name: name, NClusters: 1, // default to 1 cluster ViewsInStakingAuction: DefaultViewsInStakingAuction, @@ -440,9 +467,58 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { require.NoError(t, err) } + // add each follower to the network + for _, followerConf := range networkConf.ConsensusFollowers { + err = flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) + require.NoError(t, err) + } + return flowNetwork } +func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) error { + tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") + if err != nil { + return fmt.Errorf("could not get tmp dir: %w", err) + } + + // create a directory for the follower database + dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) + err = os.Mkdir(dataDir, 0700) + require.NoError(t, err) + + // create a follower-specific directory for the bootstrap files + followerBootstrapDir := filepath.Join(tmpdir, DefaultBootstrapDir) + err = os.Mkdir(followerBootstrapDir, 0700) + require.NoError(t, err) + + // copy bootstrap files to follower-specific bootstrap directory + err = io.CopyDirectory(bootstrapDir, followerBootstrapDir) + require.NoError(t, err) + + // consensus follower + bindPort := testingdock.RandomPort(t) + bindAddr := fmt.Sprintf("0.0.0.0:%d", bindPort) + opts := []consensus_follower.Option{ + consensus_follower.WithDataDir(dataDir), + consensus_follower.WithBootstrapDir(followerBootstrapDir), + } + + // TODO: eventually we will need upstream node's address + // + // upstreamANPort := net.ContainerByID(followerConf.upstreamNodeID).Ports[testnet.UnstakedNetworkPort] + // upstreamANAddress := fmt.Sprintf("127.0.0.1:%d", upstreamANPort) + + follower := consensus_follower.NewConsensusFollower( + followerConf.nodeID, + followerConf.upstreamNodeID, + bindAddr, + opts..., + ) + + net.ConsensusFollowers[followerConf.nodeID] = follower +} + // AddNode creates a node container with the given config and adds it to the // network. func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf ContainerConfig) error { diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 194d151b1a8..bb87daae6de 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -3,17 +3,12 @@ package access import ( "context" "testing" - "time" - "github.com/stretchr/testify/require" + "github.com/rs/zerolog" "github.com/stretchr/testify/suite" - "github.com/onflow/flow-go/engine" - ghostclient "github.com/onflow/flow-go/engine/ghost/client" "github.com/onflow/flow-go/integration/testnet" - "github.com/onflow/flow-go/integration/tests/common" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/utils/unittest" ) @@ -24,12 +19,11 @@ type UnstakedAccessSuite struct { ctx context.Context cancel context.CancelFunc - net *testnet.FlowNetwork - unstakedGhost *ghostclient.GhostClient - conGhost *ghostclient.GhostClient - stakedID flow.Identifier - unstakedID flow.Identifier - conID flow.Identifier + net *testnet.FlowNetwork + stakedID flow.Identifier + unstakedID flow.Identifier + conID flow.Identifier + follower consensus_follower.ConsensusFollower } func TestUnstakedAccessSuite(t *testing.T) { @@ -55,150 +49,62 @@ func (suite *UnstakedAccessSuite) SetupTest() { flow.RoleAccess, testnet.WithID(suite.stakedID), testnet.AsUnstakedNetworkParticipant(), - testnet.WithLogLevel(zerolog.InfoLevel) + testnet.WithLogLevel(zerolog.InfoLevel), ) nodeConfigs = append(nodeConfigs, stakedConfig) // consensus node (ghost) suite.conID = unittest.IdentifierFixture() conConfig := testnet.NewNodeConfig( - flow.RoleConsensus, - testnet.WithID(suite.conID), - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleConsensus, + testnet.WithID(suite.conID), + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, conConfig) // execution node (unused) exeConfig := testnet.NewNodeConfig( - flow.RoleExecution, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleExecution, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, exeConfig) // verification node (unused) verConfig := testnet.NewNodeConfig( - flow.RoleVerification, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleVerification, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, verConfig) // collection node (unused) collConfig := testnet.NewNodeConfig( - flow.RoleCollection, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel) + flow.RoleCollection, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), ) nodeConfigs = append(nodeConfigs, collConfig) - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs) - suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + // consensus follower + suite.unstakedID = unittest.IdentifierFixture() + followerConfigs := []testnet.ConsensusFollowerConfig{ + testnet.ConsensusFollowerConfig{ + nodeID: suite.unstakedID, + upstreamNodeID: suite.stakedID, + }, + } - suite.setupConsensusFollower() + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) + suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) // start the network suite.ctx, suite.cancel = context.WithCancel(context.Background()) suite.net.Start(suite.ctx) - - conGhost := suite.net.ContainerByID(suite.conID) - client, err = common.GetGhostClient(conGhost) - require.NoError(suite.T(), err, "could not get ghost client") - suite.conGhost = client -} - -func (suite *UnstakedAccessSuite) setupConsensusFollower() { - // create a temporary directory to store all bootstrapping files, these - // will be shared between all nodes - bootstrapDir, err := ioutil.TempDir(TmpRoot, "flow-integration-bootstrap") - require.Nil(t, err) - - // get a temporary directory in the host. On macOS the default tmp - // directory is NOT accessible to Docker by default, so we use /tmp - // instead. - tmpdir, err := ioutil.TempDir(TmpRoot, "flow-integration-node") - if err != nil { - return fmt.Errorf("could not get tmp dir: %w", err) - } - - // create a directory for the node database - dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) - err = os.Mkdir(flowDBDir, 0700) - require.NoError(t, err) - - // create a directory for the bootstrap files - // we create a node-specific bootstrap directory to enable testing nodes - // bootstrapping from different root state snapshots and epochs - followerBootstrapDir := filepath.Join(tmpdir, DefaultBootstrapDir) - err = os.Mkdir(nodeBootstrapDir, 0700) - require.NoError(t, err) - - // copy bootstrap files to node-specific bootstrap directory - err = io.CopyDirectory(bootstrapDir, followerBootstrapDir) - require.NoError(t, err) - - // consensus follower - suite.unstakedID = unittest.IdentifierFixture() - bindPort := testingdock.RandomPort(suite.T()) - bindAddr := fmt.Sprintf(":%v", bindPort) // TODO: verify this - opts := []consensus_follower.Option{ - consensus_follower.WithDataDir(dataDir), - consensus_follower.WithBootstrapDir(bootstrapDir), - } // TODO - upstreamANPort := suite.net.ContainerByID(suite.stakedID).Ports[testnet.UnstakedNetworkPort] // TODO - consensus_follower.NewConsensusFollower( - suite.unstakedID, - suite.stakedID, - bindAddr, - opts..., - ) } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - // 1. Send new block from consensus node to staked AN - // 2. Check that unstaked AN (ghost) receives it - // 3. Check that staked AN also processed the block. This can be done by calling the - // Access API on the staked AN. - - block := unittest.BlockFixture() - - proposal := &messages.BlockProposal{ - Header: block.Header, - Payload: block.Payload, - } - - // Send block proposal fron consensus node to staked AN - suite.conGhost.Send(suite.ctx, engine.PushBlocks, proposal, suite.stakedID) - - m := make(chan interface{}) - go func() { - _, msg, err := suite.unstakedReader.Next() - suite.Require().Nil(err, "could not read next message") - suite.T().Logf("unstaked ghost recv: %T", msg) - - m <- msg - }() - - // Check that the unstaked AN receives the message - select { - case msg := <-m: - suite.Assert().Equal(msg, proposal) - case <-time.After(5 * time.Second): - suite.T().Fatal("timed out waiting for next message") - } - - // TODO: Since the staked AN follower engine will perform validation on received blocks, - // the following check may not work unless we send a "valid" block. In particular we will - // probably at least need to generate a block with ParentID equal to the root block ID - // (suite.net.Root().ID()) - - // chain := suite.net.Root().Header.ChainID.Chain() - - // stakedContainer := suite.net.ContainerByID(suite.stakedID) - // stakedClient, err := testnet.NewClient(stakedContainer.Addr(testnet.AccessNodeAPIPort), chain) - // require.NoError(suite.T(), err) - - // suite.Assert().Equal(stakedClient.GetLatestBlockID(suite.ctx), block.ID()) - + suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) + // TODO } From 1306c29f20aced9f09282ef38bf11d73e2fa185c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 15:15:26 -0700 Subject: [PATCH 150/291] Update unstaked_node_test.go --- integration/tests/access/unstaked_node_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index bb87daae6de..1ee76939237 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -7,6 +7,7 @@ import ( "github.com/rs/zerolog" "github.com/stretchr/testify/suite" + consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/utils/unittest" From d5985ecd4a27d0b8a80d85e0db5ba7fe27b99899 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 16:51:26 -0700 Subject: [PATCH 151/291] Add consensus follower to integration tests --- integration/go.mod | 1 + integration/go.sum | 8 ++ integration/testnet/network.go | 77 +++++++++++++------ integration/tests/access/access_api_test.go | 2 +- .../tests/access/unstaked_node_test.go | 11 ++- 5 files changed, 69 insertions(+), 30 deletions(-) diff --git a/integration/go.mod b/integration/go.mod index 87d31b4f256..31e00992a54 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -6,6 +6,7 @@ require ( github.com/DataDog/zstd v1.4.8 // indirect github.com/HdrHistogram/hdrhistogram-go v1.0.1 // indirect github.com/dapperlabs/testingdock v0.4.3-0.20200626075145-ea23fc16bb90 + github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f // indirect github.com/dgraph-io/badger/v2 v2.2007.2 github.com/dgraph-io/ristretto v0.0.3 // indirect github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect diff --git a/integration/go.sum b/integration/go.sum index 51afd2feff8..9e9720679e5 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -215,6 +215,8 @@ github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c h1:pFUpOrbxDR github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c/go.mod h1:6UhI8N9EjYm1c2odKpFpAYeR8dsBeM7PtzQhRgxRr9U= github.com/deckarep/golang-set v0.0.0-20180603214616-504e848d77ea/go.mod h1:93vsz/8Wt4joVM7c2AVqh+YRMiUSc14yDtF28KmMOgQ= github.com/decred/dcrd/lru v1.0.0/go.mod h1:mxKOwFd7lFjN2GZYsiz/ecgqR6kkYAl+0pz0tEMk218= +github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f h1:U5y3Y5UE0w7amNe7Z5G/twsBW0KEalRQXZzf8ufSh9I= +github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f/go.mod h1:xH/i4TFMt8koVQZ6WFms69WAsDWr2XsYL3Hkl7jkoLE= github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= github.com/dgraph-io/badger v1.6.0-rc1/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= @@ -480,9 +482,13 @@ github.com/graph-gophers/graphql-go v0.0.0-20191115155744-f33e81362277/go.mod h1 github.com/graph-gophers/graphql-go v0.0.0-20201113091052-beb923fada29/go.mod h1:9CQHMSxwO4MprSdzoIEobiHpoLtHm77vfxsvsIN5Vuc= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2 h1:uxUHSMwWDJ/9jVPHNumRC8WZOi3hrBL22ObVOoLg4ww= github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2/go.mod h1:BL7w7qd2l/j9jgY6WMhYutfOFQc0I8RTVwtjpnAMoTM= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea h1:1Tk1IbruXbunEnaIZEFb+Hpv9BIZti3OxKwKn5wWyKk= github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea/go.mod h1:GugMBs30ZSAkckqXEAIEGyYdDH6EgqowG8ppA3Zt+AY= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= @@ -533,6 +539,7 @@ github.com/huin/goupnp v1.0.1-0.20200620063722-49508fba0031 h1:HarGZ5h9HD9LgEg1y github.com/huin/goupnp v1.0.1-0.20200620063722-49508fba0031/go.mod h1:nNs7wvRfN1eKaMknBydLNQU6146XQim8t4h+q90biWo= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/improbable-eng/grpc-web v0.12.0 h1:GlCS+lMZzIkfouf7CNqY+qqpowdKuJLSLLcKVfM1oLc= github.com/improbable-eng/grpc-web v0.12.0/go.mod h1:6hRR09jOEG81ADP5wCQju1z71g6OL4eEvELdran/3cs= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= @@ -1249,6 +1256,7 @@ github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.2.2/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/cors v0.0.0-20160617231935-a62a804a8a00/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/rs/xhandler v0.0.0-20160618193221-ed27b6fd6521/go.mod h1:RvLn4FgxWubrpZHtQLnOf6EwhN2hEMusxZOhcW9H3UQ= github.com/rs/xid v1.2.1/go.mod h1:+uKXf+4Djp6Md1KODXJxgGQPKngRmWyn10oCKFzNHOQ= diff --git a/integration/testnet/network.go b/integration/testnet/network.go index cfbfeed0a52..b1cc9941529 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -135,9 +135,6 @@ func (net *FlowNetwork) Start(ctx context.Context) { // makes it easier to see logs for a specific test case fmt.Println(">>>> starting network: ", net.config.Name) net.suite.Start(ctx) - for _, cf := range net.ConsensusFollowers { - go cf.Run(ctx) - } } // Remove stops the network, removes all the containers and cleans up all resources. @@ -448,17 +445,18 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { require.Nil(t, err) flowNetwork := &FlowNetwork{ - t: t, - cli: dockerClient, - config: networkConf, - suite: suite, - network: network, - Containers: make(map[string]*Container, nNodes), - AccessPorts: make(map[string]string), - root: root, - seal: seal, - result: result, - bootstrapDir: bootstrapDir, + t: t, + cli: dockerClient, + config: networkConf, + suite: suite, + network: network, + Containers: make(map[string]*Container, nNodes), + ConsensusFollowers: make(map[flow.Identifier]consensus_follower.ConsensusFollower, len(networkConf.ConsensusFollowers)), + AccessPorts: make(map[string]string), + root: root, + seal: seal, + result: result, + bootstrapDir: bootstrapDir, } // add each node to the network @@ -469,18 +467,15 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { // add each follower to the network for _, followerConf := range networkConf.ConsensusFollowers { - err = flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) - require.NoError(t, err) + flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) } return flowNetwork } -func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) error { +func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) { tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") - if err != nil { - return fmt.Errorf("could not get tmp dir: %w", err) - } + require.NoError(t, err) // create a directory for the follower database dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) @@ -498,7 +493,7 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // consensus follower bindPort := testingdock.RandomPort(t) - bindAddr := fmt.Sprintf("0.0.0.0:%d", bindPort) + bindAddr := fmt.Sprintf("0.0.0.0:%s", bindPort) opts := []consensus_follower.Option{ consensus_follower.WithDataDir(dataDir), consensus_follower.WithBootstrapDir(followerBootstrapDir), @@ -721,6 +716,37 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { require.NoError(net.t, err) } +func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { + var nodeInfos []bootstrap.NodeInfo + + // get networking keys for all followers + networkKeys, err := unittest.NetworkingKeys(len(confs)) + if err != nil { + return nil, err + } + + // get staking keys for all followers + stakingKeys, err := unittest.StakingKeys(len(confs)) + if err != nil { + return nil, err + } + + for i, conf := range confs { + info := bootstrap.NewPrivateNodeInfo( + conf.nodeID, + flow.RoleAccess, // use Access role + "", // no address + 0, // no stake + networkKeys[i], + stakingKeys[i], + ) + + nodeInfos = append(nodeInfos, info) + } + + return nodeInfos, nil +} + func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Block, *flow.ExecutionResult, *flow.Seal, []ContainerConfig, error) { chainID := flow.Localnet chain := chainID.Chain() @@ -746,9 +772,14 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo stakedConfs := filterContainerConfigs(allConfs, func(config ContainerConfig) bool { return !config.Unstaked }) - fmt.Println(len(stakedConfs)) - fmt.Println(len(allConfs)) + + followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) + if err != nil { + return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) + } + allNodeInfos := toNodeInfos(allConfs) + allNodeInfos = append(allNodeInfos, followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices // IMPORTANT: these nodes infos must include exactly the identity table diff --git a/integration/tests/access/access_api_test.go b/integration/tests/access/access_api_test.go index 56e437afc26..24fd024f720 100644 --- a/integration/tests/access/access_api_test.go +++ b/integration/tests/access/access_api_test.go @@ -66,7 +66,7 @@ func (suite *AccessSuite) SetupTest() { collConfig := testnet.NewNodeConfig(flow.RoleCollection, testnet.WithID(collID)) nodeConfigs = append(nodeConfigs, collConfig) - conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs) + conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs, nil) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) // start the network diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 1ee76939237..f5259c853aa 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -91,21 +91,20 @@ func (suite *UnstakedAccessSuite) SetupTest() { // consensus follower suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.ConsensusFollowerConfig{ - nodeID: suite.unstakedID, - upstreamNodeID: suite.stakedID, - }, + testnet.NewConsensusFollowerConfig(suite.unstakedID, suite.stakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) + // start the network suite.ctx, suite.cancel = context.WithCancel(context.Background()) suite.net.Start(suite.ctx) } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) - // TODO + go suite.follower.Run(suite.ctx) + // TODO: to be implemented later } From bc16f5a22bd8e03891c35cecb3292779c9185db2 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 16:58:07 -0700 Subject: [PATCH 152/291] Update staked_access_node_builder.go --- cmd/access/node_builder/staked_access_node_builder.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 37428de013d..a9c31c686bb 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -81,10 +81,12 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { - anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) - return node.Network, nil - }) + if anb.ParticipatesInUnstakedNetwork() { + anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) + return node.Network, nil + }) + } anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { From f19a1bebe1e9a80065c15743fc12f512400fdc55 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:21:51 -0700 Subject: [PATCH 153/291] update to reflect new design --- cmd/node_builder.go | 33 +++++----- cmd/scaffold.go | 2 - integration/testnet/network.go | 114 +++++++++++---------------------- 3 files changed, 54 insertions(+), 95 deletions(-) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index b62b9752ee5..f0e1b39219a 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -120,23 +120,22 @@ type BaseConfig struct { type NodeConfig struct { Cancel context.CancelFunc // cancel function for the context that is passed to the networking layer BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware network.Middleware - Network p2p.ReadyDoneAwareNetwork - SubscriptionManager network.SubscriptionManager - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware network.Middleware + Network p2p.ReadyDoneAwareNetwork + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // ID providers IdentityProvider id.IdentityProvider diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a7e9a52b20b..b3acf676cc5 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -191,8 +191,6 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Logger, fnb.State, ) - node.SubscriptionManager = subscriptionManager - if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) } diff --git a/integration/testnet/network.go b/integration/testnet/network.go index b1cc9941529..40225acc9e1 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -14,33 +14,31 @@ import ( "testing" "time" - "github.com/onflow/flow-go/cmd/bootstrap/utils" - "github.com/docker/docker/api/types/container" dockerclient "github.com/docker/docker/client" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/onflow/flow-go-sdk/crypto" - "github.com/onflow/flow-go/model/encodable" - "github.com/onflow/flow-go/model/flow/order" - "github.com/onflow/flow-go/utils/io" - "github.com/dapperlabs/testingdock" "github.com/onflow/cadence" + "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/cmd/bootstrap/run" + "github.com/onflow/flow-go/cmd/bootstrap/utils" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" dkgmod "github.com/onflow/flow-go/model/dkg" + "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" clusterstate "github.com/onflow/flow-go/state/cluster" "github.com/onflow/flow-go/state/protocol/inmem" + "github.com/onflow/flow-go/utils/io" "github.com/onflow/flow-go/utils/unittest" ) @@ -67,19 +65,14 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" - // UnstakedNetworkPort is the name used for the access node unstaked libp2p network port. - UnstakedNetworkPort = "access-unstaked-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" // ExeNodeMetricsPort ExeNodeMetricsPort = "exe-metrics-port" - // default staked network port - DefaultStakedFlowPort = 2137 - - // default unstaked network port - DefaultUnstakedFlowPort = 7312 + // default network port + DefaultFlowPort = 2137 DefaultViewsInStakingAuction uint64 = 5 DefaultViewsInDKGPhase uint64 = 50 @@ -312,17 +305,14 @@ func (n *NetworkConfig) Swap(i, j int) { // NodeConfig defines the input config for a particular node, specified prior // to network creation. type NodeConfig struct { - Role flow.Role - Stake uint64 - Identifier flow.Identifier - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - // Unstaked - only applicable to Access Node. Access nodes can be staked or unstaked. - // Unstaked nodes are not part of the identity table - Unstaked bool // only applicable to Access node - ParticipatesInUnstakedNetwork bool + Role flow.Role + Stake uint64 + Identifier flow.Identifier + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + ParticipatesInPublicNetwork bool // only applicable to Access node } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -398,9 +388,9 @@ func AsGhost() func(config *NodeConfig) { } } -func AsUnstakedNetworkParticipant() func(config *NodeConfig) { +func AsPublicNetworkParticipant() func(config *NodeConfig) { return func(config *NodeConfig) { - config.ParticipatesInUnstakedNetwork = true + config.ParticipatesInPublicNetwork = true } } @@ -499,11 +489,8 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, consensus_follower.WithBootstrapDir(followerBootstrapDir), } - // TODO: eventually we will need upstream node's address - // - // upstreamANPort := net.ContainerByID(followerConf.upstreamNodeID).Ports[testnet.UnstakedNetworkPort] - // upstreamANAddress := fmt.Sprintf("127.0.0.1:%d", upstreamANPort) - + // TODO: update consensus follower to just accept a networking key instead of a node ID + // it should be able to figure out the rest on its own. follower := consensus_follower.NewConsensusFollower( followerConf.nodeID, followerConf.upstreamNodeID, @@ -649,13 +636,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort - if nodeConf.ParticipatesInUnstakedNetwork { - hostUnstakedPort := testingdock.RandomPort(t) - containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) - nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) - nodeContainer.addFlag("unstaked-bind-addr", fmt.Sprintf("%s:%d", nodeContainer.Name(), DefaultUnstakedFlowPort)) - nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort - net.AccessPorts[UnstakedNetworkPort] = hostUnstakedPort + if nodeConf.ParticipatesInPublicNetwork { + // TODO: define this flag for Access node + nodeContainer.addFlag("public-network-participant", "true") } case flow.RoleConsensus: @@ -678,18 +661,12 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort - if nodeConf.ParticipatesInUnstakedNetwork { - hostUnstakedPort := testingdock.RandomPort(t) - containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) - - // TODO: Currently, it is not possible to create a staked ghost AN which - // participates on the unstaked network, because the ghost node only joins - // a single network during startup. The ghost node needs to support the - // "unstaked-bind-addr" flag which can be used to specify a bind address - // for the unstaked network. - - nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) - nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + if nodeConf.ParticipatesInPublicNetwork { + // TODO: Currently, it is not possible to create a ghost AN which participates + // in the public network, because connection gating is enabled by default and + // therefore the ghost node will deny incoming connections from all consensus + // followers. A flag for the ghost node will need to be created to enable + // overriding the default behavior. } } @@ -761,29 +738,19 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo sort.Sort(&networkConf) // generate staking and networking keys for each configured node - // NOTE: this includes unstaked access nodes, which need private keys written - // but should not be included in the identity table - allConfs, err := setupKeys(networkConf) + stakedConfs, err := setupKeys(networkConf) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to setup keys: %w", err) } - // only staked configs - this only includes identity table members - stakedConfs := filterContainerConfigs(allConfs, func(config ContainerConfig) bool { - return !config.Unstaked - }) - followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) } - allNodeInfos := toNodeInfos(allConfs) - allNodeInfos = append(allNodeInfos, followerInfos...) + allNodeInfos = append(toNodeInfos(stakedConfs), followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices - // IMPORTANT: these nodes infos must include exactly the identity table - // members (no unstaked access nodes) stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), order.Canonical) // run DKG for all consensus nodes @@ -963,11 +930,7 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { // define the node's name _ and address : name := fmt.Sprintf("%s_%d", conf.Role.String(), roleCounter[conf.Role]+1) - flowPort := DefaultStakedFlowPort - if conf.Unstaked { - flowPort = DefaultUnstakedFlowPort - } - addr := fmt.Sprintf("%s:%d", name, flowPort) + addr := fmt.Sprintf("%s:%d", name, DefaultFlowPort) roleCounter[conf.Role]++ info := bootstrap.NewPrivateNodeInfo( @@ -980,14 +943,13 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - Unstaked: conf.Unstaked, - ParticipatesInUnstakedNetwork: conf.ParticipatesInUnstakedNetwork, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + ParticipatesInPublicNetwork: conf.ParticipatesInPublicNetwork, } confs = append(confs, containerConf) From 2acd77505ca0de825d44795a85b544cfcb109096 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:22:49 -0700 Subject: [PATCH 154/291] Update staked_access_node_builder.go --- .../staked_access_node_builder.go | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a9c31c686bb..e880875b10e 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -81,12 +81,6 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { - if anb.ParticipatesInUnstakedNetwork() { - anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) - return node.Network, nil - }) - } anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -104,23 +98,7 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { } return ping, nil }) - if anb.ParticipatesInUnstakedNetwork() { - // create relay engine - anb.Component("relay engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - channels := node.SubscriptionManager.Channels() - if len(channels) == 0 { - return nil, fmt.Errorf("no subscribed channels to relay") - } - relayEngine, err := relay.New(node.Logger, channels, node.Network, anb.UnstakedNetwork) - - if err != nil { - return nil, fmt.Errorf("could not create relay engine: %w", err) - } - - return relayEngine, nil - }) - } return anb } From bc7322c35048976d11bb50460a49a3ed484f5ef8 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:31:53 -0700 Subject: [PATCH 155/291] Update consensus_follower.go --- follower/consensus_follower.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index a34b207537e..85e98c475b8 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -24,7 +24,7 @@ type ConsensusFollower interface { // Config contains the configurable fields for a `ConsensusFollower`. type Config struct { - nodeID flow.Identifier // the node ID of this node + networkPubKey crypto.PublicKey // the network public key of this node bootstrapNodes []BootstrapNodeInfo // the bootstrap nodes to use bindAddr string // address to bind on dataDir string // directory to store the protocol state @@ -69,14 +69,14 @@ func getAccessNodeOptions(config *Config) []access.Option { ids := bootstrapIdentities(config.bootstrapNodes) return []access.Option{ access.WithBootStrapPeers(ids...), - access.WithUnstakedNetworkBindAddr(config.bindAddr), + access.WithBindAddr(config.bindAddr), access.WithBaseOptions(getBaseOptions(config)), } } func getBaseOptions(config *Config) []cmd.Option { options := []cmd.Option{ - cmd.WithNodeID(config.nodeID), + cmd.WithNetworkPublicKey(config.networkPubKey), cmd.WithMetricsEnabled(false), } if config.bootstrapDir != "" { @@ -107,13 +107,13 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - nodeID flow.Identifier, + networkPublicKey crypto.PublicKey, bootstapIdentities []BootstrapNodeInfo, bindAddr string, opts ...Option, ) (*ConsensusFollowerImpl, error) { config := &Config{ - nodeID: nodeID, + networkPublicKey: networkPublicKey, bootstrapNodes: bootstapIdentities, bindAddr: bindAddr, } From a2d490cf04c47d2364e7ba8c3fe88f7f24b4469e Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:51:21 -0700 Subject: [PATCH 156/291] f --- follower/consensus_follower.go | 2 +- integration/testnet/network.go | 12 ++++++++---- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 85e98c475b8..82062be2361 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -107,7 +107,7 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - networkPublicKey crypto.PublicKey, + networkPublicKey crypto.PublicKey, // TODO: make this optional. if not explicitly supplied, we can auto-generate one for them. bootstapIdentities []BootstrapNodeInfo, bindAddr string, opts ...Option, diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 40225acc9e1..ee1f90c2be7 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -27,6 +27,7 @@ import ( "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/cmd/bootstrap/run" "github.com/onflow/flow-go/cmd/bootstrap/utils" + fcrypto "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" @@ -216,13 +217,13 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier + networkKey fcrypto.PrivateKey upstreamNodeID flow.Identifier } -func NewConsensusFollowerConfig(nodeID flow.Identifier, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - nodeID: nodeID, + networkKey: fcrypto.PrivateKey, upstreamNodeID: upstreamNodeID, } } @@ -492,12 +493,13 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. follower := consensus_follower.NewConsensusFollower( - followerConf.nodeID, + followerConf.networkKey, followerConf.upstreamNodeID, bindAddr, opts..., ) + // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower } @@ -696,6 +698,7 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { var nodeInfos []bootstrap.NodeInfo + // TODO: remove this, networking keys should be provided by the consensus follower config. // get networking keys for all followers networkKeys, err := unittest.NetworkingKeys(len(confs)) if err != nil { @@ -710,6 +713,7 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for i, conf := range confs { info := bootstrap.NewPrivateNodeInfo( + // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address From 245526a2b783c2304bfe43ba46f29b431d3c9687 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Sun, 22 Aug 2021 21:19:41 -0700 Subject: [PATCH 157/291] removing unstaked network --- .../node_builder/access_node_builder.go | 52 ++++++------ .../staked_access_node_builder.go | 79 ++++++++++++++----- .../unstaked_access_node_builder.go | 18 ++--- cmd/node_builder.go | 4 +- cmd/scaffold.go | 12 ++- follower/consensus_follower.go | 4 +- 6 files changed, 110 insertions(+), 59 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 75e87a708dd..47233445a6a 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -73,9 +73,9 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // ParticipatesInUnstakedNetwork returns True if this an Access Node which participates in the unstaked network, - // False otherwise - ParticipatesInUnstakedNetwork() bool + // SupportsUnstakedNetwork returns True if this is a staked Access node which also supports + // unstaked access nodes/unstaked consensus follower engines, False otherwise. + SupportsUnstakedNetwork() bool // Build defines all of the Access node's components and modules. Build() AccessNodeBuilder @@ -89,7 +89,7 @@ type AccessNodeConfig struct { bootstrapNodeAddresses []string bootstrapNodePublicKeys []string bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes - unstakedNetworkBindAddr string + supportsUnstakedFollower bool collectionGRPCPort uint executionGRPCPort uint pingEnabled bool @@ -137,7 +137,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { staked: true, bootstrapNodeAddresses: []string{}, bootstrapNodePublicKeys: []string{}, - unstakedNetworkBindAddr: cmd.NotSet, + supportsUnstakedFollower: false, } } @@ -149,9 +149,7 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedLibP2PNode *p2p.Node - UnstakedNetwork p2p.ReadyDoneAwareNetwork - unstakedMiddleware network.Middleware + LibP2PNode *p2p.Node FollowerState protocol.MutableState SyncCore *synchronization.Core RpcEng *rpc.Engine @@ -496,9 +494,9 @@ func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { } } -func WithUnstakedNetworkBindAddr(bindAddr string) Option { +func SupportsUnstakedFollower(enable bool) Option { return func(config *AccessNodeConfig) { - config.unstakedNetworkBindAddr = bindAddr + config.supportsUnstakedFollower = enable } } @@ -524,8 +522,14 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { return builder.staked } -func (builder *FlowAccessNodeBuilder) ParticipatesInUnstakedNetwork() bool { - return builder.unstakedNetworkBindAddr != cmd.NotSet +func (builder *FlowAccessNodeBuilder) SupportsUnstakedNetwork() bool { + // unstaked access nodes can't be upstream of other unstaked access nodes for now + if !builder.IsStaked() { + return false + } + + // a staked access node may or may not support unstaked follower + return builder.supportsUnstakedFollower } func (builder *FlowAccessNodeBuilder) ParseFlags() { @@ -566,7 +570,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") - flags.StringVar(&builder.unstakedNetworkBindAddr, "unstaked-bind-addr", defaultConfig.unstakedNetworkBindAddr, "address to bind on for the unstaked network") + flags.BoolVar(&builder.supportsUnstakedFollower, "supports-unstaked-follower", defaultConfig.supportsUnstakedFollower, "true if this staked access node supports unstaked follower (not applicable for unstaked nodes)") }) } @@ -586,19 +590,24 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, dhtOptions = append(dhtOptions, bootstrapPeersOpt) } + myAddr := builder.NodeConfig.Me.Address() + if builder.BaseConfig.BindAddr != cmd.NotSet { + myAddr = builder.BaseConfig.BindAddr + } + return func() (*p2p.Node, error) { - libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.unstakedNetworkBindAddr, networkKey). + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). SetRootBlockID(builder.RootBlock.ID().String()). - // unlike the staked network where currently all the node addresses are known upfront, - // for the unstaked network the nodes need to discover each other using DHT Discovery. + // unlike the staked side of the network where currently all the node addresses are known upfront, + // for the unstaked side of the network, the nodes need to discover each other using DHT Discovery. SetDHTOptions(dhtOptions...). SetLogger(builder.Logger). Build(ctx) if err != nil { return nil, err } - builder.UnstakedLibP2PNode = libp2pNode - return builder.UnstakedLibP2PNode, nil + builder.LibP2PNode = libp2pNode + return builder.LibP2PNode, nil }, nil } @@ -607,9 +616,8 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - validators ...network.MessageValidator) *network.Middleware { - builder.unstakedMiddleware = p2p.NewMiddleware( + builder.Middleware = p2p.NewMiddleware( builder.Logger, factoryFunc, nodeID, @@ -623,7 +631,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, p2p.WithMessageValidators(validators...), // use default identifier provider ) - return builder.unstakedMiddleware + return builder.Middleware } // initNetwork creates the network.Network implementation with the given metrics, middleware, initial list of network @@ -643,7 +651,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, builder.Logger, codec, nodeID, - builder.unstakedMiddleware, + builder.Middleware, p2p.DefaultCacheSize, topology, subscriptionManager, diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index e880875b10e..251a431b921 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -4,7 +4,10 @@ import ( "context" "fmt" + dht "github.com/libp2p/go-libp2p-kad-dht" + "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/crypto" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/engine/access/relay" splitternetwork "github.com/onflow/flow-go/engine/common/splitter/network" @@ -31,6 +34,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, node.ProtocolEvents) if err != nil { return err @@ -45,9 +49,10 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { ), idCache, ) + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // TODO: NetworkingIdentifierProvidzer should be the same as the one used in scaffold.go if this AN // doesn't participate in unstaked network. // If it does, then we can just use the default one (peerstoreProvider) @@ -67,8 +72,10 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network - if builder.ParticipatesInUnstakedNetwork() { + if builder.SupportsUnstakedNetwork() { builder.enqueueUnstakedNetworkInit(ctx) + } else { + builder.EnqueueNetworkInit(ctx) } builder.EnqueueMetricsServerInit() @@ -107,36 +114,70 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // NodeID for the staked node on the unstaked network - // TODO: set a different node ID of the staked access node on the unstaked network - unstakedNodeID := builder.NodeID // currently set the same as the staked NodeID - - // Networking key - // TODO: set a different networking key of the staked access node on the unstaked network - unstakedNetworkKey := builder.NetworkKey - - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) + msgValidators := unstakedNetworkMsgValidators(builder.NodeID) // Network Metrics // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics // TODO: define new network metrics for the unstaked network unstakedNetworkMetrics := metrics.NewNoopCollector() - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + middleware := builder.initMiddleware(builder.NodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront - top := topology.EmptyListTopology{} + top, err := topology.NewTopicBasedTopology( + builder.NodeID, + builder.Logger, + builder.State, + ) + if err != nil { + return nil, fmt.Errorf("could not create topology: %w", err) + } + topologyCache := topology.NewCache(builder.Logger, top) - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, top) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, topologyCache) builder.MustNot(err) - builder.UnstakedNetwork = network - builder.unstakedMiddleware = middleware + builder.Network = network + builder.Middleware = middleware - node.Logger.Info().Msgf("unstaked network will run on address: %s", builder.unstakedNetworkBindAddr) - return builder.UnstakedNetwork, err + node.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + return builder.Network, err }) } + +// initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. +// The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance +func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, + nodeID flow.Identifier, + networkKey crypto.PrivateKey) (p2p.LibP2PFactoryFunc, error) { + + // The staked nodes act as the DHT servers + dhtOptions := []dht.Option{p2p.AsServer(builder.IsStaked())} + + myAddr := builder.NodeConfig.Me.Address() + if builder.BaseConfig.BindAddr != cmd.NotSet { + myAddr = builder.BaseConfig.BindAddr + } + + connManager := p2p.NewConnManager(builder.Logger, builder.Metrics.Network) + + return func() (*p2p.Node, error) { + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). + SetRootBlockID(builder.RootBlock.ID().String()). + // no connection gater + SetConnectionManager(connManager). + // act as a DHT server + SetDHTOptions(dhtOptions...). + SetPubsubOptions(p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize)...). + SetLogger(builder.Logger). + Build(ctx) + if err != nil { + return nil, err + } + builder.LibP2PNode = libp2pNode + return builder.LibP2PNode, nil + }, nil +} diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index a4e0085eaca..b0a5b0479e3 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -63,12 +63,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (builder *UnstakedAccessNodeBuilder) validateParams() { - - // for an unstaked access node, the unstaked network bind address must be provided - if builder.unstakedNetworkBindAddr == cmd.NotSet { - builder.Logger.Fatal().Msg("unstaked bind address not set") - } - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") } @@ -93,7 +87,7 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N NetworkPubKey: node.NetworkKey.PublicKey(), StakingPubKey: nil, // no staking key needed for the unstaked node Role: flow.RoleAccess, // unstaked node can only run as an access node - Address: builder.unstakedNetworkBindAddr, + Address: builder.BindAddr, } me, err := local.New(self, nil) @@ -139,16 +133,16 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) builder.MustNot(err) - builder.UnstakedNetwork = network - builder.unstakedMiddleware = middleware + builder.Network = network + builder.Middleware = middleware // for an unstaked node, the staked network and middleware is set to the same as the unstaked network and middlware builder.Network = network builder.Middleware = middleware - builder.Logger.Info().Msgf("unstaked network will run on address: %s", builder.unstakedNetworkBindAddr) + builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) - return builder.UnstakedNetwork, err + return builder.Network, err }) } @@ -160,6 +154,6 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.UnstakedLibP2PNode, builder.Logger), nil + return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil }) } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index f0e1b39219a..63f64381a2f 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -95,7 +95,7 @@ type NodeBuilder interface { // while for a node running as a library, the config fields are expected to be initialized by the caller. type BaseConfig struct { nodeIDHex string - bindAddr string + BindAddr string NodeRole string timeout time.Duration datadir string @@ -156,7 +156,7 @@ func DefaultBaseConfig() *BaseConfig { datadir := filepath.Join(homedir, ".flow", "database") return &BaseConfig{ nodeIDHex: NotSet, - bindAddr: NotSet, + BindAddr: NotSet, BootstrapDir: "bootstrap", timeout: 1 * time.Minute, datadir: datadir, diff --git a/cmd/scaffold.go b/cmd/scaffold.go index b3acf676cc5..d3530588fc0 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -111,7 +111,7 @@ func (fnb *FlowNodeBuilder) BaseFlags() { // bind configuration parameters fnb.flags.StringVar(&fnb.BaseConfig.nodeIDHex, "nodeid", defaultConfig.nodeIDHex, "identity of our node") - fnb.flags.StringVar(&fnb.BaseConfig.bindAddr, "bind", defaultConfig.bindAddr, "address to bind on") + fnb.flags.StringVar(&fnb.BaseConfig.BindAddr, "bind", defaultConfig.BindAddr, "address to bind on") fnb.flags.StringVarP(&fnb.BaseConfig.BootstrapDir, "bootstrapdir", "b", defaultConfig.BootstrapDir, "path to the bootstrap directory") fnb.flags.DurationVarP(&fnb.BaseConfig.timeout, "timeout", "t", defaultConfig.timeout, "node startup / shutdown timeout") fnb.flags.StringVarP(&fnb.BaseConfig.datadir, "datadir", "d", defaultConfig.datadir, "directory to store the protocol state") @@ -138,8 +138,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { codec := cborcodec.NewCodec() myAddr := fnb.NodeConfig.Me.Address() - if fnb.BaseConfig.bindAddr != NotSet { - myAddr = fnb.BaseConfig.bindAddr + if fnb.BaseConfig.BindAddr != NotSet { + myAddr = fnb.BaseConfig.BindAddr } // setup the Ping provider to return the software version and the sealed block height @@ -704,6 +704,12 @@ func WithNodeID(nodeID flow.Identifier) Option { } } +func WithBindAddress(bindAddress string) Option { + return func(config *BaseConfig) { + config.BindAddr = bindAddress + } +} + func WithDataDir(dataDir string) Option { return func(config *BaseConfig) { config.datadir = dataDir diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 82062be2361..409870a9409 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -69,7 +69,6 @@ func getAccessNodeOptions(config *Config) []access.Option { ids := bootstrapIdentities(config.bootstrapNodes) return []access.Option{ access.WithBootStrapPeers(ids...), - access.WithBindAddr(config.bindAddr), access.WithBaseOptions(getBaseOptions(config)), } } @@ -85,6 +84,9 @@ func getBaseOptions(config *Config) []cmd.Option { if config.dataDir != "" { options = append(options, cmd.WithDataDir(config.dataDir)) } + if config.bindAddr != "" { + options = append(options, cmd.WithBindAddress(config.bindAddr)) + } return options } From a350740a7c59294852c8b4231b5aa11636962f56 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Sun, 22 Aug 2021 21:44:52 -0700 Subject: [PATCH 158/291] wip --- cmd/access/node_builder/staked_access_node_builder.go | 7 ++----- cmd/access/node_builder/unstaked_access_node_builder.go | 4 +++- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 251a431b921..2c97847a945 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -67,14 +67,11 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() - // for the staked access node, initialize the network used to communicate with the other staked flow nodes - // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - builder.EnqueueNetworkInit(ctx) - - // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network + // if this is an access node that supports unstaked followers, enqueue the unstaked network if builder.SupportsUnstakedNetwork() { builder.enqueueUnstakedNetworkInit(ctx) } else { + // otherwise, enqueue the regular network builder.EnqueueNetworkInit(ctx) } diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index b0a5b0479e3..ac7fca155ec 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -96,6 +96,8 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N } } +// Build enqueues the sync engine and the follower engine for the unstaked access node. +// Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { @@ -103,7 +105,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() return nil }) - anb.FlowAccessNodeBuilder.Build() + anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } From bfb8bb2f706efdcfd3acac66f335606ea416ac28 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 12:40:17 -0700 Subject: [PATCH 159/291] wip - integration tests for consensus follower --- .../node_builder/access_node_builder.go | 19 ++++++--- .../staked_access_node_builder.go | 4 +- .../unstaked_access_node_builder.go | 8 +++- cmd/node_builder.go | 3 ++ cmd/scaffold.go | 10 +---- follower/consensus_follower.go | 10 ++--- integration/testnet/network.go | 39 ++++++++----------- .../tests/access/unstaked_node_test.go | 10 +++-- 8 files changed, 55 insertions(+), 48 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 47233445a6a..731b3982453 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -73,9 +73,9 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // SupportsUnstakedNetwork returns True if this is a staked Access node which also supports + // SupportsUnstakedNode returns True if this is a staked Access node which also supports // unstaked access nodes/unstaked consensus follower engines, False otherwise. - SupportsUnstakedNetwork() bool + SupportsUnstakedNode() bool // Build defines all of the Access node's components and modules. Build() AccessNodeBuilder @@ -89,6 +89,7 @@ type AccessNodeConfig struct { bootstrapNodeAddresses []string bootstrapNodePublicKeys []string bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes + NetworkKey crypto.PrivateKey supportsUnstakedFollower bool collectionGRPCPort uint executionGRPCPort uint @@ -494,12 +495,18 @@ func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { } } -func SupportsUnstakedFollower(enable bool) Option { +func SupportsUnstakedNode(enable bool) Option { return func(config *AccessNodeConfig) { config.supportsUnstakedFollower = enable } } +func WithNetworkKey(key crypto.PrivateKey) Option { + return func(config *AccessNodeConfig) { + config.NetworkKey = key + } +} + func WithBaseOptions(baseOptions []cmd.Option) Option { return func(config *AccessNodeConfig) { config.baseOptions = baseOptions @@ -522,7 +529,7 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { return builder.staked } -func (builder *FlowAccessNodeBuilder) SupportsUnstakedNetwork() bool { +func (builder *FlowAccessNodeBuilder) SupportsUnstakedNode() bool { // unstaked access nodes can't be upstream of other unstaked access nodes for now if !builder.IsStaked() { return false @@ -570,7 +577,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") - flags.BoolVar(&builder.supportsUnstakedFollower, "supports-unstaked-follower", defaultConfig.supportsUnstakedFollower, "true if this staked access node supports unstaked follower (not applicable for unstaked nodes)") + flags.BoolVar(&builder.supportsUnstakedFollower, "supports-unstaked-node", defaultConfig.supportsUnstakedFollower, "true if this staked access node supports unstaked node") }) } @@ -616,7 +623,7 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - validators ...network.MessageValidator) *network.Middleware { + validators ...network.MessageValidator) network.Middleware { builder.Middleware = p2p.NewMiddleware( builder.Logger, factoryFunc, diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 2c97847a945..fa19144d167 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -9,8 +9,6 @@ import ( "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/crypto" pingeng "github.com/onflow/flow-go/engine/access/ping" - "github.com/onflow/flow-go/engine/access/relay" - splitternetwork "github.com/onflow/flow-go/engine/common/splitter/network" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -68,7 +66,7 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() // if this is an access node that supports unstaked followers, enqueue the unstaked network - if builder.SupportsUnstakedNetwork() { + if builder.SupportsUnstakedNode() { builder.enqueueUnstakedNetworkInit(ctx) } else { // otherwise, enqueue the regular network diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index ac7fca155ec..a04c9f6d0b5 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -21,6 +21,12 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } +func (fnb *UnstakedAccessNodeBuilder) InitNodeInfo() { + fnb.NodeID = flow.ZeroID // TODO: extract node id from networking key + fnb.NodeConfig.NetworkKey = fnb.NetworkKey // use the networking that has been passed in + fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node +} + func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() @@ -55,7 +61,7 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.RegisterBadgerMetrics() - builder.EnqueueTracer() + builder.EnqxueueTracer() builder.PreInit(builder.initUnstakedLocal()) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 63f64381a2f..2189975390a 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -44,6 +44,9 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() + // InitNodeInfo initializes the node id, staking key and networking key + InitNodeInfo() + // InitIDProviders initializes the ID providers needed by various components InitIDProviders() diff --git a/cmd/scaffold.go b/cmd/scaffold.go index d3530588fc0..45966f4948a 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -258,7 +258,7 @@ func (fnb *FlowNodeBuilder) PrintBuildVersionDetails() { fnb.Logger.Info().Str("version", build.Semver()).Str("commit", build.Commit()).Msg("build details") } -func (fnb *FlowNodeBuilder) initNodeInfo() { +func (fnb *FlowNodeBuilder) InitNodeInfo() { if fnb.BaseConfig.nodeIDHex == NotSet { fnb.Logger.Fatal().Msg("cannot start without node ID") } @@ -698,12 +698,6 @@ func WithBootstrapDir(bootstrapDir string) Option { } } -func WithNodeID(nodeID flow.Identifier) Option { - return func(config *BaseConfig) { - config.nodeIDHex = nodeID.String() - } -} - func WithBindAddress(bindAddress string) Option { return func(config *BaseConfig) { config.BindAddr = bindAddress @@ -811,7 +805,7 @@ func (fnb *FlowNodeBuilder) Ready() <-chan struct{} { // seed random generator rand.Seed(time.Now().UnixNano()) - fnb.initNodeInfo() + fnb.InitNodeInfo() fnb.initLogger() diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 409870a9409..33a52064bf6 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -24,7 +24,7 @@ type ConsensusFollower interface { // Config contains the configurable fields for a `ConsensusFollower`. type Config struct { - networkPubKey crypto.PublicKey // the network public key of this node + networkPrivKey crypto.PrivateKey // the network private key of this node bootstrapNodes []BootstrapNodeInfo // the bootstrap nodes to use bindAddr string // address to bind on dataDir string // directory to store the protocol state @@ -70,12 +70,12 @@ func getAccessNodeOptions(config *Config) []access.Option { return []access.Option{ access.WithBootStrapPeers(ids...), access.WithBaseOptions(getBaseOptions(config)), + access.WithNetworkKey(config.networkPrivKey), } } func getBaseOptions(config *Config) []cmd.Option { options := []cmd.Option{ - cmd.WithNetworkPublicKey(config.networkPubKey), cmd.WithMetricsEnabled(false), } if config.bootstrapDir != "" { @@ -109,13 +109,13 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - networkPublicKey crypto.PublicKey, // TODO: make this optional. if not explicitly supplied, we can auto-generate one for them. - bootstapIdentities []BootstrapNodeInfo, + networkPrivKey crypto.PrivateKey, bindAddr string, + bootstapIdentities []BootstrapNodeInfo, opts ...Option, ) (*ConsensusFollowerImpl, error) { config := &Config{ - networkPublicKey: networkPublicKey, + networkPrivKey: networkPrivKey, bootstrapNodes: bootstapIdentities, bindAddr: bindAddr, } diff --git a/integration/testnet/network.go b/integration/testnet/network.go index ee1f90c2be7..bfdfa4c70b1 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -218,13 +218,13 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { type ConsensusFollowerConfig struct { networkKey fcrypto.PrivateKey - upstreamNodeID flow.Identifier + bootstrapNodes []consensus_follower.BootstrapNodeInfo } -func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, bootstrapNodes []consensus_follower.BootstrapNodeInfo) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - networkKey: fcrypto.PrivateKey, - upstreamNodeID: upstreamNodeID, + networkKey: networkKey, + bootstrapNodes: bootstrapNodes, } } @@ -306,14 +306,14 @@ func (n *NetworkConfig) Swap(i, j int) { // NodeConfig defines the input config for a particular node, specified prior // to network creation. type NodeConfig struct { - Role flow.Role - Stake uint64 - Identifier flow.Identifier - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - ParticipatesInPublicNetwork bool // only applicable to Access node + Role flow.Role + Stake uint64 + Identifier flow.Identifier + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + SupportsUnstakedNodes bool // only applicable to Access node } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -389,9 +389,9 @@ func AsGhost() func(config *NodeConfig) { } } -func AsPublicNetworkParticipant() func(config *NodeConfig) { +func SupportsUnstakedNodes() func(config *NodeConfig) { return func(config *NodeConfig) { - config.ParticipatesInPublicNetwork = true + config.SupportsUnstakedNodes = true } } @@ -492,12 +492,7 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower := consensus_follower.NewConsensusFollower( - followerConf.networkKey, - followerConf.upstreamNodeID, - bindAddr, - opts..., - ) + follower, err := consensus_follower.NewConsensusFollower(followerConf.networkKey, bindAddr, followerConf.bootstrapNodes, opts...) // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower @@ -752,7 +747,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) } - allNodeInfos = append(toNodeInfos(stakedConfs), followerInfos...) + allNodeInfos := append(toNodeInfos(stakedConfs), followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), order.Canonical) @@ -953,7 +948,7 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { Ghost: conf.Ghost, AdditionalFlags: conf.AdditionalFlags, Debug: conf.Debug, - ParticipatesInPublicNetwork: conf.ParticipatesInPublicNetwork, + ParticipatesInPublicNetwork: conf.SupportsUnstakedNodes, } confs = append(confs, containerConf) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index f5259c853aa..16de0d53db4 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/rs/zerolog" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" consensus_follower "github.com/onflow/flow-go/follower" @@ -49,7 +50,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { stakedConfig := testnet.NewNodeConfig( flow.RoleAccess, testnet.WithID(suite.stakedID), - testnet.AsUnstakedNetworkParticipant(), + testnet.SupportsUnstakedNodes(), testnet.WithLogLevel(zerolog.InfoLevel), ) nodeConfigs = append(nodeConfigs, stakedConfig) @@ -89,9 +90,12 @@ func (suite *UnstakedAccessSuite) SetupTest() { nodeConfigs = append(nodeConfigs, collConfig) // consensus follower - suite.unstakedID = unittest.IdentifierFixture() + unstakedKey, err := unittest.NetworkingKey() + require.NoError(suite.T(), err) + + followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(suite.unstakedID, suite.stakedID), + testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) From 1e09384a66bccbc102a50406b2b1cd81c78e0c20 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 15:48:03 -0700 Subject: [PATCH 160/291] wip --- .../unstaked_access_node_builder.go | 20 ++--- integration/testnet/container.go | 14 +-- integration/testnet/network.go | 87 ++++++++++++------- .../tests/access/unstaked_node_test.go | 4 +- 4 files changed, 72 insertions(+), 53 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index a04c9f6d0b5..ba602757fe1 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -57,12 +57,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() - builder.EnqueueMetricsServerInit() - - builder.RegisterBadgerMetrics() - - builder.EnqxueueTracer() - builder.PreInit(builder.initUnstakedLocal()) return builder @@ -105,13 +99,13 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { - anb. - Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - // use the default identifier provider - node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() - return nil - }) - anb.FlowAccessNodeBuilder.BuildConsensusFollower() + //anb. + // Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + // // use the default identifier provider + // node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() + // return nil + // }) + //anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } diff --git a/integration/testnet/container.go b/integration/testnet/container.go index 666c54d349a..8b60384ef58 100644 --- a/integration/testnet/container.go +++ b/integration/testnet/container.go @@ -38,13 +38,13 @@ func init() { // ContainerConfig represents configuration for a node container in the network. type ContainerConfig struct { bootstrap.NodeInfo - ContainerName string - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - Unstaked bool - ParticipatesInUnstakedNetwork bool + ContainerName string + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + Unstaked bool + SupportsUnstakedNodes bool } // ImageName returns the Docker image name for the given config. diff --git a/integration/testnet/network.go b/integration/testnet/network.go index bfdfa4c70b1..bccc9c80c86 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -25,9 +25,9 @@ import ( "github.com/onflow/cadence" "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go/cmd/bootstrap/run" "github.com/onflow/flow-go/cmd/bootstrap/utils" - fcrypto "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" @@ -217,14 +217,16 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - networkKey fcrypto.PrivateKey - bootstrapNodes []consensus_follower.BootstrapNodeInfo + nodeID flow.Identifier + networkingPrivKey crypto.PrivateKey + stakedNodeID flow.Identifier } -func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, bootstrapNodes []consensus_follower.BootstrapNodeInfo) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier, nodeID flow.Identifier) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - networkKey: networkKey, - bootstrapNodes: bootstrapNodes, + networkingPrivKey: networkingPrivKey, + stakedNodeID: stakedNodeID, + nodeID: nodeID, // TODO: remove this and derive it from the key instead } } @@ -458,13 +460,13 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { // add each follower to the network for _, followerConf := range networkConf.ConsensusFollowers { - flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) + flowNetwork.addConsensusFollower(t, bootstrapDir, followerConf, confs) } return flowNetwork } -func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) { +func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig, containers []ContainerConfig) { tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") require.NoError(t, err) @@ -490,9 +492,35 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, consensus_follower.WithBootstrapDir(followerBootstrapDir), } + var stakedANContainer *ContainerConfig + // find the upstream Access node container for this follower engine + for _, cont := range containers { + if cont.NodeID == followerConf.stakedNodeID { + stakedANContainer = &cont + break + } + } + require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) + + hostPort := strings.Split(stakedANContainer.Address, ":") + require.Len(t, hostPort, 2, "invalid address for staked AN %s", stakedANContainer.Address) + + host := hostPort[0] + portStr := hostPort[1] + portU64, err := strconv.ParseUint(portStr, 10, 32) + require.NoError(t, err) + port := uint(portU64) + + bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ + Host: host, + Port: port, + NetworkPublicKey: stakedANContainer.NetworkPubKey(), + } + // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower, err := consensus_follower.NewConsensusFollower(followerConf.networkKey, bindAddr, followerConf.bootstrapNodes, opts...) + follower, err := consensus_follower.NewConsensusFollower(followerConf.networkingPrivKey, bindAddr, + []consensus_follower.BootstrapNodeInfo{bootstrapNodeInfo}, opts...) // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower @@ -633,9 +661,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort - if nodeConf.ParticipatesInPublicNetwork { + if nodeConf.SupportsUnstakedNodes { // TODO: define this flag for Access node - nodeContainer.addFlag("public-network-participant", "true") + nodeContainer.addFlag("supports-unstaked-node", "true") } case flow.RoleConsensus: @@ -658,7 +686,7 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort - if nodeConf.ParticipatesInPublicNetwork { + if nodeConf.SupportsUnstakedNodes { // TODO: Currently, it is not possible to create a ghost AN which participates // in the public network, because connection gating is enabled by default and // therefore the ghost node will deny incoming connections from all consensus @@ -693,28 +721,22 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { var nodeInfos []bootstrap.NodeInfo - // TODO: remove this, networking keys should be provided by the consensus follower config. - // get networking keys for all followers - networkKeys, err := unittest.NetworkingKeys(len(confs)) + // TODO: currently just stashing a dummy key as staking key to prevent the nodeinfo.Type() function from + // returning an error. Eventually, a new key type NodeInfoTypePrivateUnstaked needs to be defined + dummyStakingKey, err := unittest.StakingKey() if err != nil { return nil, err } - // get staking keys for all followers - stakingKeys, err := unittest.StakingKeys(len(confs)) - if err != nil { - return nil, err - } - - for i, conf := range confs { + for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address 0, // no stake - networkKeys[i], - stakingKeys[i], + conf.networkingPrivKey, + dummyStakingKey, ) nodeInfos = append(nodeInfos, info) @@ -742,6 +764,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, fmt.Errorf("failed to setup keys: %w", err) } + // generate the follower node keys (follow nodes do not run as docker containers) followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) @@ -897,7 +920,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, err } - return root, result, seal, allConfs, nil + return root, result, seal, stakedConfs, nil } // setupKeys generates private staking and networking keys for each configured @@ -942,13 +965,13 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - ParticipatesInPublicNetwork: conf.SupportsUnstakedNodes, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + SupportsUnstakedNodes: conf.SupportsUnstakedNodes, } confs = append(confs, containerConf) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 16de0d53db4..cc1033c4288 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -92,10 +92,12 @@ func (suite *UnstakedAccessSuite) SetupTest() { // consensus follower unstakedKey, err := unittest.NetworkingKey() require.NoError(suite.T(), err) + // TODO: derive node id from the key + suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID), + testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) From 318c2a415888ada4689a9d34bb4f77a286c2a61e Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 17:10:13 -0700 Subject: [PATCH 161/291] wip --- cmd/access/node_builder/access_node_builder.go | 6 +++--- .../node_builder/staked_access_node_builder.go | 12 +++--------- .../unstaked_access_node_builder.go | 17 +++++++++++++---- cmd/node_builder.go | 3 --- cmd/scaffold.go | 7 +++++-- 5 files changed, 24 insertions(+), 21 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 731b3982453..b4bdf3f6d36 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -41,7 +41,7 @@ import ( "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/network" - jsoncodec "github.com/onflow/flow-go/network/codec/json" + cborcodec "github.com/onflow/flow-go/network/codec/cbor" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/validator" "github.com/onflow/flow-go/state/protocol" @@ -89,7 +89,7 @@ type AccessNodeConfig struct { bootstrapNodeAddresses []string bootstrapNodePublicKeys []string bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes - NetworkKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // the networking key passed in by the caller when being used as a library supportsUnstakedFollower bool collectionGRPCPort uint executionGRPCPort uint @@ -649,7 +649,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, middleware network.Middleware, topology network.Topology) (*p2p.Network, error) { - codec := jsoncodec.NewCodec() + codec := cborcodec.NewCodec() subscriptionManager := p2p.NewChannelSubscriptionManager(middleware) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index fa19144d167..9e5ffb4cd01 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -13,7 +13,6 @@ import ( "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" - "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" ) @@ -109,17 +108,12 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NetworkKey) + libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NodeConfig.NetworkKey) builder.MustNot(err) msgValidators := unstakedNetworkMsgValidators(builder.NodeID) - // Network Metrics - // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics - // TODO: define new network metrics for the unstaked network - unstakedNetworkMetrics := metrics.NewNoopCollector() - - middleware := builder.initMiddleware(builder.NodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront top, err := topology.NewTopicBasedTopology( @@ -132,7 +126,7 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C } topologyCache := topology.NewCache(builder.Logger, top) - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, topologyCache) + network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, topologyCache) builder.MustNot(err) builder.Network = network diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index ba602757fe1..78706ba010a 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -21,9 +21,13 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } -func (fnb *UnstakedAccessNodeBuilder) InitNodeInfo() { - fnb.NodeID = flow.ZeroID // TODO: extract node id from networking key - fnb.NodeConfig.NetworkKey = fnb.NetworkKey // use the networking that has been passed in +func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { + // use the networking key that has been passed in the config + networkingKey := fnb.AccessNodeConfig.NetworkKey + nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + fnb.MustNot(err) + fnb.NodeID = nodeID + fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } @@ -49,6 +53,11 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() + // if a network key has been passed in the init node info here + if builder.AccessNodeConfig.NetworkKey != nil { + builder.initNodeInfo() + } + builder.InitIDProviders() builder.deriveBootstrapPeerIdentities() @@ -105,7 +114,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { // node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() // return nil // }) - //anb.FlowAccessNodeBuilder.BuildConsensusFollower() + anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 2189975390a..63f64381a2f 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -44,9 +44,6 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() - // InitNodeInfo initializes the node id, staking key and networking key - InitNodeInfo() - // InitIDProviders initializes the ID providers needed by various components InitIDProviders() diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 45966f4948a..4eb4718c2ad 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -258,7 +258,7 @@ func (fnb *FlowNodeBuilder) PrintBuildVersionDetails() { fnb.Logger.Info().Str("version", build.Semver()).Str("commit", build.Commit()).Msg("build details") } -func (fnb *FlowNodeBuilder) InitNodeInfo() { +func (fnb *FlowNodeBuilder) initNodeInfo() { if fnb.BaseConfig.nodeIDHex == NotSet { fnb.Logger.Fatal().Msg("cannot start without node ID") } @@ -805,7 +805,10 @@ func (fnb *FlowNodeBuilder) Ready() <-chan struct{} { // seed random generator rand.Seed(time.Now().UnixNano()) - fnb.InitNodeInfo() + // init nodeinfo by reading the private bootstrap file if not already set + if fnb.NodeID == flow.ZeroID { + fnb.initNodeInfo() + } fnb.initLogger() From 76c2ccd84fd36ce91502ff9e909f99a73100ecb7 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 17:21:26 -0700 Subject: [PATCH 162/291] fixing unstaked_node_test integration test --- cmd/access/node_builder/unstaked_access_node_builder.go | 6 +++--- integration/testnet/network.go | 9 +++++++-- integration/tests/access/unstaked_node_test.go | 2 +- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 78706ba010a..fb6a2b17492 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -24,11 +24,11 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config networkingKey := fnb.AccessNodeConfig.NetworkKey - nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this fnb.MustNot(err) fnb.NodeID = nodeID - fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig - fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node + fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig + fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { diff --git a/integration/testnet/network.go b/integration/testnet/network.go index bccc9c80c86..98f5b660184 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -243,10 +243,9 @@ type NetworkConfig struct { type NetworkConfigOpt func(*NetworkConfig) -func NewNetworkConfig(name string, nodes []NodeConfig, followers []ConsensusFollowerConfig, opts ...NetworkConfigOpt) NetworkConfig { +func NewNetworkConfig(name string, nodes []NodeConfig, opts ...NetworkConfigOpt) NetworkConfig { c := NetworkConfig{ Nodes: nodes, - ConsensusFollowers: followers, Name: name, NClusters: 1, // default to 1 cluster ViewsInStakingAuction: DefaultViewsInStakingAuction, @@ -285,6 +284,12 @@ func WithClusters(n uint) func(*NetworkConfig) { } } +func WithConsensusFollowers(followers ...ConsensusFollowerConfig) func(*NetworkConfig) { + return func(conf *NetworkConfig) { + conf.ConsensusFollowers = followers + } +} + func (n *NetworkConfig) Len() int { return len(n.Nodes) } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index cc1033c4288..680050bdc07 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -100,7 +100,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) From 54520e20c0bd7eb6fd42c8eddc7e2688c1043937 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 18:04:50 -0700 Subject: [PATCH 163/291] making the unstaked node test similar to the mvp test to allow block generation --- .../tests/access/unstaked_node_test.go | 90 ++++++++++--------- 1 file changed, 46 insertions(+), 44 deletions(-) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 680050bdc07..0a18d01617a 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -2,7 +2,9 @@ package access import ( "context" + "fmt" "testing" + "time" "github.com/rs/zerolog" "github.com/stretchr/testify/require" @@ -43,7 +45,23 @@ func (suite *UnstakedAccessSuite) TearDownTest() { } func (suite *UnstakedAccessSuite) SetupTest() { - nodeConfigs := []testnet.NodeConfig{} + suite.buildNetworkConfig() + // start the network + suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.net.Start(suite.ctx) +} + +func (suite *UnstakedAccessSuite) TestReceiveBlocks() { + //go suite.follower.Run(suite.ctx) + // TODO: to be implemented later + time.Sleep(time.Second * 30) +} + +func (suite *UnstakedAccessSuite) OnBlockFinalizedConsumer(finalizedBlockID flow.Identifier) { + fmt.Println(finalizedBlockID.String()) +} + +func (suite *UnstakedAccessSuite) buildNetworkConfig() { // staked access node suite.stakedID = unittest.IdentifierFixture() @@ -53,41 +71,34 @@ func (suite *UnstakedAccessSuite) SetupTest() { testnet.SupportsUnstakedNodes(), testnet.WithLogLevel(zerolog.InfoLevel), ) - nodeConfigs = append(nodeConfigs, stakedConfig) - - // consensus node (ghost) - suite.conID = unittest.IdentifierFixture() - conConfig := testnet.NewNodeConfig( - flow.RoleConsensus, - testnet.WithID(suite.conID), - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, conConfig) - // execution node (unused) - exeConfig := testnet.NewNodeConfig( - flow.RoleExecution, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, exeConfig) + collectionConfigs := []func(*testnet.NodeConfig){ + testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), + testnet.WithAdditionalFlag("--block-rate-delay=100ms"), + testnet.WithLogLevel(zerolog.WarnLevel), + // TODO replace these with actual values + testnet.WithAdditionalFlag("--access-address=null"), + } - // verification node (unused) - verConfig := testnet.NewNodeConfig( - flow.RoleVerification, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, verConfig) + consensusConfigs := []func(config *testnet.NodeConfig){ + testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), + testnet.WithAdditionalFlag("--block-rate-delay=100ms"), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 1)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 1)), + testnet.WithLogLevel(zerolog.WarnLevel), + } - // collection node (unused) - collConfig := testnet.NewNodeConfig( - flow.RoleCollection, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, collConfig) + net := []testnet.NodeConfig{ + testnet.NewNodeConfig(flow.RoleCollection, collectionConfigs...), + testnet.NewNodeConfig(flow.RoleCollection, collectionConfigs...), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.WarnLevel)), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.WarnLevel)), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.WarnLevel), testnet.WithDebugImage(false)), + stakedConfig, + } // consensus follower unstakedKey, err := unittest.NetworkingKey() @@ -95,22 +106,13 @@ func (suite *UnstakedAccessSuite) SetupTest() { // TODO: derive node id from the key suite.unstakedID = unittest.IdentifierFixture() - followerConfigs := []testnet.ConsensusFollowerConfig{ testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, testnet.WithConsensusFollowers(followerConfigs...)) + conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) - - // start the network - suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.net.Start(suite.ctx) -} - -func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - go suite.follower.Run(suite.ctx) - // TODO: to be implemented later + suite.follower.AddOnBlockFinalizedConsumer(suite.OnBlockFinalizedConsumer) } From 227bbb9f3055c4cb47acdf78079e7f66d47af6b3 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 18:30:12 -0700 Subject: [PATCH 164/291] adding default size for cache --- cmd/access/node_builder/unstaked_access_node_builder.go | 6 ++++++ cmd/node_builder.go | 3 +++ integration/tests/access/unstaked_node_test.go | 2 +- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index fb6a2b17492..23a305adfd6 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -66,6 +66,12 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() + //builder.EnqueueMetricsServerInit() + // + //builder.RegisterBadgerMetrics() + // + //builder.EnqueueTracer() + builder.PreInit(builder.initUnstakedLocal()) return builder diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 63f64381a2f..ae844cc1f2c 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -21,6 +21,7 @@ import ( "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/state/protocol/events" + bstorage "github.com/onflow/flow-go/storage/badger" ) const NotSet = "not set" @@ -170,5 +171,7 @@ func DefaultBaseConfig() *BaseConfig { profilerDuration: 10 * time.Second, tracerEnabled: false, metricsEnabled: true, + receiptsCacheSize: bstorage.DefaultCacheSize, + guaranteesCacheSize: bstorage.DefaultCacheSize, } } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 0a18d01617a..daddce23936 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -52,7 +52,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - //go suite.follower.Run(suite.ctx) + go suite.follower.Run(suite.ctx) // TODO: to be implemented later time.Sleep(time.Second * 30) } From 2eaf3be24b9ff5d9797f426f088a0f4a057feada Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 19:42:07 -0700 Subject: [PATCH 165/291] moving builder.deriveBootstrapPeerIdentities to access_node_builder from unstaked_access_node_builder --- cmd/access/node_builder/access_node_builder.go | 10 ++++++++++ .../node_builder/unstaked_access_node_builder.go | 12 +----------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b4bdf3f6d36..1c53c4155d3 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -546,6 +546,8 @@ func (builder *FlowAccessNodeBuilder) ParseFlags() { builder.extraFlags() builder.ParseAndPrintFlags() + + builder.deriveBootstrapPeerIdentities() } func (builder *FlowAccessNodeBuilder) extraFlags() { @@ -581,6 +583,14 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { }) } +// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstrap peers from the parameters. +// These are the identities of the staked and unstaked ANs also acting as the DHT bootstrap server +func (builder *FlowAccessNodeBuilder) deriveBootstrapPeerIdentities() { + ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) + builder.MustNot(err) + builder.bootstrapIdentites = ids +} + // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. // The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 23a305adfd6..25657415037 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -60,8 +60,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() - builder.deriveBootstrapPeerIdentities() - builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() @@ -83,14 +81,6 @@ func (builder *UnstakedAccessNodeBuilder) validateParams() { } } -// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstreap peers from the parameters. -// These are the identity of the staked and unstaked AN also acting as the DHT bootstrap server -func (builder *UnstakedAccessNodeBuilder) deriveBootstrapPeerIdentities() { - ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) - builder.MustNot(err) - builder.bootstrapIdentites = ids -} - // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files @@ -170,7 +160,7 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil }) } From e04819ac070127c1d490606f75e32a7fe3a7b9ce Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 22:14:53 -0700 Subject: [PATCH 166/291] exposing libp2p port externally in integration test docker container for consensus follower to access --- cmd/access/node_builder/upstream_connector.go | 2 +- integration/testnet/network.go | 15 ++++++++------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index fdc608cfadb..233614d1f87 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -40,7 +40,7 @@ func (connector *upstreamConnector) Ready() <-chan struct{} { defer close(resultChan) // a shorter context for the connection worker - workerCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + workerCtx, cancel := context.WithTimeout(ctx, 30*time.Second) defer cancel() // spawn a connect worker for each bootstrap node diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 98f5b660184..9b14f56bdc4 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -66,6 +66,8 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" + // AccessNodeExternalNetworkPort is the name used for the access node network port accessible from outside any docker container + AccessNodeExternalNetworkPort = "access-external-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" @@ -507,17 +509,13 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, } require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) - hostPort := strings.Split(stakedANContainer.Address, ":") - require.Len(t, hostPort, 2, "invalid address for staked AN %s", stakedANContainer.Address) - - host := hostPort[0] - portStr := hostPort[1] + portStr := net.AccessPorts[AccessNodeExternalNetworkPort] portU64, err := strconv.ParseUint(portStr, 10, 32) require.NoError(t, err) port := uint(portU64) bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ - Host: host, + Host: "localhost", Port: port, NetworkPublicKey: stakedANContainer.NetworkPubKey(), } @@ -667,7 +665,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort if nodeConf.SupportsUnstakedNodes { - // TODO: define this flag for Access node + hostExternalNetworkPort := testingdock.RandomPort(t) + nodeContainer.bindPort(hostExternalNetworkPort, fmt.Sprintf("%s/tcp", strconv.Itoa(DefaultFlowPort))) + net.AccessPorts[AccessNodeExternalNetworkPort] = hostExternalNetworkPort nodeContainer.addFlag("supports-unstaked-node", "true") } @@ -697,6 +697,7 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont // therefore the ghost node will deny incoming connections from all consensus // followers. A flag for the ghost node will need to be created to enable // overriding the default behavior. + return fmt.Errorf("currently ghost node for an access node which supports unstaked node is not implemented") } } From 87bba78afc6a8b9c84f6f79c511fd69c7abb136c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:23:29 -0700 Subject: [PATCH 167/291] update node id logic --- cmd/access/node_builder/access_node_builder.go | 8 ++++---- .../unstaked_access_node_builder.go | 18 +++++++++++------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 1c53c4155d3..b85c49e5bed 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -88,7 +88,7 @@ type AccessNodeConfig struct { staked bool bootstrapNodeAddresses []string bootstrapNodePublicKeys []string - bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes + bootstrapIdentities flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes NetworkKey crypto.PrivateKey // the networking key passed in by the caller when being used as a library supportsUnstakedFollower bool collectionGRPCPort uint @@ -491,7 +491,7 @@ type Option func(*AccessNodeConfig) func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { return func(config *AccessNodeConfig) { - config.bootstrapIdentites = bootstrapNodes + config.bootstrapIdentities = bootstrapNodes } } @@ -588,7 +588,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { func (builder *FlowAccessNodeBuilder) deriveBootstrapPeerIdentities() { ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) builder.MustNot(err) - builder.bootstrapIdentites = ids + builder.bootstrapIdentities = ids } // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. @@ -602,7 +602,7 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, // if this is an unstaked access node, then seed the DHT with the boostrap identities if !builder.IsStaked() { - bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentites) + bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentities) builder.MustNot(err) dhtOptions = append(dhtOptions, bootstrapPeersOpt) } diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 25657415037..7024a7d3003 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -3,6 +3,7 @@ package node_builder import ( "context" + "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -24,17 +25,20 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config networkingKey := fnb.AccessNodeConfig.NetworkKey - nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + pubKey, err := p2p.LibP2PPublicKeyFromFlow(networkingKey.PublicKey()) + fnb.MustNot(err) + peerID, err := peer.IDFromPublicKey(pubKey) + fnb.MustNot(err) + fnb.NodeID, err = fnb.IDTranslator.GetFlowID(peerID) fnb.MustNot(err) - fnb.NodeID = nodeID fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err @@ -53,13 +57,13 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() + builder.InitIDProviders() + // if a network key has been passed in the init node info here if builder.AccessNodeConfig.NetworkKey != nil { builder.initNodeInfo() } - builder.InitIDProviders() - builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() @@ -161,6 +165,6 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil + return newUpstreamConnector(builder.bootstrapIdentities, builder.LibP2PNode, builder.Logger), nil }) } From 77a3daf2bb0e07960dc66db48ddc807c7e0df2e6 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:31:44 -0700 Subject: [PATCH 168/291] Add NetworkingIdentifierProider to staked node --- cmd/access/node_builder/staked_access_node_builder.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 9e5ffb4cd01..8910c8da79d 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -49,9 +49,9 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: NetworkingIdentifierProvidzer should be the same as the one used in scaffold.go if this AN - // doesn't participate in unstaked network. - // If it does, then we can just use the default one (peerstoreProvider) + if !fnb.SupportsUnstakedNode() { + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) + } return nil }) From e5b2800a3eeaab1e7568531c27efdf56336eded1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:35:19 -0700 Subject: [PATCH 169/291] Use hierarchical translator for unstaked node --- .../node_builder/unstaked_access_node_builder.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 7024a7d3003..05fcbb6da85 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -29,15 +29,13 @@ func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { fnb.MustNot(err) peerID, err := peer.IDFromPublicKey(pubKey) fnb.MustNot(err) - fnb.NodeID, err = fnb.IDTranslator.GetFlowID(peerID) + fnb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) fnb.MustNot(err) fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { @@ -46,6 +44,8 @@ func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) + return nil }) } @@ -57,13 +57,13 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() - builder.InitIDProviders() - // if a network key has been passed in the init node info here if builder.AccessNodeConfig.NetworkKey != nil { builder.initNodeInfo() } + builder.InitIDProviders() + builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() From dad1ae974b442402277cf3349420b0e6454d083f Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:38:46 -0700 Subject: [PATCH 170/291] Add identity delta for staked and unstaked AN's --- cmd/access/node_builder/staked_access_node_builder.go | 4 ++++ cmd/access/node_builder/unstaked_access_node_builder.go | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 8910c8da79d..4d00a44895c 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" + "github.com/onflow/flow-go/state/protocol/events/gadgets" ) // StakedAccessNodeBuilder builds a staked access node. The staked access node can optionally participate in the @@ -132,6 +133,9 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Network = network builder.Middleware = middleware + idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) + builder.ProtocolEvents.AddConsumer(idEvents) + node.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) return builder.Network, err }) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 05fcbb6da85..31cda8ac094 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/state/protocol/events/gadgets" ) type UnstakedAccessNodeBuilder struct { @@ -153,6 +154,9 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) + builder.ProtocolEvents.AddConsumer(idEvents) + return builder.Network, err }) } From 923fe2d16d4688dd2da62e835ef8303365374909 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 00:19:57 -0700 Subject: [PATCH 171/291] Implement proper message validators for unstaked network. --- .../node_builder/access_node_builder.go | 9 +++++- .../staked_access_node_builder.go | 2 +- .../unstaked_access_node_builder.go | 2 +- network/validator/anyValidator.go | 28 +++++++++++++++++++ network/validator/originValidator.go | 24 ++++++++++++++++ 5 files changed, 62 insertions(+), 3 deletions(-) create mode 100644 network/validator/anyValidator.go create mode 100644 network/validator/originValidator.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b85c49e5bed..9436f1fdb82 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -9,6 +9,7 @@ import ( dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow/protobuf/go/flow/access" + "github.com/rs/zerolog" "github.com/spf13/pflag" "google.golang.org/grpc" "google.golang.org/grpc/credentials" @@ -682,10 +683,16 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, return net, nil } -func unstakedNetworkMsgValidators(selfID flow.Identifier) []network.MessageValidator { +func unstakedNetworkMsgValidators(log zerolog.Logger, idProvider id.IdentityProvider, selfID flow.Identifier) []network.MessageValidator { return []network.MessageValidator{ // filter out messages sent by this node itself validator.ValidateNotSender(selfID), + validator.NewAnyValidator( + validator.NewOriginValidator( + id.NewFilteredIdentifierProvider(filter.IsValidCurrentEpochParticipant, idProvider), + ), + validator.ValidateTarget(log, selfID), + ), } } diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 4d00a44895c..827f056072e 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -112,7 +112,7 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NodeConfig.NetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(builder.NodeID) + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, builder.NodeID) middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 31cda8ac094..ea50d7e93a7 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -137,7 +137,7 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) diff --git a/network/validator/anyValidator.go b/network/validator/anyValidator.go new file mode 100644 index 00000000000..0639de95bbb --- /dev/null +++ b/network/validator/anyValidator.go @@ -0,0 +1,28 @@ +package validator + +import ( + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" +) + +var _ network.MessageValidator = &AnyValidator{} + +// AnyValidator returns true if any of the given validators returns true +type AnyValidator struct { + validators []network.MessageValidator +} + +func NewAnyValidator(validators ...network.MessageValidator) network.MessageValidator { + return &AnyValidator{ + validators: validators, + } +} + +func (v AnyValidator) Validate(msg message.Message) bool { + for _, validator := range v.validators { + if validator.Validate(msg) { + return true + } + } + return false +} diff --git a/network/validator/originValidator.go b/network/validator/originValidator.go new file mode 100644 index 00000000000..250f464d6d5 --- /dev/null +++ b/network/validator/originValidator.go @@ -0,0 +1,24 @@ +package validator + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" +) + +var _ network.MessageValidator = &OriginValidator{} + +// OriginValidator returns true if the sender of the message is among the set of identifiers +// returned by the given IdentifierProvider +type OriginValidator struct { + idProvider id.IdentifierProvider +} + +func NewOriginValidator(provider id.IdentifierProvider) network.MessageValidator { + return &OriginValidator{provider} +} + +func (v OriginValidator) Validate(msg message.Message) bool { + return v.idProvider.Identifiers().Contains(flow.HashToID(msg.OriginID)) +} From 2ac44b7b833fe22c942637e6a68891bd009598e2 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 00:31:13 -0700 Subject: [PATCH 172/291] recover sync engine participants provider --- .../unstaked_access_node_builder.go | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index ea50d7e93a7..54313ce2b21 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -2,6 +2,7 @@ package node_builder import ( "context" + "errors" "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" @@ -109,12 +110,17 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { - //anb. - // Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - // // use the default identifier provider - // node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() - // return nil - // }) + anb. + Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + middleware, ok := node.Middleware.(*p2p.Middleware) + if !ok { + return errors.New("middleware was of unexpected type") + } + // use the default identifier provider + anb.SyncEngineParticipantsProvider = middleware.IdentifierProvider() + return nil + }) + anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } From f1cc3c429d6243d287ad489cb02347c7c606e87e Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 11:02:50 -0700 Subject: [PATCH 173/291] using the unstaked networking key for the consesus follower --- .../unstaked_access_node_builder.go | 6 ------ .../tests/access/unstaked_node_test.go | 19 ++++++++++++++++--- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 54313ce2b21..1ce91142d40 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -70,12 +70,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() - //builder.EnqueueMetricsServerInit() - // - //builder.RegisterBadgerMetrics() - // - //builder.EnqueueTracer() - builder.PreInit(builder.initUnstakedLocal()) return builder diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index daddce23936..6bdac10eb30 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -2,6 +2,7 @@ package access import ( "context" + "crypto/rand" "fmt" "testing" "time" @@ -10,6 +11,8 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" @@ -69,7 +72,7 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { flow.RoleAccess, testnet.WithID(suite.stakedID), testnet.SupportsUnstakedNodes(), - testnet.WithLogLevel(zerolog.InfoLevel), + testnet.WithLogLevel(zerolog.TraceLevel), ) collectionConfigs := []func(*testnet.NodeConfig){ @@ -100,8 +103,7 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { stakedConfig, } - // consensus follower - unstakedKey, err := unittest.NetworkingKey() + unstakedKey, err := UnstakedNetworkingKey() require.NoError(suite.T(), err) // TODO: derive node id from the key suite.unstakedID = unittest.IdentifierFixture() @@ -110,9 +112,20 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } + // consensus follower conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) suite.follower.AddOnBlockFinalizedConsumer(suite.OnBlockFinalizedConsumer) } + +// TODO: Move this to unittest and resolve the circular dependency issue +func UnstakedNetworkingKey() (crypto.PrivateKey, error) { + seed := make([]byte, crypto.KeyGenSeedMinLenECDSASecp256k1) + n, err := rand.Read(seed) + if err != nil || n != crypto.KeyGenSeedMinLenECDSASecp256k1 { + return nil, err + } + return utils.GenerateUnstakedNetworkingKey(unittest.SeedFixture(n)) +} From 290e0e8ecdbc7b8df3b2e529e5775c99ffb0e98a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:06:10 -0700 Subject: [PATCH 174/291] chance consensus follower to use build function --- follower/consensus_follower.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 33a52064bf6..f35ab03689e 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -96,7 +96,7 @@ func buildAccessNode(accessNodeOptions []access.Option) *access.UnstakedAccessNo nodeBuilder := access.NewUnstakedAccessNodeBuilder(anb) nodeBuilder.Initialize() - nodeBuilder.BuildConsensusFollower() + nodeBuilder.Build() return nodeBuilder } From 15a20bbb33f4dbdc9d3cfd9600a80048ddd63953 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:27:30 -0700 Subject: [PATCH 175/291] derive node id from key --- integration/testnet/network.go | 19 ++++++++++++------- .../tests/access/unstaked_node_test.go | 4 +--- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 9b14f56bdc4..2a4c4f34f92 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -37,6 +37,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" + "github.com/onflow/flow-go/network/p2p" clusterstate "github.com/onflow/flow-go/state/cluster" "github.com/onflow/flow-go/state/protocol/inmem" "github.com/onflow/flow-go/utils/io" @@ -219,16 +220,20 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier + nodeID flow.Identifier networkingPrivKey crypto.PrivateKey - stakedNodeID flow.Identifier + stakedNodeID flow.Identifier } -func NewConsensusFollowerConfig(networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier, nodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier) ConsensusFollowerConfig { + pid, err := p2p.ExtractPeerID(networkingPrivKey.PublicKey()) + assert.NoError(t, err) + nodeID, err := p2p.NewUnstakedNetworkIDTranslator().GetFlowID(pid) + assert.NoError(t, err) return ConsensusFollowerConfig{ networkingPrivKey: networkingPrivKey, - stakedNodeID: stakedNodeID, - nodeID: nodeID, // TODO: remove this and derive it from the key instead + stakedNodeID: stakedNodeID, + nodeID: nodeID, } } @@ -515,8 +520,8 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, port := uint(portU64) bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ - Host: "localhost", - Port: port, + Host: "localhost", + Port: port, NetworkPublicKey: stakedANContainer.NetworkPubKey(), } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 6bdac10eb30..c4ff35d242b 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -105,11 +105,9 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { unstakedKey, err := UnstakedNetworkingKey() require.NoError(suite.T(), err) - // TODO: derive node id from the key - suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), + testnet.NewConsensusFollowerConfig(suite.T(), unstakedKey, suite.stakedID), } // consensus follower From 5c10daa3f002ba7c8d6c8eabd6374c99ef30506a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:30:05 -0700 Subject: [PATCH 176/291] remove comment --- integration/testnet/network.go | 1 - 1 file changed, 1 deletion(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 2a4c4f34f92..a27410ddef7 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -741,7 +741,6 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( - // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address From 1175e0724b7b5bf23c6f9b5163efc07529762b90 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:42:09 -0700 Subject: [PATCH 177/291] remove topology --- .../node_builder/staked_access_node_builder.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 827f056072e..10068570bb9 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -117,17 +117,9 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront - top, err := topology.NewTopicBasedTopology( - builder.NodeID, - builder.Logger, - builder.State, - ) - if err != nil { - return nil, fmt.Errorf("could not create topology: %w", err) - } - topologyCache := topology.NewCache(builder.Logger, top) + top := topology.EmptyListTopology{} - network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, topologyCache) + network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, top) builder.MustNot(err) builder.Network = network From 5bef8e7a812e7056459085d6c5ec8804db0d74f1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:47:51 -0700 Subject: [PATCH 178/291] fix node ID --- integration/testnet/network.go | 24 +++++++++---------- .../tests/access/unstaked_node_test.go | 2 ++ 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index a27410ddef7..26489f23982 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -220,9 +220,9 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier - networkingPrivKey crypto.PrivateKey - stakedNodeID flow.Identifier + NodeID flow.Identifier + NetworkingPrivKey crypto.PrivateKey + StakedNodeID flow.Identifier } func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier) ConsensusFollowerConfig { @@ -231,9 +231,9 @@ func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKe nodeID, err := p2p.NewUnstakedNetworkIDTranslator().GetFlowID(pid) assert.NoError(t, err) return ConsensusFollowerConfig{ - networkingPrivKey: networkingPrivKey, - stakedNodeID: stakedNodeID, - nodeID: nodeID, + NetworkingPrivKey: networkingPrivKey, + StakedNodeID: stakedNodeID, + NodeID: nodeID, } } @@ -507,12 +507,12 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, var stakedANContainer *ContainerConfig // find the upstream Access node container for this follower engine for _, cont := range containers { - if cont.NodeID == followerConf.stakedNodeID { + if cont.NodeID == followerConf.StakedNodeID { stakedANContainer = &cont break } } - require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) + require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.NodeID.String()) portStr := net.AccessPorts[AccessNodeExternalNetworkPort] portU64, err := strconv.ParseUint(portStr, 10, 32) @@ -527,11 +527,11 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower, err := consensus_follower.NewConsensusFollower(followerConf.networkingPrivKey, bindAddr, + follower, err := consensus_follower.NewConsensusFollower(followerConf.NetworkingPrivKey, bindAddr, []consensus_follower.BootstrapNodeInfo{bootstrapNodeInfo}, opts...) // TODO: convert key to node ID? or just store with the network key as map key - net.ConsensusFollowers[followerConf.nodeID] = follower + net.ConsensusFollowers[followerConf.NodeID] = follower } // AddNode creates a node container with the given config and adds it to the @@ -741,11 +741,11 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( - conf.nodeID, + conf.NodeID, flow.RoleAccess, // use Access role "", // no address 0, // no stake - conf.networkingPrivKey, + conf.NetworkingPrivKey, dummyStakingKey, ) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index c4ff35d242b..b48db55734f 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -110,6 +110,8 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.NewConsensusFollowerConfig(suite.T(), unstakedKey, suite.stakedID), } + suite.unstakedID = followerConfigs[0].NodeID + // consensus follower conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) From 2464eb4eabeb74a67653e601e8491c64ac7c5717 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 11:53:34 -0700 Subject: [PATCH 179/291] removing unneeded flags --- integration/tests/access/unstaked_node_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index b48db55734f..03a9d1f8af6 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -79,15 +79,13 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), testnet.WithAdditionalFlag("--block-rate-delay=100ms"), testnet.WithLogLevel(zerolog.WarnLevel), - // TODO replace these with actual values - testnet.WithAdditionalFlag("--access-address=null"), } consensusConfigs := []func(config *testnet.NodeConfig){ testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), testnet.WithAdditionalFlag("--block-rate-delay=100ms"), - testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 1)), - testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 1)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 0)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 0)), testnet.WithLogLevel(zerolog.WarnLevel), } From b906c0469f4e08194c49028f249f8549eb45cf42 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 13:09:35 -0700 Subject: [PATCH 180/291] initializing middleware before initiliazing the sync follower for the unstaked node --- .../unstaked_access_node_builder.go | 134 ++++++++++-------- 1 file changed, 73 insertions(+), 61 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 1ce91142d40..044c58564a0 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -5,6 +5,7 @@ import ( "errors" "github.com/libp2p/go-libp2p-core/peer" + "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -24,67 +25,69 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } -func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { +func (anb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config - networkingKey := fnb.AccessNodeConfig.NetworkKey + networkingKey := anb.AccessNodeConfig.NetworkKey pubKey, err := p2p.LibP2PPublicKeyFromFlow(networkingKey.PublicKey()) - fnb.MustNot(err) + anb.MustNot(err) peerID, err := peer.IDFromPublicKey(pubKey) - fnb.MustNot(err) - fnb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) - fnb.MustNot(err) - fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig - fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node + anb.MustNot(err) + anb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) + anb.MustNot(err) + anb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig + anb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } -func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) +func (anb *UnstakedAccessNodeBuilder) InitIDProviders() { + anb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, anb.ProtocolEvents) if err != nil { return err } - fnb.IdentityProvider = idCache + anb.IdentityProvider = idCache - fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) + anb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) return nil }) } -func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { +func (anb *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) - builder.Cancel = cancel + anb.Cancel = cancel - builder.validateParams() + anb.validateParams() // if a network key has been passed in the init node info here - if builder.AccessNodeConfig.NetworkKey != nil { - builder.initNodeInfo() + if anb.AccessNodeConfig.NetworkKey != nil { + anb.initNodeInfo() } - builder.InitIDProviders() + anb.InitIDProviders() + + anb.enqueueMiddleware(ctx) - builder.enqueueUnstakedNetworkInit(ctx) + anb.enqueueUnstakedNetworkInit(ctx) - builder.enqueueConnectWithStakedAN() + anb.enqueueConnectWithStakedAN() - builder.PreInit(builder.initUnstakedLocal()) + anb.PreInit(anb.initUnstakedLocal()) - return builder + return anb } -func (builder *UnstakedAccessNodeBuilder) validateParams() { - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { - builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") +func (anb *UnstakedAccessNodeBuilder) validateParams() { + if len(anb.bootstrapNodeAddresses) != len(anb.bootstrapNodePublicKeys) { + anb.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") } } // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files -func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.NodeBuilder, node *cmd.NodeConfig) { +func (anb *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.NodeBuilder, node *cmd.NodeConfig) { return func(_ cmd.NodeBuilder, node *cmd.NodeConfig) { // for an unstaked node, set the identity here explicitly since it will not be found in the protocol state self := &flow.Identity{ @@ -92,21 +95,48 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N NetworkPubKey: node.NetworkKey.PublicKey(), StakingPubKey: nil, // no staking key needed for the unstaked node Role: flow.RoleAccess, // unstaked node can only run as an access node - Address: builder.BindAddr, + Address: anb.BindAddr, } me, err := local.New(self, nil) - builder.MustNot(err).Msg("could not initialize local") + anb.MustNot(err).Msg("could not initialize local") node.Me = me } } +// enqueueMiddleware enqueues the creation of the network middleware +// this needs to be done before sync engine participants module +func (anb *UnstakedAccessNodeBuilder) enqueueMiddleware(ctx context.Context) { + anb. + Module("network middleware", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) error { + + // NodeID for the unstaked node on the unstaked network + unstakedNodeID := node.NodeID + + // Networking key + unstakedNetworkKey := node.NetworkKey + + // Network Metrics + // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics + unstakedNetworkMetrics := metrics.NewNoopCollector() + + libP2PFactory, err := anb.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + anb.MustNot(err) + + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) + + anb.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + + return nil + }) +} + // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. - Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - middleware, ok := node.Middleware.(*p2p.Middleware) + Module("sync engine participants provider", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) error { + middleware, ok := anb.Middleware.(*p2p.Middleware) if !ok { return errors.New("middleware was of unexpected type") } @@ -120,44 +150,26 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { } // enqueueUnstakedNetworkInit enqueues the unstaked network component initialized for the unstaked node -func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { - - builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { +func (anb *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { - // NodeID for the unstaked node on the unstaked network - unstakedNodeID := node.NodeID - - // Networking key - unstakedNetworkKey := node.NetworkKey + anb.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { // Network Metrics // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) - builder.MustNot(err) - - msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) - - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) - builder.MustNot(err) - - builder.Network = network - builder.Middleware = middleware + network, err := anb.initNetwork(anb.Me, unstakedNetworkMetrics, anb.Middleware, nil) + anb.MustNot(err) - // for an unstaked node, the staked network and middleware is set to the same as the unstaked network and middlware - builder.Network = network - builder.Middleware = middleware + anb.Network = network - builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + anb.Logger.Info().Msgf("network will run on address: %s", anb.BindAddr) - idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) - builder.ProtocolEvents.AddConsumer(idEvents) + idEvents := gadgets.NewIdentityDeltas(anb.Middleware.UpdateNodeAddresses) + anb.ProtocolEvents.AddConsumer(idEvents) - return builder.Network, err + return anb.Network, err }) } @@ -167,8 +179,8 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // (https://github.com/libp2p/go-libp2p-pubsub/issues/442). This means that an unstaked AN could end up not being // discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need // of an explicit connect to the staked AN before the node attempts to subscribe to topics. -func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentities, builder.LibP2PNode, builder.Logger), nil +func (anb *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { + anb.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return newUpstreamConnector(anb.bootstrapIdentities, anb.LibP2PNode, anb.Logger), nil }) } From d3f684539f98447f68d8155eee96aedf77cce1f4 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 13:59:55 -0700 Subject: [PATCH 181/291] using a factory method for the SyncEngineParticipantsProvider to wait for middleware to have started for the unstaked an --- .../node_builder/access_node_builder.go | 37 ++++++++++--------- .../staked_access_node_builder.go | 19 ++++++---- .../unstaked_access_node_builder.go | 3 +- 3 files changed, 32 insertions(+), 27 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 9436f1fdb82..08f6155acbd 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -151,23 +151,24 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - LibP2PNode *p2p.Node - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + LibP2PNode *p2p.Node + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower + SyncEngineParticipantsProviderFactory func() id.IdentifierProvider // engines IngestEng *ingestion.Engine @@ -320,7 +321,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - node.SyncEngineIdentifierProvider, + builder.SyncEngineParticipantsProviderFactory(), ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 10068570bb9..22c05ee0f7b 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -39,14 +39,17 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { } fnb.IdentityProvider = idCache - fnb.SyncEngineIdentifierProvider = id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - p2p.NotEjectedFilter, - ), - idCache, - ) + // translator + fnb.SyncEngineParticipantsProviderFactory = func() id.IdentifierProvider { + return id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NotEjectedFilter, + ), + idCache, + ) + } fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 044c58564a0..eddfcb7ca47 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -9,6 +9,7 @@ import ( "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" @@ -141,7 +142,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { return errors.New("middleware was of unexpected type") } // use the default identifier provider - anb.SyncEngineParticipantsProvider = middleware.IdentifierProvider() + anb.SyncEngineParticipantsProviderFactory = func() id.IdentifierProvider { return middleware.IdentifierProvider() } return nil }) From c2d6f7caafad624e772c2df18458e245064f22c6 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 14:06:43 -0700 Subject: [PATCH 182/291] adding comment --- cmd/access/node_builder/access_node_builder.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 08f6155acbd..0f67a9ddf51 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -168,6 +168,9 @@ type FlowAccessNodeBuilder struct { Finalized *flow.Header Pending []*flow.Header FollowerCore module.HotStuffFollower + // for the untsaked access node, the sync engine participants provider is the libp2p peer store which is not + // available until after the network has started. Hence, a factory function that needs to be called just before + // creating the sync engine SyncEngineParticipantsProviderFactory func() id.IdentifierProvider // engines From 37300b2fc337b34e3bdd805617a7d7e0391dd1fd Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 16:46:41 -0700 Subject: [PATCH 183/291] wip --- .../node_builder/access_node_builder.go | 34 +++++++++---------- cmd/scaffold.go | 1 - integration/tests/access/access_api_test.go | 2 +- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 0f67a9ddf51..0c1392bfe80 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -151,23 +151,23 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - LibP2PNode *p2p.Node - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + LibP2PNode *p2p.Node + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower // for the untsaked access node, the sync engine participants provider is the libp2p peer store which is not // available until after the network has started. Hence, a factory function that needs to be called just before // creating the sync engine diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 4eb4718c2ad..b73ac13998e 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -21,7 +21,6 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" diff --git a/integration/tests/access/access_api_test.go b/integration/tests/access/access_api_test.go index 24fd024f720..56e437afc26 100644 --- a/integration/tests/access/access_api_test.go +++ b/integration/tests/access/access_api_test.go @@ -66,7 +66,7 @@ func (suite *AccessSuite) SetupTest() { collConfig := testnet.NewNodeConfig(flow.RoleCollection, testnet.WithID(collID)) nodeConfigs = append(nodeConfigs, collConfig) - conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs, nil) + conf := testnet.NewNetworkConfig("access_api_test", nodeConfigs) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) // start the network From 45c2d3e6fc965b076bf958c0aa0b0d14a2500e17 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 16:49:49 -0700 Subject: [PATCH 184/291] go mod tidy & lints --- .../node_builder/access_node_builder.go | 1 + integration/go.sum | 10 + module/mocks/network.go | 64 +++--- storage/mocks/storage.go | 208 +++++++++--------- 4 files changed, 145 insertions(+), 138 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 0c1392bfe80..a0a320778be 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -37,6 +37,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" diff --git a/integration/go.sum b/integration/go.sum index 9e9720679e5..08030e135cc 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -88,6 +88,7 @@ github.com/VictoriaMetrics/fastcache v1.5.7/go.mod h1:ptDBkNMQI4RtmVo8VS/XwRY6Ro github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= +github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af h1:wVe6/Ea46ZMeNkQjjBW6xcqyQA/j5e0D6GytH95g0gQ= github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -127,6 +128,7 @@ github.com/blang/semver v3.1.0+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnweb github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40/go.mod h1:8rLXio+WjiTceGBHIoTvn60HIbs7Hm7bcHjyrSqYB9c= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= +github.com/bsipos/thist v1.0.0 h1:vZ3W5/ZnT54s4LHeonTCbnzCb20ERlJUnhiwXoGpsbY= github.com/bsipos/thist v1.0.0/go.mod h1:7i0xwRua1/bmUxcxi2xAxaFL895rLtOpKUwnw3NrT8I= github.com/btcsuite/btcd v0.0.0-20171128150713-2e60448ffcc6/go.mod h1:Dmm/EzmjnCiweXmzRIAiUWCInVmPgjkzgv5k4tVyXiQ= github.com/btcsuite/btcd v0.0.0-20190213025234-306aecffea32/go.mod h1:DrZx5ec/dmnfpw9KyYoQyYo7d0KEvTkk/5M/vbZjAr8= @@ -287,6 +289,7 @@ github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5/go.mod h1:VvhXpOYNQvB+ github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= github.com/flynn/noise v1.0.0 h1:DlTHqmzmvcEiKj+4RYo/imoswx/4r6iBlCMfVtrMXpQ= github.com/flynn/noise v1.0.0/go.mod h1:xbMo+0i6+IGbYdJhF31t2eR1BIU0CYc12+BNAKwUTag= +github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90 h1:WXb3TSNmHp2vHoCroCIB1foO/yQ36swABL8aOVeDpgg= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= @@ -375,6 +378,7 @@ github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXP github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 h1:DACJavvAHhabrF08vX0COfcOBJRhZ8lUbR+ZWIs0Y5g= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -643,6 +647,7 @@ github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfV github.com/julienschmidt/httprouter v1.1.1-0.20170430222011-975b5c4c7c21/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5 h1:PJr+ZMXIecYc1Ey2zucXdR73SMBtgjPgwa31099IMv0= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jwilder/encoding v0.0.0-20170811194829-b4e1701a28ef/go.mod h1:Ct9fl0F6iIOGgxJ5npU/IUOhOhqlVrGjyIZc8/MagT0= github.com/k0kubun/go-ansi v0.0.0-20180517002512-3bf9e2903213/go.mod h1:vNUNkEQ1e29fT/6vq2aBdFsgNPmy8qMdSay1npru+Sw= @@ -1081,6 +1086,7 @@ github.com/multiformats/go-varint v0.0.5/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXS github.com/multiformats/go-varint v0.0.6 h1:gk85QWKxh3TazbLxED/NlDVv8+q+ReFJk7Y2W/KhfNY= github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/naoina/go-stringutil v0.1.0/go.mod h1:XJ2SJL9jCtBh+P9q5btrd/Ylo8XwT/h1USek5+NqSA0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416/go.mod h1:NBIhNtsFMo3G2szEBne+bO4gS192HuIYRqfvOWb4i1E= @@ -1521,6 +1527,7 @@ golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6 h1:QE6XYQK6naiK1EPAe1g/ILLxN golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b h1:+qEpEAPhDZ1o0x3tHzZTQDArnOixOzGD9HUJfcg0mb4= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -1715,6 +1722,7 @@ golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744 h1:yhBbb4IRs2HS9PPlAg6DMC6mU golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d h1:SZxvLBoTP5yHO3Frd4z4vrF+DBX9vMVanchswa69toE= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1812,6 +1820,7 @@ gonum.org/v1/gonum v0.6.1/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0 h1:OE9mWmgKkjJyEmDAAtGMPjXu+YNeGvK9VTSHY6+Qihc= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= +gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b h1:Qh4dB5D/WpoUUp3lSod7qgoyEHbDGPUWjIbnqdqqe1k= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= @@ -1978,6 +1987,7 @@ k8s.io/kubernetes v1.13.0/go.mod h1:ocZa8+6APFNC2tX1DZASIbocyYT5jHzqFVsY5aoB7Jk= pgregory.net/rapid v0.4.7 h1:MTNRktPuv5FNqOO151TM9mDTa+XHcX6ypYeISDVD14g= pgregory.net/rapid v0.4.7/go.mod h1:UYpPVyjFHzYBGHIxLFoupi8vwk6rXNzRY9OMvVxFIOU= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/pdf v0.1.1 h1:k1MczvYDUvJBe93bYd7wrZLLUEcLZAuF824/I4e5Xr4= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/module/mocks/network.go b/module/mocks/network.go index 305cee9f346..38f65a552ce 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -5,40 +5,38 @@ package mocks import ( - reflect "reflect" - gomock "github.com/golang/mock/gomock" - crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" network "github.com/onflow/flow-go/network" + reflect "reflect" ) -// MockNetwork is a mock of Network interface. +// MockNetwork is a mock of Network interface type MockNetwork struct { ctrl *gomock.Controller recorder *MockNetworkMockRecorder } -// MockNetworkMockRecorder is the mock recorder for MockNetwork. +// MockNetworkMockRecorder is the mock recorder for MockNetwork type MockNetworkMockRecorder struct { mock *MockNetwork } -// NewMockNetwork creates a new mock instance. +// NewMockNetwork creates a new mock instance func NewMockNetwork(ctrl *gomock.Controller) *MockNetwork { mock := &MockNetwork{ctrl: ctrl} mock.recorder = &MockNetworkMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockNetwork) EXPECT() *MockNetworkMockRecorder { return m.recorder } -// Register mocks base method. +// Register mocks base method func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (network.Conduit, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Register", arg0, arg1) @@ -47,36 +45,36 @@ func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (netwo return ret0, ret1 } -// Register indicates an expected call of Register. +// Register indicates an expected call of Register func (mr *MockNetworkMockRecorder) Register(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Register", reflect.TypeOf((*MockNetwork)(nil).Register), arg0, arg1) } -// MockLocal is a mock of Local interface. +// MockLocal is a mock of Local interface type MockLocal struct { ctrl *gomock.Controller recorder *MockLocalMockRecorder } -// MockLocalMockRecorder is the mock recorder for MockLocal. +// MockLocalMockRecorder is the mock recorder for MockLocal type MockLocalMockRecorder struct { mock *MockLocal } -// NewMockLocal creates a new mock instance. +// NewMockLocal creates a new mock instance func NewMockLocal(ctrl *gomock.Controller) *MockLocal { mock := &MockLocal{ctrl: ctrl} mock.recorder = &MockLocalMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockLocal) EXPECT() *MockLocalMockRecorder { return m.recorder } -// Address mocks base method. +// Address mocks base method func (m *MockLocal) Address() string { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Address") @@ -84,13 +82,13 @@ func (m *MockLocal) Address() string { return ret0 } -// Address indicates an expected call of Address. +// Address indicates an expected call of Address func (mr *MockLocalMockRecorder) Address() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Address", reflect.TypeOf((*MockLocal)(nil).Address)) } -// NodeID mocks base method. +// NodeID mocks base method func (m *MockLocal) NodeID() flow.Identifier { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NodeID") @@ -98,13 +96,13 @@ func (m *MockLocal) NodeID() flow.Identifier { return ret0 } -// NodeID indicates an expected call of NodeID. +// NodeID indicates an expected call of NodeID func (mr *MockLocalMockRecorder) NodeID() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NodeID", reflect.TypeOf((*MockLocal)(nil).NodeID)) } -// NotMeFilter mocks base method. +// NotMeFilter mocks base method func (m *MockLocal) NotMeFilter() flow.IdentityFilter { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NotMeFilter") @@ -112,13 +110,13 @@ func (m *MockLocal) NotMeFilter() flow.IdentityFilter { return ret0 } -// NotMeFilter indicates an expected call of NotMeFilter. +// NotMeFilter indicates an expected call of NotMeFilter func (mr *MockLocalMockRecorder) NotMeFilter() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NotMeFilter", reflect.TypeOf((*MockLocal)(nil).NotMeFilter)) } -// Sign mocks base method. +// Sign mocks base method func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Sign", arg0, arg1) @@ -127,13 +125,13 @@ func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error return ret0, ret1 } -// Sign indicates an expected call of Sign. +// Sign indicates an expected call of Sign func (mr *MockLocalMockRecorder) Sign(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sign", reflect.TypeOf((*MockLocal)(nil).Sign), arg0, arg1) } -// SignFunc mocks base method. +// SignFunc mocks base method func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.PrivateKey, []byte, hash.Hasher) (crypto.Signature, error)) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SignFunc", arg0, arg1, arg2) @@ -142,66 +140,66 @@ func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.Pri return ret0, ret1 } -// SignFunc indicates an expected call of SignFunc. +// SignFunc indicates an expected call of SignFunc func (mr *MockLocalMockRecorder) SignFunc(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignFunc", reflect.TypeOf((*MockLocal)(nil).SignFunc), arg0, arg1, arg2) } -// MockRequester is a mock of Requester interface. +// MockRequester is a mock of Requester interface type MockRequester struct { ctrl *gomock.Controller recorder *MockRequesterMockRecorder } -// MockRequesterMockRecorder is the mock recorder for MockRequester. +// MockRequesterMockRecorder is the mock recorder for MockRequester type MockRequesterMockRecorder struct { mock *MockRequester } -// NewMockRequester creates a new mock instance. +// NewMockRequester creates a new mock instance func NewMockRequester(ctrl *gomock.Controller) *MockRequester { mock := &MockRequester{ctrl: ctrl} mock.recorder = &MockRequesterMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockRequester) EXPECT() *MockRequesterMockRecorder { return m.recorder } -// EntityByID mocks base method. +// EntityByID mocks base method func (m *MockRequester) EntityByID(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "EntityByID", arg0, arg1) } -// EntityByID indicates an expected call of EntityByID. +// EntityByID indicates an expected call of EntityByID func (mr *MockRequesterMockRecorder) EntityByID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EntityByID", reflect.TypeOf((*MockRequester)(nil).EntityByID), arg0, arg1) } -// Force mocks base method. +// Force mocks base method func (m *MockRequester) Force() { m.ctrl.T.Helper() m.ctrl.Call(m, "Force") } -// Force indicates an expected call of Force. +// Force indicates an expected call of Force func (mr *MockRequesterMockRecorder) Force() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Force", reflect.TypeOf((*MockRequester)(nil).Force)) } -// Query mocks base method. +// Query mocks base method func (m *MockRequester) Query(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Query", arg0, arg1) } -// Query indicates an expected call of Query. +// Query indicates an expected call of Query func (mr *MockRequesterMockRecorder) Query(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockRequester)(nil).Query), arg0, arg1) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 969044a53ee..2967a3e5cce 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -5,39 +5,37 @@ package mocks import ( - reflect "reflect" - gomock "github.com/golang/mock/gomock" - flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" + reflect "reflect" ) -// MockBlocks is a mock of Blocks interface. +// MockBlocks is a mock of Blocks interface type MockBlocks struct { ctrl *gomock.Controller recorder *MockBlocksMockRecorder } -// MockBlocksMockRecorder is the mock recorder for MockBlocks. +// MockBlocksMockRecorder is the mock recorder for MockBlocks type MockBlocksMockRecorder struct { mock *MockBlocks } -// NewMockBlocks creates a new mock instance. +// NewMockBlocks creates a new mock instance func NewMockBlocks(ctrl *gomock.Controller) *MockBlocks { mock := &MockBlocks{ctrl: ctrl} mock.recorder = &MockBlocksMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockBlocks) EXPECT() *MockBlocksMockRecorder { return m.recorder } -// ByCollectionID mocks base method. +// ByCollectionID mocks base method func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByCollectionID", arg0) @@ -46,13 +44,13 @@ func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByCollectionID indicates an expected call of ByCollectionID. +// ByCollectionID indicates an expected call of ByCollectionID func (mr *MockBlocksMockRecorder) ByCollectionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByCollectionID", reflect.TypeOf((*MockBlocks)(nil).ByCollectionID), arg0) } -// ByHeight mocks base method. +// ByHeight mocks base method func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -61,13 +59,13 @@ func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight. +// ByHeight indicates an expected call of ByHeight func (mr *MockBlocksMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockBlocks)(nil).ByHeight), arg0) } -// ByID mocks base method. +// ByID mocks base method func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -76,13 +74,13 @@ func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID. +// ByID indicates an expected call of ByID func (mr *MockBlocksMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockBlocks)(nil).ByID), arg0) } -// GetLastFullBlockHeight mocks base method. +// GetLastFullBlockHeight mocks base method func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetLastFullBlockHeight") @@ -91,13 +89,13 @@ func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { return ret0, ret1 } -// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight. +// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight func (mr *MockBlocksMockRecorder) GetLastFullBlockHeight() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).GetLastFullBlockHeight)) } -// IndexBlockForCollections mocks base method. +// IndexBlockForCollections mocks base method func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexBlockForCollections", arg0, arg1) @@ -105,13 +103,13 @@ func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow. return ret0 } -// IndexBlockForCollections indicates an expected call of IndexBlockForCollections. +// IndexBlockForCollections indicates an expected call of IndexBlockForCollections func (mr *MockBlocksMockRecorder) IndexBlockForCollections(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexBlockForCollections", reflect.TypeOf((*MockBlocks)(nil).IndexBlockForCollections), arg0, arg1) } -// Store mocks base method. +// Store mocks base method func (m *MockBlocks) Store(arg0 *flow.Block) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -119,13 +117,13 @@ func (m *MockBlocks) Store(arg0 *flow.Block) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockBlocksMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockBlocks)(nil).Store), arg0) } -// StoreTx mocks base method. +// StoreTx mocks base method func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreTx", arg0) @@ -133,13 +131,13 @@ func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { return ret0 } -// StoreTx indicates an expected call of StoreTx. +// StoreTx indicates an expected call of StoreTx func (mr *MockBlocksMockRecorder) StoreTx(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreTx", reflect.TypeOf((*MockBlocks)(nil).StoreTx), arg0) } -// UpdateLastFullBlockHeight mocks base method. +// UpdateLastFullBlockHeight mocks base method func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "UpdateLastFullBlockHeight", arg0) @@ -147,36 +145,36 @@ func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { return ret0 } -// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight. +// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight func (mr *MockBlocksMockRecorder) UpdateLastFullBlockHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).UpdateLastFullBlockHeight), arg0) } -// MockHeaders is a mock of Headers interface. +// MockHeaders is a mock of Headers interface type MockHeaders struct { ctrl *gomock.Controller recorder *MockHeadersMockRecorder } -// MockHeadersMockRecorder is the mock recorder for MockHeaders. +// MockHeadersMockRecorder is the mock recorder for MockHeaders type MockHeadersMockRecorder struct { mock *MockHeaders } -// NewMockHeaders creates a new mock instance. +// NewMockHeaders creates a new mock instance func NewMockHeaders(ctrl *gomock.Controller) *MockHeaders { mock := &MockHeaders{ctrl: ctrl} mock.recorder = &MockHeadersMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockHeaders) EXPECT() *MockHeadersMockRecorder { return m.recorder } -// BatchIndexByChunkID mocks base method. +// BatchIndexByChunkID mocks base method func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchIndexByChunkID", arg0, arg1, arg2) @@ -184,13 +182,13 @@ func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 stora return ret0 } -// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID. +// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID func (mr *MockHeadersMockRecorder) BatchIndexByChunkID(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchIndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).BatchIndexByChunkID), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -199,13 +197,13 @@ func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockHeadersMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockHeaders)(nil).ByBlockID), arg0) } -// ByHeight mocks base method. +// ByHeight mocks base method func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -214,13 +212,13 @@ func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight. +// ByHeight indicates an expected call of ByHeight func (mr *MockHeadersMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockHeaders)(nil).ByHeight), arg0) } -// ByParentID mocks base method. +// ByParentID mocks base method func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByParentID", arg0) @@ -229,13 +227,13 @@ func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { return ret0, ret1 } -// ByParentID indicates an expected call of ByParentID. +// ByParentID indicates an expected call of ByParentID func (mr *MockHeadersMockRecorder) ByParentID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByParentID", reflect.TypeOf((*MockHeaders)(nil).ByParentID), arg0) } -// IDByChunkID mocks base method. +// IDByChunkID mocks base method func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IDByChunkID", arg0) @@ -244,13 +242,13 @@ func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) return ret0, ret1 } -// IDByChunkID indicates an expected call of IDByChunkID. +// IDByChunkID indicates an expected call of IDByChunkID func (mr *MockHeadersMockRecorder) IDByChunkID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDByChunkID", reflect.TypeOf((*MockHeaders)(nil).IDByChunkID), arg0) } -// IndexByChunkID mocks base method. +// IndexByChunkID mocks base method func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexByChunkID", arg0, arg1) @@ -258,13 +256,13 @@ func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { return ret0 } -// IndexByChunkID indicates an expected call of IndexByChunkID. +// IndexByChunkID indicates an expected call of IndexByChunkID func (mr *MockHeadersMockRecorder) IndexByChunkID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).IndexByChunkID), arg0, arg1) } -// Store mocks base method. +// Store mocks base method func (m *MockHeaders) Store(arg0 *flow.Header) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -272,36 +270,36 @@ func (m *MockHeaders) Store(arg0 *flow.Header) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockHeadersMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockHeaders)(nil).Store), arg0) } -// MockPayloads is a mock of Payloads interface. +// MockPayloads is a mock of Payloads interface type MockPayloads struct { ctrl *gomock.Controller recorder *MockPayloadsMockRecorder } -// MockPayloadsMockRecorder is the mock recorder for MockPayloads. +// MockPayloadsMockRecorder is the mock recorder for MockPayloads type MockPayloadsMockRecorder struct { mock *MockPayloads } -// NewMockPayloads creates a new mock instance. +// NewMockPayloads creates a new mock instance func NewMockPayloads(ctrl *gomock.Controller) *MockPayloads { mock := &MockPayloads{ctrl: ctrl} mock.recorder = &MockPayloadsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockPayloads) EXPECT() *MockPayloadsMockRecorder { return m.recorder } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -310,13 +308,13 @@ func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockPayloadsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockPayloads)(nil).ByBlockID), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -324,36 +322,36 @@ func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockPayloadsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockPayloads)(nil).Store), arg0, arg1) } -// MockCollections is a mock of Collections interface. +// MockCollections is a mock of Collections interface type MockCollections struct { ctrl *gomock.Controller recorder *MockCollectionsMockRecorder } -// MockCollectionsMockRecorder is the mock recorder for MockCollections. +// MockCollectionsMockRecorder is the mock recorder for MockCollections type MockCollectionsMockRecorder struct { mock *MockCollections } -// NewMockCollections creates a new mock instance. +// NewMockCollections creates a new mock instance func NewMockCollections(ctrl *gomock.Controller) *MockCollections { mock := &MockCollections{ctrl: ctrl} mock.recorder = &MockCollectionsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockCollections) EXPECT() *MockCollectionsMockRecorder { return m.recorder } -// ByID mocks base method. +// ByID mocks base method func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -362,13 +360,13 @@ func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID. +// ByID indicates an expected call of ByID func (mr *MockCollectionsMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockCollections)(nil).ByID), arg0) } -// LightByID mocks base method. +// LightByID mocks base method func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByID", arg0) @@ -377,13 +375,13 @@ func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection return ret0, ret1 } -// LightByID indicates an expected call of LightByID. +// LightByID indicates an expected call of LightByID func (mr *MockCollectionsMockRecorder) LightByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByID", reflect.TypeOf((*MockCollections)(nil).LightByID), arg0) } -// LightByTransactionID mocks base method. +// LightByTransactionID mocks base method func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByTransactionID", arg0) @@ -392,13 +390,13 @@ func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.Ligh return ret0, ret1 } -// LightByTransactionID indicates an expected call of LightByTransactionID. +// LightByTransactionID indicates an expected call of LightByTransactionID func (mr *MockCollectionsMockRecorder) LightByTransactionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByTransactionID", reflect.TypeOf((*MockCollections)(nil).LightByTransactionID), arg0) } -// Remove mocks base method. +// Remove mocks base method func (m *MockCollections) Remove(arg0 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Remove", arg0) @@ -406,13 +404,13 @@ func (m *MockCollections) Remove(arg0 flow.Identifier) error { return ret0 } -// Remove indicates an expected call of Remove. +// Remove indicates an expected call of Remove func (mr *MockCollectionsMockRecorder) Remove(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockCollections)(nil).Remove), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockCollections) Store(arg0 *flow.Collection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -420,13 +418,13 @@ func (m *MockCollections) Store(arg0 *flow.Collection) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockCollectionsMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCollections)(nil).Store), arg0) } -// StoreLight mocks base method. +// StoreLight mocks base method func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLight", arg0) @@ -434,13 +432,13 @@ func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { return ret0 } -// StoreLight indicates an expected call of StoreLight. +// StoreLight indicates an expected call of StoreLight func (mr *MockCollectionsMockRecorder) StoreLight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLight", reflect.TypeOf((*MockCollections)(nil).StoreLight), arg0) } -// StoreLightAndIndexByTransaction mocks base method. +// StoreLightAndIndexByTransaction mocks base method func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLightAndIndexByTransaction", arg0) @@ -448,36 +446,36 @@ func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollec return ret0 } -// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction. +// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction func (mr *MockCollectionsMockRecorder) StoreLightAndIndexByTransaction(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLightAndIndexByTransaction", reflect.TypeOf((*MockCollections)(nil).StoreLightAndIndexByTransaction), arg0) } -// MockCommits is a mock of Commits interface. +// MockCommits is a mock of Commits interface type MockCommits struct { ctrl *gomock.Controller recorder *MockCommitsMockRecorder } -// MockCommitsMockRecorder is the mock recorder for MockCommits. +// MockCommitsMockRecorder is the mock recorder for MockCommits type MockCommitsMockRecorder struct { mock *MockCommits } -// NewMockCommits creates a new mock instance. +// NewMockCommits creates a new mock instance func NewMockCommits(ctrl *gomock.Controller) *MockCommits { mock := &MockCommits{ctrl: ctrl} mock.recorder = &MockCommitsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -485,13 +483,13 @@ func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockCommitsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockCommits)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -500,13 +498,13 @@ func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, err return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockCommitsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockCommits)(nil).ByBlockID), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -514,36 +512,36 @@ func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) err return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockCommitsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCommits)(nil).Store), arg0, arg1) } -// MockEvents is a mock of Events interface. +// MockEvents is a mock of Events interface type MockEvents struct { ctrl *gomock.Controller recorder *MockEventsMockRecorder } -// MockEventsMockRecorder is the mock recorder for MockEvents. +// MockEventsMockRecorder is the mock recorder for MockEvents type MockEventsMockRecorder struct { mock *MockEvents } -// NewMockEvents creates a new mock instance. +// NewMockEvents creates a new mock instance func NewMockEvents(ctrl *gomock.Controller) *MockEvents { mock := &MockEvents{ctrl: ctrl} mock.recorder = &MockEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockEvents) EXPECT() *MockEventsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -551,13 +549,13 @@ func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, ar return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -566,13 +564,13 @@ func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockEvents)(nil).ByBlockID), arg0) } -// ByBlockIDEventType mocks base method. +// ByBlockIDEventType mocks base method func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventType) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDEventType", arg0, arg1) @@ -581,13 +579,13 @@ func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventTyp return ret0, ret1 } -// ByBlockIDEventType indicates an expected call of ByBlockIDEventType. +// ByBlockIDEventType indicates an expected call of ByBlockIDEventType func (mr *MockEventsMockRecorder) ByBlockIDEventType(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDEventType", reflect.TypeOf((*MockEvents)(nil).ByBlockIDEventType), arg0, arg1) } -// ByBlockIDTransactionID mocks base method. +// ByBlockIDTransactionID mocks base method func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -596,36 +594,36 @@ func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow. return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID func (mr *MockEventsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockEvents)(nil).ByBlockIDTransactionID), arg0, arg1) } -// MockServiceEvents is a mock of ServiceEvents interface. +// MockServiceEvents is a mock of ServiceEvents interface type MockServiceEvents struct { ctrl *gomock.Controller recorder *MockServiceEventsMockRecorder } -// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents. +// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents type MockServiceEventsMockRecorder struct { mock *MockServiceEvents } -// NewMockServiceEvents creates a new mock instance. +// NewMockServiceEvents creates a new mock instance func NewMockServiceEvents(ctrl *gomock.Controller) *MockServiceEvents { mock := &MockServiceEvents{ctrl: ctrl} mock.recorder = &MockServiceEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -633,13 +631,13 @@ func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockServiceEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockServiceEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -648,36 +646,36 @@ func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockServiceEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockServiceEvents)(nil).ByBlockID), arg0) } -// MockTransactionResults is a mock of TransactionResults interface. +// MockTransactionResults is a mock of TransactionResults interface type MockTransactionResults struct { ctrl *gomock.Controller recorder *MockTransactionResultsMockRecorder } -// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults. +// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults type MockTransactionResultsMockRecorder struct { mock *MockTransactionResults } -// NewMockTransactionResults creates a new mock instance. +// NewMockTransactionResults creates a new mock instance func NewMockTransactionResults(ctrl *gomock.Controller) *MockTransactionResults { mock := &MockTransactionResults{ctrl: ctrl} mock.recorder = &MockTransactionResultsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -685,13 +683,13 @@ func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.Tr return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockTransactionResultsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockTransactionResults)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockIDTransactionID mocks base method. +// ByBlockIDTransactionID mocks base method func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) (*flow.TransactionResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -700,7 +698,7 @@ func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifi return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID func (mr *MockTransactionResultsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockTransactionResults)(nil).ByBlockIDTransactionID), arg0, arg1) From 26c56c21980ba4cc08ddc06ded4a62d7fd1623a7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 25 Aug 2021 15:09:07 -0400 Subject: [PATCH 185/291] fix integration tests compilation --- integration/localnet/bootstrap.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration/localnet/bootstrap.go b/integration/localnet/bootstrap.go index bb042e0972c..b55c2bf8d7e 100644 --- a/integration/localnet/bootstrap.go +++ b/integration/localnet/bootstrap.go @@ -208,7 +208,7 @@ func prepareNodes() []testnet.NodeConfig { for i := 0; i < unstakedAccessCount; i++ { nodes = append(nodes, testnet.NewNodeConfig(flow.RoleAccess, func(cfg *testnet.NodeConfig) { - cfg.Unstaked = true + cfg.SupportsUnstakedNodes = true })) } From db95a990941cbde33c9c07d50a07e1d211e40d63 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Wed, 25 Aug 2021 17:46:44 -0700 Subject: [PATCH 186/291] fixing scaffold --- cmd/scaffold.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index b73ac13998e..4eb4718c2ad 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -21,6 +21,7 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" From 078346eed32160f9545e99264ad6b172a68d9f1a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 19:47:29 -0700 Subject: [PATCH 187/291] fix flakiness in peerstore provider test --- network/test/peerstore_provider_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index 093c55072a0..f0d652efa02 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -50,7 +50,7 @@ func (suite *PeerStoreProviderTestSuite) SetupTest() { require.NoError(suite.T(), err) u := p2p.NewUnstakedNetworkIDTranslator() - suite.translator = p2p.NewHierarchicalIDTranslator(u, t) + suite.translator = p2p.NewHierarchicalIDTranslator(t, u) // emulate the middleware behavior in populating the testnode's peer store libp2pPeers := make([]peer.ID, peerCount) From 8170978bf2f348048abf6ec64dcba0e38e5dcf47 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 21:29:15 -0700 Subject: [PATCH 188/291] Fix flaky middleware test --- network/test/middleware_test.go | 107 +++++++++++++----------- network/test/peerstore_provider_test.go | 2 +- network/test/testUtil.go | 23 +++-- network/topology/topology_test.go | 10 +-- 4 files changed, 77 insertions(+), 65 deletions(-) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index c177ade677f..1fbaf45645b 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -58,6 +58,7 @@ func (co *tagsObserver) OnComplete() { type MiddlewareTestSuite struct { suite.Suite + sync.RWMutex size int // used to determine number of middlewares under test mws []*p2p.Middleware // used to keep track of middlewares under test ov []*mocknetwork.Overlay @@ -70,6 +71,7 @@ type MiddlewareTestSuite struct { // TestMiddlewareTestSuit runs all the test methods in this test suit func TestMiddlewareTestSuite(t *testing.T) { + t.Parallel() suite.Run(t, new(MiddlewareTestSuite)) } @@ -111,18 +113,66 @@ func (m *MiddlewareTestSuite) SetupTest() { } } +// TestUpdateNodeAddresses tests that the UpdateNodeAddresses method correctly updates +// the addresses of the staked network participants. +func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { + // create a new staked identity + ids, libP2PNodes, _ := GenerateIDs(m.T(), m.logger, 1, false, false) + mws, providers := GenerateMiddlewares(m.T(), m.logger, ids, libP2PNodes, false) + require.Len(m.T(), ids, 1) + require.Len(m.T(), providers, 1) + require.Len(m.T(), mws, 1) + newId := ids[0] + newMw := mws[0] + // newProvider := providers[0] + defer newMw.Stop() + + overlay := m.createOverlay() + overlay.On("Receive", + m.ids[0].NodeID, + mock.AnythingOfType("*message.Message"), + ).Return(nil) + assert.NoError(m.T(), newMw.Start(overlay)) + + idList := flow.IdentityList(append(m.ids, newId)) + + // needed to enable ID translation + m.providers[0].SetIdentities(idList) + m.mws[0].UpdateAllowList() + + msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") + + // message should fail to send because no address is known yet + // for the new identity + err := m.mws[0].SendDirect(msg, newId.NodeID) + require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) + + // update the addresses + m.Lock() + m.ids = idList + m.Unlock() + // newProvider.SetIdentities(idList) + // newMw.UpdateAllowList() + m.mws[0].UpdateNodeAddresses() + + // now the message should send successfully + err = m.mws[0].SendDirect(msg, newId.NodeID) + require.NoError(m.T(), err) +} + func (m *MiddlewareTestSuite) createOverlay() *mocknetwork.Overlay { overlay := &mocknetwork.Overlay{} - - overlay.On("Identities").Maybe().Return(func() flow.IdentityList { - return flow.IdentityList(m.ids) - }, nil) - overlay.On("Topology").Maybe().Return(func() flow.IdentityList { - return flow.IdentityList(m.ids) - }, nil) + overlay.On("Identities").Maybe().Return(m.getIds, nil) + overlay.On("Topology").Maybe().Return(m.getIds, nil) return overlay } +func (m *MiddlewareTestSuite) getIds() flow.IdentityList { + m.RLock() + defer m.RUnlock() + return flow.IdentityList(m.ids) +} + func (m *MiddlewareTestSuite) TearDownTest() { m.stopMiddlewares() } @@ -174,7 +224,6 @@ func (m *MiddlewareTestSuite) TestMultiPing() { // expectID and expectPayload are what we expect the receiver side to evaluate the // incoming ping against, it can be mocked or typed data func (m *MiddlewareTestSuite) Ping(expectID, expectPayload interface{}) { - ch := make(chan struct{}) // extracts sender id based on the mock option var err error @@ -475,48 +524,6 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { require.Error(m.Suite.T(), err) } -// TestUpdateNodeAddresses tests that the UpdateNodeAddresses method correctly updates -// the addresses of the staked network participants. -func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { - // create a new staked identity - ids, mws, _, providers := GenerateIDsAndMiddlewares(m.T(), 1, false, m.logger) - require.Len(m.T(), ids, 1) - require.Len(m.T(), providers, 1) - require.Len(m.T(), mws, 1) - newId := ids[0] - newMw := mws[0] - newProvider := providers[0] - - idList := flow.IdentityList(append(m.ids, newId)) - - newProvider.SetIdentities(idList) - overlay := m.createOverlay() - overlay.On("Receive", - m.ids[0].NodeID, - mock.AnythingOfType("*message.Message"), - ).Return(nil) - assert.NoError(m.T(), newMw.Start(overlay)) - - // needed to enable ID translation - m.providers[0].SetIdentities(idList) - m.mws[0].UpdateAllowList() - - msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") - - // message should fail to send because no address is known yet - // for the new identity - err := m.mws[0].SendDirect(msg, newId.NodeID) - require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) - - // update the addresses - m.ids = idList - m.mws[0].UpdateNodeAddresses() - - // now the message should send successfully - err = m.mws[0].SendDirect(msg, newId.NodeID) - require.NoError(m.T(), err) -} - // TestUnsubscribe tests that an engine can unsubscribe from a topic it was earlier subscribed to and stop receiving // messages. func (m *MiddlewareTestSuite) TestUnsubscribe() { diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index f0d652efa02..03ef68a4785 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -45,7 +45,7 @@ func (suite *PeerStoreProviderTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) ctx := context.Background() - suite.ids, suite.nodes, _ = GenerateIDs(suite.T(), suite.logger, nodeCount, !DryRun) + suite.ids, suite.nodes, _ = GenerateIDs(suite.T(), suite.logger, nodeCount, !DryRun, true) t, err := p2p.NewFixedTableIdentityTranslator(suite.ids) require.NoError(suite.T(), err) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 13d5fd80554..0189bbc20a0 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -93,7 +93,7 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. -func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Node, []observable.Observable) { +func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode, connGating bool, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Node, []observable.Observable) { libP2PNodes := make([]*p2p.Node, n) tagObservables := make([]observable.Observable, n) @@ -107,7 +107,7 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op port := "0" if !dryRunMode { - libP2PNodes[i], tagObservables[i] = generateLibP2PNode(t, logger, *id, key) + libP2PNodes[i], tagObservables[i] = generateLibP2PNode(t, logger, *id, key, connGating) _, port, err = libP2PNodes[i].GetIPPort() require.NoError(t, err) @@ -121,7 +121,7 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*UpdatableIDProvider) { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node, enablePeerManagementAndConnectionGating bool) ([]*p2p.Middleware, []*UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) idProviders := make([]*UpdatableIDProvider, len(identities)) @@ -145,8 +145,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id rootBlockID, p2p.DefaultPeerUpdateInterval, p2p.DefaultUnicastTimeout, - true, - true, + enablePeerManagementAndConnectionGating, + enablePeerManagementAndConnectionGating, p2p.NewIdentityProviderIDTranslator(idProviders[i]), p2p.WithIdentifierProvider( idProviders[i], @@ -224,8 +224,8 @@ func GenerateIDsAndMiddlewares(t *testing.T, dryRunMode bool, logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*UpdatableIDProvider) { - ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) - mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) + ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, true, opts...) + mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes, true) return ids, mws, protectObservables, providers } @@ -257,7 +257,9 @@ func GenerateEngines(t *testing.T, nets []*p2p.Network) []*MeshEngine { func generateLibP2PNode(t *testing.T, logger zerolog.Logger, id flow.Identity, - key crypto.PrivateKey) (*p2p.Node, observable.Observable) { + key crypto.PrivateKey, + connGating bool, +) (*p2p.Node, observable.Observable) { noopMetrics := metrics.NewNoopCollector() @@ -266,7 +268,10 @@ func generateLibP2PNode(t *testing.T, pingInfoProvider.On("SealedBlockHeight").Return(uint64(1000)) ctx := context.Background() - connGater := p2p.NewConnGater(logger) + var connGater *p2p.ConnGater = nil + if connGating { + connGater = p2p.NewConnGater(logger) + } // Inject some logic to be able to observe connections of this node connManager := NewTagWatchingConnManager(logger, noopMetrics) diff --git a/network/topology/topology_test.go b/network/topology/topology_test.go index 80a7417b6fe..db2d4f5ce65 100644 --- a/network/topology/topology_test.go +++ b/network/topology/topology_test.go @@ -138,11 +138,11 @@ func (suite *TopologyTestSuite) generateSystem(acc, col, con, exe, ver, cluster flow.IdentityList, []network.SubscriptionManager) { - collector, _, _ := test.GenerateIDs(suite.T(), suite.logger, col, test.DryRun, unittest.WithRole(flow.RoleCollection)) - access, _, _ := test.GenerateIDs(suite.T(), suite.logger, acc, test.DryRun, unittest.WithRole(flow.RoleAccess)) - consensus, _, _ := test.GenerateIDs(suite.T(), suite.logger, con, test.DryRun, unittest.WithRole(flow.RoleConsensus)) - verification, _, _ := test.GenerateIDs(suite.T(), suite.logger, ver, test.DryRun, unittest.WithRole(flow.RoleVerification)) - execution, _, _ := test.GenerateIDs(suite.T(), suite.logger, exe, test.DryRun, unittest.WithRole(flow.RoleExecution)) + collector, _, _ := test.GenerateIDs(suite.T(), suite.logger, col, test.DryRun, true, unittest.WithRole(flow.RoleCollection)) + access, _, _ := test.GenerateIDs(suite.T(), suite.logger, acc, test.DryRun, true, unittest.WithRole(flow.RoleAccess)) + consensus, _, _ := test.GenerateIDs(suite.T(), suite.logger, con, test.DryRun, true, unittest.WithRole(flow.RoleConsensus)) + verification, _, _ := test.GenerateIDs(suite.T(), suite.logger, ver, test.DryRun, true, unittest.WithRole(flow.RoleVerification)) + execution, _, _ := test.GenerateIDs(suite.T(), suite.logger, exe, test.DryRun, true, unittest.WithRole(flow.RoleExecution)) ids := flow.IdentityList{} ids = ids.Union(collector) From 5826a23b1864a5669fe34470a19705e7f0b2ef71 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 21:36:33 -0700 Subject: [PATCH 189/291] Revert "Merge branch 'vishal/remove_unstaked_network' of https://github.com/onflow/flow-go into vishal/remove_unstaked_network" This reverts commit 89553073cbf194ff678c2b2bc1bc4122f5f6b4ee, reversing changes made to 589dfc2c76d1af27fac816fee75f8f221465a0b8. --- integration/go.sum | 10 - integration/localnet/bootstrap.go | 2 +- .../test => module/id}/updatable_provider.go | 2 +- module/mocks/network.go | 64 +++--- network/p2p/readSubscription.go | 43 +--- network/test/middleware_test.go | 3 +- network/test/peerstore_provider_test.go | 111 ---------- network/test/testUtil.go | 9 +- storage/mocks/storage.go | 208 +++++++++--------- 9 files changed, 148 insertions(+), 304 deletions(-) rename {network/test => module/id}/updatable_provider.go (98%) delete mode 100644 network/test/peerstore_provider_test.go diff --git a/integration/go.sum b/integration/go.sum index 08030e135cc..9e9720679e5 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -88,7 +88,6 @@ github.com/VictoriaMetrics/fastcache v1.5.7/go.mod h1:ptDBkNMQI4RtmVo8VS/XwRY6Ro github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= -github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af h1:wVe6/Ea46ZMeNkQjjBW6xcqyQA/j5e0D6GytH95g0gQ= github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -128,7 +127,6 @@ github.com/blang/semver v3.1.0+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnweb github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40/go.mod h1:8rLXio+WjiTceGBHIoTvn60HIbs7Hm7bcHjyrSqYB9c= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= -github.com/bsipos/thist v1.0.0 h1:vZ3W5/ZnT54s4LHeonTCbnzCb20ERlJUnhiwXoGpsbY= github.com/bsipos/thist v1.0.0/go.mod h1:7i0xwRua1/bmUxcxi2xAxaFL895rLtOpKUwnw3NrT8I= github.com/btcsuite/btcd v0.0.0-20171128150713-2e60448ffcc6/go.mod h1:Dmm/EzmjnCiweXmzRIAiUWCInVmPgjkzgv5k4tVyXiQ= github.com/btcsuite/btcd v0.0.0-20190213025234-306aecffea32/go.mod h1:DrZx5ec/dmnfpw9KyYoQyYo7d0KEvTkk/5M/vbZjAr8= @@ -289,7 +287,6 @@ github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5/go.mod h1:VvhXpOYNQvB+ github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= github.com/flynn/noise v1.0.0 h1:DlTHqmzmvcEiKj+4RYo/imoswx/4r6iBlCMfVtrMXpQ= github.com/flynn/noise v1.0.0/go.mod h1:xbMo+0i6+IGbYdJhF31t2eR1BIU0CYc12+BNAKwUTag= -github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90 h1:WXb3TSNmHp2vHoCroCIB1foO/yQ36swABL8aOVeDpgg= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= @@ -378,7 +375,6 @@ github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXP github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 h1:DACJavvAHhabrF08vX0COfcOBJRhZ8lUbR+ZWIs0Y5g= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -647,7 +643,6 @@ github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfV github.com/julienschmidt/httprouter v1.1.1-0.20170430222011-975b5c4c7c21/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5 h1:PJr+ZMXIecYc1Ey2zucXdR73SMBtgjPgwa31099IMv0= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jwilder/encoding v0.0.0-20170811194829-b4e1701a28ef/go.mod h1:Ct9fl0F6iIOGgxJ5npU/IUOhOhqlVrGjyIZc8/MagT0= github.com/k0kubun/go-ansi v0.0.0-20180517002512-3bf9e2903213/go.mod h1:vNUNkEQ1e29fT/6vq2aBdFsgNPmy8qMdSay1npru+Sw= @@ -1086,7 +1081,6 @@ github.com/multiformats/go-varint v0.0.5/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXS github.com/multiformats/go-varint v0.0.6 h1:gk85QWKxh3TazbLxED/NlDVv8+q+ReFJk7Y2W/KhfNY= github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/naoina/go-stringutil v0.1.0/go.mod h1:XJ2SJL9jCtBh+P9q5btrd/Ylo8XwT/h1USek5+NqSA0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416/go.mod h1:NBIhNtsFMo3G2szEBne+bO4gS192HuIYRqfvOWb4i1E= @@ -1527,7 +1521,6 @@ golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6 h1:QE6XYQK6naiK1EPAe1g/ILLxN golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= -golang.org/x/image v0.0.0-20190802002840-cff245a6509b h1:+qEpEAPhDZ1o0x3tHzZTQDArnOixOzGD9HUJfcg0mb4= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -1722,7 +1715,6 @@ golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744 h1:yhBbb4IRs2HS9PPlAg6DMC6mU golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d h1:SZxvLBoTP5yHO3Frd4z4vrF+DBX9vMVanchswa69toE= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1820,7 +1812,6 @@ gonum.org/v1/gonum v0.6.1/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0 h1:OE9mWmgKkjJyEmDAAtGMPjXu+YNeGvK9VTSHY6+Qihc= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= -gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b h1:Qh4dB5D/WpoUUp3lSod7qgoyEHbDGPUWjIbnqdqqe1k= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= @@ -1987,7 +1978,6 @@ k8s.io/kubernetes v1.13.0/go.mod h1:ocZa8+6APFNC2tX1DZASIbocyYT5jHzqFVsY5aoB7Jk= pgregory.net/rapid v0.4.7 h1:MTNRktPuv5FNqOO151TM9mDTa+XHcX6ypYeISDVD14g= pgregory.net/rapid v0.4.7/go.mod h1:UYpPVyjFHzYBGHIxLFoupi8vwk6rXNzRY9OMvVxFIOU= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= -rsc.io/pdf v0.1.1 h1:k1MczvYDUvJBe93bYd7wrZLLUEcLZAuF824/I4e5Xr4= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/integration/localnet/bootstrap.go b/integration/localnet/bootstrap.go index b55c2bf8d7e..bb042e0972c 100644 --- a/integration/localnet/bootstrap.go +++ b/integration/localnet/bootstrap.go @@ -208,7 +208,7 @@ func prepareNodes() []testnet.NodeConfig { for i := 0; i < unstakedAccessCount; i++ { nodes = append(nodes, testnet.NewNodeConfig(flow.RoleAccess, func(cfg *testnet.NodeConfig) { - cfg.SupportsUnstakedNodes = true + cfg.Unstaked = true })) } diff --git a/network/test/updatable_provider.go b/module/id/updatable_provider.go similarity index 98% rename from network/test/updatable_provider.go rename to module/id/updatable_provider.go index 7c4377a1ae5..d35a3da0566 100644 --- a/network/test/updatable_provider.go +++ b/module/id/updatable_provider.go @@ -1,4 +1,4 @@ -package test +package id import ( "sync" diff --git a/module/mocks/network.go b/module/mocks/network.go index 38f65a552ce..305cee9f346 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -5,38 +5,40 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" + crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" network "github.com/onflow/flow-go/network" - reflect "reflect" ) -// MockNetwork is a mock of Network interface +// MockNetwork is a mock of Network interface. type MockNetwork struct { ctrl *gomock.Controller recorder *MockNetworkMockRecorder } -// MockNetworkMockRecorder is the mock recorder for MockNetwork +// MockNetworkMockRecorder is the mock recorder for MockNetwork. type MockNetworkMockRecorder struct { mock *MockNetwork } -// NewMockNetwork creates a new mock instance +// NewMockNetwork creates a new mock instance. func NewMockNetwork(ctrl *gomock.Controller) *MockNetwork { mock := &MockNetwork{ctrl: ctrl} mock.recorder = &MockNetworkMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockNetwork) EXPECT() *MockNetworkMockRecorder { return m.recorder } -// Register mocks base method +// Register mocks base method. func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (network.Conduit, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Register", arg0, arg1) @@ -45,36 +47,36 @@ func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (netwo return ret0, ret1 } -// Register indicates an expected call of Register +// Register indicates an expected call of Register. func (mr *MockNetworkMockRecorder) Register(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Register", reflect.TypeOf((*MockNetwork)(nil).Register), arg0, arg1) } -// MockLocal is a mock of Local interface +// MockLocal is a mock of Local interface. type MockLocal struct { ctrl *gomock.Controller recorder *MockLocalMockRecorder } -// MockLocalMockRecorder is the mock recorder for MockLocal +// MockLocalMockRecorder is the mock recorder for MockLocal. type MockLocalMockRecorder struct { mock *MockLocal } -// NewMockLocal creates a new mock instance +// NewMockLocal creates a new mock instance. func NewMockLocal(ctrl *gomock.Controller) *MockLocal { mock := &MockLocal{ctrl: ctrl} mock.recorder = &MockLocalMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockLocal) EXPECT() *MockLocalMockRecorder { return m.recorder } -// Address mocks base method +// Address mocks base method. func (m *MockLocal) Address() string { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Address") @@ -82,13 +84,13 @@ func (m *MockLocal) Address() string { return ret0 } -// Address indicates an expected call of Address +// Address indicates an expected call of Address. func (mr *MockLocalMockRecorder) Address() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Address", reflect.TypeOf((*MockLocal)(nil).Address)) } -// NodeID mocks base method +// NodeID mocks base method. func (m *MockLocal) NodeID() flow.Identifier { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NodeID") @@ -96,13 +98,13 @@ func (m *MockLocal) NodeID() flow.Identifier { return ret0 } -// NodeID indicates an expected call of NodeID +// NodeID indicates an expected call of NodeID. func (mr *MockLocalMockRecorder) NodeID() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NodeID", reflect.TypeOf((*MockLocal)(nil).NodeID)) } -// NotMeFilter mocks base method +// NotMeFilter mocks base method. func (m *MockLocal) NotMeFilter() flow.IdentityFilter { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NotMeFilter") @@ -110,13 +112,13 @@ func (m *MockLocal) NotMeFilter() flow.IdentityFilter { return ret0 } -// NotMeFilter indicates an expected call of NotMeFilter +// NotMeFilter indicates an expected call of NotMeFilter. func (mr *MockLocalMockRecorder) NotMeFilter() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NotMeFilter", reflect.TypeOf((*MockLocal)(nil).NotMeFilter)) } -// Sign mocks base method +// Sign mocks base method. func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Sign", arg0, arg1) @@ -125,13 +127,13 @@ func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error return ret0, ret1 } -// Sign indicates an expected call of Sign +// Sign indicates an expected call of Sign. func (mr *MockLocalMockRecorder) Sign(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sign", reflect.TypeOf((*MockLocal)(nil).Sign), arg0, arg1) } -// SignFunc mocks base method +// SignFunc mocks base method. func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.PrivateKey, []byte, hash.Hasher) (crypto.Signature, error)) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SignFunc", arg0, arg1, arg2) @@ -140,66 +142,66 @@ func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.Pri return ret0, ret1 } -// SignFunc indicates an expected call of SignFunc +// SignFunc indicates an expected call of SignFunc. func (mr *MockLocalMockRecorder) SignFunc(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignFunc", reflect.TypeOf((*MockLocal)(nil).SignFunc), arg0, arg1, arg2) } -// MockRequester is a mock of Requester interface +// MockRequester is a mock of Requester interface. type MockRequester struct { ctrl *gomock.Controller recorder *MockRequesterMockRecorder } -// MockRequesterMockRecorder is the mock recorder for MockRequester +// MockRequesterMockRecorder is the mock recorder for MockRequester. type MockRequesterMockRecorder struct { mock *MockRequester } -// NewMockRequester creates a new mock instance +// NewMockRequester creates a new mock instance. func NewMockRequester(ctrl *gomock.Controller) *MockRequester { mock := &MockRequester{ctrl: ctrl} mock.recorder = &MockRequesterMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockRequester) EXPECT() *MockRequesterMockRecorder { return m.recorder } -// EntityByID mocks base method +// EntityByID mocks base method. func (m *MockRequester) EntityByID(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "EntityByID", arg0, arg1) } -// EntityByID indicates an expected call of EntityByID +// EntityByID indicates an expected call of EntityByID. func (mr *MockRequesterMockRecorder) EntityByID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EntityByID", reflect.TypeOf((*MockRequester)(nil).EntityByID), arg0, arg1) } -// Force mocks base method +// Force mocks base method. func (m *MockRequester) Force() { m.ctrl.T.Helper() m.ctrl.Call(m, "Force") } -// Force indicates an expected call of Force +// Force indicates an expected call of Force. func (mr *MockRequesterMockRecorder) Force() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Force", reflect.TypeOf((*MockRequester)(nil).Force)) } -// Query mocks base method +// Query mocks base method. func (m *MockRequester) Query(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Query", arg0, arg1) } -// Query indicates an expected call of Query +// Query indicates an expected call of Query. func (mr *MockRequesterMockRecorder) Query(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockRequester)(nil).Query), arg0, arg1) diff --git a/network/p2p/readSubscription.go b/network/p2p/readSubscription.go index 681721286cb..db87b097afa 100644 --- a/network/p2p/readSubscription.go +++ b/network/p2p/readSubscription.go @@ -2,11 +2,9 @@ package p2p import ( "context" - "fmt" "strings" "sync" - lcrypto "github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" @@ -81,7 +79,7 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { return } - pid, err := messageSigningID(rawMsg) + pid, err := peer.IDFromBytes(rawMsg.From) if err != nil { r.log.Err(err).Msg("failed to validate peer ID of incoming message") return @@ -104,42 +102,3 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { r.callback(&msg, pid) } } - -// messagePubKey extracts the public key of the envelope signer from a libp2p message. -// The location of that key depends on the type of the key, see: -// https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md -// This reproduces the exact logic of the private function doing the same decoding in libp2p: -// https://github.com/libp2p/go-libp2p-pubsub/blob/ba28f8ecfc551d4d916beb748d3384951bce3ed0/sign.go#L77 -func messageSigningID(m *pubsub.Message) (peer.ID, error) { - var pubk lcrypto.PubKey - - // m.From is the original sender of the message (versus `m.ReceivedFrom` which is the last hop which sent us this message) - pid, err := peer.IDFromBytes(m.From) - if err != nil { - return "", err - } - - if m.Key == nil { - // no attached key, it must be extractable from the source ID - pubk, err = pid.ExtractPublicKey() - if err != nil { - return "", fmt.Errorf("cannot extract signing key: %s", err.Error()) - } - if pubk == nil { - return "", fmt.Errorf("cannot extract signing key") - } - } else { - pubk, err = lcrypto.UnmarshalPublicKey(m.Key) - if err != nil { - return "", fmt.Errorf("cannot unmarshal signing key: %s", err.Error()) - } - - // verify that the source ID matches the attached key - if !pid.MatchesPublicKey(pubk) { - return "", fmt.Errorf("bad signing key; source ID %s doesn't match key", pid) - } - } - - // the pid either contains or matches the signing pubKey - return pid, nil -} diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index c177ade677f..29c9d980e4a 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -19,6 +19,7 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/observable" "github.com/onflow/flow-go/network/codec/cbor" @@ -65,7 +66,7 @@ type MiddlewareTestSuite struct { ids []*flow.Identity metrics *metrics.NoopCollector // no-op performance monitoring simulation logger zerolog.Logger - providers []*UpdatableIDProvider + providers []*id.UpdatableIDProvider } // TestMiddlewareTestSuit runs all the test methods in this test suit diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go deleted file mode 100644 index 093c55072a0..00000000000 --- a/network/test/peerstore_provider_test.go +++ /dev/null @@ -1,111 +0,0 @@ -package test - -import ( - "context" - "math/rand" - "os" - "strconv" - "testing" - - "github.com/ipfs/go-log" - "github.com/libp2p/go-libp2p-core/peer" - "github.com/multiformats/go-multiaddr" - "github.com/rs/zerolog" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - - "github.com/onflow/flow-go/cmd/bootstrap/utils" - "github.com/onflow/flow-go/crypto" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/utils/unittest" -) - -type PeerStoreProviderTestSuite struct { - suite.Suite - logger zerolog.Logger - nodes []*p2p.Node - libp2pPeersIDs []peer.ID - peerIDprovider *p2p.PeerstoreIdentifierProvider - translator *p2p.HierarchicalIDTranslator - ids flow.IdentityList -} - -func TestPeerStoreProviderTestSuite(t *testing.T) { - suite.Run(t, new(PeerStoreProviderTestSuite)) -} - -const nodeCount = 2 -const peerCount = 3 -const testNodeIndex = 0 // testNodeIndex < nodeCount - -func (suite *PeerStoreProviderTestSuite) SetupTest() { - suite.logger = zerolog.New(os.Stderr).Level(zerolog.ErrorLevel) - log.SetAllLoggers(log.LevelError) - ctx := context.Background() - - suite.ids, suite.nodes, _ = GenerateIDs(suite.T(), suite.logger, nodeCount, !DryRun) - t, err := p2p.NewFixedTableIdentityTranslator(suite.ids) - require.NoError(suite.T(), err) - - u := p2p.NewUnstakedNetworkIDTranslator() - suite.translator = p2p.NewHierarchicalIDTranslator(u, t) - - // emulate the middleware behavior in populating the testnode's peer store - libp2pPeers := make([]peer.ID, peerCount) - for i := 0; i < peerCount; i++ { - peerAddrInfo := suite.randomPeerInfoWithStubNetwork() - err := suite.nodes[testNodeIndex].AddPeer(ctx, peerAddrInfo) - // conn gater (then connection routine) will complain - require.Error(suite.T(), err) - libp2pPeers[i] = peerAddrInfo.ID - } - suite.libp2pPeersIDs = libp2pPeers - suite.peerIDprovider = p2p.NewPeerstoreIdentifierProvider(suite.logger, suite.nodes[testNodeIndex].Host(), suite.translator) - - // sanity checks - assert.Len(suite.T(), suite.nodes, nodeCount) - assert.Len(suite.T(), suite.libp2pPeersIDs, peerCount) - assert.Len(suite.T(), suite.ids, nodeCount) - -} - -func (suite *PeerStoreProviderTestSuite) TestTranslationPeers() { - - identifiers := suite.peerIDprovider.Identifiers() - - peerIDs := make([]peer.ID, len(identifiers)) - for i := 0; i < len(identifiers); i++ { - - pID, err := suite.translator.GetPeerID(identifiers[i]) - require.NoError(suite.T(), err) - peerIDs[i] = pID - } - // check we can find the libp2p peers - assert.ElementsMatch(suite.T(), peerIDs, append(suite.libp2pPeersIDs, suite.nodes[testNodeIndex].Host().ID()), "peer IDs should include those in the peer Store") - -} - -func (suite *PeerStoreProviderTestSuite) randomPeerInfoWithStubNetwork() peer.AddrInfo { - - // we don't care about network information, but those peers need an address - ip := "127.0.0.1" - port := strconv.Itoa(rand.Intn(65535 - 1024)) - - addr := p2p.MultiAddressStr(ip, port) - maddr, err := multiaddr.NewMultiaddr(addr) - require.NoError(suite.T(), err) - - privKey, err := utils.GenerateUnstakedNetworkingKey(unittest.SeedFixture(crypto.KeyGenSeedMinLenECDSASecp256k1)) - require.NoError(suite.T(), err) - - libp2pKey, err := p2p.LibP2PPublicKeyFromFlow(privKey.PublicKey()) - require.NoError(suite.T(), err) - - id, err := peer.IDFromPublicKey(libp2pKey) - require.NoError(suite.T(), err) - - pInfo := peer.AddrInfo{ID: id, Addrs: []multiaddr.Multiaddr{maddr}} - return pInfo -} diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 13d5fd80554..9fe6dfe2447 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,6 +20,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" idModule "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" @@ -121,10 +122,10 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*UpdatableIDProvider) { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*id.UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) - idProviders := make([]*UpdatableIDProvider, len(identities)) + idProviders := make([]*id.UpdatableIDProvider, len(identities)) for i, id := range identities { // casts libP2PNode instance to a local variable to avoid closure @@ -135,7 +136,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } - idProviders[i] = NewUpdatableIDProvider(identities) + idProviders[i] = idModule.NewUpdatableIDProvider(identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, @@ -222,7 +223,7 @@ func GenerateNetworks(t *testing.T, func GenerateIDsAndMiddlewares(t *testing.T, n int, dryRunMode bool, - logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*UpdatableIDProvider) { + logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*id.UpdatableIDProvider) { ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 2967a3e5cce..969044a53ee 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -5,37 +5,39 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" + flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" - reflect "reflect" ) -// MockBlocks is a mock of Blocks interface +// MockBlocks is a mock of Blocks interface. type MockBlocks struct { ctrl *gomock.Controller recorder *MockBlocksMockRecorder } -// MockBlocksMockRecorder is the mock recorder for MockBlocks +// MockBlocksMockRecorder is the mock recorder for MockBlocks. type MockBlocksMockRecorder struct { mock *MockBlocks } -// NewMockBlocks creates a new mock instance +// NewMockBlocks creates a new mock instance. func NewMockBlocks(ctrl *gomock.Controller) *MockBlocks { mock := &MockBlocks{ctrl: ctrl} mock.recorder = &MockBlocksMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockBlocks) EXPECT() *MockBlocksMockRecorder { return m.recorder } -// ByCollectionID mocks base method +// ByCollectionID mocks base method. func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByCollectionID", arg0) @@ -44,13 +46,13 @@ func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByCollectionID indicates an expected call of ByCollectionID +// ByCollectionID indicates an expected call of ByCollectionID. func (mr *MockBlocksMockRecorder) ByCollectionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByCollectionID", reflect.TypeOf((*MockBlocks)(nil).ByCollectionID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -59,13 +61,13 @@ func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockBlocksMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockBlocks)(nil).ByHeight), arg0) } -// ByID mocks base method +// ByID mocks base method. func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -74,13 +76,13 @@ func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockBlocksMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockBlocks)(nil).ByID), arg0) } -// GetLastFullBlockHeight mocks base method +// GetLastFullBlockHeight mocks base method. func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetLastFullBlockHeight") @@ -89,13 +91,13 @@ func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { return ret0, ret1 } -// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight +// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight. func (mr *MockBlocksMockRecorder) GetLastFullBlockHeight() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).GetLastFullBlockHeight)) } -// IndexBlockForCollections mocks base method +// IndexBlockForCollections mocks base method. func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexBlockForCollections", arg0, arg1) @@ -103,13 +105,13 @@ func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow. return ret0 } -// IndexBlockForCollections indicates an expected call of IndexBlockForCollections +// IndexBlockForCollections indicates an expected call of IndexBlockForCollections. func (mr *MockBlocksMockRecorder) IndexBlockForCollections(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexBlockForCollections", reflect.TypeOf((*MockBlocks)(nil).IndexBlockForCollections), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockBlocks) Store(arg0 *flow.Block) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -117,13 +119,13 @@ func (m *MockBlocks) Store(arg0 *flow.Block) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockBlocksMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockBlocks)(nil).Store), arg0) } -// StoreTx mocks base method +// StoreTx mocks base method. func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreTx", arg0) @@ -131,13 +133,13 @@ func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { return ret0 } -// StoreTx indicates an expected call of StoreTx +// StoreTx indicates an expected call of StoreTx. func (mr *MockBlocksMockRecorder) StoreTx(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreTx", reflect.TypeOf((*MockBlocks)(nil).StoreTx), arg0) } -// UpdateLastFullBlockHeight mocks base method +// UpdateLastFullBlockHeight mocks base method. func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "UpdateLastFullBlockHeight", arg0) @@ -145,36 +147,36 @@ func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { return ret0 } -// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight +// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight. func (mr *MockBlocksMockRecorder) UpdateLastFullBlockHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).UpdateLastFullBlockHeight), arg0) } -// MockHeaders is a mock of Headers interface +// MockHeaders is a mock of Headers interface. type MockHeaders struct { ctrl *gomock.Controller recorder *MockHeadersMockRecorder } -// MockHeadersMockRecorder is the mock recorder for MockHeaders +// MockHeadersMockRecorder is the mock recorder for MockHeaders. type MockHeadersMockRecorder struct { mock *MockHeaders } -// NewMockHeaders creates a new mock instance +// NewMockHeaders creates a new mock instance. func NewMockHeaders(ctrl *gomock.Controller) *MockHeaders { mock := &MockHeaders{ctrl: ctrl} mock.recorder = &MockHeadersMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockHeaders) EXPECT() *MockHeadersMockRecorder { return m.recorder } -// BatchIndexByChunkID mocks base method +// BatchIndexByChunkID mocks base method. func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchIndexByChunkID", arg0, arg1, arg2) @@ -182,13 +184,13 @@ func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 stora return ret0 } -// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID +// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID. func (mr *MockHeadersMockRecorder) BatchIndexByChunkID(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchIndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).BatchIndexByChunkID), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -197,13 +199,13 @@ func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockHeadersMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockHeaders)(nil).ByBlockID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -212,13 +214,13 @@ func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockHeadersMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockHeaders)(nil).ByHeight), arg0) } -// ByParentID mocks base method +// ByParentID mocks base method. func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByParentID", arg0) @@ -227,13 +229,13 @@ func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { return ret0, ret1 } -// ByParentID indicates an expected call of ByParentID +// ByParentID indicates an expected call of ByParentID. func (mr *MockHeadersMockRecorder) ByParentID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByParentID", reflect.TypeOf((*MockHeaders)(nil).ByParentID), arg0) } -// IDByChunkID mocks base method +// IDByChunkID mocks base method. func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IDByChunkID", arg0) @@ -242,13 +244,13 @@ func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) return ret0, ret1 } -// IDByChunkID indicates an expected call of IDByChunkID +// IDByChunkID indicates an expected call of IDByChunkID. func (mr *MockHeadersMockRecorder) IDByChunkID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDByChunkID", reflect.TypeOf((*MockHeaders)(nil).IDByChunkID), arg0) } -// IndexByChunkID mocks base method +// IndexByChunkID mocks base method. func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexByChunkID", arg0, arg1) @@ -256,13 +258,13 @@ func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { return ret0 } -// IndexByChunkID indicates an expected call of IndexByChunkID +// IndexByChunkID indicates an expected call of IndexByChunkID. func (mr *MockHeadersMockRecorder) IndexByChunkID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).IndexByChunkID), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockHeaders) Store(arg0 *flow.Header) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -270,36 +272,36 @@ func (m *MockHeaders) Store(arg0 *flow.Header) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockHeadersMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockHeaders)(nil).Store), arg0) } -// MockPayloads is a mock of Payloads interface +// MockPayloads is a mock of Payloads interface. type MockPayloads struct { ctrl *gomock.Controller recorder *MockPayloadsMockRecorder } -// MockPayloadsMockRecorder is the mock recorder for MockPayloads +// MockPayloadsMockRecorder is the mock recorder for MockPayloads. type MockPayloadsMockRecorder struct { mock *MockPayloads } -// NewMockPayloads creates a new mock instance +// NewMockPayloads creates a new mock instance. func NewMockPayloads(ctrl *gomock.Controller) *MockPayloads { mock := &MockPayloads{ctrl: ctrl} mock.recorder = &MockPayloadsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockPayloads) EXPECT() *MockPayloadsMockRecorder { return m.recorder } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -308,13 +310,13 @@ func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockPayloadsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockPayloads)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -322,36 +324,36 @@ func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockPayloadsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockPayloads)(nil).Store), arg0, arg1) } -// MockCollections is a mock of Collections interface +// MockCollections is a mock of Collections interface. type MockCollections struct { ctrl *gomock.Controller recorder *MockCollectionsMockRecorder } -// MockCollectionsMockRecorder is the mock recorder for MockCollections +// MockCollectionsMockRecorder is the mock recorder for MockCollections. type MockCollectionsMockRecorder struct { mock *MockCollections } -// NewMockCollections creates a new mock instance +// NewMockCollections creates a new mock instance. func NewMockCollections(ctrl *gomock.Controller) *MockCollections { mock := &MockCollections{ctrl: ctrl} mock.recorder = &MockCollectionsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCollections) EXPECT() *MockCollectionsMockRecorder { return m.recorder } -// ByID mocks base method +// ByID mocks base method. func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -360,13 +362,13 @@ func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockCollectionsMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockCollections)(nil).ByID), arg0) } -// LightByID mocks base method +// LightByID mocks base method. func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByID", arg0) @@ -375,13 +377,13 @@ func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection return ret0, ret1 } -// LightByID indicates an expected call of LightByID +// LightByID indicates an expected call of LightByID. func (mr *MockCollectionsMockRecorder) LightByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByID", reflect.TypeOf((*MockCollections)(nil).LightByID), arg0) } -// LightByTransactionID mocks base method +// LightByTransactionID mocks base method. func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByTransactionID", arg0) @@ -390,13 +392,13 @@ func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.Ligh return ret0, ret1 } -// LightByTransactionID indicates an expected call of LightByTransactionID +// LightByTransactionID indicates an expected call of LightByTransactionID. func (mr *MockCollectionsMockRecorder) LightByTransactionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByTransactionID", reflect.TypeOf((*MockCollections)(nil).LightByTransactionID), arg0) } -// Remove mocks base method +// Remove mocks base method. func (m *MockCollections) Remove(arg0 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Remove", arg0) @@ -404,13 +406,13 @@ func (m *MockCollections) Remove(arg0 flow.Identifier) error { return ret0 } -// Remove indicates an expected call of Remove +// Remove indicates an expected call of Remove. func (mr *MockCollectionsMockRecorder) Remove(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockCollections)(nil).Remove), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCollections) Store(arg0 *flow.Collection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -418,13 +420,13 @@ func (m *MockCollections) Store(arg0 *flow.Collection) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCollectionsMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCollections)(nil).Store), arg0) } -// StoreLight mocks base method +// StoreLight mocks base method. func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLight", arg0) @@ -432,13 +434,13 @@ func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { return ret0 } -// StoreLight indicates an expected call of StoreLight +// StoreLight indicates an expected call of StoreLight. func (mr *MockCollectionsMockRecorder) StoreLight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLight", reflect.TypeOf((*MockCollections)(nil).StoreLight), arg0) } -// StoreLightAndIndexByTransaction mocks base method +// StoreLightAndIndexByTransaction mocks base method. func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLightAndIndexByTransaction", arg0) @@ -446,36 +448,36 @@ func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollec return ret0 } -// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction +// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction. func (mr *MockCollectionsMockRecorder) StoreLightAndIndexByTransaction(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLightAndIndexByTransaction", reflect.TypeOf((*MockCollections)(nil).StoreLightAndIndexByTransaction), arg0) } -// MockCommits is a mock of Commits interface +// MockCommits is a mock of Commits interface. type MockCommits struct { ctrl *gomock.Controller recorder *MockCommitsMockRecorder } -// MockCommitsMockRecorder is the mock recorder for MockCommits +// MockCommitsMockRecorder is the mock recorder for MockCommits. type MockCommitsMockRecorder struct { mock *MockCommits } -// NewMockCommits creates a new mock instance +// NewMockCommits creates a new mock instance. func NewMockCommits(ctrl *gomock.Controller) *MockCommits { mock := &MockCommits{ctrl: ctrl} mock.recorder = &MockCommitsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -483,13 +485,13 @@ func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockCommitsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockCommits)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -498,13 +500,13 @@ func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, err return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockCommitsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockCommits)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -512,36 +514,36 @@ func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) err return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCommitsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCommits)(nil).Store), arg0, arg1) } -// MockEvents is a mock of Events interface +// MockEvents is a mock of Events interface. type MockEvents struct { ctrl *gomock.Controller recorder *MockEventsMockRecorder } -// MockEventsMockRecorder is the mock recorder for MockEvents +// MockEventsMockRecorder is the mock recorder for MockEvents. type MockEventsMockRecorder struct { mock *MockEvents } -// NewMockEvents creates a new mock instance +// NewMockEvents creates a new mock instance. func NewMockEvents(ctrl *gomock.Controller) *MockEvents { mock := &MockEvents{ctrl: ctrl} mock.recorder = &MockEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockEvents) EXPECT() *MockEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -549,13 +551,13 @@ func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, ar return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -564,13 +566,13 @@ func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockEvents)(nil).ByBlockID), arg0) } -// ByBlockIDEventType mocks base method +// ByBlockIDEventType mocks base method. func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventType) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDEventType", arg0, arg1) @@ -579,13 +581,13 @@ func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventTyp return ret0, ret1 } -// ByBlockIDEventType indicates an expected call of ByBlockIDEventType +// ByBlockIDEventType indicates an expected call of ByBlockIDEventType. func (mr *MockEventsMockRecorder) ByBlockIDEventType(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDEventType", reflect.TypeOf((*MockEvents)(nil).ByBlockIDEventType), arg0, arg1) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -594,36 +596,36 @@ func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow. return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockEventsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockEvents)(nil).ByBlockIDTransactionID), arg0, arg1) } -// MockServiceEvents is a mock of ServiceEvents interface +// MockServiceEvents is a mock of ServiceEvents interface. type MockServiceEvents struct { ctrl *gomock.Controller recorder *MockServiceEventsMockRecorder } -// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents +// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents. type MockServiceEventsMockRecorder struct { mock *MockServiceEvents } -// NewMockServiceEvents creates a new mock instance +// NewMockServiceEvents creates a new mock instance. func NewMockServiceEvents(ctrl *gomock.Controller) *MockServiceEvents { mock := &MockServiceEvents{ctrl: ctrl} mock.recorder = &MockServiceEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -631,13 +633,13 @@ func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockServiceEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockServiceEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -646,36 +648,36 @@ func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockServiceEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockServiceEvents)(nil).ByBlockID), arg0) } -// MockTransactionResults is a mock of TransactionResults interface +// MockTransactionResults is a mock of TransactionResults interface. type MockTransactionResults struct { ctrl *gomock.Controller recorder *MockTransactionResultsMockRecorder } -// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults +// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults. type MockTransactionResultsMockRecorder struct { mock *MockTransactionResults } -// NewMockTransactionResults creates a new mock instance +// NewMockTransactionResults creates a new mock instance. func NewMockTransactionResults(ctrl *gomock.Controller) *MockTransactionResults { mock := &MockTransactionResults{ctrl: ctrl} mock.recorder = &MockTransactionResultsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -683,13 +685,13 @@ func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.Tr return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockTransactionResultsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockTransactionResults)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) (*flow.TransactionResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -698,7 +700,7 @@ func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifi return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockTransactionResultsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockTransactionResults)(nil).ByBlockIDTransactionID), arg0, arg1) From 6a7160fad53ff1adebf3990b7cd4efb1445ddf5e Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 21:51:54 -0700 Subject: [PATCH 190/291] Disable queueing missing heights for unstaked sync provider --- .../staked_access_node_builder.go | 1 + engine/common/synchronization/engine.go | 2 +- .../common/synchronization/request_handler.go | 28 +++++++++++-------- 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 51ba4ccfee9..e983faafeb8 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -119,6 +119,7 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { node.Storage.Blocks, anb.SyncCore, anb.FinalizedHeader, + false, ) // register the sync request handler with the proxy engine diff --git a/engine/common/synchronization/engine.go b/engine/common/synchronization/engine.go index 1a348b44bc5..70dcc8f46f6 100644 --- a/engine/common/synchronization/engine.go +++ b/engine/common/synchronization/engine.go @@ -106,7 +106,7 @@ func New( } e.con = con - e.requestHandler = NewRequestHandlerEngine(log, metrics, con, me, blocks, core, finalizedHeader) + e.requestHandler = NewRequestHandlerEngine(log, metrics, con, me, blocks, core, finalizedHeader, true) return e, nil } diff --git a/engine/common/synchronization/request_handler.go b/engine/common/synchronization/request_handler.go index a1b5e219c7c..aff48224ece 100644 --- a/engine/common/synchronization/request_handler.go +++ b/engine/common/synchronization/request_handler.go @@ -45,6 +45,8 @@ type RequestHandlerEngine struct { pendingBatchRequests engine.MessageStore // message store for *message.BatchRequest pendingRangeRequests engine.MessageStore // message store for *message.RangeRequest requestMessageHandler *engine.MessageHandler // message handler responsible for request processing + + queueMissingHeights bool // true if missing heights should be added to download queue } func NewRequestHandlerEngine( @@ -55,17 +57,19 @@ func NewRequestHandlerEngine( blocks storage.Blocks, core module.SyncCore, finalizedHeader *FinalizedHeaderCache, + queueMissingHeights bool, ) *RequestHandlerEngine { r := &RequestHandlerEngine{ - unit: engine.NewUnit(), - lm: lifecycle.NewLifecycleManager(), - me: me, - log: log.With().Str("engine", "synchronization").Logger(), - metrics: metrics, - blocks: blocks, - core: core, - finalizedHeader: finalizedHeader, - con: con, + unit: engine.NewUnit(), + lm: lifecycle.NewLifecycleManager(), + me: me, + log: log.With().Str("engine", "synchronization").Logger(), + metrics: metrics, + blocks: blocks, + core: core, + finalizedHeader: finalizedHeader, + con: con, + queueMissingHeights: queueMissingHeights, } r.setupRequestMessageHandler() @@ -175,8 +179,10 @@ func (r *RequestHandlerEngine) setupRequestMessageHandler() { func (r *RequestHandlerEngine) onSyncRequest(originID flow.Identifier, req *messages.SyncRequest) error { final := r.finalizedHeader.Get() - // queue any missing heights as needed - r.core.HandleHeight(final, req.Height) + if r.queueMissingHeights { + // queue any missing heights as needed + r.core.HandleHeight(final, req.Height) + } // don't bother sending a response if we're within tolerance or if we're // behind the requester From ce8642f62daae17c27de9f027824fb8f09a26076 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 22:35:42 -0700 Subject: [PATCH 191/291] Update subscriptionFilter_test.go --- network/p2p/subscriptionFilter_test.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/network/p2p/subscriptionFilter_test.go b/network/p2p/subscriptionFilter_test.go index 198e02f59c1..2ccf022b87a 100644 --- a/network/p2p/subscriptionFilter_test.go +++ b/network/p2p/subscriptionFilter_test.go @@ -28,18 +28,16 @@ func TestBasicSubscriptionFilter(t *testing.T) { require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host2))) require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host3))) - topicname1 := "testtopic1" topicname2 := "testtopic2" filter := &Filter{ allowedIDs: make(map[peer.ID]struct{}), - topic: topicname2, + topic: topicname2, } filter.allowedIDs[host1.ID()] = struct{}{} filter.allowedIDs[host2.ID()] = struct{}{} - ps1, err := pubsub.NewGossipSub(ctx, host1, pubsub.WithSubscriptionFilter(filter)) require.NoError(t, err) ps2, err := pubsub.NewGossipSub(ctx, host2, pubsub.WithSubscriptionFilter(filter)) @@ -60,7 +58,7 @@ func TestBasicSubscriptionFilter(t *testing.T) { require.NoError(t, err) _, err = topic2.Subscribe() require.NoError(t, err) - topic2, err = ps2.Join(topicname2) + topic2Host2, err := ps2.Join(topicname2) require.NoError(t, err) subscriberHost2Topic2, err := topic2.Subscribe() require.NoError(t, err) @@ -97,7 +95,7 @@ func TestBasicSubscriptionFilter(t *testing.T) { fmt.Printf("\t For %s", topicname2) fmt.Println(ps3.ListPeers(topicname2)) - err = wrongTopic.Publish(ctx, []byte("hello")) + err = topic2Host2.Publish(ctx, []byte("hello")) require.NoError(t, err) time.Sleep(2 * time.Second) @@ -105,7 +103,7 @@ func TestBasicSubscriptionFilter(t *testing.T) { msg, err := subscriberHost2Topic2.Next(ctx) require.NoError(t, err) fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) - fmt.Println(msg) + fmt.Println(msg.Data) msg, err = subscriberHost1Topic2.Next(ctx) require.NoError(t, err) @@ -117,13 +115,14 @@ func TestBasicSubscriptionFilter(t *testing.T) { fmt.Println(ps2.ListPeers(topicname1)) fmt.Printf("\t For %s", topicname2) fmt.Println(ps2.ListPeers(topicname2)) + t.Fail() } - var _ pubsub.SubscriptionFilter = (*Filter)(nil) + type Filter struct { allowedIDs map[peer.ID]struct{} - topic string + topic string } func (filter *Filter) CanSubscribe(topic string) bool { From 3e39e104b7bc144fca21cf1c6f5f29ba99d77de4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 23:21:44 -0700 Subject: [PATCH 192/291] Create subscription filter --- engine/channels.go | 7 ++++ network/conduit.go | 9 +++++ network/p2p/subscription_filter.go | 64 ++++++++++++++++++++++++++++++ 3 files changed, 80 insertions(+) create mode 100644 network/p2p/subscription_filter.go diff --git a/engine/channels.go b/engine/channels.go index b121e534e72..1088ccc35f8 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -90,6 +90,13 @@ func Channels() network.ChannelList { return channels } +func UnstakedChannels() network.ChannelList { + return network.ChannelList{ + UnstakedSyncCommittee, + ReceiveBlocks, + } +} + // channels const ( diff --git a/network/conduit.go b/network/conduit.go index d4f8edb3688..e4aeb9e91c8 100644 --- a/network/conduit.go +++ b/network/conduit.go @@ -46,6 +46,15 @@ func (cl ChannelList) ID() flow.Identifier { return flow.MakeID(cl) } +func (cl ChannelList) Contains(channel Channel) bool { + for _, c := range cl { + if c == channel { + return true + } + } + return false +} + // Conduit represents the interface for engines to communicate over the // peer-to-peer network. Upon registration with the network, each engine is // assigned a conduit, which it can use to communicate across the network in diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go new file mode 100644 index 00000000000..90956d1f0c1 --- /dev/null +++ b/network/p2p/subscription_filter.go @@ -0,0 +1,64 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + pb "github.com/libp2p/go-libp2p-pubsub/pb" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network" +) + +type Filter struct { + idProvider id.IdentityProvider + idTranslator IDTranslator + myPeerID peer.ID +} + +func (f *Filter) getIdentity(pid peer.ID) *flow.Identity { + fid, err := f.idTranslator.GetFlowID(pid) + if err != nil { + // translation should always succeed for staked nodes + return nil + } + + identities := f.idProvider.Identities(filter.HasNodeID(fid)) + if len(identities) == 0 { + return nil + } + + return identities[0] +} + +func (f *Filter) allowedChannels(pid peer.ID) network.ChannelList { + id := f.getIdentity(pid) + + if id == nil { + return engine.UnstakedChannels() + } + + return engine.ChannelsByRole(id.Role) +} + +func (f *Filter) CanSubscribe(topic string) bool { + // TODO: check if this is correct + channel := network.Channel(topic) + + return f.allowedChannels(f.myPeerID).Contains(channel) +} + +func (f *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { + allowedChannels := f.allowedChannels(from) + var filtered []*pb.RPC_SubOpts + + for _, opt := range opts { + channel := network.Channel(*opt.Topicid) + if allowedChannels.Contains(channel) { + filtered = append(filtered, opt) + } + } + + return filtered, nil +} From ad94d71efdade66f387ec323104bc2115a61916b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 26 Aug 2021 00:26:46 -0700 Subject: [PATCH 193/291] Add subscription filter to pubsub options --- cmd/access/node_builder/staked_access_node_builder.go | 3 +-- cmd/scaffold.go | 1 + network/p2p/libp2pNode.go | 11 ++++++++--- network/p2p/subscription_filter.go | 8 ++++++++ 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index e983faafeb8..6ff7172a083 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -44,7 +44,6 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { } fnb.IdentityProvider = idCache - // translator fnb.SyncEngineParticipantsProviderFactory = func() id.IdentifierProvider { return id.NewFilteredIdentifierProvider( filter.And( @@ -200,7 +199,7 @@ func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, SetConnectionManager(connManager). // act as a DHT server SetDHTOptions(dhtOptions...). - SetPubsubOptions(p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize)...). + SetPubsubOptions(p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize, builder.IdentityProvider, builder.IDTranslator)...). SetLogger(builder.Logger). Build(ctx) if err != nil { diff --git a/cmd/scaffold.go b/cmd/scaffold.go index ce89713f68a..fcafc9bf65c 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -165,6 +165,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Metrics.Network, pingProvider, node.IdentityProvider, + node.IDTranslator, ) if err != nil { return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 9b9efa0c626..102eb5971f8 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -55,7 +55,7 @@ type LibP2PFactoryFunc func() (*Node, error) // DefaultLibP2PNodeFactory returns a LibP2PFactoryFunc which generates the libp2p host initialized with the // default options for the host, the pubsub and the ping service. func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.Identifier, address string, flowKey fcrypto.PrivateKey, rootBlockID string, - maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider, idProvider id.IdentityProvider) (LibP2PFactoryFunc, error) { + maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider, idProvider id.IdentityProvider, idTranslator IDTranslator) (LibP2PFactoryFunc, error) { connManager := NewConnManager(log, idProvider, metrics) @@ -66,7 +66,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize)...). + SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize, idProvider, idTranslator)...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). Build(ctx) @@ -644,7 +644,7 @@ func DefaultPubSub(ctx context.Context, host host.Host, psOption ...pubsub.Optio // PubsubOption generates a libp2p pubsub.Option from the given context and host type PubsubOption func(ctx context.Context, host host.Host) (pubsub.Option, error) -func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { +func DefaultPubsubOptions(maxPubSubMsgSize int, idProvider id.IdentityProvider, idTranslator IDTranslator) []PubsubOption { pubSubOptionFunc := func(option pubsub.Option) PubsubOption { return func(_ context.Context, _ host.Host) (pubsub.Option, error) { return option, nil @@ -658,6 +658,11 @@ func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { // set max message size limit for 1-k PubSub messaging pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), // no discovery + + // subscription filter + func(_ context.Context, h host.Host) (pubsub.Option, error) { + return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), idProvider, idTranslator)), nil + }, } } diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index 90956d1f0c1..6e1c0d8cb38 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -17,6 +17,14 @@ type Filter struct { myPeerID peer.ID } +func NewSubscriptionFilter(pid peer.ID, idProvider id.IdentityProvider, idTranslator IDTranslator) *Filter { + return &Filter{ + idProvider, + idTranslator, + pid, + } +} + func (f *Filter) getIdentity(pid peer.ID) *flow.Identity { fid, err := f.idTranslator.GetFlowID(pid) if err != nil { From 8258159830a5e066277e4fdc4c05f12a341084dc Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 00:07:25 -0700 Subject: [PATCH 194/291] initial --- integration/go.sum | 2 - module/id/id_provider.go | 148 ++++++++++++++++++++++++++++++++++++ network/p2p/idTranslator.go | 33 ++++++++ network/p2p/libp2pNode.go | 10 ++- 4 files changed, 189 insertions(+), 4 deletions(-) create mode 100644 module/id/id_provider.go create mode 100644 network/p2p/idTranslator.go diff --git a/integration/go.sum b/integration/go.sum index 0fd0299ba01..b43cd560555 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -792,13 +792,11 @@ github.com/libp2p/go-libp2p-peerstore v0.2.0/go.mod h1:N2l3eVIeAitSg3Pi2ipSrJYnq github.com/libp2p/go-libp2p-peerstore v0.2.1/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= -github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.7/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.8 h1:nJghUlUkFVvyk7ccsM67oFA6kqUkwyCM1G4WPVMCWYA= github.com/libp2p/go-libp2p-peerstore v0.2.8/go.mod h1:gGiPlXdz7mIHd2vfAsHzBNAMqSDkt2UBFwgcITgw1lA= github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6na5f0/k= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= -github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= github.com/libp2p/go-libp2p-pubsub v0.4.1 h1:j4umIg5nyus+sqNfU+FWvb9aeYFQH/A+nDFhWj+8yy8= github.com/libp2p/go-libp2p-pubsub v0.4.1/go.mod h1:izkeMLvz6Ht8yAISXjx60XUQZMq9ZMe5h2ih4dLIBIQ= github.com/libp2p/go-libp2p-quic-transport v0.10.0/go.mod h1:RfJbZ8IqXIhxBRm5hqUEJqjiiY8xmEuq3HUDS993MkA= diff --git a/module/id/id_provider.go b/module/id/id_provider.go new file mode 100644 index 00000000000..430780b6dc8 --- /dev/null +++ b/module/id/id_provider.go @@ -0,0 +1,148 @@ +package identity + +import ( + "sync" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" +) + +type IdentifierProvider interface { + Identifiers() flow.IdentifierList +} + +// TODO: rename to ProtocolStateIDProvider +type ProtocolStateIdentifierProvider struct { + events.Noop + identities flow.IdentityList // TODO: actually we *can* just use identifiers after all + state protocol.State + mu sync.RWMutex + filter flow.IdentityFilter + peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + flowIDs map[peer.ID]flow.Identifier +} + +func WithFilter(filter flow.IdentityFilter) ProtocolStateIdentifierProviderOption { + return func(provider *ProtocolStateIdentifierProvider) { + provider.filter = filter + } +} + +type ProtocolStateIdentifierProviderOption func(*ProtocolStateIdentifierProvider) + +// TODO: this one also implements IDTranslator!!! +func NewProtocolStateIdentifierProvider( + state protocol.State, + eventDistributer *events.Distributor, + opts ...ProtocolStateIdentifierProviderOption, +) (*ProtocolStateIdentifierProvider, error) { + provider := &ProtocolStateIdentifierProvider{ + state: state, + filter: filter.Any, + } + + for _, opt := range opts { + opt(provider) + } + + head, err := state.Final().Head() + if err != nil { + return nil, err // TODO: format the error + } + + provider.update(head.ID()) + eventDistributer.AddConsumer(provider) + + return provider, nil +} + +func (p *ProtocolStateIdentifierProvider) EpochTransition(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIdentifierProvider) update(blockID flow.Identifier) { + identities, err := p.state.AtBlockID(blockID).Identities(p.filter) + if err != nil { + // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + } + + nIds := identities.Count() + + peerIDs := make(map[flow.Identifier]peer.ID, nIds) + flowIDs := make(map[peer.ID]flow.Identifier, nIds) + + for _, identity := range identities { + pid, err := p2p.IdentityToPeerID(identity) + if err != nil { + // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + } + + flowIDs[pid] = identity.NodeID + peerIDs[identity.NodeID] = pid + } + + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities + p.flowIDs = flowIDs + p.peerIDs = peerIDs +} + +func (p *ProtocolStateIdentifierProvider) Identifiers() flow.IdentifierList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.NodeIDs() +} + +func (p *ProtocolStateIdentifierProvider) GetPeerID(flowID flow.Identifier) (pid peer.ID, found bool) { + p.mu.RLock() + defer p.mu.RUnlock() + + pid, found = p.peerIDs[flowID] + return +} + +func (p *ProtocolStateIdentifierProvider) GetFlowID(peerID peer.ID) (fid flow.Identifier, found bool) { + p.mu.RLock() + defer p.mu.RUnlock() + + fid, found = p.flowIDs[peerID] + return +} + +type PeerstoreIdentifierProvider struct { + store peerstore.Peerstore + idTranslator p2p.IDTranslator +} + +func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator p2p.IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} +} + +func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { + var result flow.IdentifierList + + pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? + for _, pid := range pids { + flowID, success := p.idTranslator.GetFlowID(pid) + if success { + result = append(result, flowID) + } + } + + return result +} diff --git a/network/p2p/idTranslator.go b/network/p2p/idTranslator.go new file mode 100644 index 00000000000..f9f567f1fec --- /dev/null +++ b/network/p2p/idTranslator.go @@ -0,0 +1,33 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type IDTranslator interface { + GetPeerID(flow.Identifier) (peer.ID, bool) + GetFlowID(peer.ID) (flow.Identifier, bool) +} + +type UnstakedNetworkPeerIDProvider struct { + // basically, just do the conversion from flow ID to peer ID using whatever scheme we talked about. + // whether this be, with caching or not +} + +func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { + pk, err := PublicKey(id.NetworkPubKey) + if err != nil { + // TODO: format the error + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + // TODO: format the error + return + } + + return +} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index a673cf4f81f..09fb7907b47 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -341,16 +341,22 @@ func (n *Node) RemovePeer(ctx context.Context, identity flow.Identity) error { } // CreateStream returns an existing stream connected to identity, if it exists or adds one to identity as a peer and creates a new stream with it. -func (n *Node) CreateStream(ctx context.Context, identity flow.Identity) (libp2pnet.Stream, error) { +func (n *Node) CreateStream(ctx context.Context, nodeID flow.Identifier) (libp2pnet.Stream, error) { // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) - stream, err := n.tryCreateNewStream(ctx, identity, maxConnectAttempt) + stream, err := n.tryCreateNewStream(ctx, nodeID, maxConnectAttempt) if err != nil { + n.host.Peerstore().Addrs() return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (node_id: %s, address: %s): %w", identity.NodeID.String(), identity.Address, err)) } return stream, nil } +func (n *Node) GetPeerIPPort(nodeID flow.Identifier) (string, string, error) { + // TODO: first get peer ID + x := n.host.Peerstore().Addrs() +} + // tryCreateNewStream makes at most maxAttempts to create a stream with the identity. // This was put in as a fix for #2416. PubSub and 1-1 communication compete with each other when trying to connect to // remote nodes and once in a while NewStream returns an error 'both yamux endpoints are clients' From d0bb4a54c52c0f1b820dc59550831da39018ffb1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 12 Aug 2021 14:04:49 -0700 Subject: [PATCH 195/291] Update idTranslator.go --- .../node_builder/access_node_builder.go | 182 +++++++----------- .../staked_access_node_builder.go | 36 +++- .../unstaked_access_node_builder.go | 87 +++++---- cmd/collection/main.go | 9 +- cmd/consensus/main.go | 9 +- cmd/execution/main.go | 9 +- cmd/node_builder.go | 9 + cmd/scaffold.go | 38 +++- cmd/verification/main.go | 10 +- engine/common/synchronization/engine.go | 64 +++--- go.sum | 3 - model/flow/identifierList.go | 27 +++ module/id/filtered_provider.go | 18 ++ module/id/id_provider.go | 141 +------------- network/middleware.go | 11 +- network/p2p/connGater.go | 8 +- network/p2p/idTranslator.go | 33 ---- network/p2p/id_translator.go | 12 ++ network/p2p/libp2pConnector.go | 35 ++-- network/p2p/libp2pNode.go | 148 +++----------- network/p2p/middleware.go | 178 +++++++---------- network/p2p/network.go | 98 +++++----- network/p2p/nodeIDRefresher.go | 82 -------- network/p2p/peerManager.go | 42 ++-- network/p2p/peerstore_provider.go | 32 +++ network/p2p/protocol_state_provider.go | 130 +++++++++++++ network/p2p/unstaked_translator.go | 43 +++++ network/topology.go | 8 +- network/topology/cache.go | 16 +- network/topology/fixedListTopology.go | 10 +- network/topology/randomizedTopology.go | 71 ++++--- network/topology/topicBasedTopology.go | 85 ++++---- network/topology/topology_utils.go | 4 +- utils/grpc/grpc.go | 9 +- 34 files changed, 821 insertions(+), 876 deletions(-) create mode 100644 module/id/filtered_provider.go delete mode 100644 network/p2p/idTranslator.go create mode 100644 network/p2p/id_translator.go delete mode 100644 network/p2p/nodeIDRefresher.go create mode 100644 network/p2p/peerstore_provider.go create mode 100644 network/p2p/protocol_state_provider.go create mode 100644 network/p2p/unstaked_translator.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 17791fff398..403cf14baea 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2,18 +2,17 @@ package node_builder import ( "context" - "encoding/json" "fmt" "strings" "time" - dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow/protobuf/go/flow/access" "github.com/spf13/pflag" "google.golang.org/grpc" "google.golang.org/grpc/credentials" "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/cmd/build" "github.com/onflow/flow-go/consensus" "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/committees" @@ -36,6 +35,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" @@ -43,7 +43,6 @@ import ( "github.com/onflow/flow-go/network" jsoncodec "github.com/onflow/flow-go/network/codec/json" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/validator" "github.com/onflow/flow-go/state/protocol" badgerState "github.com/onflow/flow-go/state/protocol/badger" "github.com/onflow/flow-go/state/protocol/blocktimer" @@ -86,9 +85,7 @@ type AccessNodeBuilder interface { // while for a node running as a library, the config fields are expected to be initialized by the caller. type AccessNodeConfig struct { staked bool - bootstrapNodeAddresses []string - bootstrapNodePublicKeys []string - bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes + stakedAccessNodeIDHex string unstakedNetworkBindAddr string collectionGRPCPort uint executionGRPCPort uint @@ -135,8 +132,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { apiRatelimits: nil, apiBurstlimits: nil, staked: true, - bootstrapNodeAddresses: []string{}, - bootstrapNodePublicKeys: []string{}, + stakedAccessNodeIDHex: "", unstakedNetworkBindAddr: cmd.NotSet, } } @@ -149,25 +145,25 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedLibP2PNode *p2p.Node - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + UnstakedNetwork *p2p.Network + unstakedMiddleware *p2p.Middleware + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower + SyncEngineParticipantsProvider id.IdentifierProvider // engines IngestEng *ingestion.Engine @@ -320,7 +316,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - node.State, + builder.SyncEngineParticipantsProvider, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) @@ -490,9 +486,9 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { type Option func(*AccessNodeConfig) -func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { +func WithUpstreamAccessNodeID(upstreamAccessNodeID flow.Identifier) Option { return func(config *AccessNodeConfig) { - config.bootstrapIdentites = bootstrapNodes + config.stakedAccessNodeIDHex = upstreamAccessNodeID.String() } } @@ -570,42 +566,50 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.StringToIntVar(&builder.apiRatelimits, "api-rate-limits", defaultConfig.apiRatelimits, "per second rate limits for Access API methods e.g. Ping=300,GetTransaction=500 etc.") flags.StringToIntVar(&builder.apiBurstlimits, "api-burst-limits", defaultConfig.apiBurstlimits, "burst limits for Access API methods e.g. Ping=100,GetTransaction=100 etc.") flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") - flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") - flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") + flags.StringVar(&builder.stakedAccessNodeIDHex, "staked-access-node-id", defaultConfig.stakedAccessNodeIDHex, "the node ID of the upstream staked access node if this is an unstaked access node") flags.StringVar(&builder.unstakedNetworkBindAddr, "unstaked-bind-addr", defaultConfig.unstakedNetworkBindAddr, "address to bind on for the unstaked network") }) } // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. // The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance -func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, +func (builder *FlowAccessNodeBuilder) initLibP2PFactory( + ctx context.Context, nodeID flow.Identifier, - networkKey crypto.PrivateKey) (p2p.LibP2PFactoryFunc, error) { + networkMetrics module.NetworkMetrics, + networkKey crypto.PrivateKey, +) (p2p.LibP2PFactoryFunc, error) { - // The staked nodes act as the DHT servers - dhtOptions := []dht.Option{p2p.AsServer(builder.IsStaked())} + // setup the Ping provider to return the software version and the sealed block height + pingProvider := p2p.PingInfoProviderImpl{ + SoftwareVersionFun: func() string { + return build.Semver() + }, + SealedBlockHeightFun: func() (uint64, error) { + head, err := builder.State.Sealed().Head() + if err != nil { + return 0, err + } + return head.Height, nil + }, + } - // if this is an unstaked access node, then seed the DHT with the boostrap identities - if !builder.IsStaked() { - bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentites) - builder.MustNot(err) - dhtOptions = append(dhtOptions, bootstrapPeersOpt) + libP2PNodeFactory, err := p2p.DefaultLibP2PNodeFactory( + ctx, + builder.Logger, + nodeID, + builder.unstakedNetworkBindAddr, + networkKey, + builder.RootBlock.ID().String(), + p2p.DefaultMaxPubSubMsgSize, + networkMetrics, + pingProvider, + ) + if err != nil { + return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) } - return func() (*p2p.Node, error) { - libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.unstakedNetworkBindAddr, networkKey). - SetRootBlockID(builder.RootBlock.ID().String()). - // unlike the staked network where currently all the node addresses are known upfront, - // for the unstaked network the nodes need to discover each other using DHT Discovery. - SetPubsubOptions(p2p.WithDHTDiscovery(dhtOptions...)). - SetLogger(builder.Logger). - Build(ctx) - if err != nil { - return nil, err - } - builder.UnstakedLibP2PNode = libp2pNode - return builder.UnstakedLibP2PNode, nil - }, nil + return libP2PNodeFactory, nil } // initMiddleware creates the network.Middleware implementation with the libp2p factory function, metrics, peer update @@ -613,16 +617,21 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, + peerUpdateInterval time.Duration, + unicastMessageTimeout time.Duration, + connectionGating bool, + managerPeerConnections bool, validators ...network.MessageValidator) *p2p.Middleware { builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, factoryFunc, nodeID, networkMetrics, builder.RootBlock.ID().String(), - time.Hour, // TODO: this is pretty meaningless since there is no peermanager in play. - p2p.DefaultUnicastTimeout, - false, // no connection gating for the unstaked network - false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) + peerUpdateInterval, + unicastMessageTimeout, + connectionGating, + managerPeerConnections, + builder.IDTranslator, validators...) return builder.unstakedMiddleware } @@ -633,7 +642,6 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, middleware *p2p.Middleware, - participants flow.IdentityList, topology network.Topology) (*p2p.Network, error) { codec := jsoncodec.NewCodec() @@ -641,64 +649,20 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, subscriptionManager := p2p.NewChannelSubscriptionManager(middleware) // creates network instance - net, err := p2p.NewNetwork(builder.Logger, + net, err := p2p.NewNetwork( + builder.Logger, codec, - participants, nodeID, builder.unstakedMiddleware, p2p.DefaultCacheSize, topology, subscriptionManager, - networkMetrics) + networkMetrics, + p2p.WithIdentifierProvider(builder.NetworkingIdentifierProvider), + ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) } return net, nil } - -func unstakedNetworkMsgValidators(selfID flow.Identifier) []network.MessageValidator { - return []network.MessageValidator{ - // filter out messages sent by this node itself - validator.ValidateNotSender(selfID), - } -} - -// BootstrapIdentities converts the bootstrap node addresses and keys to a Flow Identity list where -// each Flow Identity is initialized with the passed address, the networking key -// and the Node ID set to ZeroID, role set to Access, 0 stake and no staking key. -func BootstrapIdentities(addresses []string, keys []string) (flow.IdentityList, error) { - - if len(addresses) != len(keys) { - return nil, fmt.Errorf("number of addresses and keys provided for the boostrap nodes don't match") - } - - ids := make([]*flow.Identity, len(addresses)) - for i, address := range addresses { - - key := keys[i] - // json unmarshaller needs a quotes before and after the string - // the pflags.StringSliceVar does not retain quotes for the command line arg even if escaped with \" - // hence this additional check to ensure the key is indeed quoted - if !strings.HasPrefix(key, "\"") { - key = fmt.Sprintf("\"%s\"", key) - } - // networking public key - var networkKey encodable.NetworkPubKey - err := json.Unmarshal([]byte(key), &networkKey) - if err != nil { - return nil, err - } - - // create the identity of the peer by setting only the relevant fields - id := &flow.Identity{ - NodeID: flow.ZeroID, // the NodeID is the hash of the staking key and for the unstaked network it does not apply - Address: address, - Role: flow.RoleAccess, // the upstream node has to be an access node - NetworkPubKey: networkKey, - } - - ids = append(ids, id) - } - return ids, nil -} diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 451aa2f83ad..d022c886700 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -7,7 +7,9 @@ import ( "github.com/onflow/flow-go/cmd" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/topology" ) @@ -24,11 +26,39 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui } } +func (fnb *StakedAccessNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + // translator + // networking provider + fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ) + + // TODO: need special providers here + // for network, needs one that recognizes both protocl state and peerstore + // same for translator + + return nil + }) +} + func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) builder.Cancel = cancel + builder.InitIDProviders() + // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node builder.EnqueueNetworkInit(ctx) @@ -93,14 +123,10 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - // TODO: this list should be the unstaked addresses of all the staked AN that participate in the unstaked network - participants := flow.IdentityList{} - // topology returns empty list since peers are not known upfront top := topology.EmptyListTopology{} - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, top) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, top) builder.MustNot(err) builder.UnstakedNetwork = network diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 2b914d65db4..81c9f4600f7 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -2,12 +2,16 @@ package node_builder import ( "context" + "strings" + "time" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network/topology" ) type UnstakedAccessNodeBuilder struct { @@ -20,6 +24,13 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } +func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + return nil + }) +} + func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) @@ -27,12 +38,10 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() - builder.deriveBootstrapPeerIdentities() + builder.InitIDProviders() builder.enqueueUnstakedNetworkInit(ctx) - builder.enqueueConnectWithStakedAN() - builder.EnqueueMetricsServerInit() builder.RegisterBadgerMetrics() @@ -46,24 +55,17 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { func (builder *UnstakedAccessNodeBuilder) validateParams() { - // for an unstaked access node, the unstaked network bind address must be provided - if builder.unstakedNetworkBindAddr == cmd.NotSet { - builder.Logger.Fatal().Msg("unstaked bind address not set") + // for an unstaked access node, the staked access node ID must be provided + if strings.TrimSpace(builder.stakedAccessNodeIDHex) == "" { + builder.Logger.Fatal().Msg("staked access node ID not specified") } - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { - builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") + // and also the unstaked bind address + if builder.unstakedNetworkBindAddr == cmd.NotSet { + builder.Logger.Fatal().Msg("unstaked bind address not set") } } -// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstreap peers from the parameters. -// These are the identity of the staked and unstaked AN also acting as the DHT bootstrap server -func (builder *UnstakedAccessNodeBuilder) deriveBootstrapPeerIdentities() { - ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) - builder.MustNot(err) - builder.bootstrapIdentites = ids -} - // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files @@ -84,6 +86,16 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N } } +func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { + anb. + Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + anb.SyncEngineParticipantsProvider = node.Network.GetIdentifierProvider() + return nil + }) + anb.FlowAccessNodeBuilder.Build() + return anb +} + // enqueueUnstakedNetworkInit enqueues the unstaked network component initialized for the unstaked node func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { @@ -99,18 +111,35 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + // intialize the LibP2P factory with an empty metrics NoopCollector for now till we have defined the new unstaked + // network metrics + libP2PFactory, err := builder.FlowAccessNodeBuilder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkMetrics, unstakedNetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) + // use the default validators for the staked access node unstaked networks + msgValidators := p2p.DefaultValidators(builder.Logger, unstakedNodeID) - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + // don't need any peer updates since this will be taken care by the DHT discovery mechanism + peerUpdateInterval := time.Hour - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - participants := flow.IdentityList{} + middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, peerUpdateInterval, + node.UnicastMessageTimeout, + false, // no connection gating for the unstaked network + false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) + msgValidators...) - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, nil) + upstreamANIdentifier, err := flow.HexStringToIdentifier(builder.stakedAccessNodeIDHex) + builder.MustNot(err) + + // topology only consist of the upsteam staked AN + top := topology.NewFixedListTopology(upstreamANIdentifier) + + network, err := builder.initNetwork( + builder.Me, + unstakedNetworkMetrics, + middleware, + top, + ) builder.MustNot(err) builder.UnstakedNetwork = network @@ -125,15 +154,3 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context return builder.UnstakedNetwork, err }) } - -// enqueueConnectWithStakedAN enqueues the upstream connector component which connects the libp2p host of the unstaked -// AN with the staked AN. -// Currently, there is an issue with LibP2P stopping advertisements of subscribed topics if no peers are connected -// (https://github.com/libp2p/go-libp2p-pubsub/issues/442). This means that an unstaked AN could end up not being -// discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need -// of an explicit connect to the staked AN before the node attempts to subscribe to topics. -func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.UnstakedLibP2PNode, builder.Logger), nil - }) -} diff --git a/cmd/collection/main.go b/cmd/collection/main.go index 1e055d03a9c..ef364a5421a 100644 --- a/cmd/collection/main.go +++ b/cmd/collection/main.go @@ -35,6 +35,7 @@ import ( builder "github.com/onflow/flow-go/module/builder/collection" "github.com/onflow/flow-go/module/epochs" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/ingress" "github.com/onflow/flow-go/module/mempool" epochpool "github.com/onflow/flow-go/module/mempool/epochs" @@ -264,7 +265,13 @@ func main() { followerEng, mainChainSyncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) 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 524fd11ce4d..30b5d32f285 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -50,6 +50,7 @@ import ( dkgmodule "github.com/onflow/flow-go/module/dkg" "github.com/onflow/flow-go/module/epochs" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -617,7 +618,13 @@ func main() { comp, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/execution/main.go b/cmd/execution/main.go index 0f2987ba47e..d38d2e7a305 100644 --- a/cmd/execution/main.go +++ b/cmd/execution/main.go @@ -42,6 +42,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" chainsync "github.com/onflow/flow-go/module/synchronization" @@ -456,7 +457,13 @@ func main() { followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 9f1b45d13d2..f9e082556d5 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" @@ -43,6 +44,9 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() + // InitIDProviders initializes the IdentityProvider and IDTranslator + InitIDProviders() + // EnqueueNetworkInit enqueues the default network component with the given context EnqueueNetworkInit(ctx context.Context) @@ -133,6 +137,11 @@ type NodeConfig struct { StakingKey crypto.PrivateKey NetworkKey crypto.PrivateKey + // TODO: initialize these in scaffold and unstaked node + IdentityProvider id.IdentityProvider + IDTranslator p2p.IDTranslator + NetworkingIdentifierProvider id.IdentifierProvider + // root state information RootBlock *flow.Block RootQC *flow.QuorumCertificate diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 055c785b36d..c7912286602 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -22,6 +22,7 @@ import ( "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" @@ -176,15 +177,16 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.BaseConfig.UnicastMessageTimeout, true, true, + fnb.IDTranslator, fnb.MsgValidators...) - participants, err := fnb.State.Final().Identities(p2p.NetworkingSetFilter) - if err != nil { - return nil, fmt.Errorf("could not get network identities: %w", err) - } - subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) - top, err := topology.NewTopicBasedTopology(fnb.NodeID, fnb.Logger, fnb.State) + top, err := topology.NewTopicBasedTopology( + fnb.NodeID, + fnb.IdentityProvider, + fnb.Logger, + fnb.State, + ) if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) } @@ -193,21 +195,21 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { // creates network instance net, err := p2p.NewNetwork(fnb.Logger, codec, - participants, fnb.Me, fnb.Middleware, p2p.DefaultCacheSize, topologyCache, subscriptionManager, - fnb.Metrics.Network) + fnb.Metrics.Network, + p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) } fnb.Network = net - idRefresher := p2p.NewNodeIDRefresher(fnb.Logger, fnb.State, net.SetIDs) - idEvents := gadgets.NewIdentityDeltas(idRefresher.OnIdentityTableChanged) + idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) fnb.ProtocolEvents.AddConsumer(idEvents) return net, err @@ -416,6 +418,20 @@ func (fnb *FlowNodeBuilder) initStorage() { } } +func (fnb *FlowNodeBuilder) InitIDProviders() { + fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + fnb.IDTranslator = idCache + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NetworkingSetFilter, idCache) + return nil + }) +} + func (fnb *FlowNodeBuilder) initState() { fnb.ProtocolEvents = events.NewDistributor() @@ -715,6 +731,8 @@ func (fnb *FlowNodeBuilder) Initialize() NodeBuilder { fnb.ParseAndPrintFlags() + fnb.InitIDProviders() + fnb.EnqueueNetworkInit(ctx) if fnb.metricsEnabled { diff --git a/cmd/verification/main.go b/cmd/verification/main.go index 4860da30780..bbd2af01438 100644 --- a/cmd/verification/main.go +++ b/cmd/verification/main.go @@ -24,10 +24,12 @@ import ( "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/encoding" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" @@ -352,7 +354,13 @@ func main() { followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + node.IdentityProvider, + ), ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/engine/common/synchronization/engine.go b/engine/common/synchronization/engine.go index d067cc714f9..1a348b44bc5 100644 --- a/engine/common/synchronization/engine.go +++ b/engine/common/synchronization/engine.go @@ -15,14 +15,13 @@ import ( "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/events" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module" + identifier "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" synccore "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" ) @@ -43,11 +42,11 @@ type Engine struct { blocks storage.Blocks comp network.Engine // compliance layer engine - pollInterval time.Duration - scanInterval time.Duration - core module.SyncCore - state protocol.State - finalizedHeader *FinalizedHeaderCache + pollInterval time.Duration + scanInterval time.Duration + core module.SyncCore + participantsProvider identifier.IdentifierProvider + finalizedHeader *FinalizedHeaderCache requestHandler *RequestHandlerEngine // component responsible for handling requests @@ -66,7 +65,7 @@ func New( comp network.Engine, core module.SyncCore, finalizedHeader *FinalizedHeaderCache, - state protocol.State, + participantsProvider identifier.IdentifierProvider, opts ...OptionFunc, ) (*Engine, error) { @@ -81,18 +80,18 @@ func New( // initialize the propagation engine with its dependencies e := &Engine{ - unit: engine.NewUnit(), - lm: lifecycle.NewLifecycleManager(), - log: log.With().Str("engine", "synchronization").Logger(), - metrics: metrics, - me: me, - blocks: blocks, - comp: comp, - core: core, - pollInterval: opt.pollInterval, - scanInterval: opt.scanInterval, - finalizedHeader: finalizedHeader, - state: state, + unit: engine.NewUnit(), + lm: lifecycle.NewLifecycleManager(), + log: log.With().Str("engine", "synchronization").Logger(), + metrics: metrics, + me: me, + blocks: blocks, + comp: comp, + core: core, + pollInterval: opt.pollInterval, + scanInterval: opt.scanInterval, + finalizedHeader: finalizedHeader, + participantsProvider: participantsProvider, } err := e.setupResponseMessageHandler() @@ -332,7 +331,7 @@ CheckLoop: e.pollHeight() case <-scan.C: head := e.finalizedHeader.Get() - participants := e.getParticipants(head.ID()) + participants := e.participantsProvider.Identifiers() ranges, batches := e.core.ScanPending(head) e.sendRequests(participants, ranges, batches) } @@ -342,30 +341,17 @@ CheckLoop: scan.Stop() } -// getParticipants gets all of the consensus nodes from the state at the given block ID. -func (e *Engine) getParticipants(blockID flow.Identifier) flow.IdentityList { - participants, err := e.state.AtBlockID(blockID).Identities(filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(e.me.NodeID())), - )) - if err != nil { - e.log.Fatal().Err(err).Msgf("could not get consensus participants at block ID %v", blockID) - } - - return participants -} - // pollHeight will send a synchronization request to three random nodes. func (e *Engine) pollHeight() { head := e.finalizedHeader.Get() - participants := e.getParticipants(head.ID()) + participants := e.participantsProvider.Identifiers() // send the request for synchronization req := &messages.SyncRequest{ Nonce: rand.Uint64(), Height: head.Height, } - err := e.con.Multicast(req, synccore.DefaultPollNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultPollNodes, participants...) if err != nil { e.log.Warn().Err(err).Msg("sending sync request to poll heights failed") return @@ -374,7 +360,7 @@ func (e *Engine) pollHeight() { } // sendRequests sends a request for each range and batch using consensus participants from last finalized snapshot. -func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Range, batches []flow.Batch) { +func (e *Engine) sendRequests(participants flow.IdentifierList, ranges []flow.Range, batches []flow.Batch) { var errs *multierror.Error for _, ran := range ranges { @@ -383,7 +369,7 @@ func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Rang FromHeight: ran.From, ToHeight: ran.To, } - err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants...) if err != nil { errs = multierror.Append(errs, fmt.Errorf("could not submit range request: %w", err)) continue @@ -402,7 +388,7 @@ func (e *Engine) sendRequests(participants flow.IdentityList, ranges []flow.Rang Nonce: rand.Uint64(), BlockIDs: batch.BlockIDs, } - err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants.NodeIDs()...) + err := e.con.Multicast(req, synccore.DefaultBlockRequestNodes, participants...) if err != nil { errs = multierror.Append(errs, fmt.Errorf("could not submit batch request: %w", err)) continue diff --git a/go.sum b/go.sum index d219a1e0429..215ae0e4ccf 100644 --- a/go.sum +++ b/go.sum @@ -950,7 +950,6 @@ github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:v github.com/olekukonko/tablewriter v0.0.1/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.2-0.20190409134802-7e037d187b0c/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onflow/cadence v0.15.0/go.mod h1:KMzDF6cIv6nb5PJW9aITaqazbmJX8MMeibFcpPP385M= -github.com/onflow/cadence v0.15.0/go.mod h1:KMzDF6cIv6nb5PJW9aITaqazbmJX8MMeibFcpPP385M= github.com/onflow/cadence v0.17.0/go.mod h1:iR/tZpP+1YhM8iRnOBPiBIs7on5dE3hk2ZfunCRQswE= github.com/onflow/cadence v0.18.0/go.mod h1:iR/tZpP+1YhM8iRnOBPiBIs7on5dE3hk2ZfunCRQswE= github.com/onflow/cadence v0.18.1-0.20210730161646-b891a21c51fd h1:+J8msFNYPhdjgPDnQm3uH2q4TQFHAQXAxWTNOJ5VOec= @@ -963,8 +962,6 @@ github.com/onflow/flow-core-contracts/lib/go/templates v0.7.7/go.mod h1:oWNy8Wz5 github.com/onflow/flow-emulator v0.20.3 h1:qsxKp8oty1glaqEyUfRWtsY0qRgTZfejwGiFix2MYzI= github.com/onflow/flow-emulator v0.20.3/go.mod h1:xNdVsrMJiAaYJ59Dwo+xvj0ENdvk/bI14zkGN4V0ozs= github.com/onflow/flow-ft/lib/go/contracts v0.5.0 h1:Cg4gHGVblxcejfNNG5Mfj98Wf4zbY76O0Y28QB0766A= -github.com/onflow/flow-ft/lib/go/contracts v0.5.0 h1:Cg4gHGVblxcejfNNG5Mfj98Wf4zbY76O0Y28QB0766A= -github.com/onflow/flow-ft/lib/go/contracts v0.5.0/go.mod h1:1zoTjp1KzNnOPkyqKmWKerUyf0gciw+e6tAEt0Ks3JE= github.com/onflow/flow-ft/lib/go/contracts v0.5.0/go.mod h1:1zoTjp1KzNnOPkyqKmWKerUyf0gciw+e6tAEt0Ks3JE= github.com/onflow/flow-go v0.18.0/go.mod h1:cQpvFoqth9PR7tarWDa36R/dDOqUK5QYfeYzCdXPLII= github.com/onflow/flow-go-sdk v0.20.0-alpha.1/go.mod h1:52QZyLwU3p3UZ2FXOy+sRl4JPdtvJoae1spIUBOFxA8= diff --git a/model/flow/identifierList.go b/model/flow/identifierList.go index cb1d4780b49..224340b7b02 100644 --- a/model/flow/identifierList.go +++ b/model/flow/identifierList.go @@ -2,6 +2,7 @@ package flow import ( "bytes" + "math/rand" "github.com/rs/zerolog/log" ) @@ -70,6 +71,32 @@ func (il IdentifierList) Contains(target Identifier) bool { return false } +// Union returns a new identifier list containing the union of `il` and `other`. +// There are no duplicates in the output. +func (il IdentifierList) Union(other IdentifierList) IdentifierList { + // stores the output, the union of the two lists + union := make(IdentifierList, 0, len(il)+len(other)) + // efficient lookup to avoid duplicates + lookup := make(map[Identifier]struct{}) + + // add all identifiers, omitted duplicates + for _, identifier := range append(il.Copy(), other...) { + if _, exists := lookup[identifier]; exists { + continue + } + union = append(union, identifier) + lookup[identifier] = struct{}{} + } + + return union +} + +// DeterministicSample returns deterministic random sample from the `IdentifierList` using the given seed +func (il IdentifierList) DeterministicSample(size uint, seed int64) IdentifierList { + rand.Seed(seed) + return il.Sample(size) +} + // Sample returns random sample of length 'size' of the ids // [Fisher-Yates shuffle](https://en.wikipedia.org/wiki/Fisher–Yates_shuffle). func (il IdentifierList) Sample(size uint) IdentifierList { diff --git a/module/id/filtered_provider.go b/module/id/filtered_provider.go new file mode 100644 index 00000000000..d935125232f --- /dev/null +++ b/module/id/filtered_provider.go @@ -0,0 +1,18 @@ +package id + +import ( + "github.com/onflow/flow-go/model/flow" +) + +type FilteredIdentifierProvider struct { + filter flow.IdentityFilter + identityProvider IdentityProvider +} + +func NewFilteredIdentifierProvider(filter flow.IdentityFilter, identityProvider IdentityProvider) *FilteredIdentifierProvider { + return &FilteredIdentifierProvider{filter, identityProvider} +} + +func (p *FilteredIdentifierProvider) Identifiers() flow.IdentifierList { + return p.identityProvider.Identities(p.filter).NodeIDs() +} diff --git a/module/id/id_provider.go b/module/id/id_provider.go index 430780b6dc8..a663a8f9215 100644 --- a/module/id/id_provider.go +++ b/module/id/id_provider.go @@ -1,148 +1,13 @@ -package identity +package id import ( - "sync" - - "github.com/libp2p/go-libp2p-core/peer" - "github.com/libp2p/go-libp2p-core/peerstore" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/state/protocol/events" ) type IdentifierProvider interface { Identifiers() flow.IdentifierList } -// TODO: rename to ProtocolStateIDProvider -type ProtocolStateIdentifierProvider struct { - events.Noop - identities flow.IdentityList // TODO: actually we *can* just use identifiers after all - state protocol.State - mu sync.RWMutex - filter flow.IdentityFilter - peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! - flowIDs map[peer.ID]flow.Identifier -} - -func WithFilter(filter flow.IdentityFilter) ProtocolStateIdentifierProviderOption { - return func(provider *ProtocolStateIdentifierProvider) { - provider.filter = filter - } -} - -type ProtocolStateIdentifierProviderOption func(*ProtocolStateIdentifierProvider) - -// TODO: this one also implements IDTranslator!!! -func NewProtocolStateIdentifierProvider( - state protocol.State, - eventDistributer *events.Distributor, - opts ...ProtocolStateIdentifierProviderOption, -) (*ProtocolStateIdentifierProvider, error) { - provider := &ProtocolStateIdentifierProvider{ - state: state, - filter: filter.Any, - } - - for _, opt := range opts { - opt(provider) - } - - head, err := state.Final().Head() - if err != nil { - return nil, err // TODO: format the error - } - - provider.update(head.ID()) - eventDistributer.AddConsumer(provider) - - return provider, nil -} - -func (p *ProtocolStateIdentifierProvider) EpochTransition(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { - p.update(header.ID()) -} - -func (p *ProtocolStateIdentifierProvider) update(blockID flow.Identifier) { - identities, err := p.state.AtBlockID(blockID).Identities(p.filter) - if err != nil { - // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. - } - - nIds := identities.Count() - - peerIDs := make(map[flow.Identifier]peer.ID, nIds) - flowIDs := make(map[peer.ID]flow.Identifier, nIds) - - for _, identity := range identities { - pid, err := p2p.IdentityToPeerID(identity) - if err != nil { - // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping - } - - flowIDs[pid] = identity.NodeID - peerIDs[identity.NodeID] = pid - } - - p.mu.Lock() - defer p.mu.Unlock() - p.identities = identities - p.flowIDs = flowIDs - p.peerIDs = peerIDs -} - -func (p *ProtocolStateIdentifierProvider) Identifiers() flow.IdentifierList { - p.mu.RLock() - defer p.mu.RUnlock() - return p.identities.NodeIDs() -} - -func (p *ProtocolStateIdentifierProvider) GetPeerID(flowID flow.Identifier) (pid peer.ID, found bool) { - p.mu.RLock() - defer p.mu.RUnlock() - - pid, found = p.peerIDs[flowID] - return -} - -func (p *ProtocolStateIdentifierProvider) GetFlowID(peerID peer.ID) (fid flow.Identifier, found bool) { - p.mu.RLock() - defer p.mu.RUnlock() - - fid, found = p.flowIDs[peerID] - return -} - -type PeerstoreIdentifierProvider struct { - store peerstore.Peerstore - idTranslator p2p.IDTranslator -} - -func NewPeerstoreIdentifierProvider(store peerstore.Peerstore, idTranslator p2p.IDTranslator) *PeerstoreIdentifierProvider { - return &PeerstoreIdentifierProvider{store: store, idTranslator: idTranslator} -} - -func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { - var result flow.IdentifierList - - pids := p.store.PeersWithAddrs() // TODO: should we just call Peers here? - for _, pid := range pids { - flowID, success := p.idTranslator.GetFlowID(pid) - if success { - result = append(result, flowID) - } - } - - return result +type IdentityProvider interface { + Identities(flow.IdentityFilter) flow.IdentityList } diff --git a/network/middleware.go b/network/middleware.go index ed935995e6c..82928584d43 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -6,6 +6,7 @@ import ( "time" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/message" ) @@ -51,17 +52,17 @@ type Middleware interface { // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. - UpdateAllowList() error + UpdateAllowList() } // Overlay represents the interface that middleware uses to interact with the // overlay network layer. type Overlay interface { - // Topology returns an identity list of nodes which this node should be directly connected to as peers - Topology() (flow.IdentityList, error) - // Identity returns a map of all identifier to flow identity - Identity() (map[flow.Identifier]flow.Identity, error) + // Topology returns an identifier list of nodes which this node should be directly connected to as peers + Topology() (flow.IdentifierList, error) Receive(nodeID flow.Identifier, msg *message.Message) error + SetDefaultIdentifierProvider(id.IdentifierProvider) + GetIdentifierProvider() id.IdentifierProvider } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/connGater.go b/network/p2p/connGater.go index 10a944a5f04..caa05bb1f1a 100644 --- a/network/p2p/connGater.go +++ b/network/p2p/connGater.go @@ -29,14 +29,14 @@ func NewConnGater(log zerolog.Logger) *ConnGater { } // update updates the peer ID map -func (c *ConnGater) update(peerInfos []peer.AddrInfo) { +func (c *ConnGater) update(pids []peer.ID) { // create a new peer.ID map - peerIDs := make(map[peer.ID]struct{}, len(peerInfos)) + peerIDs := make(map[peer.ID]struct{}, len(pids)) // for each peer.AddrInfo, create an entry in the map for the peer.ID - for _, p := range peerInfos { - peerIDs[p.ID] = struct{}{} + for _, pid := range pids { + peerIDs[pid] = struct{}{} } // cache the new map diff --git a/network/p2p/idTranslator.go b/network/p2p/idTranslator.go deleted file mode 100644 index f9f567f1fec..00000000000 --- a/network/p2p/idTranslator.go +++ /dev/null @@ -1,33 +0,0 @@ -package p2p - -import ( - "github.com/libp2p/go-libp2p-core/peer" - - "github.com/onflow/flow-go/model/flow" -) - -type IDTranslator interface { - GetPeerID(flow.Identifier) (peer.ID, bool) - GetFlowID(peer.ID) (flow.Identifier, bool) -} - -type UnstakedNetworkPeerIDProvider struct { - // basically, just do the conversion from flow ID to peer ID using whatever scheme we talked about. - // whether this be, with caching or not -} - -func IdentityToPeerID(id *flow.Identity) (pid peer.ID, err error) { - pk, err := PublicKey(id.NetworkPubKey) - if err != nil { - // TODO: format the error - return - } - - pid, err = peer.IDFromPublicKey(pk) - if err != nil { - // TODO: format the error - return - } - - return -} diff --git a/network/p2p/id_translator.go b/network/p2p/id_translator.go new file mode 100644 index 00000000000..8edb2d500d5 --- /dev/null +++ b/network/p2p/id_translator.go @@ -0,0 +1,12 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type IDTranslator interface { + GetPeerID(flow.Identifier) (peer.ID, error) + GetFlowID(peer.ID) (flow.Identifier, error) +} diff --git a/network/p2p/libp2pConnector.go b/network/p2p/libp2pConnector.go index 71c5b550635..64e68a43170 100644 --- a/network/p2p/libp2pConnector.go +++ b/network/p2p/libp2pConnector.go @@ -65,34 +65,23 @@ func newLibp2pConnector(host host.Host, log zerolog.Logger) (*libp2pConnector, e // UpdatePeers is the implementation of the Connector.UpdatePeers function. It connects to all of the ids and // disconnects from any other connection that the libp2p node might have. -func (l *libp2pConnector) UpdatePeers(ctx context.Context, ids flow.IdentityList) error { - - // derive the peer.AddrInfo from each of the flow.Identity - pInfos, invalidIDs := peerInfosFromIDs(ids) - +func (l *libp2pConnector) UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) { // connect to each of the peer.AddrInfo in pInfos - l.connectToPeers(ctx, pInfos) + l.connectToPeers(ctx, peerIDs) // disconnect from any other peers not in pInfos - l.trimAllConnectionsExcept(pInfos) - - // if some ids didn't translate to peer.AddrInfo, return error - if len(invalidIDs) != 0 { - return NewUnconvertableIdentitiesError(invalidIDs) - } - - return nil + l.trimAllConnectionsExcept(peerIDs) } // connectToPeers connects each of the peer in pInfos -func (l *libp2pConnector) connectToPeers(ctx context.Context, pInfos []peer.AddrInfo) { +func (l *libp2pConnector) connectToPeers(ctx context.Context, peerIDs peer.IDSlice) { // create a channel of peer.AddrInfo as expected by the connector - peerCh := make(chan peer.AddrInfo, len(pInfos)) + peerCh := make(chan peer.AddrInfo, len(peerIDs)) // stuff all the peer.AddrInfo it into the channel - for _, peerInfo := range pInfos { - peerCh <- peerInfo + for _, peerID := range peerIDs { + peerCh <- peer.AddrInfo{ID: peerID} } // close the channel to ensure Connect does not block @@ -102,15 +91,15 @@ func (l *libp2pConnector) connectToPeers(ctx context.Context, pInfos []peer.Addr l.backoffConnector.Connect(ctx, peerCh) } -// trimAllConnectionsExcept trims all connections of the node from peers not part of peerInfos. +// trimAllConnectionsExcept trims all connections of the node from peers not part of peerIDs. // A node would have created such extra connections earlier when the identity list may have been different, or // it may have been target of such connections from node which have now been excluded. -func (l *libp2pConnector) trimAllConnectionsExcept(peerInfos []peer.AddrInfo) { +func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs []peer.ID) { // convert the peerInfos to a peer.ID -> bool map - peersToKeep := make(map[peer.ID]bool, len(peerInfos)) - for _, pInfo := range peerInfos { - peersToKeep[pInfo.ID] = true + peersToKeep := make(map[peer.ID]bool, len(peerIDs)) + for _, pid := range peerIDs { + peersToKeep[pid] = true } // get all current node connections diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 09fb7907b47..f713af8f62f 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -17,7 +17,6 @@ import ( libp2pnet "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/protocol" - dht "github.com/libp2p/go-libp2p-kad-dht" pubsub "github.com/libp2p/go-libp2p-pubsub" swarm "github.com/libp2p/go-libp2p-swarm" tptu "github.com/libp2p/go-libp2p-transport-upgrader" @@ -70,7 +69,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize)...). + SetPubsubOptions(psOptions...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). // SetResolver(resolver). @@ -82,7 +81,7 @@ type NodeBuilder interface { SetRootBlockID(string) NodeBuilder SetConnectionManager(TagLessConnManager) NodeBuilder SetConnectionGater(*ConnGater) NodeBuilder - SetPubsubOptions(...PubsubOption) NodeBuilder + SetPubsubOptions(...pubsub.Option) NodeBuilder SetPingInfoProvider(PingInfoProvider) NodeBuilder SetLogger(zerolog.Logger) NodeBuilder SetResolver(resolver *madns.Resolver) NodeBuilder @@ -99,7 +98,7 @@ type DefaultLibP2PNodeBuilder struct { resolver *madns.Resolver pubSubMaker func(context.Context, host.Host, ...pubsub.Option) (*pubsub.PubSub, error) hostMaker func(context.Context, ...config.Option) (host.Host, error) - pubSubOpts []PubsubOption + pubSubOpts []pubsub.Option } func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcrypto.PrivateKey) NodeBuilder { @@ -129,7 +128,7 @@ func (builder *DefaultLibP2PNodeBuilder) SetConnectionGater(connGater *ConnGater return builder } -func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...PubsubOption) NodeBuilder { +func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...pubsub.Option) NodeBuilder { builder.pubSubOpts = opts return builder } @@ -182,11 +181,6 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.connMgr = builder.connMngr } - if builder.rootBlockID == "" { - return nil, errors.New("root block ID must be provided") - } - node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) - if builder.pingInfoProvider != nil { opts = append(opts, libp2p.Ping(true)) } @@ -207,17 +201,7 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.pingService = pingService } - var libp2pPSOptions []pubsub.Option - // generate the libp2p Pubsub options from the given context and host - for _, optionGenerator := range builder.pubSubOpts { - option, err := optionGenerator(ctx, libp2pHost) - if err != nil { - return nil, err - } - libp2pPSOptions = append(libp2pPSOptions, option) - } - - ps, err := builder.pubSubMaker(ctx, libp2pHost, libp2pPSOptions...) + ps, err := builder.pubSubMaker(ctx, libp2pHost, builder.pubSubOpts...) if err != nil { return nil, err } @@ -312,13 +296,8 @@ func (n *Node) Stop() (chan struct{}, error) { } // AddPeer adds a peer to this node by adding it to this node's peerstore and connecting to it -func (n *Node) AddPeer(ctx context.Context, identity flow.Identity) error { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return fmt.Errorf("failed to add peer %s: %w", identity.String(), err) - } - - err = n.host.Connect(ctx, pInfo) +func (n *Node) AddPeer(ctx context.Context, peerID peer.ID) error { + err := n.host.Connect(ctx, peer.AddrInfo{ID: peerID}) if err != nil { return err } @@ -326,51 +305,29 @@ func (n *Node) AddPeer(ctx context.Context, identity flow.Identity) error { return nil } -// RemovePeer closes the connection with the identity. -func (n *Node) RemovePeer(ctx context.Context, identity flow.Identity) error { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return fmt.Errorf("failed to remove peer %x: %w", identity, err) - } - - err = n.host.Network().ClosePeer(pInfo.ID) +// RemovePeer closes the connection with the peer. +func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { + err := n.host.Network().ClosePeer(peerID) if err != nil { - return fmt.Errorf("failed to remove peer %s: %w", identity, err) + return fmt.Errorf("failed to remove peer %s: %w", peerID, err) } return nil } -// CreateStream returns an existing stream connected to identity, if it exists or adds one to identity as a peer and creates a new stream with it. -func (n *Node) CreateStream(ctx context.Context, nodeID flow.Identifier) (libp2pnet.Stream, error) { +// CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. +func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) - stream, err := n.tryCreateNewStream(ctx, nodeID, maxConnectAttempt) + stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) if err != nil { - n.host.Peerstore().Addrs() - return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (node_id: %s, address: %s): %w", identity.NodeID.String(), - identity.Address, err)) + return nil, flownet.NewPeerUnreachableError(fmt.Errorf("could not create stream (peer_id: %s): %w", peerID, err)) } return stream, nil } -func (n *Node) GetPeerIPPort(nodeID flow.Identifier) (string, string, error) { - // TODO: first get peer ID - x := n.host.Peerstore().Addrs() -} - -// tryCreateNewStream makes at most maxAttempts to create a stream with the identity. +// tryCreateNewStream makes at most maxAttempts to create a stream with the peer. // This was put in as a fix for #2416. PubSub and 1-1 communication compete with each other when trying to connect to // remote nodes and once in a while NewStream returns an error 'both yamux endpoints are clients' -func (n *Node) tryCreateNewStream(ctx context.Context, identity flow.Identity, maxAttempts int) (libp2pnet.Stream, error) { - _, _, key, err := networkingInfo(identity) - if err != nil { - return nil, fmt.Errorf("could not get translate identity to networking info %s: %w", identity.NodeID.String(), err) - } - - peerID, err := peer.IDFromPublicKey(key) - if err != nil { - return nil, fmt.Errorf("could not get peer ID: %w", err) - } - +func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttempts int) (libp2pnet.Stream, error) { // protect the underlying connection from being inadvertently pruned by the peer manager while the stream and // connection creation is being attempted n.connMgr.ProtectPeer(peerID) @@ -405,7 +362,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, identity flow.Identity, m time.Sleep(time.Duration(r) * time.Millisecond) } - err = n.AddPeer(ctx, identity) + err := n.AddPeer(ctx, peerID) if err != nil { // if the connection was rejected due to invalid node id, skip the re-attempt @@ -527,23 +484,18 @@ func (n *Node) Publish(ctx context.Context, topic flownet.Topic, data []byte) er } // Ping pings a remote node and returns the time it took to ping the remote node if successful or the error -func (n *Node) Ping(ctx context.Context, identity flow.Identity) (message.PingResponse, time.Duration, error) { - +func (n *Node) Ping(ctx context.Context, peerID peer.ID) (message.PingResponse, time.Duration, error) { pingError := func(err error) error { - return fmt.Errorf("failed to ping %s (%s): %w", identity.NodeID.String(), identity.Address, err) + return fmt.Errorf("failed to ping peer %s: %w", peerID, err) } - // convert the target node address to libp2p peer info - targetInfo, err := PeerAddressInfo(identity) - if err != nil { - return message.PingResponse{}, -1, pingError(err) - } + targetInfo := peer.AddrInfo{ID: peerID} n.connMgr.ProtectPeer(targetInfo.ID) defer n.connMgr.UnprotectPeer(targetInfo.ID) // connect to the target node - err = n.host.Connect(ctx, targetInfo) + err := n.host.Connect(ctx, targetInfo) if err != nil { return message.PingResponse{}, -1, pingError(err) } @@ -558,22 +510,20 @@ func (n *Node) Ping(ctx context.Context, identity flow.Identity) (message.PingRe } // UpdateAllowList allows the peer allow list to be updated. -func (n *Node) UpdateAllowList(identities flow.IdentityList) error { - // if the node was so far not under allowList +func (n *Node) UpdateAllowList(peers []peer.ID) { if n.connGater == nil { - return fmt.Errorf("could not add an allow list, this node was started without allow listing") - + n.logger.Debug().Hex("node_id", logging.ID(n.id)).Msg("skipping update allow list, connection gating is not enabled") + return } // generates peer address information for all identities - allowlist := make([]peer.AddrInfo, 0, len(identities)) - for _, identity := range identities { - addressInfo, err := PeerAddressInfo(*identity) + allowlist := make([]peer.AddrInfo, len(identities)) + var err error + for i, identity := range identities { + allowlist[i], err = PeerAddressInfo(*identity) if err != nil { - n.logger.Err(err).Str("identity", identity.String()).Msg("could not generate address info") - continue + return fmt.Errorf("could not generate address info: %w", err) } - allowlist = append(allowlist, addressInfo) } n.connGater.update(allowlist) @@ -596,13 +546,9 @@ func (n *Node) SetPingStreamHandler(handler libp2pnet.StreamHandler) { } // IsConnected returns true is address is a direct peer of this node else false -func (n *Node) IsConnected(identity flow.Identity) (bool, error) { - pInfo, err := PeerAddressInfo(identity) - if err != nil { - return false, err - } +func (n *Node) IsConnected(peerID peer.ID) (bool, error) { // query libp2p for connectedness status of this peer - isConnected := n.host.Network().Connectedness(pInfo.ID) == libp2pnet.Connected + isConnected := n.host.Network().Connectedness(peerID) == libp2pnet.Connected return isConnected, nil } @@ -674,33 +620,3 @@ func DefaultPubSub(ctx context.Context, host host.Host, psOption ...pubsub.Optio } return pubSub, nil } - -// PubsubOption generates a libp2p pubsub.Option from the given context and host -type PubsubOption func(ctx context.Context, host host.Host) (pubsub.Option, error) - -func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { - pubSubOptionFunc := func(option pubsub.Option) PubsubOption { - return func(_ context.Context, _ host.Host) (pubsub.Option, error) { - return option, nil - } - } - return []PubsubOption{ - // skip message signing - pubSubOptionFunc(pubsub.WithMessageSigning(true)), - // skip message signature - pubSubOptionFunc(pubsub.WithStrictSignatureVerification(true)), - // set max message size limit for 1-k PubSub messaging - pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), - // no discovery - } -} - -func WithDHTDiscovery(option ...dht.Option) PubsubOption { - return func(ctx context.Context, host host.Host) (pubsub.Option, error) { - dhtDiscovery, err := NewDHT(ctx, host, option...) - if err != nil { - return nil, err - } - return pubsub.WithDiscovery(dhtDiscovery), nil - } -} diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 49319fdc627..f24ad9c54a3 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -11,9 +11,9 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -78,6 +78,7 @@ type Middleware struct { unicastMessageTimeout time.Duration connectionGating bool managePeerConnections bool + idTranslator IDTranslator } // NewMiddleware creates a new middleware instance @@ -89,7 +90,8 @@ type Middleware struct { // connectionGating if set to True, restricts this node to only talk to other nodes which are part of the identity list // managePeerConnections if set to True, enables the default PeerManager which continuously updates the node's peer connections // validators are the set of the different message validators that each inbound messages is passed through -func NewMiddleware(log zerolog.Logger, +func NewMiddleware( + log zerolog.Logger, libP2PNodeFactory LibP2PFactoryFunc, flowID flow.Identifier, metrics module.NetworkMetrics, @@ -98,13 +100,9 @@ func NewMiddleware(log zerolog.Logger, unicastMessageTimeout time.Duration, connectionGating bool, managePeerConnections bool, - validators ...network.MessageValidator) *Middleware { - - if len(validators) == 0 { - // add default validators to filter out unwanted messages received by this node - validators = DefaultValidators(log, flowID) - } - + idTranslator IDTranslator, + validators ...network.MessageValidator, +) *Middleware { ctx, cancel := context.WithCancel(context.Background()) if unicastMessageTimeout <= 0 { @@ -112,7 +110,7 @@ func NewMiddleware(log zerolog.Logger, } // create the node entity and inject dependencies & config - return &Middleware{ + mw := &Middleware{ ctx: ctx, cancel: cancel, log: log, @@ -121,12 +119,19 @@ func NewMiddleware(log zerolog.Logger, libP2PNodeFactory: libP2PNodeFactory, metrics: metrics, rootBlockID: rootBlockID, - validators: validators, + validators: DefaultValidators(log, flowID), peerUpdateInterval: peerUpdateInterval, unicastMessageTimeout: unicastMessageTimeout, connectionGating: connectionGating, managePeerConnections: managePeerConnections, + idTranslator: idTranslator, } + + if len(validators) != 0 { + mw.validators = validators + } + + return mw } func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.MessageValidator { @@ -136,6 +141,35 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes } } +func (m *Middleware) topologyPeers() (peer.IDSlice, error) { + identifiers, err := m.ov.Topology() + if err != nil { + // TODO: format error + return nil, err + } + + return m.peerIDs(identifiers), nil +} + +func (m *Middleware) allPeers() peer.IDSlice { + return m.peerIDs(m.ov.GetIdentifierProvider().Identifiers()) +} + +func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { + result := make([]peer.ID, len(flowIDs)) + + for _, fid := range flowIDs { + pid, err := m.idTranslator.GetPeerID(fid) + if err != nil { + // TODO: log here + } + + result = append(result, pid) + } + + return result +} + // Me returns the flow identifier of the this middleware func (m *Middleware) Me() flow.Identifier { return m.me @@ -151,6 +185,8 @@ func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov libP2PNode, err := m.libP2PNodeFactory() + ov.SetDefaultIdentifierProvider(NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator)) + if err != nil { return fmt.Errorf("could not create libp2p node: %w", err) } @@ -158,17 +194,7 @@ func (m *Middleware) Start(ov network.Overlay) error { m.libP2PNode.SetFlowProtocolStreamHandler(m.handleIncomingStream) if m.connectionGating { - - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return fmt.Errorf("could not get identities: %w", err) - } - - err = m.libP2PNode.UpdateAllowList(identityList(idsMap)) - if err != nil { - return fmt.Errorf("could not update approved peer list: %w", err) - } + m.libP2PNode.UpdateAllowList(m.allPeers()) } if m.managePeerConnections { @@ -177,7 +203,7 @@ func (m *Middleware) Start(ov network.Overlay) error { return fmt.Errorf("failed to create libp2pConnector: %w", err) } - m.peerManager = NewPeerManager(m.log, m.ov.Topology, libp2pConnector, WithInterval(m.peerUpdateInterval)) + m.peerManager = NewPeerManager(m.log, m.topologyPeers, libp2pConnector, WithInterval(m.peerUpdateInterval)) select { case <-m.peerManager.Ready(): m.log.Debug().Msg("peer manager successfully started") @@ -222,10 +248,10 @@ func (m *Middleware) Stop() { // Dispatch should be used whenever guaranteed delivery to a specific target is required. Otherwise, Publish is // a more efficient candidate. func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) error { - // translates identifier to identity - targetIdentity, err := m.identity(targetID) + // translates identifier to peer id + peerID, err := m.idTranslator.GetPeerID(targetID) if err != nil { - return fmt.Errorf("could not find identity for target id: %w", err) + return fmt.Errorf("could not find peer id for target id: %w", err) } maxMsgSize := unicastMaxMsgSize(msg) @@ -245,9 +271,9 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) // (streams don't need to be reused and are fairly inexpensive to be created for each send. // A stream creation does NOT incur an RTT as stream negotiation happens as part of the first message // sent out the the receiver - stream, err := m.libP2PNode.CreateStream(ctx, targetIdentity) + stream, err := m.libP2PNode.CreateStream(ctx, peerID) if err != nil { - return fmt.Errorf("failed to create stream for %s :%w", targetID.String(), err) + return fmt.Errorf("failed to create stream for %s :%w", targetID, err) } // create a gogo protobuf writer @@ -256,19 +282,19 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) err = writer.WriteMsg(msg) if err != nil { - return fmt.Errorf("failed to send message to %s: %w", targetID.String(), err) + return fmt.Errorf("failed to send message to %s: %w", targetID, err) } // flush the stream err = bufw.Flush() if err != nil { - return fmt.Errorf("failed to flush stream for %s: %w", targetIdentity.String(), err) + return fmt.Errorf("failed to flush stream for %s: %w", targetID, err) } // close the stream immediately err = stream.Close() if err != nil { - return fmt.Errorf("failed to close the stream for %s: %w", targetIdentity.String(), err) + return fmt.Errorf("failed to close the stream for %s: %w", targetID, err) } // OneToOne communication metrics are reported with topic OneToOne @@ -277,36 +303,6 @@ func (m *Middleware) SendDirect(msg *message.Message, targetID flow.Identifier) return nil } -// identity returns corresponding identity of an identifier based on overlay identity list. -func (m *Middleware) identity(identifier flow.Identifier) (flow.Identity, error) { - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return flow.Identity{}, fmt.Errorf("could not get identities: %w", err) - } - - // retrieve the flow.Identity for the give flow.ID - flowIdentity, found := idsMap[identifier] - if !found { - return flow.Identity{}, fmt.Errorf("could not get node identity for %s: %w", identifier.String(), err) - } - - return flowIdentity, nil -} - -// identityList translates an identity map into an identity list. -func identityList(identityMap map[flow.Identifier]flow.Identity) flow.IdentityList { - var identities flow.IdentityList - for _, identity := range identityMap { - // casts identity into a local variable to - // avoid shallow copy of the loop variable - id := identity - identities = append(identities, &id) - - } - return identities -} - // handleIncomingStream handles an incoming stream from a remote peer // it is a callback that gets called for each incoming stream by libp2p with a new stream object func (m *Middleware) handleIncomingStream(s libp2pnetwork.Stream) { @@ -361,35 +357,6 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { return nil } -// processAuthenticatedMessage processes a message and a source (indicated by its PublicKey) and eventually passes it to the overlay -// In particular, it checks the claim of protocol authorship situated in the message against `originKey` -// The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` -// this requirement is fulfilled by e.g. the output of readConnection and readSubscription -func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { - identities, err := m.ov.Identity() - if err != nil { - m.log.Error().Err(err).Msg("failed to retrieve identities list while delivering a message") - return - } - - // check the origin of the message corresponds to the one claimed in the OriginID - originID := flow.HashToID(msg.OriginID) - - originIdentity, found := identities[originID] - if !found { - m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) - return - } else if originIdentity.NetworkPubKey == nil { - m.log.Warn().Msgf("received message with claimed originID %x, wich has no network identifiers at this node, and was dropped", originID) - return - } else if !originIdentity.NetworkPubKey.Equals(originKey) { - m.log.Warn().Msgf("received message claiming to be from nodeID %v with key %x was actually signed by %x and dropped", originID, originIdentity.NetworkPubKey, originKey) - return - } - - m.processMessage(msg) -} - // processMessage processes a message and eventually passes it to the overlay func (m *Middleware) processMessage(msg *message.Message) { @@ -443,40 +410,33 @@ func (m *Middleware) Publish(msg *message.Message, channel network.Channel) erro // Ping pings the target node and returns the ping RTT or an error func (m *Middleware) Ping(targetID flow.Identifier) (message.PingResponse, time.Duration, error) { - targetIdentity, err := m.identity(targetID) + peerID, err := m.idTranslator.GetPeerID(targetID) if err != nil { - return message.PingResponse{}, -1, fmt.Errorf("could not find identity for target id: %w", err) + return message.PingResponse{}, -1, fmt.Errorf("could not find peer id for target id: %w", err) } - return m.libP2PNode.Ping(m.ctx, targetIdentity) + return m.libP2PNode.Ping(m.ctx, peerID) } -// UpdateAllowList fetches the most recent identity of the nodes from overlay +// UpdateAllowList fetches the most recent identifiers of the nodes from overlay // and updates the underlying libp2p node. -func (m *Middleware) UpdateAllowList() error { - // get the node identity map from the overlay - idsMap, err := m.ov.Identity() - if err != nil { - return fmt.Errorf("could not get identities: %w", err) - } - +func (m *Middleware) UpdateAllowList() { // update libp2pNode's approve lists if this middleware also does connection gating if m.connectionGating { - err = m.libP2PNode.UpdateAllowList(identityList(idsMap)) - if err != nil { - return fmt.Errorf("failed to update approved peer list: %w", err) - } + m.libP2PNode.UpdateAllowList(m.allPeers()) } // update peer connections if this middleware also does peer management m.peerManagerUpdate() - - return nil } // IsConnected returns true if this node is connected to the node with id nodeID. -func (m *Middleware) IsConnected(identity flow.Identity) (bool, error) { - return m.libP2PNode.IsConnected(identity) +func (m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { + peerID, err := m.idTranslator.GetPeerID(nodeID) + if err != nil { + return false, fmt.Errorf("could not find peer id for target id: %w", err) + } + return m.libP2PNode.IsConnected(peerID) } // unicastMaxMsgSize returns the max permissible size for a unicast message diff --git a/network/p2p/network.go b/network/p2p/network.go index c4151e6ea79..0584143e5c5 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -12,7 +12,9 @@ import ( "github.com/onflow/flow-go/crypto/hash" channels "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" @@ -22,6 +24,14 @@ import ( const DefaultCacheSize = 10e6 +// NetworkingSetFilter is an identity filter that, when applied to the identity +// table at a given snapshot, returns all nodes that we should communicate with +// over the networking layer. +// +// NOTE: The protocol state includes nodes from the previous/next epoch that should +// be included in network communication. We omit any nodes that have been ejected. +var NetworkingSetFilter = filter.Not(filter.Ejected) + type ReadyDoneAwareNetwork interface { module.Network module.ReadyDoneAware @@ -31,20 +41,28 @@ type ReadyDoneAwareNetwork interface { // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { sync.RWMutex - logger zerolog.Logger - codec network.Codec - ids flow.IdentityList - me module.Local - mw network.Middleware - top network.Topology // used to determine fanout connections - metrics module.NetworkMetrics - rcache *RcvCache // used to deduplicate incoming messages - queue network.MessageQueue - ctx context.Context - cancel context.CancelFunc - subMngr network.SubscriptionManager // used to keep track of subscribed channels - lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle - ReadyDoneAwareNetwork + idProvider id.IdentifierProvider + defaultIdProvider id.IdentifierProvider + logger zerolog.Logger + codec network.Codec + me module.Local + mw network.Middleware + top network.Topology // used to determine fanout connections + metrics module.NetworkMetrics + rcache *RcvCache // used to deduplicate incoming messages + queue network.MessageQueue + ctx context.Context + cancel context.CancelFunc + subMngr network.SubscriptionManager // used to keep track of subscribed channels + lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle +} + +type NetworkOption func(*Network) + +func WithIdentifierProvider(provider id.IdentifierProvider) NetworkOption { + return func(net *Network) { + net.idProvider = provider + } } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -54,13 +72,13 @@ type Network struct { func NewNetwork( log zerolog.Logger, codec network.Codec, - ids flow.IdentityList, me module.Local, mw network.Middleware, csize int, top network.Topology, sm network.SubscriptionManager, metrics module.NetworkMetrics, + opts ...NetworkOption, ) (*Network, error) { rcache, err := newRcvCache(csize) @@ -80,7 +98,6 @@ func NewNetwork( lifecycleManager: lifecycle.NewLifecycleManager(), } o.ctx, o.cancel = context.WithCancel(context.Background()) - o.ids = ids // setup the message queue // create priority queue @@ -89,6 +106,10 @@ func NewNetwork( // create workers to read from the queue and call queueSubmitFunc queue.CreateQueueWorkers(o.ctx, queue.DefaultNumWorkers, o.queue, o.queueSubmitFunc) + for _, opt := range opts { + opt(o) + } + return o, nil } @@ -154,25 +175,27 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -// Identity returns a map of all flow.Identifier to flow identity by querying the flow state -func (n *Network) Identity() (map[flow.Identifier]flow.Identity, error) { +func (n *Network) GetIdentifierProvider() id.IdentifierProvider { + if n.idProvider != nil { + return n.idProvider + } n.RLock() defer n.RUnlock() - identifierToID := make(map[flow.Identifier]flow.Identity) - for _, id := range n.ids { - identifierToID[id.NodeID] = *id + if n.defaultIdProvider == nil { + n.logger.Fatal().Msg("TODO") + // TODO } - return identifierToID, nil + return n.defaultIdProvider } -// Topology returns the identities of a uniform subset of nodes in protocol state using the topology provided earlier. +// Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. -func (n *Network) Topology() (flow.IdentityList, error) { +func (n *Network) Topology() (flow.IdentifierList, error) { n.Lock() defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.ids, subscribedChannels) + top, err := n.top.GenerateFanout(n.GetIdentifierProvider().Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } @@ -187,25 +210,6 @@ func (n *Network) Receive(nodeID flow.Identifier, msg *message.Message) error { return nil } -// SetIDs updates the identity list cached by the network layer -func (n *Network) SetIDs(ids flow.IdentityList) error { - - // remove self from id - ids = ids.Filter(n.me.NotMeFilter()) - - n.Lock() - n.ids = ids - n.Unlock() - - // update the allow list - err := n.mw.UpdateAllowList() - if err != nil { - return fmt.Errorf("failed to update middleware allow list: %w", err) - } - - return nil -} - func (n *Network) processNetworkMessage(senderID flow.Identifier, message *message.Message) error { // checks the cache for deduplication and adds the message if not already present if n.rcache.add(message.EventID, network.Channel(message.ChannelID)) { @@ -298,6 +302,12 @@ func (n *Network) genNetworkMessage(channel network.Channel, event interface{}, return msg, nil } +func (n *Network) SetDefaultIdentifierProvider(provider id.IdentifierProvider) { + n.Lock() + n.defaultIdProvider = provider + n.Unlock() +} + // unicast sends the message in a reliable way to the given recipient. // It uses 1-1 direct messaging over the underlying network to deliver the message. // It returns an error if unicasting fails. diff --git a/network/p2p/nodeIDRefresher.go b/network/p2p/nodeIDRefresher.go deleted file mode 100644 index 219951ebbfc..00000000000 --- a/network/p2p/nodeIDRefresher.go +++ /dev/null @@ -1,82 +0,0 @@ -package p2p - -import ( - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/state/protocol" -) - -// NodeIDRefresher derives the latest list of flow identities with which the -// network should be communicating based on identity table changes in the -// protocol state. -type NodeIDRefresher struct { - logger zerolog.Logger - state protocol.State - callBack func(flow.IdentityList) error // callBack to call when the id list has changed -} - -func NewNodeIDRefresher(logger zerolog.Logger, state protocol.State, callBack func(list flow.IdentityList) error) *NodeIDRefresher { - return &NodeIDRefresher{ - logger: logger.With().Str("component", "network-refresher").Logger(), - state: state, - callBack: callBack, - } -} - -func (listener *NodeIDRefresher) getLogger(final protocol.Snapshot) zerolog.Logger { - - log := listener.logger - - // retrieve some contextual information for logging - head, err := final.Head() - if err != nil { - log.Error().Err(err).Msg("failed to get finalized header") - return log - } - log = log.With().Uint64("final_height", head.Height).Logger() - - phase, err := listener.state.Final().Phase() - if err != nil { - log.Error().Err(err).Msg("failed to get epoch phase") - return log - } - log = log.With().Str("epoch_phase", phase.String()).Logger() - - return log -} - -// OnIdentityTableChanged updates the networking layer's list of nodes to connect -// to when the identity table changes in the protocol state. -func (listener *NodeIDRefresher) OnIdentityTableChanged() { - - final := listener.state.Final() - log := listener.getLogger(final) - - log.Info().Msg("updating network ids upon identity table change") - - // get the new set of IDs - newIDs, err := final.Identities(NetworkingSetFilter) - if err != nil { - log.Err(err).Msg("failed to determine new identity table after identity table change") - return - } - - // call the registered callback - err = listener.callBack(newIDs) - if err != nil { - log.Err(err).Msg("failed to update network ids on identity table change") - return - } - - log.Info().Msg("successfully updated network ids upon identity table change") -} - -// NetworkingSetFilter is an identity filter that, when applied to the identity -// table at a given snapshot, returns all nodes that we should communicate with -// over the networking layer. -// -// NOTE: The protocol state includes nodes from the previous/next epoch that should -// be included in network communication. We omit any nodes that have been ejected. -var NetworkingSetFilter = filter.Not(filter.Ejected) diff --git a/network/p2p/peerManager.go b/network/p2p/peerManager.go index 71049031237..3145f73081f 100644 --- a/network/p2p/peerManager.go +++ b/network/p2p/peerManager.go @@ -5,20 +5,20 @@ import ( "fmt" "time" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" ) // Connector connects to peer and disconnects from peer using the underlying networking library type Connector interface { - // UpdatePeers connects to the given flow.Identities and returns a map of identifiers which failed. It also + // UpdatePeers connects to the given peer.IDs and returns a map of peers which failed. It also // disconnects from any other peers with which it may have previously established connection. // UpdatePeers implementation should be idempotent such that multiple calls to connect to the same peer should not // return an error or create multiple connections - UpdatePeers(ctx context.Context, ids flow.IdentityList) error + UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) } // DefaultPeerUpdateInterval is default duration for which the peer manager waits in between attempts to update peer connections @@ -28,10 +28,10 @@ var DefaultPeerUpdateInterval = 10 * time.Minute type PeerManager struct { unit *engine.Unit logger zerolog.Logger - idsProvider func() (flow.IdentityList, error) // callback to retrieve list of peers to connect to - peerRequestQ chan struct{} // a channel to queue a peer update request - connector Connector // connector to connect or disconnect from peers - peerUpdateInterval time.Duration // interval the peer manager runs on + peersProvider func() (peer.IDSlice, error) // callback to retrieve list of peers to connect to + peerRequestQ chan struct{} // a channel to queue a peer update request + connector Connector // connector to connect or disconnect from peers + peerUpdateInterval time.Duration // interval the peer manager runs on } // Option represents an option for the peer manager. @@ -43,16 +43,16 @@ func WithInterval(period time.Duration) Option { } } -// NewPeerManager creates a new peer manager which calls the idsProvider callback to get a list of peers to connect to +// NewPeerManager creates a new peer manager which calls the peersProvider callback to get a list of peers to connect to // and it uses the connector to actually connect or disconnect from peers. -func NewPeerManager(logger zerolog.Logger, idsProvider func() (flow.IdentityList, error), +func NewPeerManager(logger zerolog.Logger, peersProvider func() (peer.IDSlice, error), connector Connector, options ...Option) *PeerManager { pm := &PeerManager{ - unit: engine.NewUnit(), - logger: logger, - idsProvider: idsProvider, - connector: connector, - peerRequestQ: make(chan struct{}, 1), + unit: engine.NewUnit(), + logger: logger, + peersProvider: peersProvider, + connector: connector, + peerRequestQ: make(chan struct{}, 1), } // apply options for _, o := range options { @@ -101,25 +101,21 @@ func (pm *PeerManager) RequestPeerUpdate() { } } -// updatePeers updates the peers by connecting to all the nodes provided by the idsProvider callback and disconnecting from +// updatePeers updates the peers by connecting to all the nodes provided by the peersProvider callback and disconnecting from // previous nodes that are no longer in the new list of nodes. func (pm *PeerManager) updatePeers() { - // get all the ids to connect to - ids, err := pm.idsProvider() + // get all the peer ids to connect to + peers, err := pm.peersProvider() if err != nil { pm.logger.Error().Err(err).Msg("failed to update peers") return } pm.logger.Trace(). - Str("peers", fmt.Sprintf("%v", ids.NodeIDs())). + Str("peers", fmt.Sprintf("%v", peers)). Msg("connecting to peers") // ask the connector to connect to all peers in the list - err = pm.connector.UpdatePeers(pm.unit.Ctx(), ids) - if err != nil { - // one of more identities in the identity table could not be connected to - pm.logger.Error().Err(err).Msg("failed to connect to one or more peers") - } + pm.connector.UpdatePeers(pm.unit.Ctx(), peers) } diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go new file mode 100644 index 00000000000..cea16d466a9 --- /dev/null +++ b/network/p2p/peerstore_provider.go @@ -0,0 +1,32 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p-core/host" + + "github.com/onflow/flow-go/model/flow" +) + +type PeerstoreIdentifierProvider struct { + host host.Host + idTranslator IDTranslator +} + +func NewPeerstoreIdentifierProvider(host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{host: host, idTranslator: idTranslator} +} + +func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { + var result flow.IdentifierList + + pids := p.host.Peerstore().PeersWithAddrs() // TODO: should we just call Peers here? + for _, pid := range pids { + flowID, err := p.idTranslator.GetFlowID(pid) + if err != nil { + // TODO: log error + } else { + result = append(result, flowID) + } + } + + return result +} diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go new file mode 100644 index 00000000000..d47542c3b18 --- /dev/null +++ b/network/p2p/protocol_state_provider.go @@ -0,0 +1,130 @@ +package p2p + +import ( + "fmt" + "sync" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" +) + +type ProtocolStateIDCache struct { + events.Noop + identities flow.IdentityList + state protocol.State + mu sync.RWMutex + peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + flowIDs map[peer.ID]flow.Identifier +} + +func NewProtocolStateIDCache( + state protocol.State, + eventDistributer *events.Distributor, +) (*ProtocolStateIDCache, error) { + provider := &ProtocolStateIDCache{ + state: state, + } + + head, err := state.Final().Head() + if err != nil { + return nil, err // TODO: format the error + } + + provider.update(head.ID()) + eventDistributer.AddConsumer(provider) + + return provider, nil +} + +func (p *ProtocolStateIDCache) EpochTransition(_ uint64, header *flow.Header) { + // TODO: maybe we actually want to log the epoch information from the arguments here (epoch phase, etc) + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { + p.update(header.ID()) +} + +func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { + // TODO: log status here + identities, err := p.state.AtBlockID(blockID).Identities(filter.Any) + if err != nil { + // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + } + + nIds := identities.Count() + + peerIDs := make(map[flow.Identifier]peer.ID, nIds) + flowIDs := make(map[peer.ID]flow.Identifier, nIds) + + for _, identity := range identities { + pid, err := ExtractPeerID(identity.NetworkPubKey) + if err != nil { + // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + } + + flowIDs[pid] = identity.NodeID + peerIDs[identity.NodeID] = pid + } + + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities + p.flowIDs = flowIDs + p.peerIDs = peerIDs +} + +func (p *ProtocolStateIDCache) Identities(filter flow.IdentityFilter) flow.IdentityList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.Filter(filter) +} + +func (p *ProtocolStateIDCache) GetPeerID(flowID flow.Identifier) (pid peer.ID, err error) { + p.mu.RLock() + defer p.mu.RUnlock() + + pid, found := p.peerIDs[flowID] + if !found { + err = fmt.Errorf("flow ID %v was not found in cached identity list", flowID) + } + + return +} + +func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, err error) { + p.mu.RLock() + defer p.mu.RUnlock() + + fid, found := p.flowIDs[peerID] + if !found { + err = fmt.Errorf("peer ID %v was not found in cached identity list", peerID) + } + + return +} + +func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { + pk, err := PublicKey(networkPubKey) + if err != nil { + // TODO: format the error + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + // TODO: format the error + return + } + + return +} diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go new file mode 100644 index 00000000000..5e320ccb236 --- /dev/null +++ b/network/p2p/unstaked_translator.go @@ -0,0 +1,43 @@ +package p2p + +import ( + crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/multiformats/go-multihash" + + "github.com/onflow/flow-go/model/flow" +) + +type UnstakedNetworkIDTranslator struct{} + +func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { + return &UnstakedNetworkIDTranslator{} +} + +func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + data := append([]byte{0x02}, flowID[:]...) + mh, err := multihash.Sum(data, multihash.IDENTITY, -1) + if err != nil { + // TODO: return error + } + + return peer.ID(mh), nil +} + +func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { + pk, err := peerID.ExtractPublicKey() + if err != nil { + // return error + } + + if pk.Type() != crypto_pb.KeyType_ECDSA { + // fail + } + + data, err := pk.Raw() + if err != nil || data[0] != 0x02 { // TODO: check if this is the right byte to check + // fail + } + + return flow.HashToID(data[1:]), nil +} diff --git a/network/topology.go b/network/topology.go index 1d746c70896..1d0faf16212 100644 --- a/network/topology.go +++ b/network/topology.go @@ -6,9 +6,9 @@ import ( // Topology provides a subset of nodes which a given node should directly connect to for 1-k messaging. type Topology interface { - // GenerateFanout receives IdentityList of entire network, and list of channels the node is subscribing to. - // It constructs and returns the fanout IdentityList of node. - // A node directly communicates with its fanout IdentityList on epidemic dissemination + // GenerateFanout receives IdentifierList of entire network, and list of channels the node is subscribing to. + // It constructs and returns the fanout IdentifierList of node. + // A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. @@ -19,5 +19,5 @@ type Topology interface { // // GenerateFanout is not concurrency safe. It is responsibility of caller to lock for it. // with the channels argument, it allows the returned topology to be cached, which is necessary for randomized topology. - GenerateFanout(ids flow.IdentityList, channels ChannelList) (flow.IdentityList, error) + GenerateFanout(ids flow.IdentifierList, channels ChannelList) (flow.IdentifierList, error) } diff --git a/network/topology/cache.go b/network/topology/cache.go index c305140abed..a738e9e9453 100644 --- a/network/topology/cache.go +++ b/network/topology/cache.go @@ -19,10 +19,10 @@ import ( // in a concurrency safe way, i.e., the caller should lock for it. type Cache struct { log zerolog.Logger - top network.Topology // instance of underlying topology. - cachedFanout flow.IdentityList // most recently generated fanout list by invoking underlying topology. - idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. - chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. + top network.Topology // instance of underlying topology. + cachedFanout flow.IdentifierList // most recently generated fanout list by invoking underlying topology. + idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. + chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. } //NewCache creates and returns a topology Cache given an instance of topology implementation. @@ -36,19 +36,19 @@ func NewCache(log zerolog.Logger, top network.Topology) *Cache { } } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList // of this instance. // It caches the most recently generated fanout list, so as long as the input list is the same, it returns // the same output. It invalidates and updates its internal cache the first time input list changes. -// A node directly communicates with its fanout IdentityList on epidemic dissemination +// A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. // // Note that this implementation of GenerateFanout preserves same output as long as input is the same. This // should not be assumed as a 1-1 mapping between input and output. -func (c *Cache) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { - inputIdsFP := ids.Fingerprint() +func (c *Cache) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { + inputIdsFP := flow.MerkleRoot(ids...) inputChansFP := channels.ID() log := c.log.With(). diff --git a/network/topology/fixedListTopology.go b/network/topology/fixedListTopology.go index 1b171ce8919..6091da61d38 100644 --- a/network/topology/fixedListTopology.go +++ b/network/topology/fixedListTopology.go @@ -2,7 +2,7 @@ package topology import ( "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" "github.com/onflow/flow-go/network" ) @@ -17,14 +17,14 @@ func NewFixedListTopology(nodeID flow.Identifier) FixedListTopology { } } -func (r FixedListTopology) GenerateFanout(ids flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { - return ids.Filter(filter.HasNodeID(r.fixedNodeID)), nil +func (r FixedListTopology) GenerateFanout(ids flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { + return ids.Filter(idFilter.Is(r.fixedNodeID)), nil } // EmptyListTopology always returns an empty list as the fanout type EmptyListTopology struct { } -func (r EmptyListTopology) GenerateFanout(_ flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { - return flow.IdentityList{}, nil +func (r EmptyListTopology) GenerateFanout(_ flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { + return flow.IdentifierList{}, nil } diff --git a/network/topology/randomizedTopology.go b/network/topology/randomizedTopology.go index c1bc959ed9e..55be3545025 100644 --- a/network/topology/randomizedTopology.go +++ b/network/topology/randomizedTopology.go @@ -9,6 +9,8 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -17,15 +19,16 @@ import ( // By random topology we mean a node is connected to any other co-channel nodes with some // edge probability. type RandomizedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - chance uint64 // used to translate connectedness probability into a number in [0, 100] - rng random.Rand // used as a stateful random number generator to sample edges - logger zerolog.Logger + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + chance uint64 // used to translate connectedness probability into a number in [0, 100] + rng random.Rand // used as a stateful random number generator to sample edges + logger zerolog.Logger + idProvider id.IdentityProvider } // NewRandomizedTopology returns an instance of the RandomizedTopology. -func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { +func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { // edge probability should be a positive value between 0 and 1. However, // we like it to be strictly greater than zero. Also, at the current scale of // Flow, we need it to be greater than 0.01 to support probabilistic connectedness. @@ -44,33 +47,34 @@ func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgePr } t := &RandomizedTopology{ - myNodeID: nodeID, - state: state, - chance: uint64(100 * edgeProb), - rng: rng, - logger: logger.With().Str("component:", "randomized-topology").Logger(), + myNodeID: nodeID, + state: state, + chance: uint64(100 * edgeProb), + rng: rng, + logger: logger.With().Str("component:", "randomized-topology").Logger(), + idProvider: idProvider, } return t, nil } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList -// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. This should be done with a very high probability // in randomized topology. -func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { +func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. r.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } - var myFanout flow.IdentityList + var myFanout flow.IdentifierList // generates a randomized subgraph per channel for _, myChannel := range myUniqueChannels { @@ -90,12 +94,12 @@ func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels netwo return myFanout, nil } -// subsetChannel returns a random subset of the identity list that is passed. -// Returned identities should all subscribed to the specified `channel`. -// Note: this method should not include identity of its executor. -func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +// subsetChannel returns a random subset of the identifier list that is passed. +// Returned identifiers should all subscribed to the specified `channel`. +// Note: this method should not include identifier of its executor. +func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { // excludes node itself - sampleSpace := ids.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) + sampleSpace := ids.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) // samples a random graph based on whether channel is cluster-based or not. if _, ok := engine.ClusterChannel(channel); ok { @@ -109,12 +113,12 @@ func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network // Independent invocations of this method over different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityList, error) { +func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.IdentifierList, error) { if len(ids) == 0 { - return nil, fmt.Errorf("empty identity list") + return nil, fmt.Errorf("empty identifier list") } - fanout := flow.IdentityList{} + fanout := flow.IdentifierList{} for _, id := range ids { // tosses a biased coin and adds id to fanout accordingly. // biased coin follows the edge probability distribution. @@ -127,7 +131,7 @@ func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityLi } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.IdentityList, error) { +func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow.IdentifierList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(r.myNodeID, r.state) if err != nil { @@ -135,10 +139,10 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.I } // excludes node itself from cluster - clusterPeers = clusterPeers.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) + clusterPeers = clusterPeers.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) + nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } @@ -148,7 +152,7 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.I } // clusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. -func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -160,7 +164,16 @@ func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, chan } // samples fanout among interacting roles - return r.sampleFanout(ids.Filter(filter.HasRole(roles...))) + return r.sampleFanout( + r.idProvider. + Identities( + filter.And( + filter.HasNodeID(ids...), + filter.HasRole(roles...), + ), + ). + NodeIDs(), + ) } // tossBiasedBit returns true with probability equal `r.chance/100`, and returns false otherwise. diff --git a/network/topology/topicBasedTopology.go b/network/topology/topicBasedTopology.go index 9d14e34049b..fdc55f43b1e 100644 --- a/network/topology/topicBasedTopology.go +++ b/network/topology/topicBasedTopology.go @@ -8,6 +8,8 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -15,14 +17,15 @@ import ( // TopicBasedTopology is a deterministic topology mapping that creates a connected graph component among the nodes // involved in each topic. type TopicBasedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - logger zerolog.Logger - seed int64 + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + logger zerolog.Logger + seed int64 + identityProvider id.IdentityProvider } // NewTopicBasedTopology returns an instance of the TopicBasedTopology. -func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { +func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { seed, err := intSeedFromID(nodeID) if err != nil { return nil, fmt.Errorf("could not generate seed from id:%w", err) @@ -38,19 +41,19 @@ func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state return t, nil } -// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList -// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination +// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. -func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { +func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. t.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } // finds all interacting roles with this node @@ -64,7 +67,7 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels netwo } // builds a connected component per role this node interact with, - var myFanout flow.IdentityList + var myFanout flow.IdentifierList for _, role := range myInteractingRoles { if role == flow.RoleCollection { // we do not build connected component for collection nodes based on their role @@ -98,32 +101,36 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels netwo return myFanout, nil } -// subsetChannel returns a random subset of the identity list that is passed. `shouldHave` represents set of -// identities that should be included in the returned subset. -// Returned identities should all subscribed to the specified `channel`. -// Note: this method should not include identity of its executor. -func (t *TopicBasedTopology) subsetChannel(ids flow.IdentityList, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +// subsetChannel returns a random subset of the identifier list that is passed. `shouldHave` represents set of +// identifiers that should be included in the returned subset. +// Returned identifiers should all subscribed to the specified `channel`. +// Note: this method should not include identifier of its executor. +func (t *TopicBasedTopology) subsetChannel(ids flow.IdentifierList, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return t.clusterChannelHandler(ids, shouldHave) } return t.nonClusterChannelHandler(ids, shouldHave, channel) } -// subsetRole returns a random subset of the identity list that is passed. `shouldHave` represents set of -// identities that should be included in the returned subset. -// Returned identities should all be of one of the specified `roles`. -// Note: this method should not include identity of its executor. -func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.IdentityList, roles flow.RoleList) (flow.IdentityList, error) { +// subsetRole returns a random subset of the identifier list that is passed. `shouldHave` represents set of +// identifiers that should be included in the returned subset. +// Returned identifiers should all be of one of the specified `roles`. +// Note: this method should not include the identifier of its executor. +func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow.IdentifierList, roles flow.RoleList) (flow.IdentifierList, error) { // excludes irrelevant roles and the node itself from both should have and ids set - shouldHave = shouldHave.Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )) - - ids = ids.Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )) + shouldHave = t.identityProvider. + Identities(filter.HasNodeID(shouldHave...)). + Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )).NodeIDs() + + ids = t.identityProvider. + Identities(filter.HasNodeID(ids...)). + Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )).NodeIDs() sample, err := t.sampleConnectedGraph(ids, shouldHave) if err != nil { @@ -138,11 +145,11 @@ func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.Id // different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHave flow.IdentityList) (flow.IdentityList, error) { +func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { if len(all) == 0 { t.logger.Debug().Msg("skips sampling connected graph with zero nodes") - return flow.IdentityList{}, nil + return flow.IdentifierList{}, nil } if len(shouldHave) == 0 { @@ -153,9 +160,9 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa } // checks `shouldHave` be a subset of `all` - nonMembers := shouldHave.Filter(filter.Not(filter.In(all))) + nonMembers := shouldHave.Filter(idFilter.Not(idFilter.In(all...))) if len(nonMembers) != 0 { - return nil, fmt.Errorf("should have identities is not a subset of all: %v", nonMembers) + return nil, fmt.Errorf("should have identifiers is not a subset of all: %v", nonMembers) } // total sample size @@ -170,7 +177,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa subsetSize := totalSize - len(shouldHave) // others are all excluding should have ones - others := all.Filter(filter.Not(filter.In(shouldHave))) + others := all.Filter(idFilter.Not(idFilter.In(shouldHave...))) others = others.DeterministicSample(uint(subsetSize), t.seed) return others.Union(shouldHave), nil @@ -178,7 +185,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHa } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityList) (flow.IdentityList, error) { +func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(t.myNodeID, t.state) if err != nil { @@ -186,18 +193,18 @@ func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityL } // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) + nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } // samples a connected graph topology from the cluster peers - return t.subsetRole(clusterPeers, shouldHave.Filter(filter.HasNodeID(clusterPeers.NodeIDs()...)), flow.RoleList{flow.RoleCollection}) + return t.subsetRole(clusterPeers, shouldHave.Filter(idFilter.In(clusterPeers...)), flow.RoleList{flow.RoleCollection}) } // nonClusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. // The returned sample contains `shouldHave` ones that also subscribed to `channel`. -func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { +func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -209,5 +216,5 @@ func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.Identi } // samples a connected graph topology - return t.subsetRole(ids.Filter(filter.HasRole(roles...)), shouldHave, roles) + return t.subsetRole(ids, shouldHave, roles) } diff --git a/network/topology/topology_utils.go b/network/topology/topology_utils.go index bf1d2b43986..3668d8557a6 100644 --- a/network/topology/topology_utils.go +++ b/network/topology/topology_utils.go @@ -51,7 +51,7 @@ func byteSeedFromID(id flow.Identifier) ([]byte, error) { } // clusterPeers returns the list of other nodes within the same cluster as specified identifier. -func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, error) { +func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList, error) { currentEpoch := state.Final().Epochs().Current() clusterList, err := currentEpoch.Clustering() if err != nil { @@ -63,5 +63,5 @@ func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, return nil, fmt.Errorf("failed to find the cluster for node ID %s", id.String()) } - return myCluster, nil + return myCluster.NodeIDs(), nil } diff --git a/utils/grpc/grpc.go b/utils/grpc/grpc.go index 2fe22c59bde..b22144a7d40 100644 --- a/utils/grpc/grpc.go +++ b/utils/grpc/grpc.go @@ -7,7 +7,6 @@ import ( "fmt" lcrypto "github.com/libp2p/go-libp2p-core/crypto" - "github.com/libp2p/go-libp2p-core/peer" libp2ptls "github.com/libp2p/go-libp2p-tls" "github.com/onflow/flow-go/crypto" @@ -102,13 +101,9 @@ func DefaultClientTLSConfig(publicKey crypto.PublicKey) (*tls.Config, error) { func verifyPeerCertificateFunc(expectedPublicKey crypto.PublicKey) (func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error, error) { // convert the Flow.crypto key to LibP2P key for easy comparision using LibP2P TLS utils - expectedLibP2PKey, err := p2p.LibP2PPublicKeyFromFlow(expectedPublicKey) + remotePeerLibP2PID, err := p2p.ExtractPeerID(expectedPublicKey) if err != nil { - return nil, fmt.Errorf("failed to generate a libp2p key from a Flow key: %w", err) - } - remotePeerLibP2PID, err := peer.IDFromPublicKey(expectedLibP2PKey) - if err != nil { - return nil, fmt.Errorf("failed to derive the libp2p Peer ID from the libp2p public key: %w", err) + return nil, fmt.Errorf("failed to derive the libp2p Peer ID from the Flow key: %w", err) } // We're using InsecureSkipVerify, so the verifiedChains parameter will always be empty. From 130a859fbdaceda7b7ac445c3ef531146b0aa355 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 13:52:26 -0700 Subject: [PATCH 196/291] add dht peer lookup --- .../node_builder/access_node_builder.go | 134 +++++++++++------- cmd/scaffold.go | 2 + network/p2p/dht.go | 6 +- network/p2p/libp2pNode.go | 103 +++++++++++--- 4 files changed, 176 insertions(+), 69 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 403cf14baea..cfb6e5f71a4 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2,17 +2,18 @@ package node_builder import ( "context" + "encoding/json" "fmt" "strings" "time" + dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow/protobuf/go/flow/access" "github.com/spf13/pflag" "google.golang.org/grpc" "google.golang.org/grpc/credentials" "github.com/onflow/flow-go/cmd" - "github.com/onflow/flow-go/cmd/build" "github.com/onflow/flow-go/consensus" "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/committees" @@ -43,6 +44,7 @@ import ( "github.com/onflow/flow-go/network" jsoncodec "github.com/onflow/flow-go/network/codec/json" "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network/validator" "github.com/onflow/flow-go/state/protocol" badgerState "github.com/onflow/flow-go/state/protocol/badger" "github.com/onflow/flow-go/state/protocol/blocktimer" @@ -85,7 +87,9 @@ type AccessNodeBuilder interface { // while for a node running as a library, the config fields are expected to be initialized by the caller. type AccessNodeConfig struct { staked bool - stakedAccessNodeIDHex string + bootstrapNodeAddresses []string + bootstrapNodePublicKeys []string + bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes unstakedNetworkBindAddr string collectionGRPCPort uint executionGRPCPort uint @@ -132,7 +136,8 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { apiRatelimits: nil, apiBurstlimits: nil, staked: true, - stakedAccessNodeIDHex: "", + bootstrapNodeAddresses: []string{}, + bootstrapNodePublicKeys: []string{}, unstakedNetworkBindAddr: cmd.NotSet, } } @@ -438,7 +443,6 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, - node.Storage.Results, node.RootChainID, anb.TransactionMetrics, anb.collectionGRPCPort, @@ -467,7 +471,7 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { return nil, fmt.Errorf("could not create requester engine: %w", err) } - anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Results, node.Storage.Receipts, anb.TransactionMetrics, + anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, anb.TransactionMetrics, anb.CollectionsToMarkFinalized, anb.CollectionsToMarkExecuted, anb.BlocksToMarkExecuted, anb.RpcEng) anb.RequestEng.WithHandle(anb.IngestEng.OnCollection) anb.FinalizationDistributor.AddConsumer(anb.IngestEng) @@ -486,9 +490,9 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { type Option func(*AccessNodeConfig) -func WithUpstreamAccessNodeID(upstreamAccessNodeID flow.Identifier) Option { +func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { return func(config *AccessNodeConfig) { - config.stakedAccessNodeIDHex = upstreamAccessNodeID.String() + config.bootstrapIdentites = bootstrapNodes } } @@ -566,50 +570,42 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.StringToIntVar(&builder.apiRatelimits, "api-rate-limits", defaultConfig.apiRatelimits, "per second rate limits for Access API methods e.g. Ping=300,GetTransaction=500 etc.") flags.StringToIntVar(&builder.apiBurstlimits, "api-burst-limits", defaultConfig.apiBurstlimits, "burst limits for Access API methods e.g. Ping=100,GetTransaction=100 etc.") flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") - flags.StringVar(&builder.stakedAccessNodeIDHex, "staked-access-node-id", defaultConfig.stakedAccessNodeIDHex, "the node ID of the upstream staked access node if this is an unstaked access node") + flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") + flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") flags.StringVar(&builder.unstakedNetworkBindAddr, "unstaked-bind-addr", defaultConfig.unstakedNetworkBindAddr, "address to bind on for the unstaked network") }) } // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. // The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance -func (builder *FlowAccessNodeBuilder) initLibP2PFactory( - ctx context.Context, +func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, nodeID flow.Identifier, - networkMetrics module.NetworkMetrics, - networkKey crypto.PrivateKey, -) (p2p.LibP2PFactoryFunc, error) { + networkKey crypto.PrivateKey) (p2p.LibP2PFactoryFunc, error) { - // setup the Ping provider to return the software version and the sealed block height - pingProvider := p2p.PingInfoProviderImpl{ - SoftwareVersionFun: func() string { - return build.Semver() - }, - SealedBlockHeightFun: func() (uint64, error) { - head, err := builder.State.Sealed().Head() - if err != nil { - return 0, err - } - return head.Height, nil - }, - } + // The staked nodes act as the DHT servers + dhtOptions := []dht.Option{p2p.AsServer(builder.IsStaked())} - libP2PNodeFactory, err := p2p.DefaultLibP2PNodeFactory( - ctx, - builder.Logger, - nodeID, - builder.unstakedNetworkBindAddr, - networkKey, - builder.RootBlock.ID().String(), - p2p.DefaultMaxPubSubMsgSize, - networkMetrics, - pingProvider, - ) - if err != nil { - return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) + // if this is an unstaked access node, then seed the DHT with the boostrap identities + if !builder.IsStaked() { + bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentites) + builder.MustNot(err) + dhtOptions = append(dhtOptions, bootstrapPeersOpt) } - return libP2PNodeFactory, nil + return func() (*p2p.Node, error) { + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.unstakedNetworkBindAddr, networkKey). + SetRootBlockID(builder.RootBlock.ID().String()). + // unlike the staked network where currently all the node addresses are known upfront, + // for the unstaked network the nodes need to discover each other using DHT Discovery. + SetDHTOptions(dhtOptions...). + SetLogger(builder.Logger). + Build(ctx) + if err != nil { + return nil, err + } + builder.UnstakedLibP2PNode = libp2pNode + return builder.UnstakedLibP2PNode, nil + }, nil } // initMiddleware creates the network.Middleware implementation with the libp2p factory function, metrics, peer update @@ -617,20 +613,16 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory( func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - peerUpdateInterval time.Duration, - unicastMessageTimeout time.Duration, - connectionGating bool, - managerPeerConnections bool, validators ...network.MessageValidator) *p2p.Middleware { builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, factoryFunc, nodeID, networkMetrics, builder.RootBlock.ID().String(), - peerUpdateInterval, - unicastMessageTimeout, - connectionGating, - managerPeerConnections, + time.Hour, // TODO: this is pretty meaningless since there is no peermanager in play. + p2p.DefaultUnicastTimeout, + false, // no connection gating for the unstaked network + false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) builder.IDTranslator, validators...) return builder.unstakedMiddleware @@ -666,3 +658,49 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, return net, nil } + +func unstakedNetworkMsgValidators(selfID flow.Identifier) []network.MessageValidator { + return []network.MessageValidator{ + // filter out messages sent by this node itself + validator.ValidateNotSender(selfID), + } +} + +// BootstrapIdentities converts the bootstrap node addresses and keys to a Flow Identity list where +// each Flow Identity is initialized with the passed address, the networking key +// and the Node ID set to ZeroID, role set to Access, 0 stake and no staking key. +func BootstrapIdentities(addresses []string, keys []string) (flow.IdentityList, error) { + + if len(addresses) != len(keys) { + return nil, fmt.Errorf("number of addresses and keys provided for the boostrap nodes don't match") + } + + ids := make([]*flow.Identity, len(addresses)) + for i, address := range addresses { + + key := keys[i] + // json unmarshaller needs a quotes before and after the string + // the pflags.StringSliceVar does not retain quotes for the command line arg even if escaped with \" + // hence this additional check to ensure the key is indeed quoted + if !strings.HasPrefix(key, "\"") { + key = fmt.Sprintf("\"%s\"", key) + } + // networking public key + var networkKey encodable.NetworkPubKey + err := json.Unmarshal([]byte(key), &networkKey) + if err != nil { + return nil, err + } + + // create the identity of the peer by setting only the relevant fields + id := &flow.Identity{ + NodeID: flow.ZeroID, // the NodeID is the hash of the staking key and for the unstaked network it does not apply + Address: address, + Role: flow.RoleAccess, // the upstream node has to be an access node + NetworkPubKey: networkKey, + } + + ids = append(ids, id) + } + return ids, nil +} diff --git a/cmd/scaffold.go b/cmd/scaffold.go index c7912286602..d57fb218282 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -211,6 +211,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) fnb.ProtocolEvents.AddConsumer(idEvents) + // TODO: add a consumer which implements setting the permanent peer addresses, + // and expiring the old permanent ones return net, err }) diff --git a/network/p2p/dht.go b/network/p2p/dht.go index 8cd702c5a9a..b21211a9adb 100644 --- a/network/p2p/dht.go +++ b/network/p2p/dht.go @@ -5,7 +5,6 @@ import ( "github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p-core/peer" - discovery "github.com/libp2p/go-libp2p-discovery" dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow-go/model/flow" @@ -13,7 +12,7 @@ import ( // This produces a new IPFS DHT // on the name, see https://github.com/libp2p/go-libp2p-kad-dht/issues/337 -func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*discovery.RoutingDiscovery, error) { +func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*dht.IpfsDHT, error) { defaultOptions := defaultDHTOptions() allOptions := append(defaultOptions, options...) @@ -27,8 +26,7 @@ func NewDHT(ctx context.Context, host host.Host, options ...dht.Option) (*discov return nil, err } - routingDiscovery := discovery.NewRoutingDiscovery(kdht) - return routingDiscovery, nil + return kdht, nil } // DHT defaults to ModeAuto which will automatically switch the DHT between Server and Client modes based on diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index f713af8f62f..5702bade9b3 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -17,6 +17,8 @@ import ( libp2pnet "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/protocol" + discovery "github.com/libp2p/go-libp2p-discovery" + dht "github.com/libp2p/go-libp2p-kad-dht" pubsub "github.com/libp2p/go-libp2p-pubsub" swarm "github.com/libp2p/go-libp2p-swarm" tptu "github.com/libp2p/go-libp2p-transport-upgrader" @@ -69,7 +71,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(psOptions...). + SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize)...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). // SetResolver(resolver). @@ -81,8 +83,9 @@ type NodeBuilder interface { SetRootBlockID(string) NodeBuilder SetConnectionManager(TagLessConnManager) NodeBuilder SetConnectionGater(*ConnGater) NodeBuilder - SetPubsubOptions(...pubsub.Option) NodeBuilder + SetPubsubOptions(...PubsubOption) NodeBuilder SetPingInfoProvider(PingInfoProvider) NodeBuilder + SetDHTOptions(...dht.Option) NodeBuilder SetLogger(zerolog.Logger) NodeBuilder SetResolver(resolver *madns.Resolver) NodeBuilder Build(context.Context) (*Node, error) @@ -98,7 +101,8 @@ type DefaultLibP2PNodeBuilder struct { resolver *madns.Resolver pubSubMaker func(context.Context, host.Host, ...pubsub.Option) (*pubsub.PubSub, error) hostMaker func(context.Context, ...config.Option) (host.Host, error) - pubSubOpts []pubsub.Option + pubSubOpts []PubsubOption + dhtOpts []dht.Option } func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcrypto.PrivateKey) NodeBuilder { @@ -113,6 +117,11 @@ func NewDefaultLibP2PNodeBuilder(id flow.Identifier, address string, flowKey fcr } } +func (builder *DefaultLibP2PNodeBuilder) SetDHTOptions(opts ...dht.Option) NodeBuilder { + builder.dhtOpts = opts + return builder +} + func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId string) NodeBuilder { builder.rootBlockID = rootBlockId return builder @@ -128,7 +137,7 @@ func (builder *DefaultLibP2PNodeBuilder) SetConnectionGater(connGater *ConnGater return builder } -func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...pubsub.Option) NodeBuilder { +func (builder *DefaultLibP2PNodeBuilder) SetPubsubOptions(opts ...PubsubOption) NodeBuilder { builder.pubSubOpts = opts return builder } @@ -181,6 +190,11 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.connMgr = builder.connMngr } + if builder.rootBlockID == "" { + return nil, errors.New("root block ID must be provided") + } + node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) + if builder.pingInfoProvider != nil { opts = append(opts, libp2p.Ping(true)) } @@ -195,13 +209,32 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro } node.host = libp2pHost + if len(builder.dhtOpts) != 0 { + kdht, err := NewDHT(ctx, node.host, builder.dhtOpts...) + if err != nil { + return nil, err + } + node.dht = kdht + builder.pubSubOpts = append(builder.pubSubOpts, WithDHTDiscovery(kdht)) + } + if builder.pingInfoProvider != nil { pingLibP2PProtocolID := generatePingProtcolID(builder.rootBlockID) pingService := NewPingService(libp2pHost, pingLibP2PProtocolID, builder.pingInfoProvider, node.logger) node.pingService = pingService } - ps, err := builder.pubSubMaker(ctx, libp2pHost, builder.pubSubOpts...) + var libp2pPSOptions []pubsub.Option + // generate the libp2p Pubsub options from the given context and host + for _, optionGenerator := range builder.pubSubOpts { + option, err := optionGenerator(ctx, libp2pHost) + if err != nil { + return nil, err + } + libp2pPSOptions = append(libp2pPSOptions, option) + } + + ps, err := builder.pubSubMaker(ctx, libp2pHost, libp2pPSOptions...) if err != nil { return nil, err } @@ -233,6 +266,7 @@ type Node struct { flowLibP2PProtocolID protocol.ID // the unique protocol ID pingService *PingService connMgr TagLessConnManager + dht *dht.IpfsDHT } // Stop stops the libp2p node. @@ -349,6 +383,22 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp // the desired behaviour for pub-sub (1-k style of communication) for 1-1 style we want to retry the connection // immediately without backing off and fail-fast. // Hence, explicitly cancel the dial back off (if any) and try connecting again + + if len(n.host.Peerstore().Addrs(peerID)) == 0 { + if n.dht != nil { + // TODO: adjust timeout + timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) + // try to find the peer + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + } + } + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } + + // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() if swm, ok := network.(*swarm.Swarm); ok { swm.Backoff().Clear(peerID) @@ -516,18 +566,7 @@ func (n *Node) UpdateAllowList(peers []peer.ID) { return } - // generates peer address information for all identities - allowlist := make([]peer.AddrInfo, len(identities)) - var err error - for i, identity := range identities { - allowlist[i], err = PeerAddressInfo(*identity) - if err != nil { - return fmt.Errorf("could not generate address info: %w", err) - } - } - - n.connGater.update(allowlist) - return nil + n.connGater.update(peers) } // Host returns pointer to host object of node. @@ -620,3 +659,33 @@ func DefaultPubSub(ctx context.Context, host host.Host, psOption ...pubsub.Optio } return pubSub, nil } + +// PubsubOption generates a libp2p pubsub.Option from the given context and host +type PubsubOption func(ctx context.Context, host host.Host) (pubsub.Option, error) + +func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { + pubSubOptionFunc := func(option pubsub.Option) PubsubOption { + return func(_ context.Context, _ host.Host) (pubsub.Option, error) { + return option, nil + } + } + return []PubsubOption{ + // skip message signing + pubSubOptionFunc(pubsub.WithMessageSigning(true)), + // skip message signature + pubSubOptionFunc(pubsub.WithStrictSignatureVerification(true)), + // set max message size limit for 1-k PubSub messaging + pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), + // no discovery + } +} + +func WithDHTDiscovery(option ...dht.Option) PubsubOption { + return func(ctx context.Context, host host.Host) (pubsub.Option, error) { + dhtDiscovery, err := NewDHT(ctx, host, option...) + if err != nil { + return nil, err + } + return pubsub.WithDiscovery(dhtDiscovery), nil + } + } From e2529aa09769e658e9baa786ff8a33ebab39319b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 15:42:48 -0700 Subject: [PATCH 197/291] Add dht lookup and default peerstore addresses --- .../unstaked_access_node_builder.go | 77 +++++++++++-------- cmd/scaffold.go | 7 +- network/middleware.go | 5 +- network/p2p/libp2pNode.go | 16 ++-- network/p2p/middleware.go | 18 ++++- network/p2p/network.go | 15 +++- 6 files changed, 89 insertions(+), 49 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 81c9f4600f7..88ce203fe65 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -2,8 +2,6 @@ package node_builder import ( "context" - "strings" - "time" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" @@ -11,7 +9,6 @@ import ( "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/topology" ) type UnstakedAccessNodeBuilder struct { @@ -27,6 +24,14 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + + idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + if err != nil { + return err + } + + fnb.IdentityProvider = idCache + return nil }) } @@ -40,8 +45,12 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() + builder.deriveBootstrapPeerIdentities() + builder.enqueueUnstakedNetworkInit(ctx) + builder.enqueueConnectWithStakedAN() + builder.EnqueueMetricsServerInit() builder.RegisterBadgerMetrics() @@ -55,15 +64,22 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { func (builder *UnstakedAccessNodeBuilder) validateParams() { - // for an unstaked access node, the staked access node ID must be provided - if strings.TrimSpace(builder.stakedAccessNodeIDHex) == "" { - builder.Logger.Fatal().Msg("staked access node ID not specified") - } - - // and also the unstaked bind address + // for an unstaked access node, the unstaked network bind address must be provided if builder.unstakedNetworkBindAddr == cmd.NotSet { builder.Logger.Fatal().Msg("unstaked bind address not set") } + + if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { + builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") + } +} + +// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstreap peers from the parameters. +// These are the identity of the staked and unstaked AN also acting as the DHT bootstrap server +func (builder *UnstakedAccessNodeBuilder) deriveBootstrapPeerIdentities() { + ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) + builder.MustNot(err) + builder.bootstrapIdentites = ids } // initUnstakedLocal initializes the unstaked node ID, network key and network address @@ -111,35 +127,18 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - // intialize the LibP2P factory with an empty metrics NoopCollector for now till we have defined the new unstaked - // network metrics - libP2PFactory, err := builder.FlowAccessNodeBuilder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkMetrics, unstakedNetworkKey) + libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) builder.MustNot(err) - // use the default validators for the staked access node unstaked networks - msgValidators := p2p.DefaultValidators(builder.Logger, unstakedNodeID) + msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) - // don't need any peer updates since this will be taken care by the DHT discovery mechanism - peerUpdateInterval := time.Hour + middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, peerUpdateInterval, - node.UnicastMessageTimeout, - false, // no connection gating for the unstaked network - false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) - msgValidators...) + // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront + participants := flow.IdentityList{} - upstreamANIdentifier, err := flow.HexStringToIdentifier(builder.stakedAccessNodeIDHex) - builder.MustNot(err) - - // topology only consist of the upsteam staked AN - top := topology.NewFixedListTopology(upstreamANIdentifier) - - network, err := builder.initNetwork( - builder.Me, - unstakedNetworkMetrics, - middleware, - top, - ) + // topology is nil since its automatically managed by libp2p + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, nil) builder.MustNot(err) builder.UnstakedNetwork = network @@ -154,3 +153,15 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context return builder.UnstakedNetwork, err }) } + +// enqueueConnectWithStakedAN enqueues the upstream connector component which connects the libp2p host of the unstaked +// AN with the staked AN. +// Currently, there is an issue with LibP2P stopping advertisements of subscribed topics if no peers are connected +// (https://github.com/libp2p/go-libp2p-pubsub/issues/442). This means that an unstaked AN could end up not being +// discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need +// of an explicit connect to the staked AN before the node attempts to subscribe to topics. +func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { + builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return newUpstreamConnector(builder.bootstrapIdentites, builder.UnstakedLibP2PNode, builder.Logger), nil + }) +} diff --git a/cmd/scaffold.go b/cmd/scaffold.go index d57fb218282..71d9386107f 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -209,10 +209,11 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Network = net - idEvents := gadgets.NewIdentityDeltas(fnb.Middleware.UpdateAllowList) + idEvents := gadgets.NewIdentityDeltas(func() { + fnb.Middleware.UpdateNodeAddresses() + fnb.Middleware.UpdateAllowList() + }) fnb.ProtocolEvents.AddConsumer(idEvents) - // TODO: add a consumer which implements setting the permanent peer addresses, - // and expiring the old permanent ones return net, err }) diff --git a/network/middleware.go b/network/middleware.go index 82928584d43..8bd89167549 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -53,6 +53,8 @@ type Middleware interface { // UpdateAllowList fetches the most recent identity of the nodes from overlay // and updates the underlying libp2p node. UpdateAllowList() + + UpdateNodeAddresses() } // Overlay represents the interface that middleware uses to interact with the @@ -62,7 +64,8 @@ type Overlay interface { Topology() (flow.IdentifierList, error) Receive(nodeID flow.Identifier, msg *message.Message) error SetDefaultIdentifierProvider(id.IdentifierProvider) - GetIdentifierProvider() id.IdentifierProvider + Identifiers() flow.IdentifierList + Identities() flow.IdentityList } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 5702bade9b3..5fd1c3d04bb 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -385,6 +385,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp // Hence, explicitly cancel the dial back off (if any) and try connecting again if len(n.host.Peerstore().Addrs(peerID)) == 0 { + // TODO: add bunch of logging here if n.dht != nil { // TODO: adjust timeout timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) @@ -680,12 +681,9 @@ func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { } } -func WithDHTDiscovery(option ...dht.Option) PubsubOption { - return func(ctx context.Context, host host.Host) (pubsub.Option, error) { - dhtDiscovery, err := NewDHT(ctx, host, option...) - if err != nil { - return nil, err - } - return pubsub.WithDiscovery(dhtDiscovery), nil - } - } +func WithDHTDiscovery(kdht *dht.IpfsDHT) PubsubOption { + return func(ctx context.Context, host host.Host) (pubsub.Option, error) { + routingDiscovery := discovery.NewRoutingDiscovery(kdht) + return pubsub.WithDiscovery(routingDiscovery), nil + } +} diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index f24ad9c54a3..ab9b8ed54a2 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -12,11 +12,13 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" @@ -79,6 +81,7 @@ type Middleware struct { connectionGating bool managePeerConnections bool idTranslator IDTranslator + idProvider id.IdentityProvider } // NewMiddleware creates a new middleware instance @@ -101,6 +104,7 @@ func NewMiddleware( connectionGating bool, managePeerConnections bool, idTranslator IDTranslator, + idProvider id.IdentityProvider, validators ...network.MessageValidator, ) *Middleware { ctx, cancel := context.WithCancel(context.Background()) @@ -125,6 +129,7 @@ func NewMiddleware( connectionGating: connectionGating, managePeerConnections: managePeerConnections, idTranslator: idTranslator, + idProvider: idProvider, } if len(validators) != 0 { @@ -152,7 +157,7 @@ func (m *Middleware) topologyPeers() (peer.IDSlice, error) { } func (m *Middleware) allPeers() peer.IDSlice { - return m.peerIDs(m.ov.GetIdentifierProvider().Identifiers()) + return m.peerIDs(m.ov.Identifiers()) } func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { @@ -180,6 +185,15 @@ func (m *Middleware) GetIPPort() (string, string, error) { return m.libP2PNode.GetIPPort() } +func (m *Middleware) UpdateNodeAddresses() { + ids := m.ov.Identities() + infos, _ := peerInfosFromIDs(ids) + + for _, info := range infos { + m.libP2PNode.host.Peerstore().SetAddrs(info.ID, info.Addrs, peerstore.PermanentAddrTTL) + } +} + // Start will start the middleware. func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov @@ -193,6 +207,8 @@ func (m *Middleware) Start(ov network.Overlay) error { m.libP2PNode = libP2PNode m.libP2PNode.SetFlowProtocolStreamHandler(m.handleIncomingStream) + m.UpdateNodeAddresses() + if m.connectionGating { m.libP2PNode.UpdateAllowList(m.allPeers()) } diff --git a/network/p2p/network.go b/network/p2p/network.go index 0584143e5c5..b87957774c4 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -43,6 +43,7 @@ type Network struct { sync.RWMutex idProvider id.IdentifierProvider defaultIdProvider id.IdentifierProvider + identityProvider id.IdentityProvider logger zerolog.Logger codec network.Codec me module.Local @@ -78,6 +79,7 @@ func NewNetwork( top network.Topology, sm network.SubscriptionManager, metrics module.NetworkMetrics, + identityProvider id.IdentityProvider, opts ...NetworkOption, ) (*Network, error) { @@ -96,6 +98,7 @@ func NewNetwork( metrics: metrics, subMngr: sm, lifecycleManager: lifecycle.NewLifecycleManager(), + identityProvider: identityProvider, } o.ctx, o.cancel = context.WithCancel(context.Background()) @@ -175,7 +178,7 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -func (n *Network) GetIdentifierProvider() id.IdentifierProvider { +func (n *Network) getIdentifierProvider() id.IdentifierProvider { if n.idProvider != nil { return n.idProvider } @@ -188,6 +191,14 @@ func (n *Network) GetIdentifierProvider() id.IdentifierProvider { return n.defaultIdProvider } +func (n *Network) Identifiers() flow.IdentifierList { + return n.getIdentifierProvider().Identifiers() +} + +func (n *Network) Identities() flow.IdentityList { + return n.identityProvider.Identities(NetworkingSetFilter) +} + // Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. func (n *Network) Topology() (flow.IdentifierList, error) { @@ -195,7 +206,7 @@ func (n *Network) Topology() (flow.IdentifierList, error) { defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.GetIdentifierProvider().Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } From b81c627416a3b9e5cc921ee529d3658ed116b729 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 15:48:37 -0700 Subject: [PATCH 198/291] undo topology changes --- .../node_builder/access_node_builder.go | 13 ++- .../staked_access_node_builder.go | 1 + .../unstaked_access_node_builder.go | 8 +- cmd/access/node_builder/upstream_connector.go | 7 +- cmd/scaffold.go | 10 ++- network/middleware.go | 7 +- network/p2p/middleware.go | 38 ++++++--- network/p2p/network.go | 70 ++++----------- network/topology.go | 8 +- network/topology/cache.go | 16 ++-- network/topology/fixedListTopology.go | 10 +-- network/topology/randomizedTopology.go | 71 +++++++--------- network/topology/topicBasedTopology.go | 85 +++++++++---------- network/topology/topology_utils.go | 4 +- 14 files changed, 158 insertions(+), 190 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index cfb6e5f71a4..64c8480d79c 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -150,6 +150,7 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components + UnstakedLibP2PNode *p2p.Node UnstakedNetwork *p2p.Network unstakedMiddleware *p2p.Middleware FollowerState protocol.MutableState @@ -443,6 +444,7 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, + node.Storage.Results, node.RootChainID, anb.TransactionMetrics, anb.collectionGRPCPort, @@ -471,7 +473,7 @@ func (anb *FlowAccessNodeBuilder) Build() AccessNodeBuilder { return nil, fmt.Errorf("could not create requester engine: %w", err) } - anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Receipts, anb.TransactionMetrics, + anb.IngestEng, err = ingestion.New(node.Logger, node.Network, node.State, node.Me, anb.RequestEng, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, node.Storage.Transactions, node.Storage.Results, node.Storage.Receipts, anb.TransactionMetrics, anb.CollectionsToMarkFinalized, anb.CollectionsToMarkExecuted, anb.BlocksToMarkExecuted, anb.RpcEng) anb.RequestEng.WithHandle(anb.IngestEng.OnCollection) anb.FinalizationDistributor.AddConsumer(anb.IngestEng) @@ -614,7 +616,8 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, validators ...network.MessageValidator) *p2p.Middleware { - builder.unstakedMiddleware = p2p.NewMiddleware(builder.Logger, + builder.unstakedMiddleware = p2p.NewMiddleware( + builder.Logger, factoryFunc, nodeID, networkMetrics, @@ -624,7 +627,9 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, false, // no connection gating for the unstaked network false, // no peer management for the unstaked network (peer discovery will be done via LibP2P discovery mechanism) builder.IDTranslator, - validators...) + p2p.WithMessageValidators(validators...), + // use default identifier provider + ) return builder.unstakedMiddleware } @@ -650,7 +655,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, topology, subscriptionManager, networkMetrics, - p2p.WithIdentifierProvider(builder.NetworkingIdentifierProvider), + builder.IdentityProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index d022c886700..a8e5a9731c1 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -11,6 +11,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" ) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 88ce203fe65..d2f9d8e02d0 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -105,7 +105,8 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - anb.SyncEngineParticipantsProvider = node.Network.GetIdentifierProvider() + // use the default identifier provider + anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() return nil }) anb.FlowAccessNodeBuilder.Build() @@ -134,11 +135,8 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // empty list of unstaked network participants since they will be discovered dynamically and are not known upfront - participants := flow.IdentityList{} - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, participants, nil) + network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) builder.MustNot(err) builder.UnstakedNetwork = network diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index 0a4c3405dbd..aabfa446426 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -105,8 +105,13 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f return default: } + peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) + if err != nil { + // TODO: return formatted error + } + // try and connect to the bootstrap server - err := connector.unstakedNode.AddPeer(ctx, bootstrapPeer) + err = connector.unstakedNode.AddPeer(ctx, peerID) resultChan <- result{ id: bootstrapPeer, err: err, diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 71d9386107f..8689399ba75 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -168,7 +168,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) } - fnb.Middleware = p2p.NewMiddleware(fnb.Logger.Level(zerolog.ErrorLevel), + fnb.Middleware = p2p.NewMiddleware( + fnb.Logger.Level(zerolog.ErrorLevel), libP2PNodeFactory, fnb.Me.NodeID(), fnb.Metrics.Network, @@ -178,12 +179,13 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { true, true, fnb.IDTranslator, - fnb.MsgValidators...) + p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + p2p.WithMessageValidators(fnb.MsgValidators...), + ) subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) top, err := topology.NewTopicBasedTopology( fnb.NodeID, - fnb.IdentityProvider, fnb.Logger, fnb.State, ) @@ -201,7 +203,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { topologyCache, subscriptionManager, fnb.Metrics.Network, - p2p.WithIdentifierProvider(fnb.NetworkingIdentifierProvider), + fnb.IdentityProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize network: %w", err) diff --git a/network/middleware.go b/network/middleware.go index 8bd89167549..b6d538da53a 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -6,7 +6,6 @@ import ( "time" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/message" ) @@ -60,11 +59,9 @@ type Middleware interface { // Overlay represents the interface that middleware uses to interact with the // overlay network layer. type Overlay interface { - // Topology returns an identifier list of nodes which this node should be directly connected to as peers - Topology() (flow.IdentifierList, error) + // Topology returns an identity list of nodes which this node should be directly connected to as peers + Topology() (flow.IdentityList, error) Receive(nodeID flow.Identifier, msg *message.Message) error - SetDefaultIdentifierProvider(id.IdentifierProvider) - Identifiers() flow.IdentifierList Identities() flow.IdentityList } diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index ab9b8ed54a2..e43d7674c08 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -81,7 +81,21 @@ type Middleware struct { connectionGating bool managePeerConnections bool idTranslator IDTranslator - idProvider id.IdentityProvider + idProvider id.IdentifierProvider +} + +type MiddlewareOption func(*Middleware) + +func WithIdentifierProvider(provider id.IdentifierProvider) MiddlewareOption { + return func(mw *Middleware) { + mw.idProvider = provider + } +} + +func WithMessageValidators(validators ...network.MessageValidator) MiddlewareOption { + return func(mw *Middleware) { + mw.validators = validators + } } // NewMiddleware creates a new middleware instance @@ -104,8 +118,7 @@ func NewMiddleware( connectionGating bool, managePeerConnections bool, idTranslator IDTranslator, - idProvider id.IdentityProvider, - validators ...network.MessageValidator, + opts ...MiddlewareOption, ) *Middleware { ctx, cancel := context.WithCancel(context.Background()) @@ -129,11 +142,10 @@ func NewMiddleware( connectionGating: connectionGating, managePeerConnections: managePeerConnections, idTranslator: idTranslator, - idProvider: idProvider, } - if len(validators) != 0 { - mw.validators = validators + for _, opt := range opts { + opt(mw) } return mw @@ -147,17 +159,17 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes } func (m *Middleware) topologyPeers() (peer.IDSlice, error) { - identifiers, err := m.ov.Topology() + identities, err := m.ov.Topology() if err != nil { // TODO: format error return nil, err } - return m.peerIDs(identifiers), nil + return m.peerIDs(identities.NodeIDs()), nil } func (m *Middleware) allPeers() peer.IDSlice { - return m.peerIDs(m.ov.Identifiers()) + return m.peerIDs(m.idProvider.Identifiers()) } func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { @@ -199,7 +211,9 @@ func (m *Middleware) Start(ov network.Overlay) error { m.ov = ov libP2PNode, err := m.libP2PNodeFactory() - ov.SetDefaultIdentifierProvider(NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator)) + if m.idProvider == nil { + m.idProvider = NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator) + } if err != nil { return fmt.Errorf("could not create libp2p node: %w", err) @@ -446,6 +460,10 @@ func (m *Middleware) UpdateAllowList() { m.peerManagerUpdate() } +func (m *Middleware) IdentifierProvider() id.IdentifierProvider { + return m.idProvider +} + // IsConnected returns true if this node is connected to the node with id nodeID. func (m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { peerID, err := m.idTranslator.GetPeerID(nodeID) diff --git a/network/p2p/network.go b/network/p2p/network.go index b87957774c4..82235bd4938 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -41,29 +41,19 @@ type ReadyDoneAwareNetwork interface { // the protocols for handshakes, authentication, gossiping and heartbeats. type Network struct { sync.RWMutex - idProvider id.IdentifierProvider - defaultIdProvider id.IdentifierProvider - identityProvider id.IdentityProvider - logger zerolog.Logger - codec network.Codec - me module.Local - mw network.Middleware - top network.Topology // used to determine fanout connections - metrics module.NetworkMetrics - rcache *RcvCache // used to deduplicate incoming messages - queue network.MessageQueue - ctx context.Context - cancel context.CancelFunc - subMngr network.SubscriptionManager // used to keep track of subscribed channels - lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle -} - -type NetworkOption func(*Network) - -func WithIdentifierProvider(provider id.IdentifierProvider) NetworkOption { - return func(net *Network) { - net.idProvider = provider - } + identityProvider id.IdentityProvider + logger zerolog.Logger + codec network.Codec + me module.Local + mw network.Middleware + top network.Topology // used to determine fanout connections + metrics module.NetworkMetrics + rcache *RcvCache // used to deduplicate incoming messages + queue network.MessageQueue + ctx context.Context + cancel context.CancelFunc + subMngr network.SubscriptionManager // used to keep track of subscribed channels + lifecycleManager *lifecycle.LifecycleManager // used to manage the network's start-stop lifecycle } // NewNetwork creates a new naive overlay network, using the given middleware to @@ -80,7 +70,6 @@ func NewNetwork( sm network.SubscriptionManager, metrics module.NetworkMetrics, identityProvider id.IdentityProvider, - opts ...NetworkOption, ) (*Network, error) { rcache, err := newRcvCache(csize) @@ -109,10 +98,6 @@ func NewNetwork( // create workers to read from the queue and call queueSubmitFunc queue.CreateQueueWorkers(o.ctx, queue.DefaultNumWorkers, o.queue, o.queueSubmitFunc) - for _, opt := range opts { - opt(o) - } - return o, nil } @@ -178,35 +163,18 @@ func (n *Network) unregister(channel network.Channel) error { return nil } -func (n *Network) getIdentifierProvider() id.IdentifierProvider { - if n.idProvider != nil { - return n.idProvider - } - n.RLock() - defer n.RUnlock() - if n.defaultIdProvider == nil { - n.logger.Fatal().Msg("TODO") - // TODO - } - return n.defaultIdProvider -} - -func (n *Network) Identifiers() flow.IdentifierList { - return n.getIdentifierProvider().Identifiers() -} - func (n *Network) Identities() flow.IdentityList { return n.identityProvider.Identities(NetworkingSetFilter) } -// Topology returns the identifiers of a uniform subset of nodes in protocol state using the topology provided earlier. +// Topology returns the identitiess of a uniform subset of nodes in protocol state using the topology provided earlier. // Independent invocations of Topology on different nodes collectively constructs a connected network graph. -func (n *Network) Topology() (flow.IdentifierList, error) { +func (n *Network) Topology() (flow.IdentityList, error) { n.Lock() defer n.Unlock() subscribedChannels := n.subMngr.Channels() - top, err := n.top.GenerateFanout(n.Identifiers(), subscribedChannels) + top, err := n.top.GenerateFanout(n.Identities(), subscribedChannels) if err != nil { return nil, fmt.Errorf("could not generate topology: %w", err) } @@ -313,12 +281,6 @@ func (n *Network) genNetworkMessage(channel network.Channel, event interface{}, return msg, nil } -func (n *Network) SetDefaultIdentifierProvider(provider id.IdentifierProvider) { - n.Lock() - n.defaultIdProvider = provider - n.Unlock() -} - // unicast sends the message in a reliable way to the given recipient. // It uses 1-1 direct messaging over the underlying network to deliver the message. // It returns an error if unicasting fails. diff --git a/network/topology.go b/network/topology.go index 1d0faf16212..1d746c70896 100644 --- a/network/topology.go +++ b/network/topology.go @@ -6,9 +6,9 @@ import ( // Topology provides a subset of nodes which a given node should directly connect to for 1-k messaging. type Topology interface { - // GenerateFanout receives IdentifierList of entire network, and list of channels the node is subscribing to. - // It constructs and returns the fanout IdentifierList of node. - // A node directly communicates with its fanout IdentifierList on epidemic dissemination + // GenerateFanout receives IdentityList of entire network, and list of channels the node is subscribing to. + // It constructs and returns the fanout IdentityList of node. + // A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. @@ -19,5 +19,5 @@ type Topology interface { // // GenerateFanout is not concurrency safe. It is responsibility of caller to lock for it. // with the channels argument, it allows the returned topology to be cached, which is necessary for randomized topology. - GenerateFanout(ids flow.IdentifierList, channels ChannelList) (flow.IdentifierList, error) + GenerateFanout(ids flow.IdentityList, channels ChannelList) (flow.IdentityList, error) } diff --git a/network/topology/cache.go b/network/topology/cache.go index a738e9e9453..c305140abed 100644 --- a/network/topology/cache.go +++ b/network/topology/cache.go @@ -19,10 +19,10 @@ import ( // in a concurrency safe way, i.e., the caller should lock for it. type Cache struct { log zerolog.Logger - top network.Topology // instance of underlying topology. - cachedFanout flow.IdentifierList // most recently generated fanout list by invoking underlying topology. - idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. - chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. + top network.Topology // instance of underlying topology. + cachedFanout flow.IdentityList // most recently generated fanout list by invoking underlying topology. + idsFP flow.Identifier // unique fingerprint of input IdentityList for cached fanout. + chansFP flow.Identifier // unique fingerprint of input ChannelsList for cached fanout. } //NewCache creates and returns a topology Cache given an instance of topology implementation. @@ -36,19 +36,19 @@ func NewCache(log zerolog.Logger, top network.Topology) *Cache { } } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList // of this instance. // It caches the most recently generated fanout list, so as long as the input list is the same, it returns // the same output. It invalidates and updates its internal cache the first time input list changes. -// A node directly communicates with its fanout IdentifierList on epidemic dissemination +// A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. // // Note that this implementation of GenerateFanout preserves same output as long as input is the same. This // should not be assumed as a 1-1 mapping between input and output. -func (c *Cache) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { - inputIdsFP := flow.MerkleRoot(ids...) +func (c *Cache) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { + inputIdsFP := ids.Fingerprint() inputChansFP := channels.ID() log := c.log.With(). diff --git a/network/topology/fixedListTopology.go b/network/topology/fixedListTopology.go index 6091da61d38..1b171ce8919 100644 --- a/network/topology/fixedListTopology.go +++ b/network/topology/fixedListTopology.go @@ -2,7 +2,7 @@ package topology import ( "github.com/onflow/flow-go/model/flow" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/network" ) @@ -17,14 +17,14 @@ func NewFixedListTopology(nodeID flow.Identifier) FixedListTopology { } } -func (r FixedListTopology) GenerateFanout(ids flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { - return ids.Filter(idFilter.Is(r.fixedNodeID)), nil +func (r FixedListTopology) GenerateFanout(ids flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { + return ids.Filter(filter.HasNodeID(r.fixedNodeID)), nil } // EmptyListTopology always returns an empty list as the fanout type EmptyListTopology struct { } -func (r EmptyListTopology) GenerateFanout(_ flow.IdentifierList, _ network.ChannelList) (flow.IdentifierList, error) { - return flow.IdentifierList{}, nil +func (r EmptyListTopology) GenerateFanout(_ flow.IdentityList, _ network.ChannelList) (flow.IdentityList, error) { + return flow.IdentityList{}, nil } diff --git a/network/topology/randomizedTopology.go b/network/topology/randomizedTopology.go index 55be3545025..c1bc959ed9e 100644 --- a/network/topology/randomizedTopology.go +++ b/network/topology/randomizedTopology.go @@ -9,8 +9,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -19,16 +17,15 @@ import ( // By random topology we mean a node is connected to any other co-channel nodes with some // edge probability. type RandomizedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - chance uint64 // used to translate connectedness probability into a number in [0, 100] - rng random.Rand // used as a stateful random number generator to sample edges - logger zerolog.Logger - idProvider id.IdentityProvider + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + chance uint64 // used to translate connectedness probability into a number in [0, 100] + rng random.Rand // used as a stateful random number generator to sample edges + logger zerolog.Logger } // NewRandomizedTopology returns an instance of the RandomizedTopology. -func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { +func NewRandomizedTopology(nodeID flow.Identifier, logger zerolog.Logger, edgeProb float64, state protocol.State) (*RandomizedTopology, error) { // edge probability should be a positive value between 0 and 1. However, // we like it to be strictly greater than zero. Also, at the current scale of // Flow, we need it to be greater than 0.01 to support probabilistic connectedness. @@ -47,34 +44,33 @@ func NewRandomizedTopology(nodeID flow.Identifier, idProvider id.IdentityProvide } t := &RandomizedTopology{ - myNodeID: nodeID, - state: state, - chance: uint64(100 * edgeProb), - rng: rng, - logger: logger.With().Str("component:", "randomized-topology").Logger(), - idProvider: idProvider, + myNodeID: nodeID, + state: state, + chance: uint64(100 * edgeProb), + rng: rng, + logger: logger.With().Str("component:", "randomized-topology").Logger(), } return t, nil } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList -// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. This should be done with a very high probability // in randomized topology. -func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { +func (r RandomizedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. r.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } - var myFanout flow.IdentifierList + var myFanout flow.IdentityList // generates a randomized subgraph per channel for _, myChannel := range myUniqueChannels { @@ -94,12 +90,12 @@ func (r RandomizedTopology) GenerateFanout(ids flow.IdentifierList, channels net return myFanout, nil } -// subsetChannel returns a random subset of the identifier list that is passed. -// Returned identifiers should all subscribed to the specified `channel`. -// Note: this method should not include identifier of its executor. -func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +// subsetChannel returns a random subset of the identity list that is passed. +// Returned identities should all subscribed to the specified `channel`. +// Note: this method should not include identity of its executor. +func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { // excludes node itself - sampleSpace := ids.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) + sampleSpace := ids.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) // samples a random graph based on whether channel is cluster-based or not. if _, ok := engine.ClusterChannel(channel); ok { @@ -113,12 +109,12 @@ func (r RandomizedTopology) subsetChannel(ids flow.IdentifierList, channel netwo // Independent invocations of this method over different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.IdentifierList, error) { +func (r RandomizedTopology) sampleFanout(ids flow.IdentityList) (flow.IdentityList, error) { if len(ids) == 0 { - return nil, fmt.Errorf("empty identifier list") + return nil, fmt.Errorf("empty identity list") } - fanout := flow.IdentifierList{} + fanout := flow.IdentityList{} for _, id := range ids { // tosses a biased coin and adds id to fanout accordingly. // biased coin follows the edge probability distribution. @@ -131,7 +127,7 @@ func (r RandomizedTopology) sampleFanout(ids flow.IdentifierList) (flow.Identifi } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow.IdentifierList, error) { +func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.IdentityList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(r.myNodeID, r.state) if err != nil { @@ -139,10 +135,10 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow } // excludes node itself from cluster - clusterPeers = clusterPeers.Filter(idFilter.Not(idFilter.Is(r.myNodeID))) + clusterPeers = clusterPeers.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) + nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } @@ -152,7 +148,7 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentifierList) (flow } // clusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. -func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -164,16 +160,7 @@ func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentifierList, ch } // samples fanout among interacting roles - return r.sampleFanout( - r.idProvider. - Identities( - filter.And( - filter.HasNodeID(ids...), - filter.HasRole(roles...), - ), - ). - NodeIDs(), - ) + return r.sampleFanout(ids.Filter(filter.HasRole(roles...))) } // tossBiasedBit returns true with probability equal `r.chance/100`, and returns false otherwise. diff --git a/network/topology/topicBasedTopology.go b/network/topology/topicBasedTopology.go index fdc55f43b1e..9d14e34049b 100644 --- a/network/topology/topicBasedTopology.go +++ b/network/topology/topicBasedTopology.go @@ -8,8 +8,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - idFilter "github.com/onflow/flow-go/model/flow/filter/id" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -17,15 +15,14 @@ import ( // TopicBasedTopology is a deterministic topology mapping that creates a connected graph component among the nodes // involved in each topic. type TopicBasedTopology struct { - myNodeID flow.Identifier // used to keep identifier of the node - state protocol.State // used to keep a read only protocol state - logger zerolog.Logger - seed int64 - identityProvider id.IdentityProvider + myNodeID flow.Identifier // used to keep identifier of the node + state protocol.State // used to keep a read only protocol state + logger zerolog.Logger + seed int64 } // NewTopicBasedTopology returns an instance of the TopicBasedTopology. -func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvider, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { +func NewTopicBasedTopology(nodeID flow.Identifier, logger zerolog.Logger, state protocol.State) (*TopicBasedTopology, error) { seed, err := intSeedFromID(nodeID) if err != nil { return nil, fmt.Errorf("could not generate seed from id:%w", err) @@ -41,19 +38,19 @@ func NewTopicBasedTopology(nodeID flow.Identifier, idProvider id.IdentityProvide return t, nil } -// GenerateFanout receives IdentifierList of entire network and constructs the fanout IdentifierList -// of this instance. A node directly communicates with its fanout IdentifierList on epidemic dissemination +// GenerateFanout receives IdentityList of entire network and constructs the fanout IdentityList +// of this instance. A node directly communicates with its fanout IdentityList on epidemic dissemination // of the messages (i.e., publish and multicast). // Independent invocations of GenerateFanout on different nodes collaboratively must construct a cohesive // connected graph of nodes that enables them talking to each other. -func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels network.ChannelList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels network.ChannelList) (flow.IdentityList, error) { myUniqueChannels := engine.UniqueChannels(channels) if len(myUniqueChannels) == 0 { // no subscribed channel, hence skip topology creation // we do not return an error at this state as invocation of MakeTopology may happen before // node subscribing to all its channels. t.logger.Warn().Msg("skips generating fanout with no subscribed channels") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } // finds all interacting roles with this node @@ -67,7 +64,7 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels net } // builds a connected component per role this node interact with, - var myFanout flow.IdentifierList + var myFanout flow.IdentityList for _, role := range myInteractingRoles { if role == flow.RoleCollection { // we do not build connected component for collection nodes based on their role @@ -101,36 +98,32 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentifierList, channels net return myFanout, nil } -// subsetChannel returns a random subset of the identifier list that is passed. `shouldHave` represents set of -// identifiers that should be included in the returned subset. -// Returned identifiers should all subscribed to the specified `channel`. -// Note: this method should not include identifier of its executor. -func (t *TopicBasedTopology) subsetChannel(ids flow.IdentifierList, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +// subsetChannel returns a random subset of the identity list that is passed. `shouldHave` represents set of +// identities that should be included in the returned subset. +// Returned identities should all subscribed to the specified `channel`. +// Note: this method should not include identity of its executor. +func (t *TopicBasedTopology) subsetChannel(ids flow.IdentityList, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return t.clusterChannelHandler(ids, shouldHave) } return t.nonClusterChannelHandler(ids, shouldHave, channel) } -// subsetRole returns a random subset of the identifier list that is passed. `shouldHave` represents set of -// identifiers that should be included in the returned subset. -// Returned identifiers should all be of one of the specified `roles`. -// Note: this method should not include the identifier of its executor. -func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow.IdentifierList, roles flow.RoleList) (flow.IdentifierList, error) { +// subsetRole returns a random subset of the identity list that is passed. `shouldHave` represents set of +// identities that should be included in the returned subset. +// Returned identities should all be of one of the specified `roles`. +// Note: this method should not include identity of its executor. +func (t TopicBasedTopology) subsetRole(ids flow.IdentityList, shouldHave flow.IdentityList, roles flow.RoleList) (flow.IdentityList, error) { // excludes irrelevant roles and the node itself from both should have and ids set - shouldHave = t.identityProvider. - Identities(filter.HasNodeID(shouldHave...)). - Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )).NodeIDs() - - ids = t.identityProvider. - Identities(filter.HasNodeID(ids...)). - Filter(filter.And( - filter.HasRole(roles...), - filter.Not(filter.HasNodeID(t.myNodeID)), - )).NodeIDs() + shouldHave = shouldHave.Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )) + + ids = ids.Filter(filter.And( + filter.HasRole(roles...), + filter.Not(filter.HasNodeID(t.myNodeID)), + )) sample, err := t.sampleConnectedGraph(ids, shouldHave) if err != nil { @@ -145,11 +138,11 @@ func (t TopicBasedTopology) subsetRole(ids flow.IdentifierList, shouldHave flow. // different nodes, should create a connected graph. // Fanout is the set of nodes that this instance should get connected to in order to create a // connected graph. -func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentityList, shouldHave flow.IdentityList) (flow.IdentityList, error) { if len(all) == 0 { t.logger.Debug().Msg("skips sampling connected graph with zero nodes") - return flow.IdentifierList{}, nil + return flow.IdentityList{}, nil } if len(shouldHave) == 0 { @@ -160,9 +153,9 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should } // checks `shouldHave` be a subset of `all` - nonMembers := shouldHave.Filter(idFilter.Not(idFilter.In(all...))) + nonMembers := shouldHave.Filter(filter.Not(filter.In(all))) if len(nonMembers) != 0 { - return nil, fmt.Errorf("should have identifiers is not a subset of all: %v", nonMembers) + return nil, fmt.Errorf("should have identities is not a subset of all: %v", nonMembers) } // total sample size @@ -177,7 +170,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should subsetSize := totalSize - len(shouldHave) // others are all excluding should have ones - others := all.Filter(idFilter.Not(idFilter.In(shouldHave...))) + others := all.Filter(filter.Not(filter.In(shouldHave))) others = others.DeterministicSample(uint(subsetSize), t.seed) return others.Union(shouldHave), nil @@ -185,7 +178,7 @@ func (t TopicBasedTopology) sampleConnectedGraph(all flow.IdentifierList, should } // clusterChannelHandler returns a connected graph fanout of peers in the same cluster as executor of this instance. -func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentifierList) (flow.IdentifierList, error) { +func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityList) (flow.IdentityList, error) { // extracts cluster peer ids to which the node belongs to. clusterPeers, err := clusterPeers(t.myNodeID, t.state) if err != nil { @@ -193,18 +186,18 @@ func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.Identifie } // checks all cluster peers belong to the passed ids list - nonMembers := clusterPeers.Filter(idFilter.Not(idFilter.In(ids...))) + nonMembers := clusterPeers.Filter(filter.Not(filter.In(ids))) if len(nonMembers) > 0 { return nil, fmt.Errorf("cluster peers not belonged to sample space: %v", nonMembers) } // samples a connected graph topology from the cluster peers - return t.subsetRole(clusterPeers, shouldHave.Filter(idFilter.In(clusterPeers...)), flow.RoleList{flow.RoleCollection}) + return t.subsetRole(clusterPeers, shouldHave.Filter(filter.HasNodeID(clusterPeers.NodeIDs()...)), flow.RoleList{flow.RoleCollection}) } // nonClusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. // The returned sample contains `shouldHave` ones that also subscribed to `channel`. -func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentifierList, channel network.Channel) (flow.IdentifierList, error) { +func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { if _, ok := engine.ClusterChannel(channel); ok { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } @@ -216,5 +209,5 @@ func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.Identi } // samples a connected graph topology - return t.subsetRole(ids, shouldHave, roles) + return t.subsetRole(ids.Filter(filter.HasRole(roles...)), shouldHave, roles) } diff --git a/network/topology/topology_utils.go b/network/topology/topology_utils.go index 3668d8557a6..bf1d2b43986 100644 --- a/network/topology/topology_utils.go +++ b/network/topology/topology_utils.go @@ -51,7 +51,7 @@ func byteSeedFromID(id flow.Identifier) ([]byte, error) { } // clusterPeers returns the list of other nodes within the same cluster as specified identifier. -func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList, error) { +func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentityList, error) { currentEpoch := state.Final().Epochs().Current() clusterList, err := currentEpoch.Clustering() if err != nil { @@ -63,5 +63,5 @@ func clusterPeers(id flow.Identifier, state protocol.State) (flow.IdentifierList return nil, fmt.Errorf("failed to find the cluster for node ID %s", id.String()) } - return myCluster.NodeIDs(), nil + return myCluster, nil } From 16479998b4b6e0dc25113d999774fd3b834804aa Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 17 Aug 2021 16:50:41 -0700 Subject: [PATCH 199/291] add custom id translator --- .../staked_access_node_builder.go | 11 ++++-- network/p2p/hierarchical_translator.go | 37 +++++++++++++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) create mode 100644 network/p2p/hierarchical_translator.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a8e5a9731c1..b7bb142d921 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -36,18 +36,20 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache // translator - // networking provider fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NetworkingSetFilter, ), idCache, ) + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) // TODO: need special providers here - // for network, needs one that recognizes both protocl state and peerstore - // same for translator + // NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // doesn't participate in unstaked network. + // If it does, then we can just use the default one (peerstoreProvider) return nil }) @@ -62,7 +64,8 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - builder.EnqueueNetworkInit(ctx) + // TODO: we should remove this call since we are no longer instantiating two networks + // builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network if builder.ParticipatesInUnstakedNetwork() { diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go new file mode 100644 index 00000000000..28dbdb03c31 --- /dev/null +++ b/network/p2p/hierarchical_translator.go @@ -0,0 +1,37 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type HierarchicalIDTranslator struct { + translators []IDTranslator +} + +func NewHierarchicalIDTranslator(translators ...IDTranslator) *HierarchicalIDTranslator { + return &HierarchicalIDTranslator{translators} +} + +func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + for _, translator := range t.translators { + pid, err := translator.GetPeerID(flowID) + if err == nil { + return pid, nil + } + } + return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v", flowID) +} + +func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { + for _, translator := range t.translators { + fid, err := translator.GetFlowID(peerID) + if err == nil { + return fid, nil + } + } + return flow.ZeroID, fmt.Errorf("could not find corresponding flow ID for peer ID %v", peerID) +} From 098c07151a8b56ea4ca46e39b13f5ced74c474fd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 18 Aug 2021 11:10:53 -0400 Subject: [PATCH 200/291] validate ID translation --- network/p2p/unstaked_translator.go | 12 +-- network/p2p/unstaked_translator_test.go | 98 +++++++++++++++++++++++++ 2 files changed, 105 insertions(+), 5 deletions(-) create mode 100644 network/p2p/unstaked_translator_test.go diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 5e320ccb236..a3211c1fc68 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -1,6 +1,8 @@ package p2p import ( + "fmt" + crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/libp2p/go-libp2p-core/peer" "github.com/multiformats/go-multihash" @@ -27,16 +29,16 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { pk, err := peerID.ExtractPublicKey() if err != nil { - // return error + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) } - if pk.Type() != crypto_pb.KeyType_ECDSA { - // fail + if pk.Type() != crypto_pb.KeyType_Secp256k1 { + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) } data, err := pk.Raw() - if err != nil || data[0] != 0x02 { // TODO: check if this is the right byte to check - // fail + if err != nil || data[0] != 0x02 { + return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) } return flow.HashToID(data[1:]), nil diff --git a/network/p2p/unstaked_translator_test.go b/network/p2p/unstaked_translator_test.go new file mode 100644 index 00000000000..46a3b1dd58c --- /dev/null +++ b/network/p2p/unstaked_translator_test.go @@ -0,0 +1,98 @@ +package p2p + +import ( + "crypto/rand" + "math" + "testing" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/stretchr/testify/require" + + fcrypto "github.com/onflow/flow-go/crypto" +) + +// This test shows we can't use ECDSA P-256 keys for libp2p and expect PeerID <=> PublicKey bijections +func TestIDTranslationP256(t *testing.T) { + loops := 50 + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSAP256) + + // check that we can not extract the public key back + // This makes sense: the x509 serialization of ECDSA P-256 keys in uncompressed form is 64 + 2 bytes, + // and libp2p uses multihash.IDENTITY only on serializations of less than 42 bytes + _, err := pID.ExtractPublicKey() + require.NotNil(t, err) + + } +} + +// This test shows we can use ECDSA Secp256k1 keys for libp2p and expect PeerID <=> PublicKey bijections +func TestIDTranslationSecp256k1(t *testing.T) { + loops := 50 + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) + + // check that we can extract the public key back + // This makes sense: the compressed serialization of ECDSA Secp256k1 keys is 33 + 2 bytes, + // and libp2p uses multihash.IDENTITY on serializations of less than 42 bytes + _, err := pID.ExtractPublicKey() + require.NoError(t, err) + + } +} + +func TestUnstakedTranslationRoundTrip(t *testing.T) { + loops := 50 + unstakedTranslator := NewUnstakedNetworkIDTranslator() + for i := 0; i < loops; i++ { + pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) + + pk, err := pID.ExtractPublicKey() + require.NoError(t, err) + + // for a secp256k1 key, this is compressed representation preceded by 00 bits + // indicating the multihash.IDENTITY + pkBytes, err := pk.Raw() + require.NoError(t, err) + + // key is positive, roundtrip should be possible + if pkBytes[0] == 0x02 { + flowID, err := unstakedTranslator.GetFlowID(pID) + require.NoError(t, err) + retrievedPeerID, err := unstakedTranslator.GetPeerID(flowID) + require.NoError(t, err) + require.Equal(t, pID, retrievedPeerID) + } + + } +} + +func createPeerIDFromAlgo(t *testing.T, sa fcrypto.SigningAlgorithm) peer.ID { + seed := createSeed(t) + + // this matches GenerateNetworkingKeys, and is intended to validate the choices in cmd/bootstrap + key, err := fcrypto.GeneratePrivateKey(sa, seed) + require.NoError(t, err) + + // get the public key + pubKey := key.PublicKey() + + // extract the corresponding libp2p public Key + libp2pPubKey, err := LibP2PPublicKeyFromFlow(pubKey) + require.NoError(t, err) + + // obtain the PeerID based on libp2p's own rules + pID, err := peer.IDFromPublicKey(libp2pPubKey) + require.NoError(t, err) + + return pID +} + +func createSeed(t *testing.T) []byte { + seedLen := int(math.Max(fcrypto.KeyGenSeedMinLenECDSAP256, fcrypto.KeyGenSeedMinLenECDSASecp256k1)) + seed := make([]byte, seedLen) + n, err := rand.Read(seed) + require.NoError(t, err) + require.Equal(t, n, seedLen) + return seed +} From 066e1889f75f7a7be8163e26aa541bdc9ad02a0f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 18 Aug 2021 13:14:42 -0400 Subject: [PATCH 201/291] [network] make sure TestUnstakedTranslationRoundTrip runs 50 times --- network/p2p/unstaked_translator_test.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/network/p2p/unstaked_translator_test.go b/network/p2p/unstaked_translator_test.go index 46a3b1dd58c..9850cfa775a 100644 --- a/network/p2p/unstaked_translator_test.go +++ b/network/p2p/unstaked_translator_test.go @@ -11,6 +11,9 @@ import ( fcrypto "github.com/onflow/flow-go/crypto" ) +// For these test, refer to https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md for libp2p +// PeerID specifications and how they relate to keys. + // This test shows we can't use ECDSA P-256 keys for libp2p and expect PeerID <=> PublicKey bijections func TestIDTranslationP256(t *testing.T) { loops := 50 @@ -42,21 +45,25 @@ func TestIDTranslationSecp256k1(t *testing.T) { } func TestUnstakedTranslationRoundTrip(t *testing.T) { - loops := 50 + max_iterations := 50 unstakedTranslator := NewUnstakedNetworkIDTranslator() - for i := 0; i < loops; i++ { + + tested_vectors := 0 + + for ok := true; ok; ok = tested_vectors < max_iterations { pID := createPeerIDFromAlgo(t, fcrypto.ECDSASecp256k1) pk, err := pID.ExtractPublicKey() require.NoError(t, err) - // for a secp256k1 key, this is compressed representation preceded by 00 bits - // indicating the multihash.IDENTITY + // for a secp256k1 key, this is just the compressed representation pkBytes, err := pk.Raw() require.NoError(t, err) // key is positive, roundtrip should be possible if pkBytes[0] == 0x02 { + tested_vectors++ + flowID, err := unstakedTranslator.GetFlowID(pID) require.NoError(t, err) retrievedPeerID, err := unstakedTranslator.GetPeerID(flowID) From 13507d740ce82b8739f7b90ad2e689fd39a883de Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 18 Aug 2021 11:47:34 -0700 Subject: [PATCH 202/291] TODOs update mocks, fix a test --- .../staked_access_node_builder.go | 5 +- .../unstaked_access_node_builder.go | 2 +- cmd/access/node_builder/upstream_connector.go | 6 +- cmd/node_builder.go | 2 +- cmd/scaffold.go | 2 +- consensus/integration/nodes_test.go | 22 ++- module/id/fixed_provider.go | 17 +++ network/mocknetwork/connector.go | 18 +-- network/mocknetwork/middleware.go | 16 +-- network/mocknetwork/overlay.go | 19 +-- network/p2p/connGater.go | 2 +- network/p2p/dht_test.go | 2 +- network/p2p/libp2pConnector.go | 2 +- network/p2p/libp2pNode.go | 42 +++--- network/p2p/libp2pNode_test.go | 131 ++++++++++-------- network/p2p/middleware.go | 8 +- network/p2p/peerstore_provider.go | 19 ++- network/p2p/protocol_state_provider.go | 35 +++-- network/p2p/unstaked_translator.go | 8 +- 19 files changed, 207 insertions(+), 151 deletions(-) create mode 100644 module/id/fixed_provider.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index b7bb142d921..233cb56819d 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -29,7 +29,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } @@ -46,8 +46,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { ) fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: need special providers here - // NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // TODO: NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN // doesn't participate in unstaked network. // If it does, then we can just use the default one (peerstoreProvider) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index d2f9d8e02d0..f6fd01e63d2 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -25,7 +25,7 @@ func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index aabfa446426..00d4f7a7f3d 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -105,9 +105,13 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f return default: } + peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) if err != nil { - // TODO: return formatted error + resultChan <- result{ + id: flow.Identity{}, + err: err, + } } // try and connect to the bootstrap server diff --git a/cmd/node_builder.go b/cmd/node_builder.go index f9e082556d5..ab027fceda8 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -137,7 +137,7 @@ type NodeConfig struct { StakingKey crypto.PrivateKey NetworkKey crypto.PrivateKey - // TODO: initialize these in scaffold and unstaked node + // ID providers IdentityProvider id.IdentityProvider IDTranslator p2p.IDTranslator NetworkingIdentifierProvider id.IdentifierProvider diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 8689399ba75..0defbdee53d 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -425,7 +425,7 @@ func (fnb *FlowNodeBuilder) initStorage() { func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err } diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index e25254c252e..9320579bce8 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -24,6 +24,7 @@ import ( "github.com/onflow/flow-go/module/buffer" builder "github.com/onflow/flow-go/module/builder/consensus" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -127,7 +128,7 @@ func createNode( setupsDB := storage.NewEpochSetups(metrics, db) commitsDB := storage.NewEpochCommits(metrics, db) statusesDB := storage.NewEpochStatuses(metrics, db) - consumer := events.NewNoop() + consumer := events.NewDistributor() state, err := bprotocol.Bootstrap(metrics, db, headersDB, sealsDB, resultsDB, blocksDB, setupsDB, commitsDB, statusesDB, rootSnapshot) require.NoError(t, err) @@ -230,8 +231,25 @@ func createNode( finalizedHeader, err := synceng.NewFinalizedHeaderCache(log, state, pubsub.NewFinalizationDistributor()) require.NoError(t, err) + identities, err := state.Final().Identities(filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(me.NodeID())), + )) + require.NoError(t, err) + idProvider := id.NewFixedIdentifierProvider(identities.NodeIDs()) + // initialize the synchronization engine - sync, err := synceng.New(log, metrics, net, me, blocksDB, comp, syncCore, finalizedHeader, state) + sync, err := synceng.New( + log, + metrics, + net, + me, + blocksDB, + comp, + syncCore, + finalizedHeader, + idProvider, + ) require.NoError(t, err) pending := []*flow.Header{} diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go new file mode 100644 index 00000000000..dbdf10b9403 --- /dev/null +++ b/module/id/fixed_provider.go @@ -0,0 +1,17 @@ +package id + +import ( + "github.com/onflow/flow-go/model/flow" +) + +type FixedIdentifierProvider struct { + identifiers flow.IdentifierList +} + +func NewFixedIdentifierProvider(identifiers flow.IdentifierList) *FixedIdentifierProvider { + return &FixedIdentifierProvider{identifiers} +} + +func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { + return p.identifiers +} diff --git a/network/mocknetwork/connector.go b/network/mocknetwork/connector.go index 02f12add1b5..3f32fad44f6 100644 --- a/network/mocknetwork/connector.go +++ b/network/mocknetwork/connector.go @@ -5,8 +5,9 @@ package mocknetwork import ( context "context" - flow "github.com/onflow/flow-go/model/flow" mock "github.com/stretchr/testify/mock" + + peer "github.com/libp2p/go-libp2p-core/peer" ) // Connector is an autogenerated mock type for the Connector type @@ -14,16 +15,7 @@ type Connector struct { mock.Mock } -// UpdatePeers provides a mock function with given fields: ctx, ids -func (_m *Connector) UpdatePeers(ctx context.Context, ids flow.IdentityList) error { - ret := _m.Called(ctx, ids) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, flow.IdentityList) error); ok { - r0 = rf(ctx, ids) - } else { - r0 = ret.Error(0) - } - - return r0 +// UpdatePeers provides a mock function with given fields: ctx, peerIDs +func (_m *Connector) UpdatePeers(ctx context.Context, peerIDs peer.IDSlice) { + _m.Called(ctx, peerIDs) } diff --git a/network/mocknetwork/middleware.go b/network/mocknetwork/middleware.go index 232990ff4cf..8d0a57067f6 100644 --- a/network/mocknetwork/middleware.go +++ b/network/mocknetwork/middleware.go @@ -122,15 +122,11 @@ func (_m *Middleware) Unsubscribe(channel network.Channel) error { } // UpdateAllowList provides a mock function with given fields: -func (_m *Middleware) UpdateAllowList() error { - ret := _m.Called() - - var r0 error - if rf, ok := ret.Get(0).(func() error); ok { - r0 = rf() - } else { - r0 = ret.Error(0) - } +func (_m *Middleware) UpdateAllowList() { + _m.Called() +} - return r0 +// UpdateNodeAddresses provides a mock function with given fields: +func (_m *Middleware) UpdateNodeAddresses() { + _m.Called() } diff --git a/network/mocknetwork/overlay.go b/network/mocknetwork/overlay.go index c096adf32b1..7c98d0b8fe5 100644 --- a/network/mocknetwork/overlay.go +++ b/network/mocknetwork/overlay.go @@ -14,27 +14,20 @@ type Overlay struct { mock.Mock } -// Identity provides a mock function with given fields: -func (_m *Overlay) Identity() (map[flow.Identifier]flow.Identity, error) { +// Identities provides a mock function with given fields: +func (_m *Overlay) Identities() flow.IdentityList { ret := _m.Called() - var r0 map[flow.Identifier]flow.Identity - if rf, ok := ret.Get(0).(func() map[flow.Identifier]flow.Identity); ok { + var r0 flow.IdentityList + if rf, ok := ret.Get(0).(func() flow.IdentityList); ok { r0 = rf() } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(map[flow.Identifier]flow.Identity) + r0 = ret.Get(0).(flow.IdentityList) } } - var r1 error - if rf, ok := ret.Get(1).(func() error); ok { - r1 = rf() - } else { - r1 = ret.Error(1) - } - - return r0, r1 + return r0 } // Receive provides a mock function with given fields: nodeID, msg diff --git a/network/p2p/connGater.go b/network/p2p/connGater.go index caa05bb1f1a..64ebe02eed0 100644 --- a/network/p2p/connGater.go +++ b/network/p2p/connGater.go @@ -29,7 +29,7 @@ func NewConnGater(log zerolog.Logger) *ConnGater { } // update updates the peer ID map -func (c *ConnGater) update(pids []peer.ID) { +func (c *ConnGater) update(pids peer.IDSlice) { // create a new peer.ID map peerIDs := make(map[peer.ID]struct{}, len(pids)) diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index 8d2134f0591..dc300338ac4 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -180,7 +180,7 @@ func (suite *DHTTestSuite) CreateNodes(count int, dhtServer bool) (nodes []*Node n, err := NewDefaultLibP2PNodeBuilder(flow.Identifier{}, "0.0.0.0:0", key). SetRootBlockID(rootBlockID). SetConnectionManager(connManager). - SetPubsubOptions(WithDHTDiscovery(AsServer(dhtServer))). + SetDHTOptions(AsServer(dhtServer)). SetPingInfoProvider(pingInfoProvider). SetLogger(logger). Build(suite.ctx) diff --git a/network/p2p/libp2pConnector.go b/network/p2p/libp2pConnector.go index 64e68a43170..d9e16853aec 100644 --- a/network/p2p/libp2pConnector.go +++ b/network/p2p/libp2pConnector.go @@ -94,7 +94,7 @@ func (l *libp2pConnector) connectToPeers(ctx context.Context, peerIDs peer.IDSli // trimAllConnectionsExcept trims all connections of the node from peers not part of peerIDs. // A node would have created such extra connections earlier when the identity list may have been different, or // it may have been target of such connections from node which have now been excluded. -func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs []peer.ID) { +func (l *libp2pConnector) trimAllConnectionsExcept(peerIDs peer.IDSlice) { // convert the peerInfos to a peer.ID -> bool map peersToKeep := make(map[peer.ID]bool, len(peerIDs)) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 5fd1c3d04bb..e18efe9f394 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -46,6 +46,10 @@ const ( // maximum number of milliseconds to wait between attempts for a 1-1 direct connection maxConnectAttemptSleepDuration = 5 + + // timeout for FindPeer queries to the DHT + // TODO: is this a sensible value? + findPeerQueryTimeout = 15 * time.Second ) // LibP2PFactoryFunc is a factory function type for generating libp2p Node instances. @@ -330,8 +334,8 @@ func (n *Node) Stop() (chan struct{}, error) { } // AddPeer adds a peer to this node by adding it to this node's peerstore and connecting to it -func (n *Node) AddPeer(ctx context.Context, peerID peer.ID) error { - err := n.host.Connect(ctx, peer.AddrInfo{ID: peerID}) +func (n *Node) AddPeer(ctx context.Context, peerInfo peer.AddrInfo) error { + err := n.host.Connect(ctx, peerInfo) if err != nil { return err } @@ -350,6 +354,21 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { // CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { + if len(n.host.Peerstore().Addrs(peerID)) == 0 { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") + if n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + // try to find the peer using the dht + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + } + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + } + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) if err != nil { @@ -384,21 +403,6 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp // immediately without backing off and fail-fast. // Hence, explicitly cancel the dial back off (if any) and try connecting again - if len(n.host.Peerstore().Addrs(peerID)) == 0 { - // TODO: add bunch of logging here - if n.dht != nil { - // TODO: adjust timeout - timedCtx, cancel := context.WithTimeout(ctx, 15*time.Second) - // try to find the peer - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) - } - } - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) - } - // cancel the dial back off (if any), since we want to connect immediately network := n.host.Network() if swm, ok := network.(*swarm.Swarm); ok { @@ -413,7 +417,7 @@ func (n *Node) tryCreateNewStream(ctx context.Context, peerID peer.ID, maxAttemp time.Sleep(time.Duration(r) * time.Millisecond) } - err := n.AddPeer(ctx, peerID) + err := n.AddPeer(ctx, peer.AddrInfo{ID: peerID}) if err != nil { // if the connection was rejected due to invalid node id, skip the re-attempt @@ -561,7 +565,7 @@ func (n *Node) Ping(ctx context.Context, peerID peer.ID) (message.PingResponse, } // UpdateAllowList allows the peer allow list to be updated. -func (n *Node) UpdateAllowList(peers []peer.ID) { +func (n *Node) UpdateAllowList(peers peer.IDSlice) { if n.connGater == nil { n.logger.Debug().Hex("node_id", logging.ID(n.id)).Msg("skipping update allow list, connection gating is not enabled") return diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index 6e78e040c40..da31fdf9e41 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -15,6 +15,7 @@ import ( golog "github.com/ipfs/go-log" addrutil "github.com/libp2p/go-addr-util" "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" swarm "github.com/libp2p/go-libp2p-swarm" "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr/net" @@ -141,22 +142,13 @@ func (suite *LibP2PNodeTestSuite) TestAddPeers() { // add the remaining nodes to the first node as its set of peers for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, *identity)) + peerInfo, err := PeerAddressInfo(*identity) + require.NoError(suite.T(), err) + require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, peerInfo)) } // Checks if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Peerstore().Peers(), count) - - // Checks whether the first node is connected to the rest - for _, peer := range nodes[0].host.Peerstore().Peers() { - // A node is also a peer to itself but not marked as connected, hence skip checking that. - if nodes[0].host.ID().String() == peer.String() { - continue - } - assert.Eventuallyf(suite.T(), func() bool { - return network.Connected == nodes[0].host.Network().Connectedness(peer) - }, 2*time.Second, tickForAssertEventually, fmt.Sprintf(" first node is not connected to %s", peer.String())) - } + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) } // TestAddPeers checks if nodes can be added as peers to a given node @@ -166,39 +158,27 @@ func (suite *LibP2PNodeTestSuite) TestRemovePeers() { // create nodes nodes, identities := suite.NodesFixture(count, nil, false) + peerInfos, errs := peerInfosFromIDs(identities) + assert.Len(suite.T(), errs, 0) defer StopNodes(suite.T(), nodes) // add nodes two and three to the first node as its peers - for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, *identity)) + for _, pInfo := range peerInfos[1:] { + require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, pInfo)) } // check if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Peerstore().Peers(), count) - - // check whether the first node is connected to the rest - for _, peer := range nodes[0].host.Peerstore().Peers() { - // A node is also a peer to itself but not marked as connected, hence skip checking that. - if nodes[0].host.ID().String() == peer.String() { - continue - } - assert.Eventually(suite.T(), func() bool { - return network.Connected == nodes[0].host.Network().Connectedness(peer) - }, 2*time.Second, tickForAssertEventually) - } + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) // disconnect from each peer and assert that the connection no longer exists - for _, identity := range identities[1:] { - require.NoError(suite.T(), nodes[0].RemovePeer(suite.ctx, *identity)) - pInfo, err := PeerAddressInfo(*identity) - assert.NoError(suite.T(), err) + for _, pInfo := range peerInfos[1:] { + require.NoError(suite.T(), nodes[0].RemovePeer(suite.ctx, pInfo.ID)) assert.Equal(suite.T(), network.NotConnected, nodes[0].host.Network().Connectedness(pInfo.ID)) } } // TestCreateStreams checks if a new streams is created each time when CreateStream is called and an existing stream is not reused func (suite *LibP2PNodeTestSuite) TestCreateStream() { - count := 2 // Creates nodes @@ -214,7 +194,10 @@ func (suite *LibP2PNodeTestSuite) TestCreateStream() { // Now attempt to create another 100 outbound stream to the same destination by calling CreateStream var streams []network.Stream for i := 0; i < 100; i++ { - anotherStream, err := nodes[0].CreateStream(context.Background(), *id2) + pInfo, err := PeerAddressInfo(*id2) + require.NoError(suite.T(), err) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo)) + anotherStream, err := nodes[0].CreateStream(context.Background(), pInfo.ID) // Assert that a stream was returned without error require.NoError(suite.T(), err) require.NotNil(suite.T(), anotherStream) @@ -256,12 +239,13 @@ func (suite *LibP2PNodeTestSuite) TestNoBackoffWhenCreatingStream() { defer StopNode(suite.T(), node1) id2 := identities[1] - + pInfo, err := PeerAddressInfo(*id2) + require.NoError(suite.T(), err) + nodes[0].host.Peerstore().AddAddrs(pInfo.ID, pInfo.Addrs, peerstore.AddressTTL) maxTimeToWait := maxConnectAttempt * maxConnectAttemptSleepDuration * time.Millisecond - var err error unittest.RequireReturnsBefore(suite.T(), func() { - _, err = node1.CreateStream(context.Background(), *id2) + _, err = node1.CreateStream(context.Background(), pInfo.ID) }, maxTimeToWait, fmt.Sprintf("create stream did not error within %d ms", maxTimeToWait)) require.Error(suite.T(), err) @@ -292,9 +276,14 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { id1 := *identities[0] id2 := *identities[1] + pInfo1, err := PeerAddressInfo(id1) + require.NoError(suite.T(), err) + pInfo2, err := PeerAddressInfo(id2) + require.NoError(suite.T(), err) // Create stream from node 1 to node 2 - s1, err := nodes[0].CreateStream(context.Background(), id2) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo2)) + s1, err := nodes[0].CreateStream(context.Background(), pInfo2.ID) assert.NoError(suite.T(), err) rw := bufio.NewReadWriter(bufio.NewReader(s1), bufio.NewWriter(s1)) @@ -315,7 +304,8 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } // Create stream from node 2 to node 1 - s2, err := nodes[1].CreateStream(context.Background(), id1) + require.NoError(suite.T(), nodes[1].AddPeer(context.Background(), pInfo1)) + s2, err := nodes[1].CreateStream(context.Background(), pInfo1.ID) assert.NoError(suite.T(), err) rw = bufio.NewReadWriter(bufio.NewReader(s2), bufio.NewWriter(s2)) @@ -350,16 +340,19 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() require.NoError(suite.T(), listener.Close()) }() + silentNodeInfo, err := PeerAddressInfo(silentNodeId) + require.NoError(suite.T(), err) + // setup the context to expire after the default timeout ctx, cancel := context.WithTimeout(context.Background(), DefaultUnicastTimeout) defer cancel() // attempt to create a stream from node 1 to node 2 and assert that it fails after timeout grace := 1 * time.Second - var err error unittest.AssertReturnsBefore(suite.T(), func() { - _, err = nodes[0].CreateStream(ctx, silentNodeId) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), silentNodeInfo)) + _, err = nodes[0].CreateStream(ctx, silentNodeInfo.ID) }, DefaultUnicastTimeout+grace) assert.Error(suite.T(), err) @@ -372,17 +365,22 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { goodNodes, goodNodeIds := suite.NodesFixture(2, nil, false) defer StopNodes(suite.T(), goodNodes) require.Len(suite.T(), goodNodeIds, 2) + goodNodeInfo1, err := PeerAddressInfo(*goodNodeIds[1]) + require.NoError(suite.T(), err) // create a silent node which never replies listener, silentNodeId := silentNodeFixture(suite.T()) defer func() { require.NoError(suite.T(), listener.Close()) }() + silentNodeInfo, err := PeerAddressInfo(silentNodeId) + require.NoError(suite.T(), err) // creates a stream to unresponsive node and makes sure that the stream creation is blocked blockedCallCh := unittest.RequireNeverReturnBefore(suite.T(), func() { - _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeId) // this call will block + require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), silentNodeInfo)) + _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeInfo.ID) // this call will block }, 1*time.Second, "CreateStream attempt to the unresponsive peer did not block") @@ -390,7 +388,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // requires same peer can still connect to the other regular peer without being blocked unittest.RequireReturnsBefore(suite.T(), func() { - _, err := goodNodes[0].CreateStream(suite.ctx, *goodNodeIds[1]) + require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), goodNodeInfo1)) + _, err := goodNodes[0].CreateStream(suite.ctx, goodNodeInfo1.ID) require.NoError(suite.T(), err) }, 1*time.Second, "creating stream to a responsive node failed while concurrently blocked on unresponsive node") @@ -409,6 +408,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { nodes, identities := suite.NodesFixture(2, nil, false) defer StopNodes(suite.T(), nodes) require.Len(suite.T(), identities, 2) + nodeInfo1, err := PeerAddressInfo(*identities[1]) + require.NoError(suite.T(), err) wg := sync.WaitGroup{} @@ -417,7 +418,8 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { createStream := func() { <-gate - _, err := nodes[0].CreateStream(suite.ctx, *identities[1]) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + _, err := nodes[0].CreateStream(suite.ctx, nodeInfo1.ID) assert.NoError(suite.T(), err) // assert that stream was successfully created wg.Done() } @@ -477,10 +479,13 @@ func (suite *LibP2PNodeTestSuite) TestStreamClosing() { // Creates nodes nodes, identities := suite.NodesFixture(2, handler, false) defer StopNodes(suite.T(), nodes) + nodeInfo1, err := PeerAddressInfo(*identities[1]) + require.NoError(suite.T(), err) for i := 0; i < count; i++ { // Create stream from node 1 to node 2 (reuse if one already exists) - s, err := nodes[0].CreateStream(context.Background(), *identities[1]) + require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + s, err := nodes[0].CreateStream(context.Background(), nodeInfo1.ID) assert.NoError(suite.T(), err) w := bufio.NewWriter(s) @@ -537,7 +542,9 @@ func (suite *LibP2PNodeTestSuite) TestPing() { func testPing(t *testing.T, source *Node, target flow.Identity, expectedVersion string, expectedHeight uint64) { pctx, cancel := context.WithCancel(context.Background()) defer cancel() - resp, rtt, err := source.Ping(pctx, target) + pid, err := ExtractPeerID(target.NetworkPubKey) + assert.NoError(t, err) + resp, rtt, err := source.Ping(pctx, pid) assert.NoError(t, err) assert.NotZero(t, rtt) assert.Equal(t, expectedVersion, resp.Version) @@ -553,10 +560,14 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { node1 := nodes[0] node1Id := *identities[0] defer StopNode(suite.T(), node1) + node1Info, err := PeerAddressInfo(node1Id) + assert.NoError(suite.T(), err) node2 := nodes[1] node2Id := *identities[1] defer StopNode(suite.T(), node2) + node2Info, err := PeerAddressInfo(node2Id) + assert.NoError(suite.T(), err) requireError := func(err error) { require.Error(suite.T(), err) @@ -565,38 +576,40 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { suite.Run("outbound connection to a not-allowed node is rejected", func() { // node1 and node2 both have no allowListed peers - _, err := node1.CreateStream(suite.ctx, node2Id) + requireError(node1.AddPeer(context.Background(), node2Info)) + _, err := node1.CreateStream(suite.ctx, node2Info.ID) requireError(err) - _, err = node2.CreateStream(suite.ctx, node1Id) + requireError(node2.AddPeer(context.Background(), node1Info)) + _, err = node2.CreateStream(suite.ctx, node1Info.ID) requireError(err) }) suite.Run("inbound connection from an allowed node is rejected", func() { // node1 allowlists node2 but node2 does not allowlists node1 - err := node1.UpdateAllowList(flow.IdentityList{&node2Id}) - require.NoError(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node2Info.ID}) // node1 attempts to connect to node2 // node2 should reject the inbound connection - _, err = node1.CreateStream(suite.ctx, node2Id) + require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.Error(suite.T(), err) }) suite.Run("outbound connection to an approved node is allowed", func() { // node1 allowlists node2 - err := node1.UpdateAllowList(flow.IdentityList{&node2Id}) - require.NoError(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node2Info.ID}) // node2 allowlists node1 - err = node2.UpdateAllowList(flow.IdentityList{&node1Id}) - require.NoError(suite.T(), err) + node2.UpdateAllowList(peer.IDSlice{node1Info.ID}) // node1 should be allowed to connect to node2 - _, err = node1.CreateStream(suite.ctx, node2Id) + require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.NoError(suite.T(), err) // node2 should be allowed to connect to node1 - _, err = node2.CreateStream(suite.ctx, node1Id) + require.NoError(suite.T(), node2.AddPeer(context.Background(), node1Info)) + _, err = node2.CreateStream(suite.ctx, node1Info.ID) require.NoError(suite.T(), err) }) } @@ -607,12 +620,12 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGatingBootstrap() { node1 := node[0] node1Id := identity[0] defer StopNode(suite.T(), node1) + node1Info, err := PeerAddressInfo(*node1Id) + assert.NoError(suite.T(), err) suite.Run("updating allowlist of node w/o ConnGater does not crash", func() { - // node1 allowlists node1 - err := node1.UpdateAllowList(flow.IdentityList{node1Id}) - require.Error(suite.T(), err) + node1.UpdateAllowList(peer.IDSlice{node1Info.ID}) }) } diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index e43d7674c08..8f55761a386 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -161,7 +161,6 @@ func DefaultValidators(log zerolog.Logger, flowID flow.Identifier) []network.Mes func (m *Middleware) topologyPeers() (peer.IDSlice, error) { identities, err := m.ov.Topology() if err != nil { - // TODO: format error return nil, err } @@ -178,7 +177,10 @@ func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { for _, fid := range flowIDs { pid, err := m.idTranslator.GetPeerID(fid) if err != nil { - // TODO: log here + // We probably don't need to fail the entire function here, since the other + // translations may still succeed + m.log.Err(err).Str("flowID", fid.String()).Msg("failed to translate to peer ID") + continue } result = append(result, pid) @@ -212,7 +214,7 @@ func (m *Middleware) Start(ov network.Overlay) error { libP2PNode, err := m.libP2PNodeFactory() if m.idProvider == nil { - m.idProvider = NewPeerstoreIdentifierProvider(libP2PNode.host, m.idTranslator) + m.idProvider = NewPeerstoreIdentifierProvider(m.log, libP2PNode.host, m.idTranslator) } if err != nil { diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go index cea16d466a9..c31b2cf6917 100644 --- a/network/p2p/peerstore_provider.go +++ b/network/p2p/peerstore_provider.go @@ -2,6 +2,7 @@ package p2p import ( "github.com/libp2p/go-libp2p-core/host" + "github.com/rs/zerolog" "github.com/onflow/flow-go/model/flow" ) @@ -9,23 +10,29 @@ import ( type PeerstoreIdentifierProvider struct { host host.Host idTranslator IDTranslator + logger zerolog.Logger } -func NewPeerstoreIdentifierProvider(host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { - return &PeerstoreIdentifierProvider{host: host, idTranslator: idTranslator} +func NewPeerstoreIdentifierProvider(logger zerolog.Logger, host host.Host, idTranslator IDTranslator) *PeerstoreIdentifierProvider { + return &PeerstoreIdentifierProvider{ + logger: logger.With().Str("component", "peerstore-id-provider").Logger(), + host: host, + idTranslator: idTranslator, + } } func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { var result flow.IdentifierList - pids := p.host.Peerstore().PeersWithAddrs() // TODO: should we just call Peers here? + pids := p.host.Peerstore().PeersWithAddrs() for _, pid := range pids { flowID, err := p.idTranslator.GetFlowID(pid) if err != nil { - // TODO: log error - } else { - result = append(result, flowID) + p.logger.Err(err).Str("peerID", pid.Pretty()).Msg("failed to translate to Flow ID") + continue } + + result = append(result, flowID) } return result diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index d47542c3b18..9290a860e06 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -5,6 +5,7 @@ import ( "sync" "github.com/libp2p/go-libp2p-core/peer" + "github.com/rs/zerolog" "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" @@ -18,21 +19,24 @@ type ProtocolStateIDCache struct { identities flow.IdentityList state protocol.State mu sync.RWMutex - peerIDs map[flow.Identifier]peer.ID // TODO: need to initialize these in constructor!!! + peerIDs map[flow.Identifier]peer.ID flowIDs map[peer.ID]flow.Identifier + logger zerolog.Logger } func NewProtocolStateIDCache( + logger zerolog.Logger, state protocol.State, eventDistributer *events.Distributor, ) (*ProtocolStateIDCache, error) { provider := &ProtocolStateIDCache{ - state: state, + state: state, + logger: logger.With().Str("component", "protocol-state-id-cache").Logger(), } head, err := state.Final().Head() if err != nil { - return nil, err // TODO: format the error + return nil, fmt.Errorf("failed to get latest state header: %w", err) } provider.update(head.ID()) @@ -41,24 +45,28 @@ func NewProtocolStateIDCache( return provider, nil } -func (p *ProtocolStateIDCache) EpochTransition(_ uint64, header *flow.Header) { - // TODO: maybe we actually want to log the epoch information from the arguments here (epoch phase, etc) +func (p *ProtocolStateIDCache) EpochTransition(newEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("newEpochCounter", newEpochCounter).Msg("epoch transition") p.update(header.ID()) } -func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(_ uint64, header *flow.Header) { +func (p *ProtocolStateIDCache) EpochSetupPhaseStarted(currentEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("currentEpochCounter", currentEpochCounter).Msg("epoch setup phase started") p.update(header.ID()) } -func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(_ uint64, header *flow.Header) { +func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(currentEpochCounter uint64, header *flow.Header) { + p.logger.Info().Uint64("currentEpochCounter", currentEpochCounter).Msg("epoch committed phase started") p.update(header.ID()) } func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { - // TODO: log status here + p.logger.Info().Str("blockID", blockID.String()).Msg("updating cached identities") + identities, err := p.state.AtBlockID(blockID).Identities(filter.Any) if err != nil { - // TODO: log fatal. Reasoning here is, we don't want to continue with an expired list. + // We don't want to continue with an expired identity list. + p.logger.Fatal().Err(err).Msg("failed to fetch new identities") } nIds := identities.Count() @@ -67,9 +75,12 @@ func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { flowIDs := make(map[peer.ID]flow.Identifier, nIds) for _, identity := range identities { + p.logger.Debug().Interface("identity", identity).Msg("extracting peer ID from network key") + pid, err := ExtractPeerID(identity.NetworkPubKey) if err != nil { - // maybe don't log fatal here. It's probably okay if we miss some ppl in our mapping + p.logger.Err(err).Interface("identity", identity).Msg("failed to extract peer ID from network key") + continue } flowIDs[pid] = identity.NodeID @@ -116,13 +127,13 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { pk, err := PublicKey(networkPubKey) if err != nil { - // TODO: format the error + err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) return } pid, err = peer.IDFromPublicKey(pk) if err != nil { - // TODO: format the error + err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) return } diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index a3211c1fc68..62f5bde9668 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -20,7 +20,7 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID data := append([]byte{0x02}, flowID[:]...) mh, err := multihash.Sum(data, multihash.IDENTITY, -1) if err != nil { - // TODO: return error + return "", fmt.Errorf("failed to compute multihash: %w", err) } return peer.ID(mh), nil @@ -29,16 +29,16 @@ func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { pk, err := peerID.ExtractPublicKey() if err != nil { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not extractible from PeerID", peerID) } if pk.Type() != crypto_pb.KeyType_Secp256k1 { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is not a %v key", peerID, crypto_pb.KeyType_name[(int32)(crypto_pb.KeyType_Secp256k1)]) } data, err := pk.Raw() if err != nil || data[0] != 0x02 { - return flow.ZeroID, fmt.Errorf("Cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) + return flow.ZeroID, fmt.Errorf("cannot generate an unstaked FlowID for peerID %v: corresponding libp2p key is invalid or negative", peerID) } return flow.HashToID(data[1:]), nil From 22eb78e204e9f8b58c1d820b68c56362f7c7f0ac Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 13:01:56 -0700 Subject: [PATCH 203/291] Update peerManager_test.go Update sporking_test.go Update protocol_state_provider.go --- network/p2p/peerManager_test.go | 42 +++++++++++++++++++++----- network/p2p/protocol_state_provider.go | 2 +- network/p2p/sporking_test.go | 30 +++++++++++------- 3 files changed, 54 insertions(+), 20 deletions(-) diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 65cfa184c71..571a4a8108b 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -7,9 +7,11 @@ import ( "time" "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/model/flow" @@ -41,8 +43,14 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { currentIDs := unittest.IdentityListFixture(10) // setup a ID provider callback to return currentIDs - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } // track IDs that should be disconnected @@ -105,8 +113,14 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } connector := new(mocknetwork.Connector) @@ -137,8 +151,14 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } // chooses peer interval rate deliberately long to capture on demand peer update @@ -180,8 +200,14 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { currentIDs := unittest.IdentityListFixture(10) - idProvider := func() (flow.IdentityList, error) { - return currentIDs, nil + idProvider := func() (peer.IDSlice, error) { + pids := peer.IDSlice{} + for _, id := range currentIDs { + pid, err := ExtractPeerID(id.NetworkPubKey) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + return pids, nil } connector := new(mocknetwork.Connector) diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 9290a860e06..6f223a4d0fe 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -125,7 +125,7 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e } func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { - pk, err := PublicKey(networkPubKey) + pk, err := LibP2PPublicKeyFromFlow(networkPubKey) if err != nil { err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) return diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 935ac173a54..429f522bb0f 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -6,6 +6,7 @@ import ( "time" golog "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" @@ -13,7 +14,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/utils/unittest" ) @@ -50,9 +50,11 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { // create and start node 2 on localhost and random port node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootBlockID, nil, false, defaultAddress) + peerInfo2, err := PeerAddressInfo(id2) + require.NoError(suite.T(), err) // create stream from node 1 to node 2 - testOneToOneMessagingSucceeds(suite.T(), node1, id2) + testOneToOneMessagingSucceeds(suite.T(), node1, peerInfo2) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain @@ -71,7 +73,7 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { // attempt to create a stream from node 1 (old chain) to node 2 (new chain) // this time it should fail since node 2 is using a different public key // (and therefore has a different libp2p node id) - testOneToOneMessagingFails(suite.T(), node1, id2) + testOneToOneMessagingFails(suite.T(), node1, peerInfo2) } // TestOneToOneCrosstalkPrevention tests that a node from the old chain cannot talk directly to a node in the new chain @@ -85,13 +87,15 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { node1key := generateNetworkingKey(suite.T()) node1, id1 := NodeFixture(suite.T(), suite.logger, node1key, rootID1, nil, false, defaultAddress) defer StopNode(suite.T(), node1) + peerInfo1, err := PeerAddressInfo(id1) + require.NoError(suite.T(), err) // create and start node 2 on localhost and random port node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootID1, nil, false, defaultAddress) // create stream from node 2 to node 1 - testOneToOneMessagingSucceeds(suite.T(), node2, id1) + testOneToOneMessagingSucceeds(suite.T(), node2, peerInfo1) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain @@ -110,7 +114,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { // attempt to create a stream from node 2 (new chain) to node 1 (old chain) // this time it should fail since node 2 is listening on a different protocol - testOneToOneMessagingFails(suite.T(), node2, id1) + testOneToOneMessagingFails(suite.T(), node2, peerInfo1) } // TestOneToKCrosstalkPrevention tests that a node from the old chain cannot talk to a node in the new chain via PubSub @@ -128,7 +132,9 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { // create and start node 2 on localhost and random port with the same root block ID node2key := generateNetworkingKey(suite.T()) node2, id2 := NodeFixture(suite.T(), suite.logger, node2key, rootIDBeforeSpork, nil, false, defaultAddress) + pInfo2, err := PeerAddressInfo(id2) defer StopNode(suite.T(), node2) + require.NoError(suite.T(), err) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -137,13 +143,13 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { topicBeforeSpork := engine.TopicFromChannel(engine.TestNetwork, rootIDBeforeSpork) // both nodes are initially on the same spork and subscribed to the same topic - _, err := node1.Subscribe(ctx, topicBeforeSpork) + _, err = node1.Subscribe(ctx, topicBeforeSpork) require.NoError(suite.T(), err) sub2, err := node2.Subscribe(ctx, topicBeforeSpork) require.NoError(suite.T(), err) // add node 2 as a peer of node 1 - err = node1.AddPeer(ctx, id2) + err = node1.AddPeer(ctx, pInfo2) require.NoError(suite.T(), err) // let the two nodes form the mesh @@ -170,17 +176,19 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { testOneToKMessagingFails(ctx, suite.T(), node1, sub2, topicAfterSpork) } -func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, targetId flow.Identity) { +func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { + require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from node 1 to node 2 - s, err := sourceNode.CreateStream(context.Background(), targetId) + s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation succeeded require.NoError(t, err) assert.NotNil(t, s) } -func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, targetId flow.Identity) { +func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { + require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from source node to destination address - _, err := sourceNode.CreateStream(context.Background(), targetId) + _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation failed assert.Error(t, err) // assert that it failed with the expected error From 95844b94ef513df731a8872017d8f7d71d5cbb6b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 19:29:35 -0700 Subject: [PATCH 204/291] Add new DHT test --- module/mocks/network.go | 63 ++++++------ network/p2p/dht.go | 2 +- network/p2p/dht_test.go | 46 +++++++++ network/p2p/libp2pNode.go | 23 +++-- storage/mocks/storage.go | 207 +++++++++++++++++++------------------- 5 files changed, 195 insertions(+), 146 deletions(-) diff --git a/module/mocks/network.go b/module/mocks/network.go index 38f65a552ce..b680ae44c89 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -5,38 +5,39 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" network "github.com/onflow/flow-go/network" - reflect "reflect" ) -// MockNetwork is a mock of Network interface +// MockNetwork is a mock of Network interface. type MockNetwork struct { ctrl *gomock.Controller recorder *MockNetworkMockRecorder } -// MockNetworkMockRecorder is the mock recorder for MockNetwork +// MockNetworkMockRecorder is the mock recorder for MockNetwork. type MockNetworkMockRecorder struct { mock *MockNetwork } -// NewMockNetwork creates a new mock instance +// NewMockNetwork creates a new mock instance. func NewMockNetwork(ctrl *gomock.Controller) *MockNetwork { mock := &MockNetwork{ctrl: ctrl} mock.recorder = &MockNetworkMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockNetwork) EXPECT() *MockNetworkMockRecorder { return m.recorder } -// Register mocks base method +// Register mocks base method. func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (network.Conduit, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Register", arg0, arg1) @@ -45,36 +46,36 @@ func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (netwo return ret0, ret1 } -// Register indicates an expected call of Register +// Register indicates an expected call of Register. func (mr *MockNetworkMockRecorder) Register(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Register", reflect.TypeOf((*MockNetwork)(nil).Register), arg0, arg1) } -// MockLocal is a mock of Local interface +// MockLocal is a mock of Local interface. type MockLocal struct { ctrl *gomock.Controller recorder *MockLocalMockRecorder } -// MockLocalMockRecorder is the mock recorder for MockLocal +// MockLocalMockRecorder is the mock recorder for MockLocal. type MockLocalMockRecorder struct { mock *MockLocal } -// NewMockLocal creates a new mock instance +// NewMockLocal creates a new mock instance. func NewMockLocal(ctrl *gomock.Controller) *MockLocal { mock := &MockLocal{ctrl: ctrl} mock.recorder = &MockLocalMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockLocal) EXPECT() *MockLocalMockRecorder { return m.recorder } -// Address mocks base method +// Address mocks base method. func (m *MockLocal) Address() string { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Address") @@ -82,13 +83,13 @@ func (m *MockLocal) Address() string { return ret0 } -// Address indicates an expected call of Address +// Address indicates an expected call of Address. func (mr *MockLocalMockRecorder) Address() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Address", reflect.TypeOf((*MockLocal)(nil).Address)) } -// NodeID mocks base method +// NodeID mocks base method. func (m *MockLocal) NodeID() flow.Identifier { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NodeID") @@ -96,13 +97,13 @@ func (m *MockLocal) NodeID() flow.Identifier { return ret0 } -// NodeID indicates an expected call of NodeID +// NodeID indicates an expected call of NodeID. func (mr *MockLocalMockRecorder) NodeID() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NodeID", reflect.TypeOf((*MockLocal)(nil).NodeID)) } -// NotMeFilter mocks base method +// NotMeFilter mocks base method. func (m *MockLocal) NotMeFilter() flow.IdentityFilter { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NotMeFilter") @@ -110,13 +111,13 @@ func (m *MockLocal) NotMeFilter() flow.IdentityFilter { return ret0 } -// NotMeFilter indicates an expected call of NotMeFilter +// NotMeFilter indicates an expected call of NotMeFilter. func (mr *MockLocalMockRecorder) NotMeFilter() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NotMeFilter", reflect.TypeOf((*MockLocal)(nil).NotMeFilter)) } -// Sign mocks base method +// Sign mocks base method. func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Sign", arg0, arg1) @@ -125,13 +126,13 @@ func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error return ret0, ret1 } -// Sign indicates an expected call of Sign +// Sign indicates an expected call of Sign. func (mr *MockLocalMockRecorder) Sign(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sign", reflect.TypeOf((*MockLocal)(nil).Sign), arg0, arg1) } -// SignFunc mocks base method +// SignFunc mocks base method. func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.PrivateKey, []byte, hash.Hasher) (crypto.Signature, error)) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SignFunc", arg0, arg1, arg2) @@ -140,66 +141,66 @@ func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.Pri return ret0, ret1 } -// SignFunc indicates an expected call of SignFunc +// SignFunc indicates an expected call of SignFunc. func (mr *MockLocalMockRecorder) SignFunc(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignFunc", reflect.TypeOf((*MockLocal)(nil).SignFunc), arg0, arg1, arg2) } -// MockRequester is a mock of Requester interface +// MockRequester is a mock of Requester interface. type MockRequester struct { ctrl *gomock.Controller recorder *MockRequesterMockRecorder } -// MockRequesterMockRecorder is the mock recorder for MockRequester +// MockRequesterMockRecorder is the mock recorder for MockRequester. type MockRequesterMockRecorder struct { mock *MockRequester } -// NewMockRequester creates a new mock instance +// NewMockRequester creates a new mock instance. func NewMockRequester(ctrl *gomock.Controller) *MockRequester { mock := &MockRequester{ctrl: ctrl} mock.recorder = &MockRequesterMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockRequester) EXPECT() *MockRequesterMockRecorder { return m.recorder } -// EntityByID mocks base method +// EntityByID mocks base method. func (m *MockRequester) EntityByID(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "EntityByID", arg0, arg1) } -// EntityByID indicates an expected call of EntityByID +// EntityByID indicates an expected call of EntityByID. func (mr *MockRequesterMockRecorder) EntityByID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EntityByID", reflect.TypeOf((*MockRequester)(nil).EntityByID), arg0, arg1) } -// Force mocks base method +// Force mocks base method. func (m *MockRequester) Force() { m.ctrl.T.Helper() m.ctrl.Call(m, "Force") } -// Force indicates an expected call of Force +// Force indicates an expected call of Force. func (mr *MockRequesterMockRecorder) Force() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Force", reflect.TypeOf((*MockRequester)(nil).Force)) } -// Query mocks base method +// Query mocks base method. func (m *MockRequester) Query(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Query", arg0, arg1) } -// Query indicates an expected call of Query +// Query indicates an expected call of Query. func (mr *MockRequesterMockRecorder) Query(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockRequester)(nil).Query), arg0, arg1) diff --git a/network/p2p/dht.go b/network/p2p/dht.go index b21211a9adb..a5b82657cb9 100644 --- a/network/p2p/dht.go +++ b/network/p2p/dht.go @@ -39,7 +39,7 @@ func AsServer(enable bool) dht.Option { if enable { return dht.Mode(dht.ModeServer) } - return dht.Mode(dht.ModeAuto) + return dht.Mode(dht.ModeClient) } func WithBootstrapPeers(bootstrapNodes flow.IdentityList) (dht.Option, error) { diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index dc300338ac4..cc01eedaafb 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -43,6 +43,52 @@ func (suite *DHTTestSuite) TearDownTest() { suite.cancel() } +func (suite *DHTTestSuite) TestFindPeerWithDHT() { + count := 10 + golog.SetAllLoggers(golog.LevelFatal) // change this to Debug if libp2p logs are needed + + dhtServerNodes := suite.CreateNodes(2, true) + require.Len(suite.T(), dhtServerNodes, 2) + + dhtClientNodes := suite.CreateNodes(count-2, false) + + nodes := append(dhtServerNodes, dhtClientNodes...) + defer suite.StopNodes(nodes) + + getDhtServerAddr := func(i uint) peer.AddrInfo { + return peer.AddrInfo{ID: dhtServerNodes[i].host.ID(), Addrs: dhtServerNodes[i].host.Addrs()} + } + + for i, clientNode := range dhtClientNodes { + err := clientNode.host.Connect(suite.ctx, getDhtServerAddr(uint(i%2))) + require.NoError(suite.T(), err) + } + + require.Eventually(suite.T(), func() bool { + for i, clientNode := range dhtClientNodes { + if clientNode.dht.RoutingTable().Find(getDhtServerAddr(uint(i%2)).ID) == "" { + return false + } + } + return true + }, time.Second*5, tickForAssertEventually, "nodes failed to connect") + + err := dhtServerNodes[0].host.Connect(suite.ctx, getDhtServerAddr(1)) + require.NoError(suite.T(), err) + + require.Eventually(suite.T(), func() bool { + return dhtServerNodes[0].dht.RoutingTable().Find(getDhtServerAddr(1).ID) != "" + }, time.Second*5, tickForAssertEventually, "dht servers failed to connect") + + for i := 0; i < len(dhtClientNodes); i += 2 { + for j := 1; j < len(dhtClientNodes); j += 2 { + dhtClientNodes[i].host.Peerstore().ClearAddrs(dhtClientNodes[j].host.ID()) + _, err = dhtClientNodes[i].CreateStream(suite.ctx, dhtClientNodes[j].host.ID()) + require.NoError(suite.T(), err) + } + } +} + // TestPubSub checks if nodes can subscribe to a topic and send and receive a message on that topic. The DHT discovery // mechanism is used for nodes to find each other. func (suite *DHTTestSuite) TestPubSubWithDHTDiscovery() { diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index e18efe9f394..80dc1eae413 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -356,18 +356,19 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { if len(n.host.Peerstore().Addrs(peerID)) == 0 { n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - if n.dht != nil { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - // try to find the peer using the dht - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) - } - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + if n.dht == nil { + return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + } + + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + // try to find the peer using the dht + _, err := n.dht.FindPeer(timedCtx, peerID) + cancel() + if err != nil { + return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) } - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 2967a3e5cce..7a829429fc4 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -5,37 +5,38 @@ package mocks import ( + reflect "reflect" + gomock "github.com/golang/mock/gomock" flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" - reflect "reflect" ) -// MockBlocks is a mock of Blocks interface +// MockBlocks is a mock of Blocks interface. type MockBlocks struct { ctrl *gomock.Controller recorder *MockBlocksMockRecorder } -// MockBlocksMockRecorder is the mock recorder for MockBlocks +// MockBlocksMockRecorder is the mock recorder for MockBlocks. type MockBlocksMockRecorder struct { mock *MockBlocks } -// NewMockBlocks creates a new mock instance +// NewMockBlocks creates a new mock instance. func NewMockBlocks(ctrl *gomock.Controller) *MockBlocks { mock := &MockBlocks{ctrl: ctrl} mock.recorder = &MockBlocksMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockBlocks) EXPECT() *MockBlocksMockRecorder { return m.recorder } -// ByCollectionID mocks base method +// ByCollectionID mocks base method. func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByCollectionID", arg0) @@ -44,13 +45,13 @@ func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByCollectionID indicates an expected call of ByCollectionID +// ByCollectionID indicates an expected call of ByCollectionID. func (mr *MockBlocksMockRecorder) ByCollectionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByCollectionID", reflect.TypeOf((*MockBlocks)(nil).ByCollectionID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -59,13 +60,13 @@ func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockBlocksMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockBlocks)(nil).ByHeight), arg0) } -// ByID mocks base method +// ByID mocks base method. func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -74,13 +75,13 @@ func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockBlocksMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockBlocks)(nil).ByID), arg0) } -// GetLastFullBlockHeight mocks base method +// GetLastFullBlockHeight mocks base method. func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetLastFullBlockHeight") @@ -89,13 +90,13 @@ func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { return ret0, ret1 } -// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight +// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight. func (mr *MockBlocksMockRecorder) GetLastFullBlockHeight() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).GetLastFullBlockHeight)) } -// IndexBlockForCollections mocks base method +// IndexBlockForCollections mocks base method. func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexBlockForCollections", arg0, arg1) @@ -103,13 +104,13 @@ func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow. return ret0 } -// IndexBlockForCollections indicates an expected call of IndexBlockForCollections +// IndexBlockForCollections indicates an expected call of IndexBlockForCollections. func (mr *MockBlocksMockRecorder) IndexBlockForCollections(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexBlockForCollections", reflect.TypeOf((*MockBlocks)(nil).IndexBlockForCollections), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockBlocks) Store(arg0 *flow.Block) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -117,13 +118,13 @@ func (m *MockBlocks) Store(arg0 *flow.Block) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockBlocksMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockBlocks)(nil).Store), arg0) } -// StoreTx mocks base method +// StoreTx mocks base method. func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreTx", arg0) @@ -131,13 +132,13 @@ func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { return ret0 } -// StoreTx indicates an expected call of StoreTx +// StoreTx indicates an expected call of StoreTx. func (mr *MockBlocksMockRecorder) StoreTx(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreTx", reflect.TypeOf((*MockBlocks)(nil).StoreTx), arg0) } -// UpdateLastFullBlockHeight mocks base method +// UpdateLastFullBlockHeight mocks base method. func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "UpdateLastFullBlockHeight", arg0) @@ -145,36 +146,36 @@ func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { return ret0 } -// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight +// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight. func (mr *MockBlocksMockRecorder) UpdateLastFullBlockHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).UpdateLastFullBlockHeight), arg0) } -// MockHeaders is a mock of Headers interface +// MockHeaders is a mock of Headers interface. type MockHeaders struct { ctrl *gomock.Controller recorder *MockHeadersMockRecorder } -// MockHeadersMockRecorder is the mock recorder for MockHeaders +// MockHeadersMockRecorder is the mock recorder for MockHeaders. type MockHeadersMockRecorder struct { mock *MockHeaders } -// NewMockHeaders creates a new mock instance +// NewMockHeaders creates a new mock instance. func NewMockHeaders(ctrl *gomock.Controller) *MockHeaders { mock := &MockHeaders{ctrl: ctrl} mock.recorder = &MockHeadersMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockHeaders) EXPECT() *MockHeadersMockRecorder { return m.recorder } -// BatchIndexByChunkID mocks base method +// BatchIndexByChunkID mocks base method. func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchIndexByChunkID", arg0, arg1, arg2) @@ -182,13 +183,13 @@ func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 stora return ret0 } -// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID +// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID. func (mr *MockHeadersMockRecorder) BatchIndexByChunkID(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchIndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).BatchIndexByChunkID), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -197,13 +198,13 @@ func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockHeadersMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockHeaders)(nil).ByBlockID), arg0) } -// ByHeight mocks base method +// ByHeight mocks base method. func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -212,13 +213,13 @@ func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight +// ByHeight indicates an expected call of ByHeight. func (mr *MockHeadersMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockHeaders)(nil).ByHeight), arg0) } -// ByParentID mocks base method +// ByParentID mocks base method. func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByParentID", arg0) @@ -227,13 +228,13 @@ func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { return ret0, ret1 } -// ByParentID indicates an expected call of ByParentID +// ByParentID indicates an expected call of ByParentID. func (mr *MockHeadersMockRecorder) ByParentID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByParentID", reflect.TypeOf((*MockHeaders)(nil).ByParentID), arg0) } -// IDByChunkID mocks base method +// IDByChunkID mocks base method. func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IDByChunkID", arg0) @@ -242,13 +243,13 @@ func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) return ret0, ret1 } -// IDByChunkID indicates an expected call of IDByChunkID +// IDByChunkID indicates an expected call of IDByChunkID. func (mr *MockHeadersMockRecorder) IDByChunkID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDByChunkID", reflect.TypeOf((*MockHeaders)(nil).IDByChunkID), arg0) } -// IndexByChunkID mocks base method +// IndexByChunkID mocks base method. func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexByChunkID", arg0, arg1) @@ -256,13 +257,13 @@ func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { return ret0 } -// IndexByChunkID indicates an expected call of IndexByChunkID +// IndexByChunkID indicates an expected call of IndexByChunkID. func (mr *MockHeadersMockRecorder) IndexByChunkID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).IndexByChunkID), arg0, arg1) } -// Store mocks base method +// Store mocks base method. func (m *MockHeaders) Store(arg0 *flow.Header) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -270,36 +271,36 @@ func (m *MockHeaders) Store(arg0 *flow.Header) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockHeadersMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockHeaders)(nil).Store), arg0) } -// MockPayloads is a mock of Payloads interface +// MockPayloads is a mock of Payloads interface. type MockPayloads struct { ctrl *gomock.Controller recorder *MockPayloadsMockRecorder } -// MockPayloadsMockRecorder is the mock recorder for MockPayloads +// MockPayloadsMockRecorder is the mock recorder for MockPayloads. type MockPayloadsMockRecorder struct { mock *MockPayloads } -// NewMockPayloads creates a new mock instance +// NewMockPayloads creates a new mock instance. func NewMockPayloads(ctrl *gomock.Controller) *MockPayloads { mock := &MockPayloads{ctrl: ctrl} mock.recorder = &MockPayloadsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockPayloads) EXPECT() *MockPayloadsMockRecorder { return m.recorder } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -308,13 +309,13 @@ func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockPayloadsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockPayloads)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -322,36 +323,36 @@ func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockPayloadsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockPayloads)(nil).Store), arg0, arg1) } -// MockCollections is a mock of Collections interface +// MockCollections is a mock of Collections interface. type MockCollections struct { ctrl *gomock.Controller recorder *MockCollectionsMockRecorder } -// MockCollectionsMockRecorder is the mock recorder for MockCollections +// MockCollectionsMockRecorder is the mock recorder for MockCollections. type MockCollectionsMockRecorder struct { mock *MockCollections } -// NewMockCollections creates a new mock instance +// NewMockCollections creates a new mock instance. func NewMockCollections(ctrl *gomock.Controller) *MockCollections { mock := &MockCollections{ctrl: ctrl} mock.recorder = &MockCollectionsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCollections) EXPECT() *MockCollectionsMockRecorder { return m.recorder } -// ByID mocks base method +// ByID mocks base method. func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -360,13 +361,13 @@ func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID +// ByID indicates an expected call of ByID. func (mr *MockCollectionsMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockCollections)(nil).ByID), arg0) } -// LightByID mocks base method +// LightByID mocks base method. func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByID", arg0) @@ -375,13 +376,13 @@ func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection return ret0, ret1 } -// LightByID indicates an expected call of LightByID +// LightByID indicates an expected call of LightByID. func (mr *MockCollectionsMockRecorder) LightByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByID", reflect.TypeOf((*MockCollections)(nil).LightByID), arg0) } -// LightByTransactionID mocks base method +// LightByTransactionID mocks base method. func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByTransactionID", arg0) @@ -390,13 +391,13 @@ func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.Ligh return ret0, ret1 } -// LightByTransactionID indicates an expected call of LightByTransactionID +// LightByTransactionID indicates an expected call of LightByTransactionID. func (mr *MockCollectionsMockRecorder) LightByTransactionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByTransactionID", reflect.TypeOf((*MockCollections)(nil).LightByTransactionID), arg0) } -// Remove mocks base method +// Remove mocks base method. func (m *MockCollections) Remove(arg0 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Remove", arg0) @@ -404,13 +405,13 @@ func (m *MockCollections) Remove(arg0 flow.Identifier) error { return ret0 } -// Remove indicates an expected call of Remove +// Remove indicates an expected call of Remove. func (mr *MockCollectionsMockRecorder) Remove(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockCollections)(nil).Remove), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCollections) Store(arg0 *flow.Collection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -418,13 +419,13 @@ func (m *MockCollections) Store(arg0 *flow.Collection) error { return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCollectionsMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCollections)(nil).Store), arg0) } -// StoreLight mocks base method +// StoreLight mocks base method. func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLight", arg0) @@ -432,13 +433,13 @@ func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { return ret0 } -// StoreLight indicates an expected call of StoreLight +// StoreLight indicates an expected call of StoreLight. func (mr *MockCollectionsMockRecorder) StoreLight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLight", reflect.TypeOf((*MockCollections)(nil).StoreLight), arg0) } -// StoreLightAndIndexByTransaction mocks base method +// StoreLightAndIndexByTransaction mocks base method. func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLightAndIndexByTransaction", arg0) @@ -446,36 +447,36 @@ func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollec return ret0 } -// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction +// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction. func (mr *MockCollectionsMockRecorder) StoreLightAndIndexByTransaction(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLightAndIndexByTransaction", reflect.TypeOf((*MockCollections)(nil).StoreLightAndIndexByTransaction), arg0) } -// MockCommits is a mock of Commits interface +// MockCommits is a mock of Commits interface. type MockCommits struct { ctrl *gomock.Controller recorder *MockCommitsMockRecorder } -// MockCommitsMockRecorder is the mock recorder for MockCommits +// MockCommitsMockRecorder is the mock recorder for MockCommits. type MockCommitsMockRecorder struct { mock *MockCommits } -// NewMockCommits creates a new mock instance +// NewMockCommits creates a new mock instance. func NewMockCommits(ctrl *gomock.Controller) *MockCommits { mock := &MockCommits{ctrl: ctrl} mock.recorder = &MockCommitsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -483,13 +484,13 @@ func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockCommitsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockCommits)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -498,13 +499,13 @@ func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, err return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockCommitsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockCommits)(nil).ByBlockID), arg0) } -// Store mocks base method +// Store mocks base method. func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -512,36 +513,36 @@ func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) err return ret0 } -// Store indicates an expected call of Store +// Store indicates an expected call of Store. func (mr *MockCommitsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCommits)(nil).Store), arg0, arg1) } -// MockEvents is a mock of Events interface +// MockEvents is a mock of Events interface. type MockEvents struct { ctrl *gomock.Controller recorder *MockEventsMockRecorder } -// MockEventsMockRecorder is the mock recorder for MockEvents +// MockEventsMockRecorder is the mock recorder for MockEvents. type MockEventsMockRecorder struct { mock *MockEvents } -// NewMockEvents creates a new mock instance +// NewMockEvents creates a new mock instance. func NewMockEvents(ctrl *gomock.Controller) *MockEvents { mock := &MockEvents{ctrl: ctrl} mock.recorder = &MockEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockEvents) EXPECT() *MockEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -549,13 +550,13 @@ func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, ar return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -564,13 +565,13 @@ func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockEvents)(nil).ByBlockID), arg0) } -// ByBlockIDEventType mocks base method +// ByBlockIDEventType mocks base method. func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventType) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDEventType", arg0, arg1) @@ -579,13 +580,13 @@ func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventTyp return ret0, ret1 } -// ByBlockIDEventType indicates an expected call of ByBlockIDEventType +// ByBlockIDEventType indicates an expected call of ByBlockIDEventType. func (mr *MockEventsMockRecorder) ByBlockIDEventType(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDEventType", reflect.TypeOf((*MockEvents)(nil).ByBlockIDEventType), arg0, arg1) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -594,36 +595,36 @@ func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow. return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockEventsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockEvents)(nil).ByBlockIDTransactionID), arg0, arg1) } -// MockServiceEvents is a mock of ServiceEvents interface +// MockServiceEvents is a mock of ServiceEvents interface. type MockServiceEvents struct { ctrl *gomock.Controller recorder *MockServiceEventsMockRecorder } -// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents +// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents. type MockServiceEventsMockRecorder struct { mock *MockServiceEvents } -// NewMockServiceEvents creates a new mock instance +// NewMockServiceEvents creates a new mock instance. func NewMockServiceEvents(ctrl *gomock.Controller) *MockServiceEvents { mock := &MockServiceEvents{ctrl: ctrl} mock.recorder = &MockServiceEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -631,13 +632,13 @@ func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockServiceEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockServiceEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method +// ByBlockID mocks base method. func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -646,36 +647,36 @@ func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID +// ByBlockID indicates an expected call of ByBlockID. func (mr *MockServiceEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockServiceEvents)(nil).ByBlockID), arg0) } -// MockTransactionResults is a mock of TransactionResults interface +// MockTransactionResults is a mock of TransactionResults interface. type MockTransactionResults struct { ctrl *gomock.Controller recorder *MockTransactionResultsMockRecorder } -// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults +// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults. type MockTransactionResultsMockRecorder struct { mock *MockTransactionResults } -// NewMockTransactionResults creates a new mock instance +// NewMockTransactionResults creates a new mock instance. func NewMockTransactionResults(ctrl *gomock.Controller) *MockTransactionResults { mock := &MockTransactionResults{ctrl: ctrl} mock.recorder = &MockTransactionResultsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use +// EXPECT returns an object that allows the caller to indicate expected use. func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } -// BatchStore mocks base method +// BatchStore mocks base method. func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -683,13 +684,13 @@ func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.Tr return ret0 } -// BatchStore indicates an expected call of BatchStore +// BatchStore indicates an expected call of BatchStore. func (mr *MockTransactionResultsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockTransactionResults)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockIDTransactionID mocks base method +// ByBlockIDTransactionID mocks base method. func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) (*flow.TransactionResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -698,7 +699,7 @@ func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifi return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. func (mr *MockTransactionResultsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockTransactionResults)(nil).ByBlockIDTransactionID), arg0, arg1) From 0b4a6af6d8843f2f48f0c31cc94d41185c3925d1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 19:44:20 -0700 Subject: [PATCH 205/291] fix test --- engine/common/synchronization/engine_test.go | 17 ++++++++++++++--- engine/testutil/nodes.go | 11 ++++++++++- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 093fa9cc2a0..92d488c787c 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -18,12 +18,15 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" synccore "github.com/onflow/flow-go/module/synchronization" netint "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/mocknetwork" + "github.com/onflow/flow-go/network/p2p" protocolint "github.com/onflow/flow-go/state/protocol" + protocolEvents "github.com/onflow/flow-go/state/protocol/events" protocol "github.com/onflow/flow-go/state/protocol/mock" storerr "github.com/onflow/flow-go/storage" storage "github.com/onflow/flow-go/storage/mock" @@ -161,7 +164,15 @@ func (ss *SyncSuite) SetupTest() { finalizedHeader, err := NewFinalizedHeaderCache(log, ss.state, pubsub.NewFinalizationDistributor()) require.NoError(ss.T(), err, "could not create finalized snapshot cache") - e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, ss.state) + idCache, err := p2p.NewProtocolStateIDCache(log, ss.state, protocolEvents.NewDistributor()) + e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(ss.me.NodeID())), + ), + idCache, + )) require.NoError(ss.T(), err, "should pass engine initialization") ss.e = e @@ -415,7 +426,7 @@ func (ss *SyncSuite) TestSendRequests() { ss.core.On("BatchRequested", batches[0]) // exclude my node ID - ss.e.sendRequests(ss.participants[1:], ranges, batches) + ss.e.sendRequests(ss.participants[1:].NodeIDs(), ranges, batches) ss.con.AssertExpectations(ss.T()) } @@ -472,6 +483,6 @@ func (ss *SyncSuite) TestOnFinalizedBlock() { err := ss.e.finalizedHeader.updateHeader() require.NoError(ss.T(), err) actualHeader := ss.e.finalizedHeader.Get() - require.ElementsMatch(ss.T(), ss.e.getParticipants(actualHeader.ID()), ss.participants[1:]) + require.ElementsMatch(ss.T(), ss.e.participantsProvider.Identifiers(), ss.participants[1:].NodeIDs()) require.Equal(ss.T(), actualHeader, &finalizedBlock) } diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index eedd08a0f8a..3688e2c6727 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -53,6 +53,7 @@ import ( "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" @@ -64,6 +65,7 @@ import ( chainsync "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/module/validation" + "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/stub" "github.com/onflow/flow-go/state/protocol" badgerstate "github.com/onflow/flow-go/state/protocol/badger" @@ -625,6 +627,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit finalizedHeader, err := synchronization.NewFinalizedHeaderCache(node.Log, node.State, finalizationDistributor) require.NoError(t, err) + idCache, err := p2p.NewProtocolStateIDCache(node.Log, node.State, events.NewDistributor()) syncEngine, err := synchronization.New( node.Log, node.Metrics, @@ -634,7 +637,13 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit followerEng, syncCore, finalizedHeader, - node.State, + id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ), synchronization.WithPollInterval(time.Duration(0)), ) require.NoError(t, err) From f84624161ebfc02c83e9861c045c4b6ebb66228b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 19 Aug 2021 20:03:22 -0700 Subject: [PATCH 206/291] Update testUtil.go --- network/test/testUtil.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index c0f19542bb1..760d591b0c9 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -143,7 +143,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultPeerUpdateInterval, p2p.DefaultUnicastTimeout, true, - true) + true, + ) } return mws } From 3640594e1a076283ea03f92b4b743263a0d30070 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Fri, 20 Aug 2021 12:51:30 -0400 Subject: [PATCH 207/291] [network] Create a simple FixedTableIdentityTranslator, fix compilation --- network/test/testUtil.go | 47 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 760d591b0c9..9317742ff60 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -90,6 +90,49 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics } } +type FixedTableIdentityTranslator struct { + flow2p2p map[flow.Identifier]peer.ID + p2p2flow map[peer.ID]flow.Identifier +} + +func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + nodeID, ok := t.p2p2flow[p] + if !ok { + return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) + } + return nodeID, nil +} + +func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + peerID, ok := t.flow2p2p[n] + if !ok { + return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) + } + return peerID, nil +} + +func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableIdentityTranslator { + flow2p2p := make(map[flow.Identifier]peer.ID) + p2p2flow := make(map[peer.ID]flow.Identifier) + + for identity := range identities { + nodeID := identity.ID() + networkKey := identity.NetworkPublicKey + peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + if err != nil { + panic("could not interpret a network public key from Flow, test identities setup problem") + } + peerID, err := peer.IDFromPublicKey(peerPK) + if err != nil { + panic("could not generate a PeerID from public Key, test identities setup problem") + } + + flow2p2p[nodeID] = peerID + p2p2flow[peerID] = nodeID + } + return &FixedTableIdentityTranslator{flow2p2p, p2p2flow} +} + // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. @@ -134,6 +177,9 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } + // create a fixed id translator for the identities + tableTranslator := NewFixedTableIdentityTranslator(identities) + // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, factory, @@ -144,6 +190,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultUnicastTimeout, true, true, + tableTranslator, ) } return mws From 65f2852771cffc1dc93ef7ff8f89bf232508bd55 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 10:13:31 -0700 Subject: [PATCH 208/291] Update libp2pNode_test.go Update middleware_test.go Update testUtil.go fixed identity provider Update testUtil.go [network] Fix the compilation of a bunch of remaining tests --- cmd/access/node_builder/upstream_connector.go | 5 +- module/id/fixed_provider.go | 12 +++++ network/p2p/dht_test.go | 2 + network/p2p/libp2pNode.go | 15 ++++-- network/p2p/libp2pNode_test.go | 54 ++++++++++--------- network/p2p/sporking_test.go | 5 +- network/test/middleware_test.go | 3 +- network/test/testUtil.go | 20 ++++--- 8 files changed, 74 insertions(+), 42 deletions(-) diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index 00d4f7a7f3d..fdc608cfadb 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -106,7 +106,8 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f default: } - peerID, err := p2p.ExtractPeerID(bootstrapPeer.NetworkPubKey) + peerAddrInfo, err := p2p.PeerAddressInfo(bootstrapPeer) + if err != nil { resultChan <- result{ id: flow.Identity{}, @@ -115,7 +116,7 @@ func (connector *upstreamConnector) connect(ctx context.Context, bootstrapPeer f } // try and connect to the bootstrap server - err = connector.unstakedNode.AddPeer(ctx, peerID) + err = connector.unstakedNode.AddPeer(ctx, peerAddrInfo) resultChan <- result{ id: bootstrapPeer, err: err, diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index dbdf10b9403..106047510b6 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -15,3 +15,15 @@ func NewFixedIdentifierProvider(identifiers flow.IdentifierList) *FixedIdentifie func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { return p.identifiers } + +type FixedIdentityProvider struct { + identities flow.IdentityList +} + +func NewFixedIdentityProvider(identities flow.IdentityList) *FixedIdentityProvider { + return &FixedIdentityProvider{identities} +} + +func (p *FixedIdentityProvider) Identities(flow.IdentityFilter) flow.IdentityList { + return p.identities +} diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index cc01eedaafb..6643a7652b0 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -29,6 +29,8 @@ type DHTTestSuite struct { cancel context.CancelFunc // used to cancel the context } +// TODO: test that dht findPeer times out + // TestDHTTestSuite test the libp2p pubsub with DHT for discovery func TestDHTTestSuite(t *testing.T) { suite.Run(t, new(DHTTestSuite)) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 80dc1eae413..683252d47e1 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -49,7 +49,7 @@ const ( // timeout for FindPeer queries to the DHT // TODO: is this a sensible value? - findPeerQueryTimeout = 15 * time.Second + findPeerQueryTimeout = 10 * time.Second ) // LibP2PFactoryFunc is a factory function type for generating libp2p Node instances. @@ -361,10 +361,15 @@ func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stre } n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - // try to find the peer using the dht - _, err := n.dht.FindPeer(timedCtx, peerID) - cancel() + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + if err != nil { return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) } diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index da31fdf9e41..8d80914f9d2 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -16,6 +16,7 @@ import ( addrutil "github.com/libp2p/go-addr-util" "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" swarm "github.com/libp2p/go-libp2p-swarm" "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr/net" @@ -147,11 +148,11 @@ func (suite *LibP2PNodeTestSuite) TestAddPeers() { require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, peerInfo)) } - // Checks if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) + // Checks if both of the other nodes have been added as peers to the first node + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count-1) } -// TestAddPeers checks if nodes can be added as peers to a given node +// TestRemovePeers checks if nodes can be removed as peers from a given node func (suite *LibP2PNodeTestSuite) TestRemovePeers() { count := 3 @@ -167,8 +168,8 @@ func (suite *LibP2PNodeTestSuite) TestRemovePeers() { require.NoError(suite.T(), nodes[0].AddPeer(suite.ctx, pInfo)) } - // check if all 3 nodes have been added as peers to the first node - assert.Len(suite.T(), nodes[0].host.Network().Peers(), count) + // check if all other nodes have been added as peers to the first node + assert.Len(suite.T(), nodes[0].host.Network().Peers(), count-1) // disconnect from each peer and assert that the connection no longer exists for _, pInfo := range peerInfos[1:] { @@ -196,7 +197,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStream() { for i := 0; i < 100; i++ { pInfo, err := PeerAddressInfo(*id2) require.NoError(suite.T(), err) - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo)) + nodes[0].host.Peerstore().AddAddrs(pInfo.ID, pInfo.Addrs, peerstore.AddressTTL) anotherStream, err := nodes[0].CreateStream(context.Background(), pInfo.ID) // Assert that a stream was returned without error require.NoError(suite.T(), err) @@ -282,7 +283,7 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { require.NoError(suite.T(), err) // Create stream from node 1 to node 2 - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), pInfo2)) + nodes[0].host.Peerstore().AddAddrs(pInfo2.ID, pInfo2.Addrs, peerstore.AddressTTL) s1, err := nodes[0].CreateStream(context.Background(), pInfo2.ID) assert.NoError(suite.T(), err) rw := bufio.NewReadWriter(bufio.NewReader(s1), bufio.NewWriter(s1)) @@ -304,7 +305,7 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } // Create stream from node 2 to node 1 - require.NoError(suite.T(), nodes[1].AddPeer(context.Background(), pInfo1)) + nodes[1].host.Peerstore().AddAddrs(pInfo1.ID, pInfo1.Addrs, peerstore.AddressTTL) s2, err := nodes[1].CreateStream(context.Background(), pInfo1.ID) assert.NoError(suite.T(), err) rw = bufio.NewReadWriter(bufio.NewReader(s2), bufio.NewWriter(s2)) @@ -325,8 +326,8 @@ func (suite *LibP2PNodeTestSuite) TestOneToOneComm() { } } -// TestCreateStreamTimeoutWithUnresponsiveNode tests that the CreateStream call does not block longer than the default -// unicast timeout interval +// TestCreateStreamTimeoutWithUnresponsiveNode tests that the CreateStream call does not block longer than the +// timeout interval func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() { // creates a regular node @@ -343,18 +344,20 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamTimeoutWithUnresponsiveNode() silentNodeInfo, err := PeerAddressInfo(silentNodeId) require.NoError(suite.T(), err) + timeout := 1 * time.Second + // setup the context to expire after the default timeout - ctx, cancel := context.WithTimeout(context.Background(), DefaultUnicastTimeout) + ctx, cancel := context.WithTimeout(context.Background(), timeout) defer cancel() // attempt to create a stream from node 1 to node 2 and assert that it fails after timeout - grace := 1 * time.Second + grace := 100 * time.Millisecond unittest.AssertReturnsBefore(suite.T(), func() { - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), silentNodeInfo)) + nodes[0].host.Peerstore().AddAddrs(silentNodeInfo.ID, silentNodeInfo.Addrs, peerstore.AddressTTL) _, err = nodes[0].CreateStream(ctx, silentNodeInfo.ID) }, - DefaultUnicastTimeout+grace) + timeout+grace) assert.Error(suite.T(), err) } @@ -379,7 +382,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // creates a stream to unresponsive node and makes sure that the stream creation is blocked blockedCallCh := unittest.RequireNeverReturnBefore(suite.T(), func() { - require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), silentNodeInfo)) + goodNodes[0].host.Peerstore().AddAddrs(silentNodeInfo.ID, silentNodeInfo.Addrs, peerstore.AddressTTL) _, _ = goodNodes[0].CreateStream(suite.ctx, silentNodeInfo.ID) // this call will block }, 1*time.Second, @@ -388,7 +391,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrent() { // requires same peer can still connect to the other regular peer without being blocked unittest.RequireReturnsBefore(suite.T(), func() { - require.NoError(suite.T(), goodNodes[0].AddPeer(context.Background(), goodNodeInfo1)) + goodNodes[0].host.Peerstore().AddAddrs(goodNodeInfo1.ID, goodNodeInfo1.Addrs, peerstore.AddressTTL) _, err := goodNodes[0].CreateStream(suite.ctx, goodNodeInfo1.ID) require.NoError(suite.T(), err) }, @@ -418,7 +421,7 @@ func (suite *LibP2PNodeTestSuite) TestCreateStreamIsConcurrencySafe() { createStream := func() { <-gate - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + nodes[0].host.Peerstore().AddAddrs(nodeInfo1.ID, nodeInfo1.Addrs, peerstore.AddressTTL) _, err := nodes[0].CreateStream(suite.ctx, nodeInfo1.ID) assert.NoError(suite.T(), err) // assert that stream was successfully created wg.Done() @@ -484,7 +487,7 @@ func (suite *LibP2PNodeTestSuite) TestStreamClosing() { for i := 0; i < count; i++ { // Create stream from node 1 to node 2 (reuse if one already exists) - require.NoError(suite.T(), nodes[0].AddPeer(context.Background(), nodeInfo1)) + nodes[0].host.Peerstore().AddAddrs(nodeInfo1.ID, nodeInfo1.Addrs, peerstore.AddressTTL) s, err := nodes[0].CreateStream(context.Background(), nodeInfo1.ID) assert.NoError(suite.T(), err) w := bufio.NewWriter(s) @@ -542,9 +545,10 @@ func (suite *LibP2PNodeTestSuite) TestPing() { func testPing(t *testing.T, source *Node, target flow.Identity, expectedVersion string, expectedHeight uint64) { pctx, cancel := context.WithCancel(context.Background()) defer cancel() - pid, err := ExtractPeerID(target.NetworkPubKey) + pInfo, err := PeerAddressInfo(target) assert.NoError(t, err) - resp, rtt, err := source.Ping(pctx, pid) + source.host.Peerstore().AddAddrs(pInfo.ID, pInfo.Addrs, peerstore.AddressTTL) + resp, rtt, err := source.Ping(pctx, pInfo.ID) assert.NoError(t, err) assert.NotZero(t, rtt) assert.Equal(t, expectedVersion, resp.Version) @@ -576,10 +580,10 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { suite.Run("outbound connection to a not-allowed node is rejected", func() { // node1 and node2 both have no allowListed peers - requireError(node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err := node1.CreateStream(suite.ctx, node2Info.ID) requireError(err) - requireError(node2.AddPeer(context.Background(), node1Info)) + node2.host.Peerstore().AddAddrs(node1Info.ID, node1Info.Addrs, peerstore.AddressTTL) _, err = node2.CreateStream(suite.ctx, node1Info.ID) requireError(err) }) @@ -591,7 +595,7 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { // node1 attempts to connect to node2 // node2 should reject the inbound connection - require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.Error(suite.T(), err) }) @@ -604,11 +608,11 @@ func (suite *LibP2PNodeTestSuite) TestConnectionGating() { node2.UpdateAllowList(peer.IDSlice{node1Info.ID}) // node1 should be allowed to connect to node2 - require.NoError(suite.T(), node1.AddPeer(context.Background(), node2Info)) + node1.host.Peerstore().AddAddrs(node2Info.ID, node2Info.Addrs, peerstore.AddressTTL) _, err = node1.CreateStream(suite.ctx, node2Info.ID) require.NoError(suite.T(), err) // node2 should be allowed to connect to node1 - require.NoError(suite.T(), node2.AddPeer(context.Background(), node1Info)) + node2.host.Peerstore().AddAddrs(node1Info.ID, node1Info.Addrs, peerstore.AddressTTL) _, err = node2.CreateStream(suite.ctx, node1Info.ID) require.NoError(suite.T(), err) }) diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 429f522bb0f..b3462e05ccb 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -7,6 +7,7 @@ import ( golog "github.com/ipfs/go-log" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" @@ -177,8 +178,8 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { } func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { - require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from node 1 to node 2 + sourceNode.host.Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation succeeded require.NoError(t, err) @@ -186,8 +187,8 @@ func testOneToOneMessagingSucceeds(t *testing.T, sourceNode *Node, peerInfo peer } func testOneToOneMessagingFails(t *testing.T, sourceNode *Node, peerInfo peer.AddrInfo) { - require.NoError(t, sourceNode.AddPeer(context.Background(), peerInfo)) // create stream from source node to destination address + sourceNode.host.Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) // assert that stream creation failed assert.Error(t, err) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 1a9d5b3fa16..3ccdc66a058 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -110,8 +110,7 @@ func (m *MiddlewareTestSuite) SetupTest() { } for i, mw := range m.mws { assert.NoError(m.T(), mw.Start(m.ov[i])) - err := mw.UpdateAllowList() - require.NoError(m.T(), err) + mw.UpdateAllowList() } } diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 9317742ff60..a00d0d3451a 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -115,10 +115,10 @@ func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableId flow2p2p := make(map[flow.Identifier]peer.ID) p2p2flow := make(map[peer.ID]flow.Identifier) - for identity := range identities { + for _, identity := range identities { nodeID := identity.ID() - networkKey := identity.NetworkPublicKey - peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + networkKey := identity.NetworkPubKey + peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) if err != nil { panic("could not interpret a network public key from Flow, test identities setup problem") } @@ -233,7 +233,17 @@ func GenerateNetworks(t *testing.T, me.On("Address").Return(ids[i].Address) // create the network - net, err := p2p.NewNetwork(log, cbor.NewCodec(), ids, me, mws[i], csize, tops[i], sms[i], metrics) + net, err := p2p.NewNetwork( + log, + cbor.NewCodec(), + me, + mws[i], + csize, + tops[i], + sms[i], + metrics, + id.NewFixedIdentityProvider(ids), + ) require.NoError(t, err) nets = append(nets, net) @@ -243,8 +253,6 @@ func GenerateNetworks(t *testing.T, if !dryRunMode { for _, net := range nets { <-net.Ready() - err := net.SetIDs(ids) - require.NoError(t, err) } } return nets From 92e13fc016b3b4ec7ec074629823c235fceeb632 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 14:00:53 -0400 Subject: [PATCH 209/291] Last testUtil fixes Update peerManager_test.go fix epoch transition test Update unstaked_translator.go fixed middleware_test Update execution_test.go --- engine/common/synchronization/engine_test.go | 6 + engine/execution/execution_test.go | 68 ++++++++-- network/p2p/hierarchical_translator_test.go | 15 +++ network/p2p/middleware.go | 26 +++- network/p2p/peerManager_test.go | 128 +++++++++++-------- network/p2p/unstaked_translator.go | 15 ++- network/test/epochtransition_test.go | 57 +++------ network/test/middleware_test.go | 8 +- network/test/testUtil.go | 16 +-- 9 files changed, 215 insertions(+), 124 deletions(-) create mode 100644 network/p2p/hierarchical_translator_test.go diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 92d488c787c..4e79938fab7 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -61,6 +61,12 @@ func (ss *SyncSuite) SetupTest() { // generate own ID ss.participants = unittest.IdentityListFixture(3, unittest.WithRole(flow.RoleConsensus)) + keys, err := unittest.NetworkingKeys(len(ss.participants)) + require.NoError(ss.T(), err) + + for i, p := range ss.participants { + p.NetworkPubKey = keys[i].PublicKey() + } ss.myID = ss.participants[0].NodeID // generate a header for the final state diff --git a/engine/execution/execution_test.go b/engine/execution/execution_test.go index c7201df0e15..76d2ae13137 100644 --- a/engine/execution/execution_test.go +++ b/engine/execution/execution_test.go @@ -37,10 +37,24 @@ func TestExecutionFlow(t *testing.T) { chainID := flow.Testnet - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exeID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - verID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) + keys, errs := unittest.NetworkingKeys(4) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exeID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) + verID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[3].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID) @@ -286,12 +300,23 @@ func TestExecutionStateSyncMultipleExecutionNodes(t *testing.T) { chainID := flow.Emulator - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exe1ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - // exe2ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) + keys, errs := unittest.NetworkingKeys(4) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exe1ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exe1ID) + identities[3].NetworkPubKey = keys[3].PublicKey() collectionNode := testutil.GenericNodeFromParticipants(t, hub, colID, identities, chainID) defer collectionNode.Done() @@ -424,11 +449,28 @@ func TestBroadcastToMultipleVerificationNodes(t *testing.T) { chainID := flow.Emulator - colID := unittest.IdentityFixture(unittest.WithRole(flow.RoleCollection)) - conID := unittest.IdentityFixture(unittest.WithRole(flow.RoleConsensus)) - exeID := unittest.IdentityFixture(unittest.WithRole(flow.RoleExecution)) - ver1ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - ver2ID := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) + keys, errs := unittest.NetworkingKeys(5) + assert.NoError(t, errs) + colID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleCollection), + unittest.WithNetworkingKey(keys[0].PublicKey()), + ) + conID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleConsensus), + unittest.WithNetworkingKey(keys[1].PublicKey()), + ) + exeID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleExecution), + unittest.WithNetworkingKey(keys[2].PublicKey()), + ) + ver1ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[3].PublicKey()), + ) + ver2ID := unittest.IdentityFixture( + unittest.WithRole(flow.RoleVerification), + unittest.WithNetworkingKey(keys[4].PublicKey()), + ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, ver1ID, ver2ID) diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go new file mode 100644 index 00000000000..e111ecd0e85 --- /dev/null +++ b/network/p2p/hierarchical_translator_test.go @@ -0,0 +1,15 @@ +package p2p + +import ( + "testing" + + "github.com/stretchr/testify/suite" +) + +type HierarchicalTranslatorTestSuite struct { + suite.Suite +} + +func TestHierarchicalTranslator(t *testing.T) { + suite.Run(t, new(HierarchicalTranslatorTestSuite)) +} diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 8f55761a386..32b3ee16ef5 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -249,7 +249,6 @@ func (m *Middleware) Start(ov network.Overlay) error { // Stop will end the execution of the middleware and wait for it to end. func (m *Middleware) Stop() { - mgr, found := m.peerMgr() if found { // stops peer manager @@ -389,6 +388,31 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { return nil } +// processAuthenticatedMessage processes a message and a source (indicated by its PublicKey) and eventually passes it to the overlay +// In particular, it checks the claim of protocol authorship situated in the message against `originKey` +// The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` +// this requirement is fulfilled by e.g. the output of readConnection and readSubscription +func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { + identities := m.ov.Identities() + + // check the origin of the message corresponds to the one claimed in the OriginID + originID := flow.HashToID(msg.OriginID) + + originIdentity, found := identities.ByNodeID(originID) + if !found { + m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) + return + } else if originIdentity.NetworkPubKey == nil { + m.log.Warn().Msgf("received message with claimed originID %x, wich has no network identifiers at this node, and was dropped", originID) + return + } else if !originIdentity.NetworkPubKey.Equals(originKey) { + m.log.Warn().Msgf("received message claiming to be from nodeID %v with key %x was actually signed by %x and dropped", originID, originIdentity.NetworkPubKey, originKey) + return + } + + m.processMessage(msg) +} + // processMessage processes a message and eventually passes it to the overlay func (m *Middleware) processMessage(msg *message.Message) { diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 571a4a8108b..5c6632fbc76 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -1,6 +1,7 @@ package p2p import ( + "math/rand" "os" "sync" "testing" @@ -15,8 +16,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/utils/unittest" ) @@ -35,33 +34,41 @@ func (suite *PeerManagerTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) } +func (suite *PeerManagerTestSuite) generateIdentities(n int) (flow.IdentityList, peer.IDSlice) { + // create some test ids + currentIDs := unittest.IdentityListFixture(n) + + // setup a ID provider callback to return currentIDs + pids := peer.IDSlice{} + for _, id := range currentIDs { + key, err := generateFlowNetworkingKey(id.NodeID) + require.NoError(suite.T(), err) + id.NetworkPubKey = key.PublicKey() + pid, err := ExtractPeerID(key.PublicKey()) + require.NoError(suite.T(), err) + pids = append(pids, pid) + } + + return currentIDs, pids +} + // TestUpdatePeers tests that updatePeers calls the connector with the expected list of ids to connect and disconnect // from. The tests are cumulative and ordered. func (suite *PeerManagerTestSuite) TestUpdatePeers() { - // create some test ids - currentIDs := unittest.IdentityListFixture(10) + _, pids := suite.generateIdentities(10) - // setup a ID provider callback to return currentIDs + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } - // track IDs that should be disconnected - var extraIDs flow.IdentityList - // create the connector mock to check ids requested for connect and disconnect connector := new(mocknetwork.Connector) - connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("flow.IdentityList")). + connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("peer.IDSlice")). Run(func(args mock.Arguments) { - idArg := args[1].(flow.IdentityList) - assertListsEqual(suite.T(), currentIDs, idArg) + idArg := args[1].(peer.IDSlice) + assertListsEqual(suite.T(), pids, idArg) }). Return(nil) @@ -77,8 +84,8 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // a subsequent call to updatePeers should request a connector.UpdatePeers to existing ids and new ids suite.Run("updatePeers connects to old and new peers", func() { // create a new id - newIDs := unittest.IdentityListFixture(1) - currentIDs = append(currentIDs, newIDs...) + _, newPIDs := suite.generateIdentities(1) + pids = append(pids, newPIDs...) pm.updatePeers() connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 2) @@ -87,8 +94,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // when ids are only excluded, connector.UpdatePeers should be called suite.Run("updatePeers disconnects from extra peers", func() { // delete an id - extraIDs = currentIDs.Sample(1) - currentIDs = currentIDs.Filter(filter.Not(filter.In(extraIDs))) + pids = removeRandomElement(pids) pm.updatePeers() connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 3) @@ -97,12 +103,12 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // addition and deletion of ids should result in a call to connector.UpdatePeers suite.Run("updatePeers connects to new peers and disconnects from extra peers", func() { // remove a couple of ids - extraIDs = currentIDs.Sample(2) - currentIDs = currentIDs.Filter(filter.Not(filter.In(extraIDs))) + pids = removeRandomElement(pids) + pids = removeRandomElement(pids) // add a couple of new ids - newIDs := unittest.IdentityListFixture(2) - currentIDs = append(currentIDs, newIDs...) + _, newPIDs := suite.generateIdentities(2) + pids = append(pids, newPIDs...) pm.updatePeers() @@ -110,16 +116,19 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { }) } +func removeRandomElement(pids peer.IDSlice) peer.IDSlice { + i := rand.Intn(len(pids)) + pids[i] = pids[len(pids)-1] + return pids[:len(pids)-1] +} + // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -150,14 +159,11 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -199,14 +205,11 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { - currentIDs := unittest.IdentityListFixture(10) + // create some test ids + _, pids := suite.generateIdentities(10) + + // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { - pids := peer.IDSlice{} - for _, id := range currentIDs { - pid, err := ExtractPeerID(id.NetworkPubKey) - require.NoError(suite.T(), err) - pids = append(pids, pid) - } return pids, nil } @@ -244,9 +247,32 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { }, 10*time.Second, 100*time.Millisecond) } -// assertListsEqual asserts that two identity list are equal ignoring the order -func assertListsEqual(t *testing.T, list1, list2 flow.IdentityList) { - list1 = list1.Sort(order.ByNodeIDAsc) - list2 = list2.Sort(order.ByNodeIDAsc) - assert.Equal(t, list1, list2) +// assertListsEqual asserts that two peer ID slices are equal ignoring the order +func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { + listsEqual := func() bool { + if len(list1) != len(list2) { + return false + } + + map1 := make(map[peer.ID]int) + map2 := make(map[peer.ID]int) + + for _, e1 := range list1 { + map1[e1]++ + } + for _, e2 := range list2 { + map2[e2]++ + } + + for key, val := range map1 { + if map2[key] != val { + return false + } + } + return true + } + + if !listsEqual() { + assert.Failf(t, "peer ID slices not equal", "list1: %v, list2: %v", list1, list2) + } } diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 62f5bde9668..400d77fd04c 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -3,9 +3,9 @@ package p2p import ( "fmt" + lcrypto "github.com/libp2p/go-libp2p-core/crypto" crypto_pb "github.com/libp2p/go-libp2p-core/crypto/pb" "github.com/libp2p/go-libp2p-core/peer" - "github.com/multiformats/go-multihash" "github.com/onflow/flow-go/model/flow" ) @@ -18,12 +18,19 @@ func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { func (t *UnstakedNetworkIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { data := append([]byte{0x02}, flowID[:]...) - mh, err := multihash.Sum(data, multihash.IDENTITY, -1) + + um := lcrypto.PubKeyUnmarshallers[crypto_pb.KeyType_Secp256k1] + key, err := um(data) + if err != nil { + return "", fmt.Errorf("failed to convert flow ID to libp2p public key: %w", err) + } + + pid, err := peer.IDFromPublicKey(key) if err != nil { - return "", fmt.Errorf("failed to compute multihash: %w", err) + return "", fmt.Errorf("failed to get peer ID from libp2p public key: %w", err) } - return peer.ID(mh), nil + return pid, nil } func (t *UnstakedNetworkIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { diff --git a/network/test/epochtransition_test.go b/network/test/epochtransition_test.go index af1c8ee7001..26de777dfa7 100644 --- a/network/test/epochtransition_test.go +++ b/network/test/epochtransition_test.go @@ -43,11 +43,10 @@ type MutableIdentityTableSuite struct { // testNode encapsulates the node state which includes its identity, middleware, network, // mesh engine and the id refresher type testNode struct { - id *flow.Identity - mw *p2p.Middleware - net *p2p.Network - engine *MeshEngine - idRefresher *p2p.NodeIDRefresher + id *flow.Identity + mw *p2p.Middleware + net *p2p.Network + engine *MeshEngine } // testNodeList encapsulates a list of test node and @@ -106,16 +105,6 @@ func (t *testNodeList) engines() []*MeshEngine { return engs } -func (t *testNodeList) idRefreshers() []*p2p.NodeIDRefresher { - t.RLock() - defer t.RUnlock() - idRefreshers := make([]*p2p.NodeIDRefresher, len(t.nodes)) - for i, node := range t.nodes { - idRefreshers[i] = node.idRefresher - } - return idRefreshers -} - func (t *testNodeList) networks() []*p2p.Network { t.RLock() defer t.RUnlock() @@ -131,6 +120,13 @@ func TestEpochTransitionTestSuite(t *testing.T) { t.Skip(fmt.Sprintf("test is flaky: %v", &MutableIdentityTableSuite{})) } +func (suite *MutableIdentityTableSuite) signalIdentityChanged() { + for _, n := range suite.testNodes.nodes { + n.mw.UpdateNodeAddresses() + n.mw.UpdateAllowList() + } +} + func (suite *MutableIdentityTableSuite) SetupTest() { suite.testNodes = newTestNodeList() suite.removedTestNodes = newTestNodeList() @@ -180,17 +176,13 @@ func (suite *MutableIdentityTableSuite) addNodes(count int) { // create the engines for the new nodes engines := GenerateEngines(suite.T(), nets) - // create the node refreshers - idRefereshers := suite.generateNodeIDRefreshers(nets) - // create the test engines for i := 0; i < count; i++ { node := testNode{ - id: ids[i], - mw: mws[i], - net: nets[i], - engine: engines[i], - idRefresher: idRefereshers[i], + id: ids[i], + mw: mws[i], + net: nets[i], + engine: engines[i], } suite.testNodes.append(node) } @@ -302,13 +294,6 @@ func (suite *MutableIdentityTableSuite) TestNodesAddedAndRemoved() { suite.assertNetworkPrimitives(remainingIDs, remainingEngs, removedIDs, removedEngines) } -// signalIdentityChanged update IDs for all the current set of nodes (simulating an epoch) -func (suite *MutableIdentityTableSuite) signalIdentityChanged() { - for _, r := range suite.testNodes.idRefreshers() { - r.OnIdentityTableChanged() - } -} - // assertConnected checks that the middleware of a node is directly connected // to at least half of the other nodes. func (suite *MutableIdentityTableSuite) assertConnected(mw *p2p.Middleware, ids flow.IdentityList) { @@ -317,7 +302,7 @@ func (suite *MutableIdentityTableSuite) assertConnected(mw *p2p.Middleware, ids require.Eventuallyf(t, func() bool { connections := 0 for _, id := range ids { - connected, err := mw.IsConnected(*id) + connected, err := mw.IsConnected(id.NodeID) require.NoError(t, err) if connected { connections++ @@ -337,7 +322,7 @@ func (suite *MutableIdentityTableSuite) assertDisconnected(mw *p2p.Middleware, i t := suite.T() require.Eventuallyf(t, func() bool { for _, id := range ids { - connected, err := mw.IsConnected(*id) + connected, err := mw.IsConnected(id.NodeID) require.NoError(t, err) if connected { return false @@ -452,11 +437,3 @@ func (suite *MutableIdentityTableSuite) sendMessage(fromID flow.Identifier, return send(event, fromEngine.con, toIDs.NodeIDs()...) } - -func (suite *MutableIdentityTableSuite) generateNodeIDRefreshers(nets []*p2p.Network) []*p2p.NodeIDRefresher { - refreshers := make([]*p2p.NodeIDRefresher, len(nets)) - for i, net := range nets { - refreshers[i] = p2p.NewNodeIDRefresher(suite.logger, suite.state, net.SetIDs) - } - return refreshers -} diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 3ccdc66a058..7f97b9ad9a1 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -65,7 +65,7 @@ type MiddlewareTestSuite struct { } // TestMiddlewareTestSuit runs all the test methods in this test suit -func TestMiddlewareTestSuit(t *testing.T) { +func TestMiddlewareTestSuite(t *testing.T) { suite.Run(t, new(MiddlewareTestSuite)) } @@ -101,11 +101,7 @@ func (m *MiddlewareTestSuite) SetupTest() { overlay := &mocknetwork.Overlay{} m.ov = append(m.ov, overlay) - identifierToID := make(map[flow.Identifier]flow.Identity) - for _, id := range m.ids { - identifierToID[id.NodeID] = *id - } - overlay.On("Identity").Maybe().Return(identifierToID, nil) + overlay.On("Identities").Maybe().Return(flow.IdentityList(m.ids), nil) overlay.On("Topology").Maybe().Return(flow.IdentityList(m.ids), nil) } for i, mw := range m.mws { diff --git a/network/test/testUtil.go b/network/test/testUtil.go index a00d0d3451a..cc27a70008a 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,6 +20,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" @@ -111,21 +112,18 @@ func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, er return peerID, nil } -func NewFixedTableIdentityTranslator(identities flow.IdentityList) *FixedTableIdentityTranslator { +func NewFixedTableIdentityTranslator(t *testing.T, identities flow.IdentityList) *FixedTableIdentityTranslator { flow2p2p := make(map[flow.Identifier]peer.ID) p2p2flow := make(map[peer.ID]flow.Identifier) for _, identity := range identities { - nodeID := identity.ID() + nodeID := identity.NodeID networkKey := identity.NetworkPubKey peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) - if err != nil { - panic("could not interpret a network public key from Flow, test identities setup problem") - } + require.NoError(t, err) + peerID, err := peer.IDFromPublicKey(peerPK) - if err != nil { - panic("could not generate a PeerID from public Key, test identities setup problem") - } + require.NoError(t, err) flow2p2p[nodeID] = peerID p2p2flow[peerID] = nodeID @@ -178,7 +176,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id } // create a fixed id translator for the identities - tableTranslator := NewFixedTableIdentityTranslator(identities) + tableTranslator := NewFixedTableIdentityTranslator(t, identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, From b3ab25051b8f33c71c15fb334355ca4d02939a82 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 16:05:51 -0700 Subject: [PATCH 210/291] add test for hierchical translator --- network/p2p/fixed_translator.go | 53 +++++++++++++++++++++ network/p2p/hierarchical_translator.go | 5 +- network/p2p/hierarchical_translator_test.go | 43 +++++++++++++++++ network/test/testUtil.go | 43 +---------------- 4 files changed, 102 insertions(+), 42 deletions(-) create mode 100644 network/p2p/fixed_translator.go diff --git a/network/p2p/fixed_translator.go b/network/p2p/fixed_translator.go new file mode 100644 index 00000000000..ecfff75fbb1 --- /dev/null +++ b/network/p2p/fixed_translator.go @@ -0,0 +1,53 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" +) + +type FixedTableIdentityTranslator struct { + flow2p2p map[flow.Identifier]peer.ID + p2p2flow map[peer.ID]flow.Identifier +} + +func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + nodeID, ok := t.p2p2flow[p] + if !ok { + return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) + } + return nodeID, nil +} + +func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + peerID, ok := t.flow2p2p[n] + if !ok { + return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) + } + return peerID, nil +} + +func NewFixedTableIdentityTranslator(identities flow.IdentityList) (*FixedTableIdentityTranslator, error) { + flow2p2p := make(map[flow.Identifier]peer.ID) + p2p2flow := make(map[peer.ID]flow.Identifier) + + for _, identity := range identities { + nodeID := identity.NodeID + networkKey := identity.NetworkPubKey + peerPK, err := LibP2PPublicKeyFromFlow(networkKey) + if err != nil { + return nil, err + } + + peerID, err := peer.IDFromPublicKey(peerPK) + if err != nil { + return nil, err + } + + flow2p2p[nodeID] = peerID + p2p2flow[peerID] = nodeID + } + return &FixedTableIdentityTranslator{flow2p2p, p2p2flow}, nil +} diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go index 28dbdb03c31..4e4d0e77cd5 100644 --- a/network/p2p/hierarchical_translator.go +++ b/network/p2p/hierarchical_translator.go @@ -3,6 +3,7 @@ package p2p import ( "fmt" + "github.com/hashicorp/go-multierror" "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/model/flow" @@ -17,13 +18,15 @@ func NewHierarchicalIDTranslator(translators ...IDTranslator) *HierarchicalIDTra } func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, error) { + var errs *multierror.Error for _, translator := range t.translators { pid, err := translator.GetPeerID(flowID) if err == nil { return pid, nil } + errs = multierror.Append(errs, err) } - return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v", flowID) + return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v: %w", flowID, errs) } func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go index e111ecd0e85..6866547f378 100644 --- a/network/p2p/hierarchical_translator_test.go +++ b/network/p2p/hierarchical_translator_test.go @@ -3,13 +3,56 @@ package p2p import ( "testing" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "gotest.tools/assert" ) type HierarchicalTranslatorTestSuite struct { suite.Suite + translator *HierarchicalIDTranslator + ids flow.IdentityList +} + +func (suite *HierarchicalTranslatorTestSuite) SetupTest() { + suite.ids = unittest.IdentityListFixture(2, unittest.WithKeys) + t1, err := NewFixedTableIdentityTranslator(suite.ids[:1]) + require.NoError(suite.T(), err) + t2, err := NewFixedTableIdentityTranslator(suite.ids[1:]) + require.NoError(suite.T(), err) + + suite.translator = NewHierarchicalIDTranslator(t1, t2) } func TestHierarchicalTranslator(t *testing.T) { suite.Run(t, new(HierarchicalTranslatorTestSuite)) } + +func (suite *HierarchicalTranslatorTestSuite) TestFirstTranslatorSuccess() { + pid, err := suite.translator.GetPeerID(suite.ids[0].NodeID) + require.NoError(suite.T(), err) + fid, err := suite.translator.GetFlowID(pid) + assert.Equal(suite.T(), fid, suite.ids[0].NodeID) +} + +func (suite *HierarchicalTranslatorTestSuite) TestSecondTranslatorSuccess() { + pid, err := suite.translator.GetPeerID(suite.ids[1].NodeID) + require.NoError(suite.T(), err) + fid, err := suite.translator.GetFlowID(pid) + assert.Equal(suite.T(), fid, suite.ids[1].NodeID) +} + +func (suite *HierarchicalTranslatorTestSuite) TestTranslationFailure() { + fid := unittest.IdentifierFixture() + _, err := suite.translator.GetPeerID(fid) + require.Error(suite.T(), err) + + key, err := LibP2PPrivKeyFromFlow(generateNetworkingKey(suite.T())) + require.NoError(suite.T(), err) + pid, err := peer.IDFromPrivateKey(key) + _, err = suite.translator.GetFlowID(pid) + require.Error(suite.T(), err) +} diff --git a/network/test/testUtil.go b/network/test/testUtil.go index cc27a70008a..207beaf1c23 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -91,46 +91,6 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics } } -type FixedTableIdentityTranslator struct { - flow2p2p map[flow.Identifier]peer.ID - p2p2flow map[peer.ID]flow.Identifier -} - -func (t *FixedTableIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { - nodeID, ok := t.p2p2flow[p] - if !ok { - return flow.ZeroID, fmt.Errorf("could not find a flow NodeID for peer %v", p) - } - return nodeID, nil -} - -func (t *FixedTableIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { - peerID, ok := t.flow2p2p[n] - if !ok { - return *new(peer.ID), fmt.Errorf("could not find a libp2p PeerID for flow NodeID %v", n) - } - return peerID, nil -} - -func NewFixedTableIdentityTranslator(t *testing.T, identities flow.IdentityList) *FixedTableIdentityTranslator { - flow2p2p := make(map[flow.Identifier]peer.ID) - p2p2flow := make(map[peer.ID]flow.Identifier) - - for _, identity := range identities { - nodeID := identity.NodeID - networkKey := identity.NetworkPubKey - peerPK, err := p2p.LibP2PPublicKeyFromFlow(networkKey) - require.NoError(t, err) - - peerID, err := peer.IDFromPublicKey(peerPK) - require.NoError(t, err) - - flow2p2p[nodeID] = peerID - p2p2flow[peerID] = nodeID - } - return &FixedTableIdentityTranslator{flow2p2p, p2p2flow} -} - // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. @@ -176,7 +136,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id } // create a fixed id translator for the identities - tableTranslator := NewFixedTableIdentityTranslator(t, identities) + tableTranslator, err := p2p.NewFixedTableIdentityTranslator(identities) + require.NoError(t, err) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, From 48b303ee63edad235ff3c6462a6554cacb5e6310 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 16:09:17 -0700 Subject: [PATCH 211/291] Update peerManager_test.go Update execution_test.go update middleware --- cmd/scaffold.go | 1 + engine/execution/execution_test.go | 34 ++++++++---------- network/p2p/middleware.go | 57 +++++++++++++++++++----------- network/p2p/peerManager_test.go | 27 ++++++-------- 4 files changed, 63 insertions(+), 56 deletions(-) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 0defbdee53d..1db9dc1e14a 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -736,6 +736,7 @@ func (fnb *FlowNodeBuilder) Initialize() NodeBuilder { fnb.ParseAndPrintFlags() + // ID providers must be initialized before the network fnb.InitIDProviders() fnb.EnqueueNetworkInit(ctx) diff --git a/engine/execution/execution_test.go b/engine/execution/execution_test.go index 76d2ae13137..d46d69a8dd7 100644 --- a/engine/execution/execution_test.go +++ b/engine/execution/execution_test.go @@ -37,23 +37,21 @@ func TestExecutionFlow(t *testing.T) { chainID := flow.Testnet - keys, errs := unittest.NetworkingKeys(4) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exeID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) verID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[3].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID) @@ -300,23 +298,23 @@ func TestExecutionStateSyncMultipleExecutionNodes(t *testing.T) { chainID := flow.Emulator - keys, errs := unittest.NetworkingKeys(4) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exe1ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exe1ID) - identities[3].NetworkPubKey = keys[3].PublicKey() + key, err := unittest.NetworkingKey() + require.NoError(t, err) + identities[3].NetworkPubKey = key.PublicKey() collectionNode := testutil.GenericNodeFromParticipants(t, hub, colID, identities, chainID) defer collectionNode.Done() @@ -449,27 +447,25 @@ func TestBroadcastToMultipleVerificationNodes(t *testing.T) { chainID := flow.Emulator - keys, errs := unittest.NetworkingKeys(5) - assert.NoError(t, errs) colID := unittest.IdentityFixture( unittest.WithRole(flow.RoleCollection), - unittest.WithNetworkingKey(keys[0].PublicKey()), + unittest.WithKeys, ) conID := unittest.IdentityFixture( unittest.WithRole(flow.RoleConsensus), - unittest.WithNetworkingKey(keys[1].PublicKey()), + unittest.WithKeys, ) exeID := unittest.IdentityFixture( unittest.WithRole(flow.RoleExecution), - unittest.WithNetworkingKey(keys[2].PublicKey()), + unittest.WithKeys, ) ver1ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[3].PublicKey()), + unittest.WithKeys, ) ver2ID := unittest.IdentityFixture( unittest.WithRole(flow.RoleVerification), - unittest.WithNetworkingKey(keys[4].PublicKey()), + unittest.WithKeys, ) identities := unittest.CompleteIdentitySet(colID, conID, exeID, ver1ID, ver2ID) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 32b3ee16ef5..611180b6c9c 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -64,24 +64,25 @@ const ( // our neighbours on the peer-to-peer network. type Middleware struct { sync.Mutex - ctx context.Context - cancel context.CancelFunc - log zerolog.Logger - ov network.Overlay - wg *sync.WaitGroup - libP2PNode *Node - libP2PNodeFactory LibP2PFactoryFunc - me flow.Identifier - metrics module.NetworkMetrics - rootBlockID string - validators []network.MessageValidator - peerManager *PeerManager - peerUpdateInterval time.Duration - unicastMessageTimeout time.Duration - connectionGating bool - managePeerConnections bool - idTranslator IDTranslator - idProvider id.IdentifierProvider + ctx context.Context + cancel context.CancelFunc + log zerolog.Logger + ov network.Overlay + wg *sync.WaitGroup + libP2PNode *Node + libP2PNodeFactory LibP2PFactoryFunc + me flow.Identifier + metrics module.NetworkMetrics + rootBlockID string + validators []network.MessageValidator + peerManager *PeerManager + peerUpdateInterval time.Duration + unicastMessageTimeout time.Duration + connectionGating bool + managePeerConnections bool + idTranslator IDTranslator + idProvider id.IdentifierProvider + previousProtocolStatePeers []peer.AddrInfo } type MiddlewareOption func(*Middleware) @@ -200,12 +201,28 @@ func (m *Middleware) GetIPPort() (string, string, error) { } func (m *Middleware) UpdateNodeAddresses() { + m.log.Info().Msg("Updating protocol state node addresses") + ids := m.ov.Identities() - infos, _ := peerInfosFromIDs(ids) + newInfos, invalid := peerInfosFromIDs(ids) + + for id, err := range invalid { + m.log.Err(err).Str("node_id", id.String()).Msg("failed to extract peer info from identity") + } + + m.Lock() + defer m.Unlock() - for _, info := range infos { + // set old addresses to expire + for _, oldInfo := range m.previousProtocolStatePeers { + m.libP2PNode.host.Peerstore().SetAddrs(oldInfo.ID, oldInfo.Addrs, peerstore.TempAddrTTL) + } + + for _, info := range newInfos { m.libP2PNode.host.Peerstore().SetAddrs(info.ID, info.Addrs, peerstore.PermanentAddrTTL) } + + m.previousProtocolStatePeers = newInfos } // Start will start the middleware. diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index 5c6632fbc76..bdaa6aca015 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -15,7 +15,6 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/utils/unittest" ) @@ -34,29 +33,23 @@ func (suite *PeerManagerTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) } -func (suite *PeerManagerTestSuite) generateIdentities(n int) (flow.IdentityList, peer.IDSlice) { - // create some test ids - currentIDs := unittest.IdentityListFixture(n) - - // setup a ID provider callback to return currentIDs +func (suite *PeerManagerTestSuite) generatePeerIDs(n int) peer.IDSlice { pids := peer.IDSlice{} - for _, id := range currentIDs { - key, err := generateFlowNetworkingKey(id.NodeID) - require.NoError(suite.T(), err) - id.NetworkPubKey = key.PublicKey() + for i := 0; i < n; i++ { + key := generateNetworkingKey(suite.T()) pid, err := ExtractPeerID(key.PublicKey()) require.NoError(suite.T(), err) pids = append(pids, pid) } - return currentIDs, pids + return pids } // TestUpdatePeers tests that updatePeers calls the connector with the expected list of ids to connect and disconnect // from. The tests are cumulative and ordered. func (suite *PeerManagerTestSuite) TestUpdatePeers() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -84,7 +77,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { // a subsequent call to updatePeers should request a connector.UpdatePeers to existing ids and new ids suite.Run("updatePeers connects to old and new peers", func() { // create a new id - _, newPIDs := suite.generateIdentities(1) + newPIDs := suite.generatePeerIDs(1) pids = append(pids, newPIDs...) pm.updatePeers() @@ -107,7 +100,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { pids = removeRandomElement(pids) // add a couple of new ids - _, newPIDs := suite.generateIdentities(2) + newPIDs := suite.generatePeerIDs(2) pids = append(pids, newPIDs...) pm.updatePeers() @@ -125,7 +118,7 @@ func removeRandomElement(pids peer.IDSlice) peer.IDSlice { // TestPeriodicPeerUpdate tests that the peer manager runs periodically func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -160,7 +153,7 @@ func (suite *PeerManagerTestSuite) TestPeriodicPeerUpdate() { // TestOnDemandPeerUpdate tests that the a peer update can be requested on demand and in between the periodic runs func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { @@ -206,7 +199,7 @@ func (suite *PeerManagerTestSuite) TestOnDemandPeerUpdate() { // TestConcurrentOnDemandPeerUpdate tests that concurrent on-demand peer update request never block func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { // create some test ids - _, pids := suite.generateIdentities(10) + pids := suite.generatePeerIDs(10) // setup a ID provider callback to return peer IDs idProvider := func() (peer.IDSlice, error) { From 02f73025f43579fad58d135a1b73142e48c9a1ba Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 20:00:32 -0700 Subject: [PATCH 212/291] Address comments --- .../staked_access_node_builder.go | 2 +- cmd/scaffold.go | 2 +- network/p2p/libp2pUtils.go | 17 +++++++++++ network/p2p/network.go | 6 ++-- network/p2p/peerManager_test.go | 28 ++----------------- network/p2p/protocol_state_provider.go | 17 ----------- 6 files changed, 24 insertions(+), 48 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 233cb56819d..a2fcec2fa82 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -40,7 +40,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), - p2p.NetworkingSetFilter, + p2p.NotEjectedFilter, ), idCache, ) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 1db9dc1e14a..5352eeada89 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -432,7 +432,7 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache fnb.IDTranslator = idCache - fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NetworkingSetFilter, idCache) + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) return nil }) } diff --git a/network/p2p/libp2pUtils.go b/network/p2p/libp2pUtils.go index ad43adbc877..04a2996c2aa 100644 --- a/network/p2p/libp2pUtils.go +++ b/network/p2p/libp2pUtils.go @@ -16,6 +16,7 @@ import ( "github.com/multiformats/go-multiaddr" "github.com/rs/zerolog" + fcrypto "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" ) @@ -231,3 +232,19 @@ func flowStream(conn network.Conn) network.Stream { } return nil } + +func ExtractPeerID(networkPubKey fcrypto.PublicKey) (pid peer.ID, err error) { + pk, err := LibP2PPublicKeyFromFlow(networkPubKey) + if err != nil { + err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) + return + } + + pid, err = peer.IDFromPublicKey(pk) + if err != nil { + err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) + return + } + + return +} diff --git a/network/p2p/network.go b/network/p2p/network.go index 82235bd4938..1c0d010a16f 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -24,13 +24,13 @@ import ( const DefaultCacheSize = 10e6 -// NetworkingSetFilter is an identity filter that, when applied to the identity +// NotEjectedFilter is an identity filter that, when applied to the identity // table at a given snapshot, returns all nodes that we should communicate with // over the networking layer. // // NOTE: The protocol state includes nodes from the previous/next epoch that should // be included in network communication. We omit any nodes that have been ejected. -var NetworkingSetFilter = filter.Not(filter.Ejected) +var NotEjectedFilter = filter.Not(filter.Ejected) type ReadyDoneAwareNetwork interface { module.Network @@ -164,7 +164,7 @@ func (n *Network) unregister(channel network.Channel) error { } func (n *Network) Identities() flow.IdentityList { - return n.identityProvider.Identities(NetworkingSetFilter) + return n.identityProvider.Identities(NotEjectedFilter) } // Topology returns the identitiess of a uniform subset of nodes in protocol state using the topology provided earlier. diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index bdaa6aca015..ad8471f8be5 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -242,30 +242,6 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { // assertListsEqual asserts that two peer ID slices are equal ignoring the order func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { - listsEqual := func() bool { - if len(list1) != len(list2) { - return false - } - - map1 := make(map[peer.ID]int) - map2 := make(map[peer.ID]int) - - for _, e1 := range list1 { - map1[e1]++ - } - for _, e2 := range list2 { - map2[e2]++ - } - - for key, val := range map1 { - if map2[key] != val { - return false - } - } - return true - } - - if !listsEqual() { - assert.Failf(t, "peer ID slices not equal", "list1: %v, list2: %v", list1, list2) - } + assert.Subset(t, list1, list2) + assert.Subset(t, list2, list1) } diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index 6f223a4d0fe..b1a5bdcf834 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -7,7 +7,6 @@ import ( "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/state/protocol" @@ -123,19 +122,3 @@ func (p *ProtocolStateIDCache) GetFlowID(peerID peer.ID) (fid flow.Identifier, e return } - -func ExtractPeerID(networkPubKey crypto.PublicKey) (pid peer.ID, err error) { - pk, err := LibP2PPublicKeyFromFlow(networkPubKey) - if err != nil { - err = fmt.Errorf("failed to convert Flow key to LibP2P key: %w", err) - return - } - - pid, err = peer.IDFromPublicKey(pk) - if err != nil { - err = fmt.Errorf("failed to convert LibP2P key to peer ID: %w", err) - return - } - - return -} From 959e6f943bcc825ba6b9e3ee970495cb89b808d5 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 21:29:16 -0700 Subject: [PATCH 213/291] protocol state provider test --- network/p2p/peerManager_test.go | 8 +- network/p2p/protocol_state_provider_test.go | 138 ++++++++++++++++++++ 2 files changed, 139 insertions(+), 7 deletions(-) create mode 100644 network/p2p/protocol_state_provider_test.go diff --git a/network/p2p/peerManager_test.go b/network/p2p/peerManager_test.go index ad8471f8be5..08b3df05193 100644 --- a/network/p2p/peerManager_test.go +++ b/network/p2p/peerManager_test.go @@ -61,7 +61,7 @@ func (suite *PeerManagerTestSuite) TestUpdatePeers() { connector.On("UpdatePeers", mock.Anything, mock.AnythingOfType("peer.IDSlice")). Run(func(args mock.Arguments) { idArg := args[1].(peer.IDSlice) - assertListsEqual(suite.T(), pids, idArg) + assert.ElementsMatch(suite.T(), pids, idArg) }). Return(nil) @@ -239,9 +239,3 @@ func (suite *PeerManagerTestSuite) TestConcurrentOnDemandPeerUpdate() { return connector.AssertNumberOfCalls(suite.T(), "UpdatePeers", 2) }, 10*time.Second, 100*time.Millisecond) } - -// assertListsEqual asserts that two peer ID slices are equal ignoring the order -func assertListsEqual(t *testing.T, list1, list2 peer.IDSlice) { - assert.Subset(t, list1, list2) - assert.Subset(t, list2, list1) -} diff --git a/network/p2p/protocol_state_provider_test.go b/network/p2p/protocol_state_provider_test.go new file mode 100644 index 00000000000..55ac4074a5e --- /dev/null +++ b/network/p2p/protocol_state_provider_test.go @@ -0,0 +1,138 @@ +package p2p + +import ( + "testing" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/state/protocol/events" + mockprotocol "github.com/onflow/flow-go/state/protocol/mock" + "github.com/onflow/flow-go/utils/unittest" +) + +type ProtocolStateProviderTestSuite struct { + suite.Suite + provider *ProtocolStateIDCache + distributor *events.Distributor + state protocol.State + snapshot protocol.Snapshot + head *flow.Header + participants flow.IdentityList + epochNum uint64 +} + +func (suite *ProtocolStateProviderTestSuite) SetupTest() { + suite.distributor = events.NewDistributor() + + // set up protocol state mock + state := &mockprotocol.State{} + state.On("Final").Return( + func() protocol.Snapshot { + return suite.snapshot + }, + ) + state.On("AtBlockID", mock.Anything).Return( + func(blockID flow.Identifier) protocol.Snapshot { + if suite.head.ID() == blockID { + return suite.snapshot + } else { + return unittest.StateSnapshotForUnknownBlock() + } + }, + ) + suite.state = state + suite.epochNum = 0 + + suite.triggerUpdate() + + provider, err := NewProtocolStateIDCache(zerolog.Logger{}, state, suite.distributor) + require.NoError(suite.T(), err) + + suite.provider = provider +} + +func (suite *ProtocolStateProviderTestSuite) triggerUpdate() { + suite.participants = unittest.IdentityListFixture(5, unittest.WithAllRoles(), unittest.WithKeys) + + block := unittest.BlockFixture() + suite.head = block.Header + + // set up protocol snapshot mock + snapshot := &mockprotocol.Snapshot{} + snapshot.On("Identities", mock.Anything).Return( + func(filter flow.IdentityFilter) flow.IdentityList { + return suite.participants.Filter(filter) + }, + nil, + ) + snapshot.On("Identity", mock.Anything).Return(func(id flow.Identifier) *flow.Identity { + for _, n := range suite.participants { + if n.ID() == id { + return n + } + } + return nil + }, nil) + snapshot.On("Head").Return( + func() *flow.Header { + return suite.head + }, + nil, + ) + suite.snapshot = snapshot + suite.epochNum += 1 + + suite.distributor.EpochTransition(suite.epochNum, suite.head) +} + +func TestProtocolStateProvider(t *testing.T) { + suite.Run(t, new(ProtocolStateProviderTestSuite)) +} + +func (suite *ProtocolStateProviderTestSuite) checkStateTransition() { + oldParticipants := suite.participants + + suite.triggerUpdate() + + assert.ElementsMatch(suite.T(), suite.participants, suite.provider.Identities(filter.Any)) + for _, participant := range suite.participants { + pid, err := suite.provider.GetPeerID(participant.NodeID) + require.NoError(suite.T(), err) + fid, err := suite.provider.GetFlowID(pid) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), fid, participant.NodeID) + } + for _, participant := range oldParticipants { + _, err := suite.provider.GetPeerID(participant.NodeID) + require.Error(suite.T(), err) + } +} + +func (suite *ProtocolStateProviderTestSuite) TestUpdateState() { + for i := 0; i < 10; i++ { + suite.checkStateTransition() + } +} + +func (suite *ProtocolStateProviderTestSuite) TestIDTranslation() { + for _, participant := range suite.participants { + pid, err := suite.provider.GetPeerID(participant.NodeID) + require.NoError(suite.T(), err) + key, err := LibP2PPublicKeyFromFlow(participant.NetworkPubKey) + require.NoError(suite.T(), err) + expectedPid, err := peer.IDFromPublicKey(key) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), expectedPid, pid) + fid, err := suite.provider.GetFlowID(pid) + require.NoError(suite.T(), err) + assert.Equal(suite.T(), fid, participant.NodeID) + } +} From d4b2e96eb29e92d4900766ff05bd27d02181cab5 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 20 Aug 2021 22:59:39 -0700 Subject: [PATCH 214/291] updating origin ID checks to use idtranslator --- network/p2p/middleware.go | 25 +++++++------- network/p2p/readConnection.go | 20 ++++-------- network/p2p/readSubscription.go | 58 +++------------------------------ 3 files changed, 22 insertions(+), 81 deletions(-) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 611180b6c9c..72866bd977d 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -405,25 +405,22 @@ func (m *Middleware) Unsubscribe(channel network.Channel) error { return nil } -// processAuthenticatedMessage processes a message and a source (indicated by its PublicKey) and eventually passes it to the overlay -// In particular, it checks the claim of protocol authorship situated in the message against `originKey` -// The assumption is that the message has been authenticated at the network level (libp2p) to origin at the network public key `originKey` +// processAuthenticatedMessage processes a message and a source (indicated by its peer ID) and eventually passes it to the overlay +// In particular, it checks the claim of protocol authorship situated in the message against `peerID` +// The assumption is that the message has been authenticated at the network level (libp2p) to originate from the peer with ID `peerID` // this requirement is fulfilled by e.g. the output of readConnection and readSubscription -func (m *Middleware) processAuthenticatedMessage(msg *message.Message, originKey crypto.PublicKey) { - identities := m.ov.Identities() +func (m *Middleware) processAuthenticatedMessage(msg *message.Message, peerID peer.ID) { + flowID, err := m.idTranslator.GetFlowID(peerID) + if err != nil { + m.log.Warn().Err(err).Msgf("received message from unknown peer %v, and was dropped", peerID.String()) + return + } // check the origin of the message corresponds to the one claimed in the OriginID originID := flow.HashToID(msg.OriginID) - originIdentity, found := identities.ByNodeID(originID) - if !found { - m.log.Warn().Msgf("received message with claimed originID %x, which is not known by this node, and was dropped", originID) - return - } else if originIdentity.NetworkPubKey == nil { - m.log.Warn().Msgf("received message with claimed originID %x, wich has no network identifiers at this node, and was dropped", originID) - return - } else if !originIdentity.NetworkPubKey.Equals(originKey) { - m.log.Warn().Msgf("received message claiming to be from nodeID %v with key %x was actually signed by %x and dropped", originID, originIdentity.NetworkPubKey, originKey) + if flowID != originID { + m.log.Warn().Msgf("received message claiming to be from nodeID %v was actually from %v and dropped", originID, flowID) return } diff --git a/network/p2p/readConnection.go b/network/p2p/readConnection.go index 5d6a0711872..c71b8bb1e25 100644 --- a/network/p2p/readConnection.go +++ b/network/p2p/readConnection.go @@ -7,10 +7,10 @@ import ( ggio "github.com/gogo/protobuf/io" libp2pnetwork "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/message" @@ -21,17 +21,17 @@ import ( type readConnection struct { ctx context.Context stream libp2pnetwork.Stream - remoteKey crypto.PublicKey + remoteID peer.ID log zerolog.Logger metrics module.NetworkMetrics maxMsgSize int - callback func(msg *message.Message, pk crypto.PublicKey) + callback func(msg *message.Message, peerID peer.ID) } // newReadConnection creates a new readConnection func newReadConnection(ctx context.Context, stream libp2pnetwork.Stream, - callback func(msg *message.Message, pubKey crypto.PublicKey), + callback func(msg *message.Message, peerID peer.ID), log zerolog.Logger, metrics module.NetworkMetrics, maxMsgSize int) *readConnection { @@ -40,18 +40,10 @@ func newReadConnection(ctx context.Context, maxMsgSize = DefaultMaxUnicastMsgSize } - remoteKey := stream.Conn().RemotePublicKey() - flowKey, err := FlowPublicKeyFromLibP2P(remoteKey) - // this should not happen if the stream was setup properly - if err != nil { - log.Err(err).Msg("failed to extract flow public key of stream libp2p key") - return nil - } - c := readConnection{ ctx: ctx, stream: stream, - remoteKey: flowKey, + remoteID: stream.Conn().RemotePeer(), callback: callback, log: log, metrics: metrics, @@ -113,7 +105,7 @@ func (rc *readConnection) receiveLoop(wg *sync.WaitGroup) { rc.metrics.NetworkMessageReceived(msg.Size(), metrics.ChannelOneToOne, msg.Type) // call the callback - rc.callback(&msg, rc.remoteKey) + rc.callback(&msg, rc.remoteID) } } diff --git a/network/p2p/readSubscription.go b/network/p2p/readSubscription.go index 3b1b66f8e69..db87b097afa 100644 --- a/network/p2p/readSubscription.go +++ b/network/p2p/readSubscription.go @@ -2,17 +2,14 @@ package p2p import ( "context" - "fmt" "strings" "sync" - lcrypto "github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/rs/zerolog" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/network/message" _ "github.com/onflow/flow-go/utils/binstat" @@ -25,13 +22,13 @@ type readSubscription struct { log zerolog.Logger sub *pubsub.Subscription metrics module.NetworkMetrics - callback func(msg *message.Message, pubKey crypto.PublicKey) + callback func(msg *message.Message, peerID peer.ID) } // newReadSubscription reads the messages coming in on the subscription func newReadSubscription(ctx context.Context, sub *pubsub.Subscription, - callback func(msg *message.Message, pubKey crypto.PublicKey), + callback func(msg *message.Message, peerID peer.ID), log zerolog.Logger, metrics module.NetworkMetrics) *readSubscription { @@ -82,16 +79,9 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { return } - // if pubsub.WithMessageSigning(true) and pubsub.WithStrictSignatureVerification(true), - // the emitter is authenticated - emitterKey, err := messagePubKey(rawMsg) + pid, err := peer.IDFromBytes(rawMsg.From) if err != nil { - r.log.Err(err).Msg("failed to extract libp2p public key of message") - return - } - flowKey, err := FlowPublicKeyFromLibP2P(emitterKey) - if err != nil { - r.log.Err(err).Msg("failed to extract flow public key of libp2p key") + r.log.Err(err).Msg("failed to validate peer ID of incoming message") return } @@ -109,44 +99,6 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { r.metrics.NetworkMessageReceived(msg.Size(), msg.ChannelID, msg.Type) // call the callback - r.callback(&msg, flowKey) - } -} - -// messagePubKey extracts the public key of the envelope signer from a libp2p message. -// The location of that key depends on the type of the key, see: -// https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md -// This reproduces the exact logic of the private function doing the same decoding in libp2p: -// https://github.com/libp2p/go-libp2p-pubsub/blob/ba28f8ecfc551d4d916beb748d3384951bce3ed0/sign.go#L77 -func messagePubKey(m *pubsub.Message) (lcrypto.PubKey, error) { - var pubk lcrypto.PubKey - - // m.From is the original sender of the message (versus `m.ReceivedFrom` which is the last hop which sent us this message) - pid, err := peer.IDFromBytes(m.From) - if err != nil { - return nil, err - } - - if m.Key == nil { - // no attached key, it must be extractable from the source ID - pubk, err = pid.ExtractPublicKey() - if err != nil { - return nil, fmt.Errorf("cannot extract signing key: %s", err.Error()) - } - if pubk == nil { - return nil, fmt.Errorf("cannot extract signing key") - } - } else { - pubk, err = lcrypto.UnmarshalPublicKey(m.Key) - if err != nil { - return nil, fmt.Errorf("cannot unmarshal signing key: %s", err.Error()) - } - - // verify that the source ID matches the attached key - if !pid.MatchesPublicKey(pubk) { - return nil, fmt.Errorf("bad signing key; source ID %s doesn't match key", pid) - } + r.callback(&msg, pid) } - - return pubk, nil } From 35353554aa6d354143bfcb5c540d48252ca367c9 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Sat, 21 Aug 2021 14:12:50 -0700 Subject: [PATCH 215/291] new providers remove comments Update hierarchical_translator_test.go --- .../staked_access_node_builder.go | 3 +- engine/common/synchronization/engine_test.go | 1 + engine/testutil/nodes.go | 1 + model/flow/filter/identity.go | 12 +++ module/id/fixed_provider.go | 4 +- module/id/updatable_provider.go | 34 +++++++++ network/p2p/dht_test.go | 2 - network/p2p/hierarchical_translator_test.go | 8 +- network/p2p/identity_provider_translator.go | 51 +++++++++++++ network/p2p/libp2pNode.go | 32 ++++---- network/test/middleware_test.go | 73 ++++++++++++++++--- network/test/testUtil.go | 25 ++++--- 12 files changed, 199 insertions(+), 47 deletions(-) create mode 100644 module/id/updatable_provider.go create mode 100644 network/p2p/identity_provider_translator.go diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a2fcec2fa82..8e5b5285e3f 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -63,8 +63,7 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { // for the staked access node, initialize the network used to communicate with the other staked flow nodes // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - // TODO: we should remove this call since we are no longer instantiating two networks - // builder.EnqueueNetworkInit(ctx) + builder.EnqueueNetworkInit(ctx) // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network if builder.ParticipatesInUnstakedNetwork() { diff --git a/engine/common/synchronization/engine_test.go b/engine/common/synchronization/engine_test.go index 4e79938fab7..5dce4ea28a5 100644 --- a/engine/common/synchronization/engine_test.go +++ b/engine/common/synchronization/engine_test.go @@ -171,6 +171,7 @@ func (ss *SyncSuite) SetupTest() { require.NoError(ss.T(), err, "could not create finalized snapshot cache") idCache, err := p2p.NewProtocolStateIDCache(log, ss.state, protocolEvents.NewDistributor()) + require.NoError(ss.T(), err, "could not create protocol state identity cache") e, err := New(log, metrics, ss.net, ss.me, ss.blocks, ss.comp, ss.core, finalizedHeader, id.NewFilteredIdentifierProvider( filter.And( diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index 3688e2c6727..ecaad824601 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -628,6 +628,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit require.NoError(t, err) idCache, err := p2p.NewProtocolStateIDCache(node.Log, node.State, events.NewDistributor()) + require.NoError(t, err, "could not create finalized snapshot cache") syncEngine, err := synchronization.New( node.Log, node.Metrics, diff --git a/model/flow/filter/identity.go b/model/flow/filter/identity.go index f55b2e5227b..dc39e4ce475 100644 --- a/model/flow/filter/identity.go +++ b/model/flow/filter/identity.go @@ -3,6 +3,7 @@ package filter import ( + "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" ) @@ -61,6 +62,17 @@ func HasNodeID(nodeIDs ...flow.Identifier) flow.IdentityFilter { } } +func HasNetworkingKey(keys ...crypto.PublicKey) flow.IdentityFilter { + return func(identity *flow.Identity) bool { + for _, key := range keys { + if key.Equals(identity.NetworkPubKey) { + return true + } + } + return false + } +} + // HasStake returns a filter for nodes with non-zero stake. func HasStake(hasStake bool) flow.IdentityFilter { return func(identity *flow.Identity) bool { diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index 106047510b6..39aac7567a1 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -24,6 +24,6 @@ func NewFixedIdentityProvider(identities flow.IdentityList) *FixedIdentityProvid return &FixedIdentityProvider{identities} } -func (p *FixedIdentityProvider) Identities(flow.IdentityFilter) flow.IdentityList { - return p.identities +func (p *FixedIdentityProvider) Identities(filter flow.IdentityFilter) flow.IdentityList { + return p.identities.Filter(filter) } diff --git a/module/id/updatable_provider.go b/module/id/updatable_provider.go new file mode 100644 index 00000000000..2288b9e6834 --- /dev/null +++ b/module/id/updatable_provider.go @@ -0,0 +1,34 @@ +package id + +import ( + "sync" + + "github.com/onflow/flow-go/model/flow" +) + +type UpdatableIDProvider struct { + mu sync.RWMutex + identities flow.IdentityList +} + +func NewUpdatableIDProvider(identities flow.IdentityList) *UpdatableIDProvider { + return &UpdatableIDProvider{identities: identities} +} + +func (p *UpdatableIDProvider) SetIdentities(identities flow.IdentityList) { + p.mu.Lock() + defer p.mu.Unlock() + p.identities = identities +} + +func (p *UpdatableIDProvider) Identities(filter flow.IdentityFilter) flow.IdentityList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.Filter(filter) +} + +func (p *UpdatableIDProvider) Identifiers() flow.IdentifierList { + p.mu.RLock() + defer p.mu.RUnlock() + return p.identities.NodeIDs() +} diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index 6643a7652b0..cc01eedaafb 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -29,8 +29,6 @@ type DHTTestSuite struct { cancel context.CancelFunc // used to cancel the context } -// TODO: test that dht findPeer times out - // TestDHTTestSuite test the libp2p pubsub with DHT for discovery func TestDHTTestSuite(t *testing.T) { suite.Run(t, new(DHTTestSuite)) diff --git a/network/p2p/hierarchical_translator_test.go b/network/p2p/hierarchical_translator_test.go index 6866547f378..90245d82a97 100644 --- a/network/p2p/hierarchical_translator_test.go +++ b/network/p2p/hierarchical_translator_test.go @@ -4,11 +4,12 @@ import ( "testing" "github.com/libp2p/go-libp2p-core/peer" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/utils/unittest" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "gotest.tools/assert" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" ) type HierarchicalTranslatorTestSuite struct { @@ -35,6 +36,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestFirstTranslatorSuccess() { pid, err := suite.translator.GetPeerID(suite.ids[0].NodeID) require.NoError(suite.T(), err) fid, err := suite.translator.GetFlowID(pid) + require.NoError(suite.T(), err) assert.Equal(suite.T(), fid, suite.ids[0].NodeID) } @@ -42,6 +44,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestSecondTranslatorSuccess() { pid, err := suite.translator.GetPeerID(suite.ids[1].NodeID) require.NoError(suite.T(), err) fid, err := suite.translator.GetFlowID(pid) + require.NoError(suite.T(), err) assert.Equal(suite.T(), fid, suite.ids[1].NodeID) } @@ -53,6 +56,7 @@ func (suite *HierarchicalTranslatorTestSuite) TestTranslationFailure() { key, err := LibP2PPrivKeyFromFlow(generateNetworkingKey(suite.T())) require.NoError(suite.T(), err) pid, err := peer.IDFromPrivateKey(key) + require.NoError(suite.T(), err) _, err = suite.translator.GetFlowID(pid) require.Error(suite.T(), err) } diff --git a/network/p2p/identity_provider_translator.go b/network/p2p/identity_provider_translator.go new file mode 100644 index 00000000000..61504b1ef05 --- /dev/null +++ b/network/p2p/identity_provider_translator.go @@ -0,0 +1,51 @@ +package p2p + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module/id" +) + +type IdentityProviderIdentityTranslator struct { + idProvider id.IdentityProvider +} + +func (t *IdentityProviderIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { + key, err := p.ExtractPublicKey() + if err != nil { + return flow.ZeroID, err + } + flowKey, err := FlowPublicKeyFromLibP2P(key) + if err != nil { + return flow.ZeroID, err + } + ids := t.idProvider.Identities(filter.HasNetworkingKey(flowKey)) + if len(ids) == 0 { + return flow.ZeroID, fmt.Errorf("could not find identity corresponding to peer id %v", p.Pretty()) + } + return ids[0].NodeID, nil +} + +func (t *IdentityProviderIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { + ids := t.idProvider.Identities(filter.HasNodeID(n)) + if len(ids) == 0 { + return "", fmt.Errorf("could not find identity with id %v", n.String()) + } + key, err := LibP2PPublicKeyFromFlow(ids[0].NetworkPubKey) + if err != nil { + return "", err + } + pid, err := peer.IDFromPublicKey(key) + if err != nil { + return "", err + } + return pid, nil +} + +func NewIdentityProviderIdentityTranslator(provider id.IdentityProvider) *IdentityProviderIdentityTranslator { + return &IdentityProviderIdentityTranslator{provider} +} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 683252d47e1..db380d94226 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -356,24 +356,24 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { if len(n.host.Peerstore().Addrs(peerID)) == 0 { n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - if n.dht == nil { - return nil, fmt.Errorf("no valid addresses exist for peer %v", peerID) - } - - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - - var err error - func() { - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - defer cancel() - // try to find the peer using the dht - _, err = n.dht.FindPeer(timedCtx, peerID) - }() - if err != nil { - return nil, fmt.Errorf("could not find address for peer %v: %w", peerID, err) + if n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + + if err != nil { + n.logger.Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") + } else { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") + } } - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) stream, err := n.tryCreateNewStream(ctx, peerID, maxConnectAttempt) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 7f97b9ad9a1..3b33df7f312 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -8,8 +8,10 @@ import ( "time" "github.com/ipfs/go-log" + swarm "github.com/libp2p/go-libp2p-swarm" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" mockery "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -17,6 +19,7 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/observable" "github.com/onflow/flow-go/network/codec/cbor" @@ -56,12 +59,14 @@ func (co *tagsObserver) OnComplete() { type MiddlewareTestSuite struct { suite.Suite - size int // used to determine number of middlewares under test - mws []*p2p.Middleware // used to keep track of middlewares under test - ov []*mocknetwork.Overlay - obs chan string // used to keep track of Protect events tagged by pubsub messages - ids []*flow.Identity - metrics *metrics.NoopCollector // no-op performance monitoring simulation + size int // used to determine number of middlewares under test + mws []*p2p.Middleware // used to keep track of middlewares under test + ov []*mocknetwork.Overlay + obs chan string // used to keep track of Protect events tagged by pubsub messages + ids []*flow.Identity + metrics *metrics.NoopCollector // no-op performance monitoring simulation + logger zerolog.Logger + providers []*id.UpdatableIDProvider } // TestMiddlewareTestSuit runs all the test methods in this test suit @@ -73,6 +78,7 @@ func TestMiddlewareTestSuite(t *testing.T) { func (m *MiddlewareTestSuite) SetupTest() { logger := zerolog.New(os.Stderr).Level(zerolog.ErrorLevel) log.SetAllLoggers(log.LevelError) + m.logger = logger m.size = 2 // operates on two middlewares m.metrics = metrics.NewNoopCollector() @@ -85,7 +91,7 @@ func (m *MiddlewareTestSuite) SetupTest() { log: logger, } - m.ids, m.mws, obs = GenerateIDsAndMiddlewares(m.T(), m.size, !DryRun, logger) + m.ids, m.mws, obs, m.providers = GenerateIDsAndMiddlewares(m.T(), m.size, !DryRun, logger) for _, observableConnMgr := range obs { observableConnMgr.Subscribe(&ob) @@ -98,11 +104,7 @@ func (m *MiddlewareTestSuite) SetupTest() { // create the mock overlays for i := 0; i < m.size; i++ { - overlay := &mocknetwork.Overlay{} - m.ov = append(m.ov, overlay) - - overlay.On("Identities").Maybe().Return(flow.IdentityList(m.ids), nil) - overlay.On("Topology").Maybe().Return(flow.IdentityList(m.ids), nil) + m.ov = append(m.ov, m.createOverlay()) } for i, mw := range m.mws { assert.NoError(m.T(), mw.Start(m.ov[i])) @@ -110,6 +112,18 @@ func (m *MiddlewareTestSuite) SetupTest() { } } +func (m *MiddlewareTestSuite) createOverlay() *mocknetwork.Overlay { + overlay := &mocknetwork.Overlay{} + + overlay.On("Identities").Maybe().Return(func() flow.IdentityList { + return flow.IdentityList(m.ids) + }, nil) + overlay.On("Topology").Maybe().Return(func() flow.IdentityList { + return flow.IdentityList(m.ids) + }, nil) + return overlay +} + func (m *MiddlewareTestSuite) TearDownTest() { m.stopMiddlewares() } @@ -462,6 +476,41 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { require.Error(m.Suite.T(), err) } +func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { + ids, mws, _, providers := GenerateIDsAndMiddlewares(m.T(), 1, false, m.logger) + require.Len(m.T(), ids, 1) + require.Len(m.T(), providers, 1) + require.Len(m.T(), mws, 1) + newId := ids[0] + newMw := mws[0] + newProvider := providers[0] + + idList := flow.IdentityList(append(m.ids, newId)) + + newProvider.SetIdentities(idList) + overlay := m.createOverlay() + overlay.On("Receive", + m.ids[0].NodeID, + mock.AnythingOfType("*message.Message"), + ).Return(nil) + assert.NoError(m.T(), newMw.Start(overlay)) + + // needed to enable ID translation + m.providers[0].SetIdentities(idList) + m.mws[0].UpdateAllowList() + + msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") + + err := m.mws[0].SendDirect(msg, newId.NodeID) + require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) + + m.ids = idList + m.mws[0].UpdateNodeAddresses() + + err = m.mws[0].SendDirect(msg, newId.NodeID) + require.NoError(m.T(), err) +} + // TestUnsubscribe tests that an engine can unsubscribe from a topic it was earlier subscribed to and stop receiving // messages. func (m *MiddlewareTestSuite) TestUnsubscribe() { diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 207beaf1c23..dae530624f5 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -21,6 +21,7 @@ import ( message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" + idModule "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/mock" @@ -122,9 +123,10 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) []*p2p.Middleware { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*id.UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) + idProviders := make([]*id.UpdatableIDProvider, len(identities)) for i, id := range identities { // casts libP2PNode instance to a local variable to avoid closure @@ -135,9 +137,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } - // create a fixed id translator for the identities - tableTranslator, err := p2p.NewFixedTableIdentityTranslator(identities) - require.NoError(t, err) + idProviders[i] = idModule.NewUpdatableIDProvider(identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, @@ -149,10 +149,13 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultUnicastTimeout, true, true, - tableTranslator, + p2p.NewIdentityProviderIdentityTranslator(idProviders[i]), + p2p.WithIdentifierProvider( + idProviders[i], + ), ) } - return mws + return mws, idProviders } // GenerateNetworks generates the network for the given middlewares @@ -201,7 +204,7 @@ func GenerateNetworks(t *testing.T, tops[i], sms[i], metrics, - id.NewFixedIdentityProvider(ids), + idModule.NewFixedIdentityProvider(ids), ) require.NoError(t, err) @@ -221,11 +224,11 @@ func GenerateNetworks(t *testing.T, func GenerateIDsAndMiddlewares(t *testing.T, n int, dryRunMode bool, - logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable) { + logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*id.UpdatableIDProvider) { ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) - mws := GenerateMiddlewares(t, logger, ids, libP2PNodes) - return ids, mws, protectObservables + mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) + return ids, mws, protectObservables, providers } func GenerateIDsMiddlewaresNetworks(t *testing.T, @@ -235,7 +238,7 @@ func GenerateIDsMiddlewaresNetworks(t *testing.T, tops []network.Topology, dryRun bool, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []*p2p.Network, []observable.Observable) { - ids, mws, observables := GenerateIDsAndMiddlewares(t, n, dryRun, log, opts...) + ids, mws, observables, _ := GenerateIDsAndMiddlewares(t, n, dryRun, log, opts...) sms := GenerateSubscriptionManagers(t, mws) networks := GenerateNetworks(t, log, ids, mws, csize, tops, sms, dryRun) return ids, mws, networks, observables From d35b2ba565928326c38e713ff0f86ba7df42d8c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Mon, 23 Aug 2021 09:57:56 -0400 Subject: [PATCH 216/291] add basic test for identity provider --- module/id/fixed_provider_test.go | 62 ++++++++++++++++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 module/id/fixed_provider_test.go diff --git a/module/id/fixed_provider_test.go b/module/id/fixed_provider_test.go new file mode 100644 index 00000000000..9fd224106da --- /dev/null +++ b/module/id/fixed_provider_test.go @@ -0,0 +1,62 @@ +package id + +import ( + "math/rand" + "testing" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + + "github.com/onflow/flow-go/utils/unittest" + "github.com/stretchr/testify/assert" +) + +func TestFixedIdentifierProvider(t *testing.T) { + identifiers := make([]flow.Identifier, 10) + for i := 0; i < len(identifiers); i++ { + identifiers[i] = unittest.IdentifierFixture() + } + + fp := NewFixedIdentifierProvider(identifiers) + + in := identifiers[rand.Intn(10)] + out := unittest.IdentifierFixture() + + assert.True(t, contains(fp.Identifiers(), in)) + assert.False(t, contains(fp.Identifiers(), out)) + +} + +func TestFixedIdentitiesProvider(t *testing.T) { + identities := make([]*flow.Identity, 10) + for i := 0; i < len(identities); i++ { + identities[i] = unittest.IdentityFixture() + } + + fp := NewFixedIdentityProvider(identities) + + in := identities[rand.Intn(10)] + out := unittest.IdentityFixture() + + assert.True(t, idContains(fp.Identities(filter.Any), in)) + assert.False(t, idContains(fp.Identities(filter.Any), out)) + +} + +func contains(a []flow.Identifier, b flow.Identifier) bool { + for _, i := range a { + if b == i { + return true + } + } + return false +} + +func idContains(a []*flow.Identity, b *flow.Identity) bool { + for _, i := range a { + if b == i { + return true + } + } + return false +} From 957dbbfb10dd7cd9943e859386dde533d2372e45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Mon, 23 Aug 2021 10:14:20 -0400 Subject: [PATCH 217/291] basic test for filtered identifier provider make goimports happy --- module/id/filtered_provider_test.go | 46 +++++++++++++++++++++++++++++ module/id/fixed_provider_test.go | 3 +- module/mocks/network.go | 1 + storage/mocks/storage.go | 1 + 4 files changed, 50 insertions(+), 1 deletion(-) create mode 100644 module/id/filtered_provider_test.go diff --git a/module/id/filtered_provider_test.go b/module/id/filtered_provider_test.go new file mode 100644 index 00000000000..52cffeec180 --- /dev/null +++ b/module/id/filtered_provider_test.go @@ -0,0 +1,46 @@ +package id + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + + "github.com/onflow/flow-go/utils/unittest" +) + +func TestFilteredIdentitiesProvider(t *testing.T) { + identities := make([]*flow.Identity, 10) + for i := 0; i < len(identities); i++ { + identities[i] = unittest.IdentityFixture() + } + identifiers := (flow.IdentityList)(identities).NodeIDs() + + oddIdentifiers := make([]flow.Identifier, 5) + for j := 0; j < 5; j++ { + oddIdentifiers[j] = identifiers[2*j+1] + } + + oddIdentities := make([]*flow.Identity, 5) + for j := 0; j < 5; j++ { + oddIdentities[j] = identities[2*j+1] + } + + ip := NewFixedIdentityProvider(identities) + fp := NewFilteredIdentifierProvider(filter.In(oddIdentities), ip) + + in := 0 + out := 0 + for _, id := range identifiers { + if contains(fp.Identifiers(), id) { + in++ + } else { + out++ + } + } + require.Equal(t, 5, in) + require.Equal(t, 5, out) + +} diff --git a/module/id/fixed_provider_test.go b/module/id/fixed_provider_test.go index 9fd224106da..b53e84c7d2d 100644 --- a/module/id/fixed_provider_test.go +++ b/module/id/fixed_provider_test.go @@ -7,8 +7,9 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/utils/unittest" "github.com/stretchr/testify/assert" + + "github.com/onflow/flow-go/utils/unittest" ) func TestFixedIdentifierProvider(t *testing.T) { diff --git a/module/mocks/network.go b/module/mocks/network.go index b680ae44c89..305cee9f346 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -8,6 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" + crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 7a829429fc4..969044a53ee 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -8,6 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" + flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" From 60f6591b9b37bdeff9c81c82b6e441a987cef97e Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 12:21:34 -0700 Subject: [PATCH 218/291] add default identifier provider for sync engine --- .../node_builder/access_node_builder.go | 42 +++++++++---------- .../staked_access_node_builder.go | 3 +- .../unstaked_access_node_builder.go | 2 +- cmd/collection/main.go | 9 +--- cmd/consensus/main.go | 9 +--- cmd/execution/main.go | 9 +--- cmd/node_builder.go | 1 + cmd/scaffold.go | 16 +++++-- cmd/verification/main.go | 10 +---- 9 files changed, 39 insertions(+), 62 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 64c8480d79c..d23024ed603 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -36,7 +36,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" @@ -150,26 +149,25 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedLibP2PNode *p2p.Node - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower - SyncEngineParticipantsProvider id.IdentifierProvider + UnstakedLibP2PNode *p2p.Node + UnstakedNetwork *p2p.Network + unstakedMiddleware *p2p.Middleware + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower // engines IngestEng *ingestion.Engine @@ -322,7 +320,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - builder.SyncEngineParticipantsProvider, + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 8e5b5285e3f..7a8023d495a 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -35,8 +35,7 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { } fnb.IdentityProvider = idCache - // translator - fnb.SyncEngineParticipantsProvider = id.NewFilteredIdentifierProvider( + fnb.SyncEngineIdentifierProvider = id.NewFilteredIdentifierProvider( filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index f6fd01e63d2..a4e0085eaca 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -106,7 +106,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { // use the default identifier provider - anb.SyncEngineParticipantsProvider = node.Middleware.IdentifierProvider() + node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() return nil }) anb.FlowAccessNodeBuilder.Build() diff --git a/cmd/collection/main.go b/cmd/collection/main.go index ef364a5421a..14b83348a65 100644 --- a/cmd/collection/main.go +++ b/cmd/collection/main.go @@ -35,7 +35,6 @@ import ( builder "github.com/onflow/flow-go/module/builder/collection" "github.com/onflow/flow-go/module/epochs" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/ingress" "github.com/onflow/flow-go/module/mempool" epochpool "github.com/onflow/flow-go/module/mempool/epochs" @@ -265,13 +264,7 @@ func main() { followerEng, mainChainSyncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) 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 30b5d32f285..5e6d7712d28 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -50,7 +50,6 @@ import ( dkgmodule "github.com/onflow/flow-go/module/dkg" "github.com/onflow/flow-go/module/epochs" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" consensusMempools "github.com/onflow/flow-go/module/mempool/consensus" "github.com/onflow/flow-go/module/mempool/stdmap" @@ -618,13 +617,7 @@ func main() { comp, syncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/execution/main.go b/cmd/execution/main.go index d38d2e7a305..4c15ca0d95a 100644 --- a/cmd/execution/main.go +++ b/cmd/execution/main.go @@ -42,7 +42,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" chainsync "github.com/onflow/flow-go/module/synchronization" @@ -457,13 +456,7 @@ func main() { followerEng, syncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not initialize synchronization engine: %w", err) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index ab027fceda8..a65435206b6 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -141,6 +141,7 @@ type NodeConfig struct { IdentityProvider id.IdentityProvider IDTranslator p2p.IDTranslator NetworkingIdentifierProvider id.IdentifierProvider + SyncEngineIdentifierProvider id.IdentifierProvider // root state information RootBlock *flow.Block diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 5352eeada89..cdab28d2495 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -21,6 +21,7 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" @@ -425,14 +426,21 @@ func (fnb *FlowNodeBuilder) initStorage() { func (fnb *FlowNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder NodeBuilder, node *NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, node.ProtocolEvents) if err != nil { return err } - fnb.IdentityProvider = idCache - fnb.IDTranslator = idCache - fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) + node.IdentityProvider = idCache + node.IDTranslator = idCache + node.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) + node.SyncEngineIdentifierProvider = id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + ), + idCache, + ) return nil }) } diff --git a/cmd/verification/main.go b/cmd/verification/main.go index bbd2af01438..fe4532b1ca0 100644 --- a/cmd/verification/main.go +++ b/cmd/verification/main.go @@ -24,12 +24,10 @@ import ( "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/encoding" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" "github.com/onflow/flow-go/module/chunks" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" @@ -354,13 +352,7 @@ func main() { followerEng, syncCore, finalizedHeader, - id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - ), - node.IdentityProvider, - ), + node.SyncEngineIdentifierProvider, ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) From 5b87c5258973a5b60bc1bd15dee474149090a8b1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 15:18:13 -0700 Subject: [PATCH 219/291] add comments --- .../staked_access_node_builder.go | 2 +- cmd/node_builder.go | 2 +- model/flow/filter/identity.go | 2 + module/id/filtered_provider.go | 2 + module/id/filtered_provider_test.go | 4 ++ module/id/fixed_provider.go | 4 ++ module/id/id_provider.go | 4 ++ module/id/updatable_provider.go | 5 +++ network/middleware.go | 7 +++- network/p2p/dht_test.go | 12 ++++++ network/p2p/fixed_translator.go | 2 + network/p2p/hierarchical_translator.go | 10 ++++- network/p2p/id_translator.go | 5 +++ network/p2p/identity_provider_translator.go | 12 +++--- network/p2p/libp2pNode.go | 37 +++++++++---------- network/p2p/libp2pUtils.go | 1 + network/p2p/peerstore_provider.go | 4 ++ network/p2p/protocol_state_provider.go | 5 +++ network/p2p/protocol_state_provider_test.go | 3 ++ network/p2p/unstaked_translator.go | 8 ++++ network/test/epochtransition_test.go | 1 + network/test/middleware_test.go | 7 ++++ network/test/testUtil.go | 2 +- 23 files changed, 111 insertions(+), 30 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 7a8023d495a..e17ce0e9d73 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -29,7 +29,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, node.ProtocolEvents) if err != nil { return err } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index a65435206b6..6f1300df68a 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -44,7 +44,7 @@ type NodeBuilder interface { // PrintBuildVersionDetails prints the node software build version PrintBuildVersionDetails() - // InitIDProviders initializes the IdentityProvider and IDTranslator + // InitIDProviders initializes the ID providers needed by various components InitIDProviders() // EnqueueNetworkInit enqueues the default network component with the given context diff --git a/model/flow/filter/identity.go b/model/flow/filter/identity.go index dc39e4ce475..0a02d15cf4c 100644 --- a/model/flow/filter/identity.go +++ b/model/flow/filter/identity.go @@ -62,6 +62,8 @@ func HasNodeID(nodeIDs ...flow.Identifier) flow.IdentityFilter { } } +// HasNetworkingKey returns a filter that returns true for any identity with a +// networking public key matching any of the inputs. func HasNetworkingKey(keys ...crypto.PublicKey) flow.IdentityFilter { return func(identity *flow.Identity) bool { for _, key := range keys { diff --git a/module/id/filtered_provider.go b/module/id/filtered_provider.go index d935125232f..5564ebe3990 100644 --- a/module/id/filtered_provider.go +++ b/module/id/filtered_provider.go @@ -4,6 +4,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// FilteredIdentifierProvider implements an IdentifierProvider which provides the identifiers +// resulting from applying a filter to an IdentityProvider. type FilteredIdentifierProvider struct { filter flow.IdentityFilter identityProvider IdentityProvider diff --git a/module/id/filtered_provider_test.go b/module/id/filtered_provider_test.go index 52cffeec180..28f4f24b94f 100644 --- a/module/id/filtered_provider_test.go +++ b/module/id/filtered_provider_test.go @@ -3,6 +3,7 @@ package id import ( "testing" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/flow" @@ -31,6 +32,9 @@ func TestFilteredIdentitiesProvider(t *testing.T) { ip := NewFixedIdentityProvider(identities) fp := NewFilteredIdentifierProvider(filter.In(oddIdentities), ip) + assert.ElementsMatch(t, fp.Identifiers(), + (flow.IdentityList)(oddIdentities).NodeIDs()) + in := 0 out := 0 for _, id := range identifiers { diff --git a/module/id/fixed_provider.go b/module/id/fixed_provider.go index 39aac7567a1..66cbed93685 100644 --- a/module/id/fixed_provider.go +++ b/module/id/fixed_provider.go @@ -4,6 +4,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// FixedIdentifierProvider implements an IdentifierProvider which provides a fixed list +// of identifiers. type FixedIdentifierProvider struct { identifiers flow.IdentifierList } @@ -16,6 +18,8 @@ func (p *FixedIdentifierProvider) Identifiers() flow.IdentifierList { return p.identifiers } +// FixedIdentityProvider implements an IdentityProvider which provides a fixed list +// of identities. type FixedIdentityProvider struct { identities flow.IdentityList } diff --git a/module/id/id_provider.go b/module/id/id_provider.go index a663a8f9215..73f1e87e587 100644 --- a/module/id/id_provider.go +++ b/module/id/id_provider.go @@ -4,10 +4,14 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// IdentifierProvider provides an interface to get a list of Identifiers representing +// a specific set of nodes in the network. type IdentifierProvider interface { Identifiers() flow.IdentifierList } +// IdentifierProvider provides an interface to get a list of Identities representing +// the set of staked participants in the Flow protocol. type IdentityProvider interface { Identities(flow.IdentityFilter) flow.IdentityList } diff --git a/module/id/updatable_provider.go b/module/id/updatable_provider.go index 2288b9e6834..d35a3da0566 100644 --- a/module/id/updatable_provider.go +++ b/module/id/updatable_provider.go @@ -6,6 +6,10 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// UpdatableIDProvider implements an IdentityProvider which can be manually updated by setting +// the IdentityList to a new value. +// It also implements an IdentifierProvider which provides the identifiers of the IdentityList. +// This is mainly used to simulate epoch transitions in tests. type UpdatableIDProvider struct { mu sync.RWMutex identities flow.IdentityList @@ -15,6 +19,7 @@ func NewUpdatableIDProvider(identities flow.IdentityList) *UpdatableIDProvider { return &UpdatableIDProvider{identities: identities} } +// SetIdentities updates the IdentityList returned by this provider. func (p *UpdatableIDProvider) SetIdentities(identities flow.IdentityList) { p.mu.Lock() defer p.mu.Unlock() diff --git a/network/middleware.go b/network/middleware.go index b6d538da53a..ea2c157eb51 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -53,6 +53,8 @@ type Middleware interface { // and updates the underlying libp2p node. UpdateAllowList() + // UpdateNodeAddresses fetches and updates the addresses of all the staked participants + // in the Flow protocol. UpdateNodeAddresses() } @@ -61,8 +63,11 @@ type Middleware interface { type Overlay interface { // Topology returns an identity list of nodes which this node should be directly connected to as peers Topology() (flow.IdentityList, error) - Receive(nodeID flow.Identifier, msg *message.Message) error + + // Identities returns a list of all Flow identities on the network Identities() flow.IdentityList + + Receive(nodeID flow.Identifier, msg *message.Message) error } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index cc01eedaafb..4d2cebcb00e 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -43,6 +43,8 @@ func (suite *DHTTestSuite) TearDownTest() { suite.cancel() } +// TestFindPeerWithDHT checks that if a node is configured to participate in the DHT, it is +// able to create new streams with peers even without knowing their address info beforehand. func (suite *DHTTestSuite) TestFindPeerWithDHT() { count := 10 golog.SetAllLoggers(golog.LevelFatal) // change this to Debug if libp2p logs are needed @@ -59,11 +61,13 @@ func (suite *DHTTestSuite) TestFindPeerWithDHT() { return peer.AddrInfo{ID: dhtServerNodes[i].host.ID(), Addrs: dhtServerNodes[i].host.Addrs()} } + // connect even numbered clients to the first DHT server, and odd number clients to the second for i, clientNode := range dhtClientNodes { err := clientNode.host.Connect(suite.ctx, getDhtServerAddr(uint(i%2))) require.NoError(suite.T(), err) } + // wait for clients to connect to DHT servers and update their routing tables require.Eventually(suite.T(), func() bool { for i, clientNode := range dhtClientNodes { if clientNode.dht.RoutingTable().Find(getDhtServerAddr(uint(i%2)).ID) == "" { @@ -73,16 +77,24 @@ func (suite *DHTTestSuite) TestFindPeerWithDHT() { return true }, time.Second*5, tickForAssertEventually, "nodes failed to connect") + // connect the two DHT servers to each other err := dhtServerNodes[0].host.Connect(suite.ctx, getDhtServerAddr(1)) require.NoError(suite.T(), err) + // wait for the first server to connect to the second and update its routing table require.Eventually(suite.T(), func() bool { return dhtServerNodes[0].dht.RoutingTable().Find(getDhtServerAddr(1).ID) != "" }, time.Second*5, tickForAssertEventually, "dht servers failed to connect") + // check that all even numbered clients can create streams with all odd numbered clients for i := 0; i < len(dhtClientNodes); i += 2 { for j := 1; j < len(dhtClientNodes); j += 2 { + // client i should not yet know the address of client j, but we clear any addresses + // here just in case. dhtClientNodes[i].host.Peerstore().ClearAddrs(dhtClientNodes[j].host.ID()) + + // Try to create a stream from client i to client j. This should resort to a DHT + // lookup since client i does not know client j's address. _, err = dhtClientNodes[i].CreateStream(suite.ctx, dhtClientNodes[j].host.ID()) require.NoError(suite.T(), err) } diff --git a/network/p2p/fixed_translator.go b/network/p2p/fixed_translator.go index ecfff75fbb1..89bae14c743 100644 --- a/network/p2p/fixed_translator.go +++ b/network/p2p/fixed_translator.go @@ -8,6 +8,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// FixedTableIdentityTranslator implements an IDTranslator which translates ID's for a +// fixed list of identities. type FixedTableIdentityTranslator struct { flow2p2p map[flow.Identifier]peer.ID p2p2flow map[peer.ID]flow.Identifier diff --git a/network/p2p/hierarchical_translator.go b/network/p2p/hierarchical_translator.go index 4e4d0e77cd5..0a3d98174ce 100644 --- a/network/p2p/hierarchical_translator.go +++ b/network/p2p/hierarchical_translator.go @@ -9,6 +9,10 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// HierarchicalIDTranslator implements an IDTranslator which combines the ID translation +// capabilities of multiple IDTranslators. +// When asked to translate an ID, it will iterate through all of the IDTranslators it was +// given and return the first successful translation. type HierarchicalIDTranslator struct { translators []IDTranslator } @@ -26,15 +30,17 @@ func (t *HierarchicalIDTranslator) GetPeerID(flowID flow.Identifier) (peer.ID, e } errs = multierror.Append(errs, err) } - return "", fmt.Errorf("could not find corresponding peer ID for flow ID %v: %w", flowID, errs) + return "", fmt.Errorf("could not translate the given flow ID: %w", errs) } func (t *HierarchicalIDTranslator) GetFlowID(peerID peer.ID) (flow.Identifier, error) { + var errs *multierror.Error for _, translator := range t.translators { fid, err := translator.GetFlowID(peerID) if err == nil { return fid, nil } + errs = multierror.Append(errs, err) } - return flow.ZeroID, fmt.Errorf("could not find corresponding flow ID for peer ID %v", peerID) + return flow.ZeroID, fmt.Errorf("could not translate the given peer ID: %w", errs) } diff --git a/network/p2p/id_translator.go b/network/p2p/id_translator.go index 8edb2d500d5..a07c23f56da 100644 --- a/network/p2p/id_translator.go +++ b/network/p2p/id_translator.go @@ -6,7 +6,12 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// IDTranslator provides an interface for converting from Flow ID's to LibP2P peer ID's +// and vice versa. type IDTranslator interface { + // GetPeerID returns the peer ID for the given Flow ID GetPeerID(flow.Identifier) (peer.ID, error) + + // GetFlowID returns the Flow ID for the given peer ID GetFlowID(peer.ID) (flow.Identifier, error) } diff --git a/network/p2p/identity_provider_translator.go b/network/p2p/identity_provider_translator.go index 61504b1ef05..61aa33ecf0f 100644 --- a/network/p2p/identity_provider_translator.go +++ b/network/p2p/identity_provider_translator.go @@ -10,11 +10,13 @@ import ( "github.com/onflow/flow-go/module/id" ) -type IdentityProviderIdentityTranslator struct { +// IdentityProviderIDTranslator implements an IDTranslator which provides ID +// translation capabilities for an IdentityProvider. +type IdentityProviderIDTranslator struct { idProvider id.IdentityProvider } -func (t *IdentityProviderIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { +func (t *IdentityProviderIDTranslator) GetFlowID(p peer.ID) (flow.Identifier, error) { key, err := p.ExtractPublicKey() if err != nil { return flow.ZeroID, err @@ -30,7 +32,7 @@ func (t *IdentityProviderIdentityTranslator) GetFlowID(p peer.ID) (flow.Identifi return ids[0].NodeID, nil } -func (t *IdentityProviderIdentityTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { +func (t *IdentityProviderIDTranslator) GetPeerID(n flow.Identifier) (peer.ID, error) { ids := t.idProvider.Identities(filter.HasNodeID(n)) if len(ids) == 0 { return "", fmt.Errorf("could not find identity with id %v", n.String()) @@ -46,6 +48,6 @@ func (t *IdentityProviderIdentityTranslator) GetPeerID(n flow.Identifier) (peer. return pid, nil } -func NewIdentityProviderIdentityTranslator(provider id.IdentityProvider) *IdentityProviderIdentityTranslator { - return &IdentityProviderIdentityTranslator{provider} +func NewIdentityProviderIDTranslator(provider id.IdentityProvider) *IdentityProviderIDTranslator { + return &IdentityProviderIDTranslator{provider} } diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index db380d94226..4522eaa0e4a 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -354,25 +354,24 @@ func (n *Node) RemovePeer(ctx context.Context, peerID peer.ID) error { // CreateStream returns an existing stream connected to the peer if it exists, or creates a new stream with it. func (n *Node) CreateStream(ctx context.Context, peerID peer.ID) (libp2pnet.Stream, error) { - if len(n.host.Peerstore().Addrs(peerID)) == 0 { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore") - - if n.dht != nil { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("searching for peer in dht") - - var err error - func() { - timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) - defer cancel() - // try to find the peer using the dht - _, err = n.dht.FindPeer(timedCtx, peerID) - }() - - if err != nil { - n.logger.Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") - } else { - n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address found") - } + // If we do not currently have any addresses for the given peer, stream creation will almost + // certainly fail. If this Node was configure with a DHT, we can try to lookup the address of + // the peer in the DHT as a last resort. + if len(n.host.Peerstore().Addrs(peerID)) == 0 && n.dht != nil { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("address not found in peerstore, searching for peer in dht") + + var err error + func() { + timedCtx, cancel := context.WithTimeout(ctx, findPeerQueryTimeout) + defer cancel() + // try to find the peer using the dht + _, err = n.dht.FindPeer(timedCtx, peerID) + }() + + if err != nil { + n.logger.Warn().Err(err).Str("peerID", peerID.Pretty()).Msg("could not find addresses") + } else { + n.logger.Info().Str("peerID", peerID.Pretty()).Msg("addresses found") } } // Open libp2p Stream with the remote peer (will use an existing TCP connection underneath if it exists) diff --git a/network/p2p/libp2pUtils.go b/network/p2p/libp2pUtils.go index 04a2996c2aa..e37b4460b1b 100644 --- a/network/p2p/libp2pUtils.go +++ b/network/p2p/libp2pUtils.go @@ -233,6 +233,7 @@ func flowStream(conn network.Conn) network.Stream { return nil } +// ExtractPeerID extracts the LibP2P peer ID associated with the given Flow public key. func ExtractPeerID(networkPubKey fcrypto.PublicKey) (pid peer.ID, err error) { pk, err := LibP2PPublicKeyFromFlow(networkPubKey) if err != nil { diff --git a/network/p2p/peerstore_provider.go b/network/p2p/peerstore_provider.go index c31b2cf6917..3f665589a10 100644 --- a/network/p2p/peerstore_provider.go +++ b/network/p2p/peerstore_provider.go @@ -7,6 +7,8 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// PeerstoreIdentifierProvider implements an IdentifierProvider which provides the identifiers +// of the peers present in the given LibP2P host's peerstore. type PeerstoreIdentifierProvider struct { host host.Host idTranslator IDTranslator @@ -24,7 +26,9 @@ func NewPeerstoreIdentifierProvider(logger zerolog.Logger, host host.Host, idTra func (p *PeerstoreIdentifierProvider) Identifiers() flow.IdentifierList { var result flow.IdentifierList + // get all peers with addresses from the peerstore pids := p.host.Peerstore().PeersWithAddrs() + for _, pid := range pids { flowID, err := p.idTranslator.GetFlowID(pid) if err != nil { diff --git a/network/p2p/protocol_state_provider.go b/network/p2p/protocol_state_provider.go index b1a5bdcf834..ad678e6babe 100644 --- a/network/p2p/protocol_state_provider.go +++ b/network/p2p/protocol_state_provider.go @@ -13,6 +13,8 @@ import ( "github.com/onflow/flow-go/state/protocol/events" ) +// ProtocolStateIDCache implements an IdentityProvider and IDTranslator for the set of staked +// Flow network participants as according to the given `protocol.State`. type ProtocolStateIDCache struct { events.Noop identities flow.IdentityList @@ -59,6 +61,9 @@ func (p *ProtocolStateIDCache) EpochCommittedPhaseStarted(currentEpochCounter ui p.update(header.ID()) } +// update updates the cached identities stored in this provider. +// This is called whenever an epoch event occurs, signaling a possible change in +// protocol state identities. func (p *ProtocolStateIDCache) update(blockID flow.Identifier) { p.logger.Info().Str("blockID", blockID.String()).Msg("updating cached identities") diff --git a/network/p2p/protocol_state_provider_test.go b/network/p2p/protocol_state_provider_test.go index 55ac4074a5e..22167d83757 100644 --- a/network/p2p/protocol_state_provider_test.go +++ b/network/p2p/protocol_state_provider_test.go @@ -59,6 +59,7 @@ func (suite *ProtocolStateProviderTestSuite) SetupTest() { suite.provider = provider } +// triggerUpdate simulates an epoch transition func (suite *ProtocolStateProviderTestSuite) triggerUpdate() { suite.participants = unittest.IdentityListFixture(5, unittest.WithAllRoles(), unittest.WithKeys) @@ -97,6 +98,8 @@ func TestProtocolStateProvider(t *testing.T) { suite.Run(t, new(ProtocolStateProviderTestSuite)) } +// checkStateTransition triggers an epoch transition and checks that the updated +// state is reflected by the provider being tested. func (suite *ProtocolStateProviderTestSuite) checkStateTransition() { oldParticipants := suite.participants diff --git a/network/p2p/unstaked_translator.go b/network/p2p/unstaked_translator.go index 400d77fd04c..50802126611 100644 --- a/network/p2p/unstaked_translator.go +++ b/network/p2p/unstaked_translator.go @@ -10,6 +10,14 @@ import ( "github.com/onflow/flow-go/model/flow" ) +// UnstakedNetworkIDTranslator implements an IDTranslator which translates IDs for peers +// on the unstaked network. +// On the unstaked network, a Flow ID is derived from a peer ID by extracting the public +// key from the peer ID, dropping the first byte (parity byte), and using the remaining +// 32 bytes as the Flow ID. +// Network keys for unstaked nodes must be generated using the Secp256k1 curve, and must +// be positive. It is assumed that these requirements are enforced during key generation, +// and any peer ID's which don't follow these conventions are considered invalid. type UnstakedNetworkIDTranslator struct{} func NewUnstakedNetworkIDTranslator() *UnstakedNetworkIDTranslator { diff --git a/network/test/epochtransition_test.go b/network/test/epochtransition_test.go index 26de777dfa7..d63351a54ef 100644 --- a/network/test/epochtransition_test.go +++ b/network/test/epochtransition_test.go @@ -120,6 +120,7 @@ func TestEpochTransitionTestSuite(t *testing.T) { t.Skip(fmt.Sprintf("test is flaky: %v", &MutableIdentityTableSuite{})) } +// signalIdentityChanged update IDs for all the current set of nodes (simulating an epoch) func (suite *MutableIdentityTableSuite) signalIdentityChanged() { for _, n := range suite.testNodes.nodes { n.mw.UpdateNodeAddresses() diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 3b33df7f312..29c9d980e4a 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -476,7 +476,10 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { require.Error(m.Suite.T(), err) } +// TestUpdateNodeAddresses tests that the UpdateNodeAddresses method correctly updates +// the addresses of the staked network participants. func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { + // create a new staked identity ids, mws, _, providers := GenerateIDsAndMiddlewares(m.T(), 1, false, m.logger) require.Len(m.T(), ids, 1) require.Len(m.T(), providers, 1) @@ -501,12 +504,16 @@ func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") + // message should fail to send because no address is known yet + // for the new identity err := m.mws[0].SendDirect(msg, newId.NodeID) require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) + // update the addresses m.ids = idList m.mws[0].UpdateNodeAddresses() + // now the message should send successfully err = m.mws[0].SendDirect(msg, newId.NodeID) require.NoError(m.T(), err) } diff --git a/network/test/testUtil.go b/network/test/testUtil.go index dae530624f5..af5ecdd65ba 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -149,7 +149,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id p2p.DefaultUnicastTimeout, true, true, - p2p.NewIdentityProviderIdentityTranslator(idProviders[i]), + p2p.NewIdentityProviderIDTranslator(idProviders[i]), p2p.WithIdentifierProvider( idProviders[i], ), From 787fedb7f1d2684bdd8b1f557e9bc599ae267575 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Tue, 24 Aug 2021 22:56:31 -0400 Subject: [PATCH 220/291] [network] test for peerstore ID provider --- network/test/peerstore_provider_test.go | 114 ++++++++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 network/test/peerstore_provider_test.go diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go new file mode 100644 index 00000000000..26fd1dfcf9b --- /dev/null +++ b/network/test/peerstore_provider_test.go @@ -0,0 +1,114 @@ +package test + +import ( + "context" + "math/rand" + "os" + "strconv" + "testing" + "time" + + "github.com/ipfs/go-log" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/multiformats/go-multiaddr" + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/utils/unittest" +) + +type PeerStoreProviderTestSuite struct { + suite.Suite + logger zerolog.Logger + nodes []*p2p.Node + libp2pPeersIDs []peer.ID + peerIDprovider *p2p.PeerstoreIdentifierProvider + translator *p2p.HierarchicalIDTranslator + ids flow.IdentityList +} + +func TestPeerStoreProviderTestSuite(t *testing.T) { + suite.Run(t, new(PeerStoreProviderTestSuite)) +} + +const nodeCount = 2 +const peerCount = 3 +const testNodeIndex = 0 // testNodeIndex < nodeCount + +func (suite *PeerStoreProviderTestSuite) SetupTest() { + suite.logger = zerolog.New(os.Stderr).Level(zerolog.ErrorLevel) + log.SetAllLoggers(log.LevelError) + ctx := context.Background() + + suite.ids, suite.nodes, _ = GenerateIDs(suite.T(), suite.logger, nodeCount, !DryRun) + t, err := p2p.NewFixedTableIdentityTranslator(suite.ids) + require.NoError(suite.T(), err) + + u := p2p.NewUnstakedNetworkIDTranslator() + suite.translator = p2p.NewHierarchicalIDTranslator(u, t) + + // emulate the middleware behavior in populating the testnode's peer store + libp2pPeers := make([]peer.ID, peerCount) + for i := 0; i < peerCount; i++ { + peerAddrInfo := suite.randomPeerInfoWithStubNetwork() + err := suite.nodes[testNodeIndex].AddPeer(ctx, peerAddrInfo) + // conn gater (then connection routine) will complain + require.Error(suite.T(), err) + libp2pPeers[i] = peerAddrInfo.ID + } + suite.libp2pPeersIDs = libp2pPeers + suite.peerIDprovider = p2p.NewPeerstoreIdentifierProvider(suite.logger, suite.nodes[testNodeIndex].Host(), suite.translator) + + // sanity checks + assert.Len(suite.T(), suite.nodes, nodeCount) + assert.Len(suite.T(), suite.libp2pPeersIDs, peerCount) + assert.Len(suite.T(), suite.ids, nodeCount) + + // give the Node the time to establish those connections + time.Sleep(2 * time.Second) +} + +func (suite *PeerStoreProviderTestSuite) TestTranslationPeers() { + + identifiers := suite.peerIDprovider.Identifiers() + + peerIDs := make([]peer.ID, len(identifiers)) + for i := 0; i < len(identifiers); i++ { + + pID, err := suite.translator.GetPeerID(identifiers[i]) + require.NoError(suite.T(), err) + peerIDs[i] = pID + } + // check we can find the libp2p peers + assert.Subset(suite.T(), peerIDs, suite.libp2pPeersIDs, "peer IDs should include those in the peer Store") + +} + +func (suite *PeerStoreProviderTestSuite) randomPeerInfoWithStubNetwork() peer.AddrInfo { + + // we don't care about network information, but those peers need an address + ip := "127.0.0.1" + port := strconv.Itoa(rand.Intn(65535 - 1024)) + + addr := p2p.MultiAddressStr(ip, port) + maddr, err := multiaddr.NewMultiaddr(addr) + require.NoError(suite.T(), err) + + privKey, err := utils.GenerateUnstakedNetworkingKey(unittest.SeedFixture(crypto.KeyGenSeedMinLenECDSASecp256k1)) + require.NoError(suite.T(), err) + + libp2pKey, err := p2p.LibP2PPublicKeyFromFlow(privKey.PublicKey()) + require.NoError(suite.T(), err) + + id, err := peer.IDFromPublicKey(libp2pKey) + require.NoError(suite.T(), err) + + pInfo := peer.AddrInfo{ID: id, Addrs: []multiaddr.Multiaddr{maddr}} + return pInfo +} From 3d7b11992fb9993b6125b657b61c0648722642ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Tue, 24 Aug 2021 23:11:38 -0400 Subject: [PATCH 221/291] [network] Correct bug in ID validation Introduced in c8bce7815e770bd5786e5c03c1476d923468c550 The From field of a message is not necessarily the depository of the authentified signing information of a message: it can be an attached public key that may or may not match (hash to) the libp2p PeerID. This reinstates the pid.MatchesPublicKey(pubk) call that checks that. --- network/p2p/readSubscription.go | 43 ++++++++++++++++++++++++++++++++- 1 file changed, 42 insertions(+), 1 deletion(-) diff --git a/network/p2p/readSubscription.go b/network/p2p/readSubscription.go index db87b097afa..681721286cb 100644 --- a/network/p2p/readSubscription.go +++ b/network/p2p/readSubscription.go @@ -2,9 +2,11 @@ package p2p import ( "context" + "fmt" "strings" "sync" + lcrypto "github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p-core/peer" pubsub "github.com/libp2p/go-libp2p-pubsub" @@ -79,7 +81,7 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { return } - pid, err := peer.IDFromBytes(rawMsg.From) + pid, err := messageSigningID(rawMsg) if err != nil { r.log.Err(err).Msg("failed to validate peer ID of incoming message") return @@ -102,3 +104,42 @@ func (r *readSubscription) receiveLoop(wg *sync.WaitGroup) { r.callback(&msg, pid) } } + +// messagePubKey extracts the public key of the envelope signer from a libp2p message. +// The location of that key depends on the type of the key, see: +// https://github.com/libp2p/specs/blob/master/peer-ids/peer-ids.md +// This reproduces the exact logic of the private function doing the same decoding in libp2p: +// https://github.com/libp2p/go-libp2p-pubsub/blob/ba28f8ecfc551d4d916beb748d3384951bce3ed0/sign.go#L77 +func messageSigningID(m *pubsub.Message) (peer.ID, error) { + var pubk lcrypto.PubKey + + // m.From is the original sender of the message (versus `m.ReceivedFrom` which is the last hop which sent us this message) + pid, err := peer.IDFromBytes(m.From) + if err != nil { + return "", err + } + + if m.Key == nil { + // no attached key, it must be extractable from the source ID + pubk, err = pid.ExtractPublicKey() + if err != nil { + return "", fmt.Errorf("cannot extract signing key: %s", err.Error()) + } + if pubk == nil { + return "", fmt.Errorf("cannot extract signing key") + } + } else { + pubk, err = lcrypto.UnmarshalPublicKey(m.Key) + if err != nil { + return "", fmt.Errorf("cannot unmarshal signing key: %s", err.Error()) + } + + // verify that the source ID matches the attached key + if !pid.MatchesPublicKey(pubk) { + return "", fmt.Errorf("bad signing key; source ID %s doesn't match key", pid) + } + } + + // the pid either contains or matches the signing pubKey + return pid, nil +} From 98bb01237334f97726e733a1ec39a3f676770a87 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 25 Aug 2021 09:36:05 -0400 Subject: [PATCH 222/291] [network] Move UpdatableIDProvider to tests --- network/test/middleware_test.go | 3 +-- network/test/testUtil.go | 9 ++++----- {module/id => network/test}/updatable_provider.go | 2 +- 3 files changed, 6 insertions(+), 8 deletions(-) rename {module/id => network/test}/updatable_provider.go (98%) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 29c9d980e4a..c177ade677f 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -19,7 +19,6 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" - "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/observable" "github.com/onflow/flow-go/network/codec/cbor" @@ -66,7 +65,7 @@ type MiddlewareTestSuite struct { ids []*flow.Identity metrics *metrics.NoopCollector // no-op performance monitoring simulation logger zerolog.Logger - providers []*id.UpdatableIDProvider + providers []*UpdatableIDProvider } // TestMiddlewareTestSuit runs all the test methods in this test suit diff --git a/network/test/testUtil.go b/network/test/testUtil.go index af5ecdd65ba..ed7acdbb946 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -20,7 +20,6 @@ import ( "github.com/onflow/flow-go/model/flow/filter" message "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/id" idModule "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" "github.com/onflow/flow-go/module/metrics" @@ -123,10 +122,10 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*id.UpdatableIDProvider) { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) - idProviders := make([]*id.UpdatableIDProvider, len(identities)) + idProviders := make([]*UpdatableIDProvider, len(identities)) for i, id := range identities { // casts libP2PNode instance to a local variable to avoid closure @@ -137,7 +136,7 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id return node, nil } - idProviders[i] = idModule.NewUpdatableIDProvider(identities) + idProviders[i] = NewUpdatableIDProvider(identities) // creating middleware of nodes mws[i] = p2p.NewMiddleware(logger, @@ -224,7 +223,7 @@ func GenerateNetworks(t *testing.T, func GenerateIDsAndMiddlewares(t *testing.T, n int, dryRunMode bool, - logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*id.UpdatableIDProvider) { + logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*UpdatableIDProvider) { ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) diff --git a/module/id/updatable_provider.go b/network/test/updatable_provider.go similarity index 98% rename from module/id/updatable_provider.go rename to network/test/updatable_provider.go index d35a3da0566..7c4377a1ae5 100644 --- a/module/id/updatable_provider.go +++ b/network/test/updatable_provider.go @@ -1,4 +1,4 @@ -package id +package test import ( "sync" From f0db946293b12a8283624b9b2583116fe9f1e5c3 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 10:06:17 -0700 Subject: [PATCH 223/291] add NotEjectedFilter to sync engine id provider --- cmd/scaffold.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index cdab28d2495..dc2d53549f6 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -438,6 +438,7 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { filter.And( filter.HasRole(flow.RoleConsensus), filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NotEjectedFilter, ), idCache, ) From 4915975e05b45db97fa9cb7a84e2cf6962bbcf3c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 10:29:31 -0700 Subject: [PATCH 224/291] update assert.Subset to assert.ElementsMatch --- network/test/peerstore_provider_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index 26fd1dfcf9b..959b0e34c36 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -86,7 +86,7 @@ func (suite *PeerStoreProviderTestSuite) TestTranslationPeers() { peerIDs[i] = pID } // check we can find the libp2p peers - assert.Subset(suite.T(), peerIDs, suite.libp2pPeersIDs, "peer IDs should include those in the peer Store") + assert.ElementsMatch(suite.T(), peerIDs, append(suite.libp2pPeersIDs, suite.nodes[testNodeIndex].Host().ID()), "peer IDs should include those in the peer Store") } From c8c15a9d75192dbb177f8a30fffb717e9e4f5121 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 10:29:49 -0700 Subject: [PATCH 225/291] remove time.Sleep --- network/test/peerstore_provider_test.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index 959b0e34c36..093c55072a0 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -6,7 +6,6 @@ import ( "os" "strconv" "testing" - "time" "github.com/ipfs/go-log" "github.com/libp2p/go-libp2p-core/peer" @@ -70,8 +69,6 @@ func (suite *PeerStoreProviderTestSuite) SetupTest() { assert.Len(suite.T(), suite.libp2pPeersIDs, peerCount) assert.Len(suite.T(), suite.ids, nodeCount) - // give the Node the time to establish those connections - time.Sleep(2 * time.Second) } func (suite *PeerStoreProviderTestSuite) TestTranslationPeers() { From a9c53406829b7c606147b21b8b973e85ad4e8f79 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 19:47:29 -0700 Subject: [PATCH 226/291] fix flakiness in peerstore provider test --- network/test/peerstore_provider_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index 093c55072a0..f0d652efa02 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -50,7 +50,7 @@ func (suite *PeerStoreProviderTestSuite) SetupTest() { require.NoError(suite.T(), err) u := p2p.NewUnstakedNetworkIDTranslator() - suite.translator = p2p.NewHierarchicalIDTranslator(u, t) + suite.translator = p2p.NewHierarchicalIDTranslator(t, u) // emulate the middleware behavior in populating the testnode's peer store libp2pPeers := make([]peer.ID, peerCount) From 9c234dc333540408c2ee4fca032ccbb7ace2f704 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Wed, 25 Aug 2021 21:29:15 -0700 Subject: [PATCH 227/291] Fix flaky middleware test --- network/test/middleware_test.go | 107 +++++++++++++----------- network/test/peerstore_provider_test.go | 2 +- network/test/testUtil.go | 23 +++-- network/topology/topology_test.go | 10 +-- 4 files changed, 77 insertions(+), 65 deletions(-) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index c177ade677f..1fbaf45645b 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -58,6 +58,7 @@ func (co *tagsObserver) OnComplete() { type MiddlewareTestSuite struct { suite.Suite + sync.RWMutex size int // used to determine number of middlewares under test mws []*p2p.Middleware // used to keep track of middlewares under test ov []*mocknetwork.Overlay @@ -70,6 +71,7 @@ type MiddlewareTestSuite struct { // TestMiddlewareTestSuit runs all the test methods in this test suit func TestMiddlewareTestSuite(t *testing.T) { + t.Parallel() suite.Run(t, new(MiddlewareTestSuite)) } @@ -111,18 +113,66 @@ func (m *MiddlewareTestSuite) SetupTest() { } } +// TestUpdateNodeAddresses tests that the UpdateNodeAddresses method correctly updates +// the addresses of the staked network participants. +func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { + // create a new staked identity + ids, libP2PNodes, _ := GenerateIDs(m.T(), m.logger, 1, false, false) + mws, providers := GenerateMiddlewares(m.T(), m.logger, ids, libP2PNodes, false) + require.Len(m.T(), ids, 1) + require.Len(m.T(), providers, 1) + require.Len(m.T(), mws, 1) + newId := ids[0] + newMw := mws[0] + // newProvider := providers[0] + defer newMw.Stop() + + overlay := m.createOverlay() + overlay.On("Receive", + m.ids[0].NodeID, + mock.AnythingOfType("*message.Message"), + ).Return(nil) + assert.NoError(m.T(), newMw.Start(overlay)) + + idList := flow.IdentityList(append(m.ids, newId)) + + // needed to enable ID translation + m.providers[0].SetIdentities(idList) + m.mws[0].UpdateAllowList() + + msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") + + // message should fail to send because no address is known yet + // for the new identity + err := m.mws[0].SendDirect(msg, newId.NodeID) + require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) + + // update the addresses + m.Lock() + m.ids = idList + m.Unlock() + // newProvider.SetIdentities(idList) + // newMw.UpdateAllowList() + m.mws[0].UpdateNodeAddresses() + + // now the message should send successfully + err = m.mws[0].SendDirect(msg, newId.NodeID) + require.NoError(m.T(), err) +} + func (m *MiddlewareTestSuite) createOverlay() *mocknetwork.Overlay { overlay := &mocknetwork.Overlay{} - - overlay.On("Identities").Maybe().Return(func() flow.IdentityList { - return flow.IdentityList(m.ids) - }, nil) - overlay.On("Topology").Maybe().Return(func() flow.IdentityList { - return flow.IdentityList(m.ids) - }, nil) + overlay.On("Identities").Maybe().Return(m.getIds, nil) + overlay.On("Topology").Maybe().Return(m.getIds, nil) return overlay } +func (m *MiddlewareTestSuite) getIds() flow.IdentityList { + m.RLock() + defer m.RUnlock() + return flow.IdentityList(m.ids) +} + func (m *MiddlewareTestSuite) TearDownTest() { m.stopMiddlewares() } @@ -174,7 +224,6 @@ func (m *MiddlewareTestSuite) TestMultiPing() { // expectID and expectPayload are what we expect the receiver side to evaluate the // incoming ping against, it can be mocked or typed data func (m *MiddlewareTestSuite) Ping(expectID, expectPayload interface{}) { - ch := make(chan struct{}) // extracts sender id based on the mock option var err error @@ -475,48 +524,6 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { require.Error(m.Suite.T(), err) } -// TestUpdateNodeAddresses tests that the UpdateNodeAddresses method correctly updates -// the addresses of the staked network participants. -func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { - // create a new staked identity - ids, mws, _, providers := GenerateIDsAndMiddlewares(m.T(), 1, false, m.logger) - require.Len(m.T(), ids, 1) - require.Len(m.T(), providers, 1) - require.Len(m.T(), mws, 1) - newId := ids[0] - newMw := mws[0] - newProvider := providers[0] - - idList := flow.IdentityList(append(m.ids, newId)) - - newProvider.SetIdentities(idList) - overlay := m.createOverlay() - overlay.On("Receive", - m.ids[0].NodeID, - mock.AnythingOfType("*message.Message"), - ).Return(nil) - assert.NoError(m.T(), newMw.Start(overlay)) - - // needed to enable ID translation - m.providers[0].SetIdentities(idList) - m.mws[0].UpdateAllowList() - - msg := createMessage(m.ids[0].NodeID, newId.NodeID, "hello") - - // message should fail to send because no address is known yet - // for the new identity - err := m.mws[0].SendDirect(msg, newId.NodeID) - require.ErrorIs(m.T(), err, swarm.ErrNoAddresses) - - // update the addresses - m.ids = idList - m.mws[0].UpdateNodeAddresses() - - // now the message should send successfully - err = m.mws[0].SendDirect(msg, newId.NodeID) - require.NoError(m.T(), err) -} - // TestUnsubscribe tests that an engine can unsubscribe from a topic it was earlier subscribed to and stop receiving // messages. func (m *MiddlewareTestSuite) TestUnsubscribe() { diff --git a/network/test/peerstore_provider_test.go b/network/test/peerstore_provider_test.go index f0d652efa02..03ef68a4785 100644 --- a/network/test/peerstore_provider_test.go +++ b/network/test/peerstore_provider_test.go @@ -45,7 +45,7 @@ func (suite *PeerStoreProviderTestSuite) SetupTest() { log.SetAllLoggers(log.LevelError) ctx := context.Background() - suite.ids, suite.nodes, _ = GenerateIDs(suite.T(), suite.logger, nodeCount, !DryRun) + suite.ids, suite.nodes, _ = GenerateIDs(suite.T(), suite.logger, nodeCount, !DryRun, true) t, err := p2p.NewFixedTableIdentityTranslator(suite.ids) require.NoError(suite.T(), err) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index ed7acdbb946..259efa1fdea 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -94,7 +94,7 @@ func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics // GenerateIDs is a test helper that generate flow identities with a valid port and libp2p nodes. // If `dryRunMode` is set to true, it returns an empty slice instead of libp2p nodes, assuming that slice is never going // to get used. -func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Node, []observable.Observable) { +func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode, connGating bool, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Node, []observable.Observable) { libP2PNodes := make([]*p2p.Node, n) tagObservables := make([]observable.Observable, n) @@ -108,7 +108,7 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op port := "0" if !dryRunMode { - libP2PNodes[i], tagObservables[i] = generateLibP2PNode(t, logger, *id, key) + libP2PNodes[i], tagObservables[i] = generateLibP2PNode(t, logger, *id, key, connGating) _, port, err = libP2PNodes[i].GetIPPort() require.NoError(t, err) @@ -122,7 +122,7 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op } // GenerateMiddlewares creates and initializes middleware instances for all the identities -func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node) ([]*p2p.Middleware, []*UpdatableIDProvider) { +func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.IdentityList, libP2PNodes []*p2p.Node, enablePeerManagementAndConnectionGating bool) ([]*p2p.Middleware, []*UpdatableIDProvider) { metrics := metrics.NewNoopCollector() mws := make([]*p2p.Middleware, len(identities)) idProviders := make([]*UpdatableIDProvider, len(identities)) @@ -146,8 +146,8 @@ func GenerateMiddlewares(t *testing.T, logger zerolog.Logger, identities flow.Id rootBlockID, p2p.DefaultPeerUpdateInterval, p2p.DefaultUnicastTimeout, - true, - true, + enablePeerManagementAndConnectionGating, + enablePeerManagementAndConnectionGating, p2p.NewIdentityProviderIDTranslator(idProviders[i]), p2p.WithIdentifierProvider( idProviders[i], @@ -225,8 +225,8 @@ func GenerateIDsAndMiddlewares(t *testing.T, dryRunMode bool, logger zerolog.Logger, opts ...func(*flow.Identity)) (flow.IdentityList, []*p2p.Middleware, []observable.Observable, []*UpdatableIDProvider) { - ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, opts...) - mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes) + ids, libP2PNodes, protectObservables := GenerateIDs(t, logger, n, dryRunMode, true, opts...) + mws, providers := GenerateMiddlewares(t, logger, ids, libP2PNodes, true) return ids, mws, protectObservables, providers } @@ -258,7 +258,9 @@ func GenerateEngines(t *testing.T, nets []*p2p.Network) []*MeshEngine { func generateLibP2PNode(t *testing.T, logger zerolog.Logger, id flow.Identity, - key crypto.PrivateKey) (*p2p.Node, observable.Observable) { + key crypto.PrivateKey, + connGating bool, +) (*p2p.Node, observable.Observable) { noopMetrics := metrics.NewNoopCollector() @@ -267,7 +269,10 @@ func generateLibP2PNode(t *testing.T, pingInfoProvider.On("SealedBlockHeight").Return(uint64(1000)) ctx := context.Background() - connGater := p2p.NewConnGater(logger) + var connGater *p2p.ConnGater = nil + if connGating { + connGater = p2p.NewConnGater(logger) + } // Inject some logic to be able to observe connections of this node connManager := NewTagWatchingConnManager(logger, noopMetrics) diff --git a/network/topology/topology_test.go b/network/topology/topology_test.go index 80a7417b6fe..db2d4f5ce65 100644 --- a/network/topology/topology_test.go +++ b/network/topology/topology_test.go @@ -138,11 +138,11 @@ func (suite *TopologyTestSuite) generateSystem(acc, col, con, exe, ver, cluster flow.IdentityList, []network.SubscriptionManager) { - collector, _, _ := test.GenerateIDs(suite.T(), suite.logger, col, test.DryRun, unittest.WithRole(flow.RoleCollection)) - access, _, _ := test.GenerateIDs(suite.T(), suite.logger, acc, test.DryRun, unittest.WithRole(flow.RoleAccess)) - consensus, _, _ := test.GenerateIDs(suite.T(), suite.logger, con, test.DryRun, unittest.WithRole(flow.RoleConsensus)) - verification, _, _ := test.GenerateIDs(suite.T(), suite.logger, ver, test.DryRun, unittest.WithRole(flow.RoleVerification)) - execution, _, _ := test.GenerateIDs(suite.T(), suite.logger, exe, test.DryRun, unittest.WithRole(flow.RoleExecution)) + collector, _, _ := test.GenerateIDs(suite.T(), suite.logger, col, test.DryRun, true, unittest.WithRole(flow.RoleCollection)) + access, _, _ := test.GenerateIDs(suite.T(), suite.logger, acc, test.DryRun, true, unittest.WithRole(flow.RoleAccess)) + consensus, _, _ := test.GenerateIDs(suite.T(), suite.logger, con, test.DryRun, true, unittest.WithRole(flow.RoleConsensus)) + verification, _, _ := test.GenerateIDs(suite.T(), suite.logger, ver, test.DryRun, true, unittest.WithRole(flow.RoleVerification)) + execution, _, _ := test.GenerateIDs(suite.T(), suite.logger, exe, test.DryRun, true, unittest.WithRole(flow.RoleExecution)) ids := flow.IdentityList{} ids = ids.Union(collector) From f5622b92c06815915ffba5dcfe36a5ba2582a587 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 6 Aug 2021 22:18:19 -0700 Subject: [PATCH 228/291] init --- .../node_builder/access_node_builder.go | 19 +- .../staked_access_node_builder.go | 23 +++ cmd/node_builder.go | 33 ++-- cmd/scaffold.go | 3 + engine/access/ping/engine.go | 6 +- engine/access/relay/engine.go | 3 +- integration/Makefile | 6 +- integration/testnet/container.go | 13 +- integration/testnet/network.go | 164 +++++++++++++++--- integration/testnet/network_test.go | 2 + .../tests/access/unstaked_node_test.go | 110 ++++++++++++ 11 files changed, 321 insertions(+), 61 deletions(-) create mode 100644 integration/tests/access/unstaked_node_test.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index d23024ed603..75e87a708dd 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -73,8 +73,8 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // ParticipatesInUnstakedNetwork returns True if this is a staked Access node which also participates - // in the unstaked network acting as an upstream for other unstaked access nodes, False otherwise. + // ParticipatesInUnstakedNetwork returns True if this an Access Node which participates in the unstaked network, + // False otherwise ParticipatesInUnstakedNetwork() bool // Build defines all of the Access node's components and modules. @@ -150,8 +150,8 @@ type FlowAccessNodeBuilder struct { // components UnstakedLibP2PNode *p2p.Node - UnstakedNetwork *p2p.Network - unstakedMiddleware *p2p.Middleware + UnstakedNetwork p2p.ReadyDoneAwareNetwork + unstakedMiddleware network.Middleware FollowerState protocol.MutableState SyncCore *synchronization.Core RpcEng *rpc.Engine @@ -525,12 +525,6 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { } func (builder *FlowAccessNodeBuilder) ParticipatesInUnstakedNetwork() bool { - // unstaked access nodes can't be upstream of other unstaked access nodes for now - if !builder.IsStaked() { - return false - } - // if an unstaked network bind address is provided, then this staked access node will act as the upstream for - // unstaked access nodes return builder.unstakedNetworkBindAddr != cmd.NotSet } @@ -613,7 +607,8 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - validators ...network.MessageValidator) *p2p.Middleware { + + validators ...network.MessageValidator) *network.Middleware { builder.unstakedMiddleware = p2p.NewMiddleware( builder.Logger, factoryFunc, @@ -636,7 +631,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, // updated by calling network.SetIDs. func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, networkMetrics module.NetworkMetrics, - middleware *p2p.Middleware, + middleware network.Middleware, topology network.Topology) (*p2p.Network, error) { codec := jsoncodec.NewCodec() diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index e17ce0e9d73..37428de013d 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -6,6 +6,8 @@ import ( "github.com/onflow/flow-go/cmd" pingeng "github.com/onflow/flow-go/engine/access/ping" + "github.com/onflow/flow-go/engine/access/relay" + splitternetwork "github.com/onflow/flow-go/engine/common/splitter/network" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -79,6 +81,10 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { + anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) + return node.Network, nil + }) anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -96,6 +102,23 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { } return ping, nil }) + if anb.ParticipatesInUnstakedNetwork() { + // create relay engine + anb.Component("relay engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + channels := node.SubscriptionManager.Channels() + if len(channels) == 0 { + return nil, fmt.Errorf("no subscribed channels to relay") + } + + relayEngine, err := relay.New(node.Logger, channels, node.Network, anb.UnstakedNetwork) + + if err != nil { + return nil, fmt.Errorf("could not create relay engine: %w", err) + } + + return relayEngine, nil + }) + } return anb } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 6f1300df68a..b62b9752ee5 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -120,22 +120,23 @@ type BaseConfig struct { type NodeConfig struct { Cancel context.CancelFunc // cancel function for the context that is passed to the networking layer BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware *p2p.Middleware - Network *p2p.Network - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware network.Middleware + Network p2p.ReadyDoneAwareNetwork + SubscriptionManager network.SubscriptionManager + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // ID providers IdentityProvider id.IdentityProvider diff --git a/cmd/scaffold.go b/cmd/scaffold.go index dc2d53549f6..530827fa56e 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -185,11 +185,14 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { ) subscriptionManager := p2p.NewChannelSubscriptionManager(fnb.Middleware) + top, err := topology.NewTopicBasedTopology( fnb.NodeID, fnb.Logger, fnb.State, ) + node.SubscriptionManager = subscriptionManager + if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) } diff --git a/engine/access/ping/engine.go b/engine/access/ping/engine.go index dfcb3c70f5a..291d54ce232 100644 --- a/engine/access/ping/engine.go +++ b/engine/access/ping/engine.go @@ -9,7 +9,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/state/protocol" ) @@ -22,7 +22,7 @@ type Engine struct { pingEnabled bool pingInterval time.Duration - middleware *p2p.Middleware + middleware network.Middleware nodeInfo map[flow.Identifier]string // additional details about a node such as operator name } @@ -32,7 +32,7 @@ func New( me module.Local, metrics module.PingMetrics, pingEnabled bool, - mw *p2p.Middleware, + mw network.Middleware, nodeInfoFile string, ) (*Engine, error) { diff --git a/engine/access/relay/engine.go b/engine/access/relay/engine.go index b850f3d48f0..6c20ab72544 100644 --- a/engine/access/relay/engine.go +++ b/engine/access/relay/engine.go @@ -99,10 +99,11 @@ func (e *Engine) process(channel network.Channel, originID flow.Identifier, even conduit, ok := e.conduits[channel] if !ok { - e.log.Trace().Interface("event", event).Str("channel", channel.String()).Str("originID", originID.String()).Msg("unknown channel") return fmt.Errorf("received message on unknown channel %s", channel) } + e.log.Trace().Interface("event", event).Str("channel", channel.String()).Str("originID", originID.String()).Msg("relaying message") + // We use a dummy target ID here so that events are broadcast to the entire network if err := conduit.Publish(event, flow.ZeroID); err != nil { return fmt.Errorf("could not relay message: %w", err) diff --git a/integration/Makefile b/integration/Makefile index 231543fd848..81276cd00c6 100644 --- a/integration/Makefile +++ b/integration/Makefile @@ -12,7 +12,11 @@ integration-test: common-tests execution-tests verification-tests collection-tes # NOTE: Currently skipping collection and consensus tests due to them not passing properly on CI .PHONY: ci-integration-test -ci-integration-test: common-tests execution-tests epoch-tests verification-tests # collection-tests # consensus-tests +ci-integration-test: common-tests execution-tests epoch-tests verification-tests access-tests # collection-tests # consensus-tests + +.PHONY: access-tests +access-tests: + GO111MODULE=on go test -tags relic -count=1 ./tests/access .PHONY: collection-tests collection-tests: diff --git a/integration/testnet/container.go b/integration/testnet/container.go index da269214da0..666c54d349a 100644 --- a/integration/testnet/container.go +++ b/integration/testnet/container.go @@ -38,12 +38,13 @@ func init() { // ContainerConfig represents configuration for a node container in the network. type ContainerConfig struct { bootstrap.NodeInfo - ContainerName string - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - Unstaked bool + ContainerName string + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + Unstaked bool + ParticipatesInUnstakedNetwork bool } // ImageName returns the Docker image name for the given config. diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 04d5c4e88a4..cfbfeed0a52 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -4,7 +4,6 @@ import ( "context" "encoding/hex" "fmt" - "github.com/onflow/flow-go/cmd/bootstrap/utils" "io/ioutil" "math/rand" "os" @@ -15,6 +14,8 @@ import ( "testing" "time" + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/docker/docker/api/types/container" dockerclient "github.com/docker/docker/client" "github.com/rs/zerolog" @@ -31,6 +32,7 @@ import ( "github.com/onflow/cadence" "github.com/onflow/flow-go/cmd/bootstrap/run" + consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" dkgmod "github.com/onflow/flow-go/model/dkg" @@ -65,12 +67,20 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" + // UnstakedNetworkPort is the name used for the access node unstaked libp2p network port. + UnstakedNetworkPort = "access-unstaked-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" // ExeNodeMetricsPort ExeNodeMetricsPort = "exe-metrics-port" + // default staked network port + DefaultStakedFlowPort = 2137 + + // default unstaked network port + DefaultUnstakedFlowPort = 7312 + DefaultViewsInStakingAuction uint64 = 5 DefaultViewsInDKGPhase uint64 = 50 DefaultViewsInEpoch uint64 = 180 @@ -82,17 +92,18 @@ func init() { // FlowNetwork represents a test network of Flow nodes running in Docker containers. type FlowNetwork struct { - t *testing.T - suite *testingdock.Suite - config NetworkConfig - cli *dockerclient.Client - network *testingdock.Network - Containers map[string]*Container - AccessPorts map[string]string - root *flow.Block - result *flow.ExecutionResult - seal *flow.Seal - bootstrapDir string + t *testing.T + suite *testingdock.Suite + config NetworkConfig + cli *dockerclient.Client + network *testingdock.Network + Containers map[string]*Container + ConsensusFollowers map[flow.Identifier]consensus_follower.ConsensusFollower + AccessPorts map[string]string + root *flow.Block + result *flow.ExecutionResult + seal *flow.Seal + bootstrapDir string } // Identities returns a list of identities, one for each node in the network. @@ -124,6 +135,9 @@ func (net *FlowNetwork) Start(ctx context.Context) { // makes it easier to see logs for a specific test case fmt.Println(">>>> starting network: ", net.config.Name) net.suite.Start(ctx) + for _, cf := range net.ConsensusFollowers { + go cf.Run(ctx) + } } // Remove stops the network, removes all the containers and cleans up all resources. @@ -195,6 +209,14 @@ func (net *FlowNetwork) ContainerByID(id flow.Identifier) *Container { return nil } +// ConsensusFollowerByID returns the ConsensusFollower with the given node ID, if it exists. +// Otherwise fails the test. +func (net *FlowNetwork) ConsensusFollowerByID(id flow.Identifier) consensus_follower.ConsensusFollower { + follower, ok := net.ConsensusFollowers[id] + require.True(net.t, ok) + return follower +} + // ContainerByName returns the container with the given name, if it exists. // Otherwise fails the test. func (net *FlowNetwork) ContainerByName(name string) *Container { @@ -203,9 +225,22 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { return container } +type ConsensusFollowerConfig struct { + nodeID flow.Identifier + upstreamNodeID flow.Identifier +} + +func NewConsensusFollowerConfig(nodeID flow.Identifier, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { + return ConsensusFollowerConfig{ + nodeID: nodeID, + upstreamNodeID: upstreamNodeID, + } +} + // NetworkConfig is the config for the network. type NetworkConfig struct { Nodes []NodeConfig + ConsensusFollowers []ConsensusFollowerConfig Name string NClusters uint ViewsInDKGPhase uint64 @@ -215,9 +250,10 @@ type NetworkConfig struct { type NetworkConfigOpt func(*NetworkConfig) -func NewNetworkConfig(name string, nodes []NodeConfig, opts ...NetworkConfigOpt) NetworkConfig { +func NewNetworkConfig(name string, nodes []NodeConfig, followers []ConsensusFollowerConfig, opts ...NetworkConfigOpt) NetworkConfig { c := NetworkConfig{ Nodes: nodes, + ConsensusFollowers: followers, Name: name, NClusters: 1, // default to 1 cluster ViewsInStakingAuction: DefaultViewsInStakingAuction, @@ -288,7 +324,8 @@ type NodeConfig struct { Debug bool // Unstaked - only applicable to Access Node. Access nodes can be staked or unstaked. // Unstaked nodes are not part of the identity table - Unstaked bool // only applicable to Access node + Unstaked bool // only applicable to Access node + ParticipatesInUnstakedNetwork bool } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -364,6 +401,12 @@ func AsGhost() func(config *NodeConfig) { } } +func AsUnstakedNetworkParticipant() func(config *NodeConfig) { + return func(config *NodeConfig) { + config.ParticipatesInUnstakedNetwork = true + } +} + // WithAdditionalFlag adds additional flags to the command func WithAdditionalFlag(flag string) func(config *NodeConfig) { return func(config *NodeConfig) { @@ -424,9 +467,58 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { require.NoError(t, err) } + // add each follower to the network + for _, followerConf := range networkConf.ConsensusFollowers { + err = flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) + require.NoError(t, err) + } + return flowNetwork } +func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) error { + tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") + if err != nil { + return fmt.Errorf("could not get tmp dir: %w", err) + } + + // create a directory for the follower database + dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) + err = os.Mkdir(dataDir, 0700) + require.NoError(t, err) + + // create a follower-specific directory for the bootstrap files + followerBootstrapDir := filepath.Join(tmpdir, DefaultBootstrapDir) + err = os.Mkdir(followerBootstrapDir, 0700) + require.NoError(t, err) + + // copy bootstrap files to follower-specific bootstrap directory + err = io.CopyDirectory(bootstrapDir, followerBootstrapDir) + require.NoError(t, err) + + // consensus follower + bindPort := testingdock.RandomPort(t) + bindAddr := fmt.Sprintf("0.0.0.0:%d", bindPort) + opts := []consensus_follower.Option{ + consensus_follower.WithDataDir(dataDir), + consensus_follower.WithBootstrapDir(followerBootstrapDir), + } + + // TODO: eventually we will need upstream node's address + // + // upstreamANPort := net.ContainerByID(followerConf.upstreamNodeID).Ports[testnet.UnstakedNetworkPort] + // upstreamANAddress := fmt.Sprintf("127.0.0.1:%d", upstreamANPort) + + follower := consensus_follower.NewConsensusFollower( + followerConf.nodeID, + followerConf.upstreamNodeID, + bindAddr, + opts..., + ) + + net.ConsensusFollowers[followerConf.nodeID] = follower +} + // AddNode creates a node container with the given config and adds it to the // network. func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf ContainerConfig) error { @@ -562,6 +654,15 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort + if nodeConf.ParticipatesInUnstakedNetwork { + hostUnstakedPort := testingdock.RandomPort(t) + containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) + nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) + nodeContainer.addFlag("unstaked-bind-addr", fmt.Sprintf("%s:%d", nodeContainer.Name(), DefaultUnstakedFlowPort)) + nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + net.AccessPorts[UnstakedNetworkPort] = hostUnstakedPort + } + case flow.RoleConsensus: // use 1 here instead of the default 5, because the integration // tests only start 1 verification node @@ -581,6 +682,20 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.addFlag("rpc-addr", fmt.Sprintf("%s:9000", nodeContainer.Name())) nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort + + if nodeConf.ParticipatesInUnstakedNetwork { + hostUnstakedPort := testingdock.RandomPort(t) + containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) + + // TODO: Currently, it is not possible to create a staked ghost AN which + // participates on the unstaked network, because the ghost node only joins + // a single network during startup. The ghost node needs to support the + // "unstaked-bind-addr" flag which can be used to specify a bind address + // for the unstaked network. + + nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) + nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + } } if nodeConf.Debug { @@ -817,7 +932,11 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { // define the node's name _ and address : name := fmt.Sprintf("%s_%d", conf.Role.String(), roleCounter[conf.Role]+1) - addr := fmt.Sprintf("%s:%d", name, 2137) + flowPort := DefaultStakedFlowPort + if conf.Unstaked { + flowPort = DefaultUnstakedFlowPort + } + addr := fmt.Sprintf("%s:%d", name, flowPort) roleCounter[conf.Role]++ info := bootstrap.NewPrivateNodeInfo( @@ -830,13 +949,14 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - Unstaked: conf.Unstaked, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + Unstaked: conf.Unstaked, + ParticipatesInUnstakedNetwork: conf.ParticipatesInUnstakedNetwork, } confs = append(confs, containerConf) diff --git a/integration/testnet/network_test.go b/integration/testnet/network_test.go index 1a3be04cbdb..fa979f50c00 100644 --- a/integration/testnet/network_test.go +++ b/integration/testnet/network_test.go @@ -33,6 +33,7 @@ func TestNetworkSetupBasic(t *testing.T) { testnet.NewNodeConfig(flow.RoleConsensus), testnet.NewNodeConfig(flow.RoleExecution), testnet.NewNodeConfig(flow.RoleVerification), + testnet.NewNodeConfig(flow.RoleAccess), } conf := testnet.NewNetworkConfig("meta_test_basic", nodes) @@ -50,6 +51,7 @@ func TestNetworkSetupBasic(t *testing.T) { {image: defaultRegistry + "/consensus:latest", name: "consensus_3", address: "consensus_3:2137"}, {image: defaultRegistry + "/execution:latest", name: "execution_1", address: "execution_1:2137"}, {image: defaultRegistry + "/verification:latest", name: "verification_1", address: "verification_1:2137"}, + {image: defaultRegistry + "/access:latest", name: "access_1", address: "access_1:2137"}, } assert.Subset(t, realData, expectedData) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go new file mode 100644 index 00000000000..bb87daae6de --- /dev/null +++ b/integration/tests/access/unstaked_node_test.go @@ -0,0 +1,110 @@ +package access + +import ( + "context" + "testing" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/integration/testnet" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" +) + +type UnstakedAccessSuite struct { + suite.Suite + + // root context for the current test + ctx context.Context + cancel context.CancelFunc + + net *testnet.FlowNetwork + stakedID flow.Identifier + unstakedID flow.Identifier + conID flow.Identifier + follower consensus_follower.ConsensusFollower +} + +func TestUnstakedAccessSuite(t *testing.T) { + suite.Run(t, new(UnstakedAccessSuite)) +} + +func (suite *UnstakedAccessSuite) TearDownTest() { + // avoid nil pointer errors for skipped tests + if suite.cancel != nil { + defer suite.cancel() + } + if suite.net != nil { + suite.net.Remove() + } +} + +func (suite *UnstakedAccessSuite) SetupTest() { + nodeConfigs := []testnet.NodeConfig{} + + // staked access node + suite.stakedID = unittest.IdentifierFixture() + stakedConfig := testnet.NewNodeConfig( + flow.RoleAccess, + testnet.WithID(suite.stakedID), + testnet.AsUnstakedNetworkParticipant(), + testnet.WithLogLevel(zerolog.InfoLevel), + ) + nodeConfigs = append(nodeConfigs, stakedConfig) + + // consensus node (ghost) + suite.conID = unittest.IdentifierFixture() + conConfig := testnet.NewNodeConfig( + flow.RoleConsensus, + testnet.WithID(suite.conID), + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), + ) + nodeConfigs = append(nodeConfigs, conConfig) + + // execution node (unused) + exeConfig := testnet.NewNodeConfig( + flow.RoleExecution, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), + ) + nodeConfigs = append(nodeConfigs, exeConfig) + + // verification node (unused) + verConfig := testnet.NewNodeConfig( + flow.RoleVerification, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), + ) + nodeConfigs = append(nodeConfigs, verConfig) + + // collection node (unused) + collConfig := testnet.NewNodeConfig( + flow.RoleCollection, + testnet.AsGhost(), + testnet.WithLogLevel(zerolog.FatalLevel), + ) + nodeConfigs = append(nodeConfigs, collConfig) + + // consensus follower + suite.unstakedID = unittest.IdentifierFixture() + followerConfigs := []testnet.ConsensusFollowerConfig{ + testnet.ConsensusFollowerConfig{ + nodeID: suite.unstakedID, + upstreamNodeID: suite.stakedID, + }, + } + + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) + suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + + // start the network + suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.net.Start(suite.ctx) +} + +func (suite *UnstakedAccessSuite) TestReceiveBlocks() { + suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) + // TODO +} From 2bcf3f93554c90dfb4e9521b1587a6af8fd71bb9 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 15:15:26 -0700 Subject: [PATCH 229/291] Update unstaked_node_test.go --- integration/tests/access/unstaked_node_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index bb87daae6de..1ee76939237 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -7,6 +7,7 @@ import ( "github.com/rs/zerolog" "github.com/stretchr/testify/suite" + consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/utils/unittest" From b314b96e260aeccde0d88948ac5c5a62a24dd2e9 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 16:51:26 -0700 Subject: [PATCH 230/291] Add consensus follower to integration tests --- cmd/scaffold.go | 1 - integration/go.mod | 1 + integration/go.sum | 8 ++ integration/testnet/network.go | 77 +++++++++++++------ .../tests/access/unstaked_node_test.go | 11 ++- 5 files changed, 68 insertions(+), 30 deletions(-) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 530827fa56e..a51f8cc611e 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -21,7 +21,6 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" diff --git a/integration/go.mod b/integration/go.mod index 87d31b4f256..31e00992a54 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -6,6 +6,7 @@ require ( github.com/DataDog/zstd v1.4.8 // indirect github.com/HdrHistogram/hdrhistogram-go v1.0.1 // indirect github.com/dapperlabs/testingdock v0.4.3-0.20200626075145-ea23fc16bb90 + github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f // indirect github.com/dgraph-io/badger/v2 v2.2007.2 github.com/dgraph-io/ristretto v0.0.3 // indirect github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect diff --git a/integration/go.sum b/integration/go.sum index b43cd560555..794f7786af0 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -215,6 +215,8 @@ github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c h1:pFUpOrbxDR github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c/go.mod h1:6UhI8N9EjYm1c2odKpFpAYeR8dsBeM7PtzQhRgxRr9U= github.com/deckarep/golang-set v0.0.0-20180603214616-504e848d77ea/go.mod h1:93vsz/8Wt4joVM7c2AVqh+YRMiUSc14yDtF28KmMOgQ= github.com/decred/dcrd/lru v1.0.0/go.mod h1:mxKOwFd7lFjN2GZYsiz/ecgqR6kkYAl+0pz0tEMk218= +github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f h1:U5y3Y5UE0w7amNe7Z5G/twsBW0KEalRQXZzf8ufSh9I= +github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f/go.mod h1:xH/i4TFMt8koVQZ6WFms69WAsDWr2XsYL3Hkl7jkoLE= github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= github.com/dgraph-io/badger v1.6.0-rc1/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= @@ -480,9 +482,13 @@ github.com/graph-gophers/graphql-go v0.0.0-20191115155744-f33e81362277/go.mod h1 github.com/graph-gophers/graphql-go v0.0.0-20201113091052-beb923fada29/go.mod h1:9CQHMSxwO4MprSdzoIEobiHpoLtHm77vfxsvsIN5Vuc= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2 h1:uxUHSMwWDJ/9jVPHNumRC8WZOi3hrBL22ObVOoLg4ww= github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2/go.mod h1:BL7w7qd2l/j9jgY6WMhYutfOFQc0I8RTVwtjpnAMoTM= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea h1:1Tk1IbruXbunEnaIZEFb+Hpv9BIZti3OxKwKn5wWyKk= github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea/go.mod h1:GugMBs30ZSAkckqXEAIEGyYdDH6EgqowG8ppA3Zt+AY= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= @@ -533,6 +539,7 @@ github.com/huin/goupnp v1.0.1-0.20200620063722-49508fba0031 h1:HarGZ5h9HD9LgEg1y github.com/huin/goupnp v1.0.1-0.20200620063722-49508fba0031/go.mod h1:nNs7wvRfN1eKaMknBydLNQU6146XQim8t4h+q90biWo= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/improbable-eng/grpc-web v0.12.0 h1:GlCS+lMZzIkfouf7CNqY+qqpowdKuJLSLLcKVfM1oLc= github.com/improbable-eng/grpc-web v0.12.0/go.mod h1:6hRR09jOEG81ADP5wCQju1z71g6OL4eEvELdran/3cs= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= @@ -1249,6 +1256,7 @@ github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.2.2/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/cors v0.0.0-20160617231935-a62a804a8a00/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/rs/xhandler v0.0.0-20160618193221-ed27b6fd6521/go.mod h1:RvLn4FgxWubrpZHtQLnOf6EwhN2hEMusxZOhcW9H3UQ= github.com/rs/xid v1.2.1/go.mod h1:+uKXf+4Djp6Md1KODXJxgGQPKngRmWyn10oCKFzNHOQ= diff --git a/integration/testnet/network.go b/integration/testnet/network.go index cfbfeed0a52..b1cc9941529 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -135,9 +135,6 @@ func (net *FlowNetwork) Start(ctx context.Context) { // makes it easier to see logs for a specific test case fmt.Println(">>>> starting network: ", net.config.Name) net.suite.Start(ctx) - for _, cf := range net.ConsensusFollowers { - go cf.Run(ctx) - } } // Remove stops the network, removes all the containers and cleans up all resources. @@ -448,17 +445,18 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { require.Nil(t, err) flowNetwork := &FlowNetwork{ - t: t, - cli: dockerClient, - config: networkConf, - suite: suite, - network: network, - Containers: make(map[string]*Container, nNodes), - AccessPorts: make(map[string]string), - root: root, - seal: seal, - result: result, - bootstrapDir: bootstrapDir, + t: t, + cli: dockerClient, + config: networkConf, + suite: suite, + network: network, + Containers: make(map[string]*Container, nNodes), + ConsensusFollowers: make(map[flow.Identifier]consensus_follower.ConsensusFollower, len(networkConf.ConsensusFollowers)), + AccessPorts: make(map[string]string), + root: root, + seal: seal, + result: result, + bootstrapDir: bootstrapDir, } // add each node to the network @@ -469,18 +467,15 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { // add each follower to the network for _, followerConf := range networkConf.ConsensusFollowers { - err = flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) - require.NoError(t, err) + flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) } return flowNetwork } -func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) error { +func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) { tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") - if err != nil { - return fmt.Errorf("could not get tmp dir: %w", err) - } + require.NoError(t, err) // create a directory for the follower database dataDir := filepath.Join(tmpdir, DefaultFlowDBDir) @@ -498,7 +493,7 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // consensus follower bindPort := testingdock.RandomPort(t) - bindAddr := fmt.Sprintf("0.0.0.0:%d", bindPort) + bindAddr := fmt.Sprintf("0.0.0.0:%s", bindPort) opts := []consensus_follower.Option{ consensus_follower.WithDataDir(dataDir), consensus_follower.WithBootstrapDir(followerBootstrapDir), @@ -721,6 +716,37 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { require.NoError(net.t, err) } +func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { + var nodeInfos []bootstrap.NodeInfo + + // get networking keys for all followers + networkKeys, err := unittest.NetworkingKeys(len(confs)) + if err != nil { + return nil, err + } + + // get staking keys for all followers + stakingKeys, err := unittest.StakingKeys(len(confs)) + if err != nil { + return nil, err + } + + for i, conf := range confs { + info := bootstrap.NewPrivateNodeInfo( + conf.nodeID, + flow.RoleAccess, // use Access role + "", // no address + 0, // no stake + networkKeys[i], + stakingKeys[i], + ) + + nodeInfos = append(nodeInfos, info) + } + + return nodeInfos, nil +} + func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Block, *flow.ExecutionResult, *flow.Seal, []ContainerConfig, error) { chainID := flow.Localnet chain := chainID.Chain() @@ -746,9 +772,14 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo stakedConfs := filterContainerConfigs(allConfs, func(config ContainerConfig) bool { return !config.Unstaked }) - fmt.Println(len(stakedConfs)) - fmt.Println(len(allConfs)) + + followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) + if err != nil { + return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) + } + allNodeInfos := toNodeInfos(allConfs) + allNodeInfos = append(allNodeInfos, followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices // IMPORTANT: these nodes infos must include exactly the identity table diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 1ee76939237..f5259c853aa 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -91,21 +91,20 @@ func (suite *UnstakedAccessSuite) SetupTest() { // consensus follower suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.ConsensusFollowerConfig{ - nodeID: suite.unstakedID, - upstreamNodeID: suite.stakedID, - }, + testnet.NewConsensusFollowerConfig(suite.unstakedID, suite.stakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) + suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) + // start the network suite.ctx, suite.cancel = context.WithCancel(context.Background()) suite.net.Start(suite.ctx) } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) - // TODO + go suite.follower.Run(suite.ctx) + // TODO: to be implemented later } From 9e3e8071b7e36ec6619fbe4882d5d7b549eba97c Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 9 Aug 2021 16:58:07 -0700 Subject: [PATCH 231/291] Update staked_access_node_builder.go --- cmd/access/node_builder/staked_access_node_builder.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 37428de013d..a9c31c686bb 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -81,10 +81,12 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { - anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) - return node.Network, nil - }) + if anb.ParticipatesInUnstakedNetwork() { + anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) + return node.Network, nil + }) + } anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { From 593fa5053268f542c964f67479525c5e2cc4c849 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:21:51 -0700 Subject: [PATCH 232/291] update to reflect new design --- cmd/node_builder.go | 33 +++++----- cmd/scaffold.go | 2 - integration/testnet/network.go | 114 +++++++++++---------------------- 3 files changed, 54 insertions(+), 95 deletions(-) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index b62b9752ee5..f0e1b39219a 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -120,23 +120,22 @@ type BaseConfig struct { type NodeConfig struct { Cancel context.CancelFunc // cancel function for the context that is passed to the networking layer BaseConfig - Logger zerolog.Logger - NodeID flow.Identifier - Me *local.Local - Tracer module.Tracer - MetricsRegisterer prometheus.Registerer - Metrics Metrics - DB *badger.DB - Storage Storage - ProtocolEvents *events.Distributor - State protocol.State - Middleware network.Middleware - Network p2p.ReadyDoneAwareNetwork - SubscriptionManager network.SubscriptionManager - MsgValidators []network.MessageValidator - FvmOptions []fvm.Option - StakingKey crypto.PrivateKey - NetworkKey crypto.PrivateKey + Logger zerolog.Logger + NodeID flow.Identifier + Me *local.Local + Tracer module.Tracer + MetricsRegisterer prometheus.Registerer + Metrics Metrics + DB *badger.DB + Storage Storage + ProtocolEvents *events.Distributor + State protocol.State + Middleware network.Middleware + Network p2p.ReadyDoneAwareNetwork + MsgValidators []network.MessageValidator + FvmOptions []fvm.Option + StakingKey crypto.PrivateKey + NetworkKey crypto.PrivateKey // ID providers IdentityProvider id.IdentityProvider diff --git a/cmd/scaffold.go b/cmd/scaffold.go index a51f8cc611e..178b31e5015 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -190,8 +190,6 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Logger, fnb.State, ) - node.SubscriptionManager = subscriptionManager - if err != nil { return nil, fmt.Errorf("could not create topology: %w", err) } diff --git a/integration/testnet/network.go b/integration/testnet/network.go index b1cc9941529..40225acc9e1 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -14,33 +14,31 @@ import ( "testing" "time" - "github.com/onflow/flow-go/cmd/bootstrap/utils" - "github.com/docker/docker/api/types/container" dockerclient "github.com/docker/docker/client" "github.com/rs/zerolog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/onflow/flow-go-sdk/crypto" - "github.com/onflow/flow-go/model/encodable" - "github.com/onflow/flow-go/model/flow/order" - "github.com/onflow/flow-go/utils/io" - "github.com/dapperlabs/testingdock" "github.com/onflow/cadence" + "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/cmd/bootstrap/run" + "github.com/onflow/flow-go/cmd/bootstrap/utils" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" dkgmod "github.com/onflow/flow-go/model/dkg" + "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" clusterstate "github.com/onflow/flow-go/state/cluster" "github.com/onflow/flow-go/state/protocol/inmem" + "github.com/onflow/flow-go/utils/io" "github.com/onflow/flow-go/utils/unittest" ) @@ -67,19 +65,14 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" - // UnstakedNetworkPort is the name used for the access node unstaked libp2p network port. - UnstakedNetworkPort = "access-unstaked-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" // ExeNodeMetricsPort ExeNodeMetricsPort = "exe-metrics-port" - // default staked network port - DefaultStakedFlowPort = 2137 - - // default unstaked network port - DefaultUnstakedFlowPort = 7312 + // default network port + DefaultFlowPort = 2137 DefaultViewsInStakingAuction uint64 = 5 DefaultViewsInDKGPhase uint64 = 50 @@ -312,17 +305,14 @@ func (n *NetworkConfig) Swap(i, j int) { // NodeConfig defines the input config for a particular node, specified prior // to network creation. type NodeConfig struct { - Role flow.Role - Stake uint64 - Identifier flow.Identifier - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - // Unstaked - only applicable to Access Node. Access nodes can be staked or unstaked. - // Unstaked nodes are not part of the identity table - Unstaked bool // only applicable to Access node - ParticipatesInUnstakedNetwork bool + Role flow.Role + Stake uint64 + Identifier flow.Identifier + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + ParticipatesInPublicNetwork bool // only applicable to Access node } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -398,9 +388,9 @@ func AsGhost() func(config *NodeConfig) { } } -func AsUnstakedNetworkParticipant() func(config *NodeConfig) { +func AsPublicNetworkParticipant() func(config *NodeConfig) { return func(config *NodeConfig) { - config.ParticipatesInUnstakedNetwork = true + config.ParticipatesInPublicNetwork = true } } @@ -499,11 +489,8 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, consensus_follower.WithBootstrapDir(followerBootstrapDir), } - // TODO: eventually we will need upstream node's address - // - // upstreamANPort := net.ContainerByID(followerConf.upstreamNodeID).Ports[testnet.UnstakedNetworkPort] - // upstreamANAddress := fmt.Sprintf("127.0.0.1:%d", upstreamANPort) - + // TODO: update consensus follower to just accept a networking key instead of a node ID + // it should be able to figure out the rest on its own. follower := consensus_follower.NewConsensusFollower( followerConf.nodeID, followerConf.upstreamNodeID, @@ -649,13 +636,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort - if nodeConf.ParticipatesInUnstakedNetwork { - hostUnstakedPort := testingdock.RandomPort(t) - containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) - nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) - nodeContainer.addFlag("unstaked-bind-addr", fmt.Sprintf("%s:%d", nodeContainer.Name(), DefaultUnstakedFlowPort)) - nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort - net.AccessPorts[UnstakedNetworkPort] = hostUnstakedPort + if nodeConf.ParticipatesInPublicNetwork { + // TODO: define this flag for Access node + nodeContainer.addFlag("public-network-participant", "true") } case flow.RoleConsensus: @@ -678,18 +661,12 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort - if nodeConf.ParticipatesInUnstakedNetwork { - hostUnstakedPort := testingdock.RandomPort(t) - containerUnstakedPort := fmt.Sprintf("%d/tcp", DefaultUnstakedFlowPort) - - // TODO: Currently, it is not possible to create a staked ghost AN which - // participates on the unstaked network, because the ghost node only joins - // a single network during startup. The ghost node needs to support the - // "unstaked-bind-addr" flag which can be used to specify a bind address - // for the unstaked network. - - nodeContainer.bindPort(hostUnstakedPort, containerUnstakedPort) - nodeContainer.Ports[UnstakedNetworkPort] = hostUnstakedPort + if nodeConf.ParticipatesInPublicNetwork { + // TODO: Currently, it is not possible to create a ghost AN which participates + // in the public network, because connection gating is enabled by default and + // therefore the ghost node will deny incoming connections from all consensus + // followers. A flag for the ghost node will need to be created to enable + // overriding the default behavior. } } @@ -761,29 +738,19 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo sort.Sort(&networkConf) // generate staking and networking keys for each configured node - // NOTE: this includes unstaked access nodes, which need private keys written - // but should not be included in the identity table - allConfs, err := setupKeys(networkConf) + stakedConfs, err := setupKeys(networkConf) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to setup keys: %w", err) } - // only staked configs - this only includes identity table members - stakedConfs := filterContainerConfigs(allConfs, func(config ContainerConfig) bool { - return !config.Unstaked - }) - followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) } - allNodeInfos := toNodeInfos(allConfs) - allNodeInfos = append(allNodeInfos, followerInfos...) + allNodeInfos = append(toNodeInfos(stakedConfs), followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices - // IMPORTANT: these nodes infos must include exactly the identity table - // members (no unstaked access nodes) stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), order.Canonical) // run DKG for all consensus nodes @@ -963,11 +930,7 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { // define the node's name _ and address : name := fmt.Sprintf("%s_%d", conf.Role.String(), roleCounter[conf.Role]+1) - flowPort := DefaultStakedFlowPort - if conf.Unstaked { - flowPort = DefaultUnstakedFlowPort - } - addr := fmt.Sprintf("%s:%d", name, flowPort) + addr := fmt.Sprintf("%s:%d", name, DefaultFlowPort) roleCounter[conf.Role]++ info := bootstrap.NewPrivateNodeInfo( @@ -980,14 +943,13 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - Unstaked: conf.Unstaked, - ParticipatesInUnstakedNetwork: conf.ParticipatesInUnstakedNetwork, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + ParticipatesInPublicNetwork: conf.ParticipatesInPublicNetwork, } confs = append(confs, containerConf) From 8c90b8d9bff399ecd1fc7b5f1025e42cbca08857 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:22:49 -0700 Subject: [PATCH 233/291] Update staked_access_node_builder.go --- .../staked_access_node_builder.go | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index a9c31c686bb..e880875b10e 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -81,12 +81,6 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { } func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { - if anb.ParticipatesInUnstakedNetwork() { - anb.Component("splitter network", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - node.Network = splitternetwork.NewNetwork(node.Network, node.Logger) - return node.Network, nil - }) - } anb.FlowAccessNodeBuilder. Build(). Component("ping engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -104,23 +98,7 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { } return ping, nil }) - if anb.ParticipatesInUnstakedNetwork() { - // create relay engine - anb.Component("relay engine", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - channels := node.SubscriptionManager.Channels() - if len(channels) == 0 { - return nil, fmt.Errorf("no subscribed channels to relay") - } - relayEngine, err := relay.New(node.Logger, channels, node.Network, anb.UnstakedNetwork) - - if err != nil { - return nil, fmt.Errorf("could not create relay engine: %w", err) - } - - return relayEngine, nil - }) - } return anb } From 135faa027d1921c8228bd89704b50b867a80e303 Mon Sep 17 00:00:00 2001 From: smnzhu Date: Sat, 14 Aug 2021 15:31:53 -0700 Subject: [PATCH 234/291] Update consensus_follower.go --- follower/consensus_follower.go | 10 +++++----- integration/testnet/network.go | 12 ++++++++---- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index a34b207537e..82062be2361 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -24,7 +24,7 @@ type ConsensusFollower interface { // Config contains the configurable fields for a `ConsensusFollower`. type Config struct { - nodeID flow.Identifier // the node ID of this node + networkPubKey crypto.PublicKey // the network public key of this node bootstrapNodes []BootstrapNodeInfo // the bootstrap nodes to use bindAddr string // address to bind on dataDir string // directory to store the protocol state @@ -69,14 +69,14 @@ func getAccessNodeOptions(config *Config) []access.Option { ids := bootstrapIdentities(config.bootstrapNodes) return []access.Option{ access.WithBootStrapPeers(ids...), - access.WithUnstakedNetworkBindAddr(config.bindAddr), + access.WithBindAddr(config.bindAddr), access.WithBaseOptions(getBaseOptions(config)), } } func getBaseOptions(config *Config) []cmd.Option { options := []cmd.Option{ - cmd.WithNodeID(config.nodeID), + cmd.WithNetworkPublicKey(config.networkPubKey), cmd.WithMetricsEnabled(false), } if config.bootstrapDir != "" { @@ -107,13 +107,13 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - nodeID flow.Identifier, + networkPublicKey crypto.PublicKey, // TODO: make this optional. if not explicitly supplied, we can auto-generate one for them. bootstapIdentities []BootstrapNodeInfo, bindAddr string, opts ...Option, ) (*ConsensusFollowerImpl, error) { config := &Config{ - nodeID: nodeID, + networkPublicKey: networkPublicKey, bootstrapNodes: bootstapIdentities, bindAddr: bindAddr, } diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 40225acc9e1..ee1f90c2be7 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -27,6 +27,7 @@ import ( "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/cmd/bootstrap/run" "github.com/onflow/flow-go/cmd/bootstrap/utils" + fcrypto "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" @@ -216,13 +217,13 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier + networkKey fcrypto.PrivateKey upstreamNodeID flow.Identifier } -func NewConsensusFollowerConfig(nodeID flow.Identifier, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - nodeID: nodeID, + networkKey: fcrypto.PrivateKey, upstreamNodeID: upstreamNodeID, } } @@ -492,12 +493,13 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. follower := consensus_follower.NewConsensusFollower( - followerConf.nodeID, + followerConf.networkKey, followerConf.upstreamNodeID, bindAddr, opts..., ) + // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower } @@ -696,6 +698,7 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { var nodeInfos []bootstrap.NodeInfo + // TODO: remove this, networking keys should be provided by the consensus follower config. // get networking keys for all followers networkKeys, err := unittest.NetworkingKeys(len(confs)) if err != nil { @@ -710,6 +713,7 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for i, conf := range confs { info := bootstrap.NewPrivateNodeInfo( + // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address From 91ead4161bd710b97562194bc4a9d7d6ab7fea84 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Sun, 22 Aug 2021 21:19:41 -0700 Subject: [PATCH 235/291] removing unstaked network wip - integration tests for consensus follower --- .../node_builder/access_node_builder.go | 65 ++++++---- .../staked_access_node_builder.go | 94 ++++++++++----- .../unstaked_access_node_builder.go | 55 +++++---- cmd/node_builder.go | 4 +- cmd/scaffold.go | 15 ++- follower/consensus_follower.go | 14 ++- integration/testnet/container.go | 14 +-- integration/testnet/network.go | 114 ++++++++++-------- .../tests/access/unstaked_node_test.go | 10 +- 9 files changed, 232 insertions(+), 153 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 75e87a708dd..b4bdf3f6d36 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -41,7 +41,7 @@ import ( "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/module/synchronization" "github.com/onflow/flow-go/network" - jsoncodec "github.com/onflow/flow-go/network/codec/json" + cborcodec "github.com/onflow/flow-go/network/codec/cbor" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/validator" "github.com/onflow/flow-go/state/protocol" @@ -73,9 +73,9 @@ type AccessNodeBuilder interface { // IsStaked returns True is this is a staked Access Node, False otherwise IsStaked() bool - // ParticipatesInUnstakedNetwork returns True if this an Access Node which participates in the unstaked network, - // False otherwise - ParticipatesInUnstakedNetwork() bool + // SupportsUnstakedNode returns True if this is a staked Access node which also supports + // unstaked access nodes/unstaked consensus follower engines, False otherwise. + SupportsUnstakedNode() bool // Build defines all of the Access node's components and modules. Build() AccessNodeBuilder @@ -89,7 +89,8 @@ type AccessNodeConfig struct { bootstrapNodeAddresses []string bootstrapNodePublicKeys []string bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes - unstakedNetworkBindAddr string + NetworkKey crypto.PrivateKey // the networking key passed in by the caller when being used as a library + supportsUnstakedFollower bool collectionGRPCPort uint executionGRPCPort uint pingEnabled bool @@ -137,7 +138,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { staked: true, bootstrapNodeAddresses: []string{}, bootstrapNodePublicKeys: []string{}, - unstakedNetworkBindAddr: cmd.NotSet, + supportsUnstakedFollower: false, } } @@ -149,9 +150,7 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - UnstakedLibP2PNode *p2p.Node - UnstakedNetwork p2p.ReadyDoneAwareNetwork - unstakedMiddleware network.Middleware + LibP2PNode *p2p.Node FollowerState protocol.MutableState SyncCore *synchronization.Core RpcEng *rpc.Engine @@ -496,9 +495,15 @@ func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { } } -func WithUnstakedNetworkBindAddr(bindAddr string) Option { +func SupportsUnstakedNode(enable bool) Option { return func(config *AccessNodeConfig) { - config.unstakedNetworkBindAddr = bindAddr + config.supportsUnstakedFollower = enable + } +} + +func WithNetworkKey(key crypto.PrivateKey) Option { + return func(config *AccessNodeConfig) { + config.NetworkKey = key } } @@ -524,8 +529,14 @@ func (builder *FlowAccessNodeBuilder) IsStaked() bool { return builder.staked } -func (builder *FlowAccessNodeBuilder) ParticipatesInUnstakedNetwork() bool { - return builder.unstakedNetworkBindAddr != cmd.NotSet +func (builder *FlowAccessNodeBuilder) SupportsUnstakedNode() bool { + // unstaked access nodes can't be upstream of other unstaked access nodes for now + if !builder.IsStaked() { + return false + } + + // a staked access node may or may not support unstaked follower + return builder.supportsUnstakedFollower } func (builder *FlowAccessNodeBuilder) ParseFlags() { @@ -566,7 +577,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.BoolVar(&builder.staked, "staked", defaultConfig.staked, "whether this node is a staked access node or not") flags.StringSliceVar(&builder.bootstrapNodeAddresses, "bootstrap-node-addresses", defaultConfig.bootstrapNodeAddresses, "the network addresses of the bootstrap access node if this is an unstaked access node e.g. access-001.mainnet.flow.org:9653,access-002.mainnet.flow.org:9653") flags.StringSliceVar(&builder.bootstrapNodePublicKeys, "bootstrap-node-public-keys", defaultConfig.bootstrapNodePublicKeys, "the networking public key of the bootstrap access node if this is an unstaked access node (in the same order as the bootstrap node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") - flags.StringVar(&builder.unstakedNetworkBindAddr, "unstaked-bind-addr", defaultConfig.unstakedNetworkBindAddr, "address to bind on for the unstaked network") + flags.BoolVar(&builder.supportsUnstakedFollower, "supports-unstaked-node", defaultConfig.supportsUnstakedFollower, "true if this staked access node supports unstaked node") }) } @@ -586,19 +597,24 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, dhtOptions = append(dhtOptions, bootstrapPeersOpt) } + myAddr := builder.NodeConfig.Me.Address() + if builder.BaseConfig.BindAddr != cmd.NotSet { + myAddr = builder.BaseConfig.BindAddr + } + return func() (*p2p.Node, error) { - libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.unstakedNetworkBindAddr, networkKey). + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). SetRootBlockID(builder.RootBlock.ID().String()). - // unlike the staked network where currently all the node addresses are known upfront, - // for the unstaked network the nodes need to discover each other using DHT Discovery. + // unlike the staked side of the network where currently all the node addresses are known upfront, + // for the unstaked side of the network, the nodes need to discover each other using DHT Discovery. SetDHTOptions(dhtOptions...). SetLogger(builder.Logger). Build(ctx) if err != nil { return nil, err } - builder.UnstakedLibP2PNode = libp2pNode - return builder.UnstakedLibP2PNode, nil + builder.LibP2PNode = libp2pNode + return builder.LibP2PNode, nil }, nil } @@ -607,9 +623,8 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, networkMetrics module.NetworkMetrics, factoryFunc p2p.LibP2PFactoryFunc, - - validators ...network.MessageValidator) *network.Middleware { - builder.unstakedMiddleware = p2p.NewMiddleware( + validators ...network.MessageValidator) network.Middleware { + builder.Middleware = p2p.NewMiddleware( builder.Logger, factoryFunc, nodeID, @@ -623,7 +638,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, p2p.WithMessageValidators(validators...), // use default identifier provider ) - return builder.unstakedMiddleware + return builder.Middleware } // initNetwork creates the network.Network implementation with the given metrics, middleware, initial list of network @@ -634,7 +649,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, middleware network.Middleware, topology network.Topology) (*p2p.Network, error) { - codec := jsoncodec.NewCodec() + codec := cborcodec.NewCodec() subscriptionManager := p2p.NewChannelSubscriptionManager(middleware) @@ -643,7 +658,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, builder.Logger, codec, nodeID, - builder.unstakedMiddleware, + builder.Middleware, p2p.DefaultCacheSize, topology, subscriptionManager, diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index e880875b10e..9e5ffb4cd01 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -4,15 +4,15 @@ import ( "context" "fmt" + dht "github.com/libp2p/go-libp2p-kad-dht" + "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/crypto" pingeng "github.com/onflow/flow-go/engine/access/ping" - "github.com/onflow/flow-go/engine/access/relay" - splitternetwork "github.com/onflow/flow-go/engine/common/splitter/network" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" - "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" ) @@ -31,6 +31,7 @@ func NewStakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *StakedAccessNodeBui func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, node.ProtocolEvents) if err != nil { return err @@ -45,9 +46,10 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { ), idCache, ) + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: NetworkingIdentifierProvider should be the same as the one used in scaffold.go if this AN + // TODO: NetworkingIdentifierProvidzer should be the same as the one used in scaffold.go if this AN // doesn't participate in unstaked network. // If it does, then we can just use the default one (peerstoreProvider) @@ -62,13 +64,12 @@ func (builder *StakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() - // for the staked access node, initialize the network used to communicate with the other staked flow nodes - // by calling the EnqueueNetworkInit on the base FlowBuilder like any other staked node - builder.EnqueueNetworkInit(ctx) - - // if this is upstream staked AN for unstaked ANs, initialize the network to communicate on the unstaked network - if builder.ParticipatesInUnstakedNetwork() { + // if this is an access node that supports unstaked followers, enqueue the unstaked network + if builder.SupportsUnstakedNode() { builder.enqueueUnstakedNetworkInit(ctx) + } else { + // otherwise, enqueue the regular network + builder.EnqueueNetworkInit(ctx) } builder.EnqueueMetricsServerInit() @@ -107,36 +108,65 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // NodeID for the staked node on the unstaked network - // TODO: set a different node ID of the staked access node on the unstaked network - unstakedNodeID := builder.NodeID // currently set the same as the staked NodeID - - // Networking key - // TODO: set a different networking key of the staked access node on the unstaked network - unstakedNetworkKey := builder.NetworkKey - - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NodeConfig.NetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) - - // Network Metrics - // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics - // TODO: define new network metrics for the unstaked network - unstakedNetworkMetrics := metrics.NewNoopCollector() + msgValidators := unstakedNetworkMsgValidators(builder.NodeID) - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront - top := topology.EmptyListTopology{} + top, err := topology.NewTopicBasedTopology( + builder.NodeID, + builder.Logger, + builder.State, + ) + if err != nil { + return nil, fmt.Errorf("could not create topology: %w", err) + } + topologyCache := topology.NewCache(builder.Logger, top) - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, top) + network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, topologyCache) builder.MustNot(err) - builder.UnstakedNetwork = network - builder.unstakedMiddleware = middleware + builder.Network = network + builder.Middleware = middleware - node.Logger.Info().Msgf("unstaked network will run on address: %s", builder.unstakedNetworkBindAddr) - return builder.UnstakedNetwork, err + node.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + return builder.Network, err }) } + +// initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. +// The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance +func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, + nodeID flow.Identifier, + networkKey crypto.PrivateKey) (p2p.LibP2PFactoryFunc, error) { + + // The staked nodes act as the DHT servers + dhtOptions := []dht.Option{p2p.AsServer(builder.IsStaked())} + + myAddr := builder.NodeConfig.Me.Address() + if builder.BaseConfig.BindAddr != cmd.NotSet { + myAddr = builder.BaseConfig.BindAddr + } + + connManager := p2p.NewConnManager(builder.Logger, builder.Metrics.Network) + + return func() (*p2p.Node, error) { + libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). + SetRootBlockID(builder.RootBlock.ID().String()). + // no connection gater + SetConnectionManager(connManager). + // act as a DHT server + SetDHTOptions(dhtOptions...). + SetPubsubOptions(p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize)...). + SetLogger(builder.Logger). + Build(ctx) + if err != nil { + return nil, err + } + builder.LibP2PNode = libp2pNode + return builder.LibP2PNode, nil + }, nil +} diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index a4e0085eaca..78706ba010a 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -21,6 +21,16 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } +func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { + // use the networking key that has been passed in the config + networkingKey := fnb.AccessNodeConfig.NetworkKey + nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + fnb.MustNot(err) + fnb.NodeID = nodeID + fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig + fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node +} + func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() @@ -43,6 +53,11 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() + // if a network key has been passed in the init node info here + if builder.AccessNodeConfig.NetworkKey != nil { + builder.initNodeInfo() + } + builder.InitIDProviders() builder.deriveBootstrapPeerIdentities() @@ -51,24 +66,12 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() - builder.EnqueueMetricsServerInit() - - builder.RegisterBadgerMetrics() - - builder.EnqueueTracer() - builder.PreInit(builder.initUnstakedLocal()) return builder } func (builder *UnstakedAccessNodeBuilder) validateParams() { - - // for an unstaked access node, the unstaked network bind address must be provided - if builder.unstakedNetworkBindAddr == cmd.NotSet { - builder.Logger.Fatal().Msg("unstaked bind address not set") - } - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") } @@ -93,7 +96,7 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N NetworkPubKey: node.NetworkKey.PublicKey(), StakingPubKey: nil, // no staking key needed for the unstaked node Role: flow.RoleAccess, // unstaked node can only run as an access node - Address: builder.unstakedNetworkBindAddr, + Address: builder.BindAddr, } me, err := local.New(self, nil) @@ -102,14 +105,16 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N } } +// Build enqueues the sync engine and the follower engine for the unstaked access node. +// Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { - anb. - Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - // use the default identifier provider - node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() - return nil - }) - anb.FlowAccessNodeBuilder.Build() + //anb. + // Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + // // use the default identifier provider + // node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() + // return nil + // }) + anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } @@ -139,16 +144,16 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) builder.MustNot(err) - builder.UnstakedNetwork = network - builder.unstakedMiddleware = middleware + builder.Network = network + builder.Middleware = middleware // for an unstaked node, the staked network and middleware is set to the same as the unstaked network and middlware builder.Network = network builder.Middleware = middleware - builder.Logger.Info().Msgf("unstaked network will run on address: %s", builder.unstakedNetworkBindAddr) + builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) - return builder.UnstakedNetwork, err + return builder.Network, err }) } @@ -160,6 +165,6 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.UnstakedLibP2PNode, builder.Logger), nil + return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil }) } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index f0e1b39219a..63f64381a2f 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -95,7 +95,7 @@ type NodeBuilder interface { // while for a node running as a library, the config fields are expected to be initialized by the caller. type BaseConfig struct { nodeIDHex string - bindAddr string + BindAddr string NodeRole string timeout time.Duration datadir string @@ -156,7 +156,7 @@ func DefaultBaseConfig() *BaseConfig { datadir := filepath.Join(homedir, ".flow", "database") return &BaseConfig{ nodeIDHex: NotSet, - bindAddr: NotSet, + BindAddr: NotSet, BootstrapDir: "bootstrap", timeout: 1 * time.Minute, datadir: datadir, diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 178b31e5015..91f6a13e6cd 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -110,7 +110,7 @@ func (fnb *FlowNodeBuilder) BaseFlags() { // bind configuration parameters fnb.flags.StringVar(&fnb.BaseConfig.nodeIDHex, "nodeid", defaultConfig.nodeIDHex, "identity of our node") - fnb.flags.StringVar(&fnb.BaseConfig.bindAddr, "bind", defaultConfig.bindAddr, "address to bind on") + fnb.flags.StringVar(&fnb.BaseConfig.BindAddr, "bind", defaultConfig.BindAddr, "address to bind on") fnb.flags.StringVarP(&fnb.BaseConfig.BootstrapDir, "bootstrapdir", "b", defaultConfig.BootstrapDir, "path to the bootstrap directory") fnb.flags.DurationVarP(&fnb.BaseConfig.timeout, "timeout", "t", defaultConfig.timeout, "node startup / shutdown timeout") fnb.flags.StringVarP(&fnb.BaseConfig.datadir, "datadir", "d", defaultConfig.datadir, "directory to store the protocol state") @@ -137,8 +137,8 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { codec := cborcodec.NewCodec() myAddr := fnb.NodeConfig.Me.Address() - if fnb.BaseConfig.bindAddr != NotSet { - myAddr = fnb.BaseConfig.bindAddr + if fnb.BaseConfig.BindAddr != NotSet { + myAddr = fnb.BaseConfig.BindAddr } // setup the Ping provider to return the software version and the sealed block height @@ -697,9 +697,9 @@ func WithBootstrapDir(bootstrapDir string) Option { } } -func WithNodeID(nodeID flow.Identifier) Option { +func WithBindAddress(bindAddress string) Option { return func(config *BaseConfig) { - config.nodeIDHex = nodeID.String() + config.BindAddr = bindAddress } } @@ -804,7 +804,10 @@ func (fnb *FlowNodeBuilder) Ready() <-chan struct{} { // seed random generator rand.Seed(time.Now().UnixNano()) - fnb.initNodeInfo() + // init nodeinfo by reading the private bootstrap file if not already set + if fnb.NodeID == flow.ZeroID { + fnb.initNodeInfo() + } fnb.initLogger() diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 82062be2361..33a52064bf6 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -24,7 +24,7 @@ type ConsensusFollower interface { // Config contains the configurable fields for a `ConsensusFollower`. type Config struct { - networkPubKey crypto.PublicKey // the network public key of this node + networkPrivKey crypto.PrivateKey // the network private key of this node bootstrapNodes []BootstrapNodeInfo // the bootstrap nodes to use bindAddr string // address to bind on dataDir string // directory to store the protocol state @@ -69,14 +69,13 @@ func getAccessNodeOptions(config *Config) []access.Option { ids := bootstrapIdentities(config.bootstrapNodes) return []access.Option{ access.WithBootStrapPeers(ids...), - access.WithBindAddr(config.bindAddr), access.WithBaseOptions(getBaseOptions(config)), + access.WithNetworkKey(config.networkPrivKey), } } func getBaseOptions(config *Config) []cmd.Option { options := []cmd.Option{ - cmd.WithNetworkPublicKey(config.networkPubKey), cmd.WithMetricsEnabled(false), } if config.bootstrapDir != "" { @@ -85,6 +84,9 @@ func getBaseOptions(config *Config) []cmd.Option { if config.dataDir != "" { options = append(options, cmd.WithDataDir(config.dataDir)) } + if config.bindAddr != "" { + options = append(options, cmd.WithBindAddress(config.bindAddr)) + } return options } @@ -107,13 +109,13 @@ type ConsensusFollowerImpl struct { // NewConsensusFollower creates a new consensus follower. func NewConsensusFollower( - networkPublicKey crypto.PublicKey, // TODO: make this optional. if not explicitly supplied, we can auto-generate one for them. - bootstapIdentities []BootstrapNodeInfo, + networkPrivKey crypto.PrivateKey, bindAddr string, + bootstapIdentities []BootstrapNodeInfo, opts ...Option, ) (*ConsensusFollowerImpl, error) { config := &Config{ - networkPublicKey: networkPublicKey, + networkPrivKey: networkPrivKey, bootstrapNodes: bootstapIdentities, bindAddr: bindAddr, } diff --git a/integration/testnet/container.go b/integration/testnet/container.go index 666c54d349a..8b60384ef58 100644 --- a/integration/testnet/container.go +++ b/integration/testnet/container.go @@ -38,13 +38,13 @@ func init() { // ContainerConfig represents configuration for a node container in the network. type ContainerConfig struct { bootstrap.NodeInfo - ContainerName string - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - Unstaked bool - ParticipatesInUnstakedNetwork bool + ContainerName string + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + Unstaked bool + SupportsUnstakedNodes bool } // ImageName returns the Docker image name for the given config. diff --git a/integration/testnet/network.go b/integration/testnet/network.go index ee1f90c2be7..bccc9c80c86 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -25,9 +25,9 @@ import ( "github.com/onflow/cadence" "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go/cmd/bootstrap/run" "github.com/onflow/flow-go/cmd/bootstrap/utils" - fcrypto "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" @@ -217,14 +217,16 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - networkKey fcrypto.PrivateKey - upstreamNodeID flow.Identifier + nodeID flow.Identifier + networkingPrivKey crypto.PrivateKey + stakedNodeID flow.Identifier } -func NewConsensusFollowerConfig(networkKey fcrypto.PrivateKey, upstreamNodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier, nodeID flow.Identifier) ConsensusFollowerConfig { return ConsensusFollowerConfig{ - networkKey: fcrypto.PrivateKey, - upstreamNodeID: upstreamNodeID, + networkingPrivKey: networkingPrivKey, + stakedNodeID: stakedNodeID, + nodeID: nodeID, // TODO: remove this and derive it from the key instead } } @@ -306,14 +308,14 @@ func (n *NetworkConfig) Swap(i, j int) { // NodeConfig defines the input config for a particular node, specified prior // to network creation. type NodeConfig struct { - Role flow.Role - Stake uint64 - Identifier flow.Identifier - LogLevel zerolog.Level - Ghost bool - AdditionalFlags []string - Debug bool - ParticipatesInPublicNetwork bool // only applicable to Access node + Role flow.Role + Stake uint64 + Identifier flow.Identifier + LogLevel zerolog.Level + Ghost bool + AdditionalFlags []string + Debug bool + SupportsUnstakedNodes bool // only applicable to Access node } func NewNodeConfig(role flow.Role, opts ...func(*NodeConfig)) NodeConfig { @@ -389,9 +391,9 @@ func AsGhost() func(config *NodeConfig) { } } -func AsPublicNetworkParticipant() func(config *NodeConfig) { +func SupportsUnstakedNodes() func(config *NodeConfig) { return func(config *NodeConfig) { - config.ParticipatesInPublicNetwork = true + config.SupportsUnstakedNodes = true } } @@ -458,13 +460,13 @@ func PrepareFlowNetwork(t *testing.T, networkConf NetworkConfig) *FlowNetwork { // add each follower to the network for _, followerConf := range networkConf.ConsensusFollowers { - flowNetwork.AddConsensusFollower(t, bootstrapDir, followerConf) + flowNetwork.addConsensusFollower(t, bootstrapDir, followerConf, confs) } return flowNetwork } -func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig) { +func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, followerConf ConsensusFollowerConfig, containers []ContainerConfig) { tmpdir, err := ioutil.TempDir(TmpRoot, "flow-consensus-follower") require.NoError(t, err) @@ -490,14 +492,35 @@ func (net *FlowNetwork) AddConsensusFollower(t *testing.T, bootstrapDir string, consensus_follower.WithBootstrapDir(followerBootstrapDir), } + var stakedANContainer *ContainerConfig + // find the upstream Access node container for this follower engine + for _, cont := range containers { + if cont.NodeID == followerConf.stakedNodeID { + stakedANContainer = &cont + break + } + } + require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) + + hostPort := strings.Split(stakedANContainer.Address, ":") + require.Len(t, hostPort, 2, "invalid address for staked AN %s", stakedANContainer.Address) + + host := hostPort[0] + portStr := hostPort[1] + portU64, err := strconv.ParseUint(portStr, 10, 32) + require.NoError(t, err) + port := uint(portU64) + + bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ + Host: host, + Port: port, + NetworkPublicKey: stakedANContainer.NetworkPubKey(), + } + // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower := consensus_follower.NewConsensusFollower( - followerConf.networkKey, - followerConf.upstreamNodeID, - bindAddr, - opts..., - ) + follower, err := consensus_follower.NewConsensusFollower(followerConf.networkingPrivKey, bindAddr, + []consensus_follower.BootstrapNodeInfo{bootstrapNodeInfo}, opts...) // TODO: convert key to node ID? or just store with the network key as map key net.ConsensusFollowers[followerConf.nodeID] = follower @@ -638,9 +661,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIPort] = hostGRPCPort net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort - if nodeConf.ParticipatesInPublicNetwork { + if nodeConf.SupportsUnstakedNodes { // TODO: define this flag for Access node - nodeContainer.addFlag("public-network-participant", "true") + nodeContainer.addFlag("supports-unstaked-node", "true") } case flow.RoleConsensus: @@ -663,7 +686,7 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont nodeContainer.bindPort(hostPort, containerPort) nodeContainer.Ports[GhostNodeAPIPort] = hostPort - if nodeConf.ParticipatesInPublicNetwork { + if nodeConf.SupportsUnstakedNodes { // TODO: Currently, it is not possible to create a ghost AN which participates // in the public network, because connection gating is enabled by default and // therefore the ghost node will deny incoming connections from all consensus @@ -698,28 +721,22 @@ func (net *FlowNetwork) WriteRootSnapshot(snapshot *inmem.Snapshot) { func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, error) { var nodeInfos []bootstrap.NodeInfo - // TODO: remove this, networking keys should be provided by the consensus follower config. - // get networking keys for all followers - networkKeys, err := unittest.NetworkingKeys(len(confs)) - if err != nil { - return nil, err - } - - // get staking keys for all followers - stakingKeys, err := unittest.StakingKeys(len(confs)) + // TODO: currently just stashing a dummy key as staking key to prevent the nodeinfo.Type() function from + // returning an error. Eventually, a new key type NodeInfoTypePrivateUnstaked needs to be defined + dummyStakingKey, err := unittest.StakingKey() if err != nil { return nil, err } - for i, conf := range confs { + for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address 0, // no stake - networkKeys[i], - stakingKeys[i], + conf.networkingPrivKey, + dummyStakingKey, ) nodeInfos = append(nodeInfos, info) @@ -747,12 +764,13 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, fmt.Errorf("failed to setup keys: %w", err) } + // generate the follower node keys (follow nodes do not run as docker containers) followerInfos, err := followerNodeInfos(networkConf.ConsensusFollowers) if err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to generate node info for consensus followers: %w", err) } - allNodeInfos = append(toNodeInfos(stakedConfs), followerInfos...) + allNodeInfos := append(toNodeInfos(stakedConfs), followerInfos...) // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), order.Canonical) @@ -902,7 +920,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string) (*flow.Blo return nil, nil, nil, nil, err } - return root, result, seal, allConfs, nil + return root, result, seal, stakedConfs, nil } // setupKeys generates private staking and networking keys for each configured @@ -947,13 +965,13 @@ func setupKeys(networkConf NetworkConfig) ([]ContainerConfig, error) { ) containerConf := ContainerConfig{ - NodeInfo: info, - ContainerName: name, - LogLevel: conf.LogLevel, - Ghost: conf.Ghost, - AdditionalFlags: conf.AdditionalFlags, - Debug: conf.Debug, - ParticipatesInPublicNetwork: conf.ParticipatesInPublicNetwork, + NodeInfo: info, + ContainerName: name, + LogLevel: conf.LogLevel, + Ghost: conf.Ghost, + AdditionalFlags: conf.AdditionalFlags, + Debug: conf.Debug, + SupportsUnstakedNodes: conf.SupportsUnstakedNodes, } confs = append(confs, containerConf) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index f5259c853aa..cc1033c4288 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/rs/zerolog" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" consensus_follower "github.com/onflow/flow-go/follower" @@ -49,7 +50,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { stakedConfig := testnet.NewNodeConfig( flow.RoleAccess, testnet.WithID(suite.stakedID), - testnet.AsUnstakedNetworkParticipant(), + testnet.SupportsUnstakedNodes(), testnet.WithLogLevel(zerolog.InfoLevel), ) nodeConfigs = append(nodeConfigs, stakedConfig) @@ -89,9 +90,14 @@ func (suite *UnstakedAccessSuite) SetupTest() { nodeConfigs = append(nodeConfigs, collConfig) // consensus follower + unstakedKey, err := unittest.NetworkingKey() + require.NoError(suite.T(), err) + // TODO: derive node id from the key suite.unstakedID = unittest.IdentifierFixture() + + followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(suite.unstakedID, suite.stakedID), + testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) From 57393f6ecb6d7ac6b9499557865cddb9f7d06be9 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 17:21:26 -0700 Subject: [PATCH 236/291] fixing unstaked_node_test integration test --- cmd/access/node_builder/unstaked_access_node_builder.go | 6 +++--- integration/testnet/network.go | 9 +++++++-- integration/tests/access/unstaked_node_test.go | 2 +- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 78706ba010a..fb6a2b17492 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -24,11 +24,11 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config networkingKey := fnb.AccessNodeConfig.NetworkKey - nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this fnb.MustNot(err) fnb.NodeID = nodeID - fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig - fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node + fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig + fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { diff --git a/integration/testnet/network.go b/integration/testnet/network.go index bccc9c80c86..98f5b660184 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -243,10 +243,9 @@ type NetworkConfig struct { type NetworkConfigOpt func(*NetworkConfig) -func NewNetworkConfig(name string, nodes []NodeConfig, followers []ConsensusFollowerConfig, opts ...NetworkConfigOpt) NetworkConfig { +func NewNetworkConfig(name string, nodes []NodeConfig, opts ...NetworkConfigOpt) NetworkConfig { c := NetworkConfig{ Nodes: nodes, - ConsensusFollowers: followers, Name: name, NClusters: 1, // default to 1 cluster ViewsInStakingAuction: DefaultViewsInStakingAuction, @@ -285,6 +284,12 @@ func WithClusters(n uint) func(*NetworkConfig) { } } +func WithConsensusFollowers(followers ...ConsensusFollowerConfig) func(*NetworkConfig) { + return func(conf *NetworkConfig) { + conf.ConsensusFollowers = followers + } +} + func (n *NetworkConfig) Len() int { return len(n.Nodes) } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index cc1033c4288..680050bdc07 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -100,7 +100,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, followerConfigs) + conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) From 74a59e8d48952b57af28c4c5c5c2224c31c9a94d Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 18:04:50 -0700 Subject: [PATCH 237/291] making the unstaked node test similar to the mvp test to allow block generation --- .../tests/access/unstaked_node_test.go | 90 ++++++++++--------- 1 file changed, 46 insertions(+), 44 deletions(-) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 680050bdc07..0a18d01617a 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -2,7 +2,9 @@ package access import ( "context" + "fmt" "testing" + "time" "github.com/rs/zerolog" "github.com/stretchr/testify/require" @@ -43,7 +45,23 @@ func (suite *UnstakedAccessSuite) TearDownTest() { } func (suite *UnstakedAccessSuite) SetupTest() { - nodeConfigs := []testnet.NodeConfig{} + suite.buildNetworkConfig() + // start the network + suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.net.Start(suite.ctx) +} + +func (suite *UnstakedAccessSuite) TestReceiveBlocks() { + //go suite.follower.Run(suite.ctx) + // TODO: to be implemented later + time.Sleep(time.Second * 30) +} + +func (suite *UnstakedAccessSuite) OnBlockFinalizedConsumer(finalizedBlockID flow.Identifier) { + fmt.Println(finalizedBlockID.String()) +} + +func (suite *UnstakedAccessSuite) buildNetworkConfig() { // staked access node suite.stakedID = unittest.IdentifierFixture() @@ -53,41 +71,34 @@ func (suite *UnstakedAccessSuite) SetupTest() { testnet.SupportsUnstakedNodes(), testnet.WithLogLevel(zerolog.InfoLevel), ) - nodeConfigs = append(nodeConfigs, stakedConfig) - - // consensus node (ghost) - suite.conID = unittest.IdentifierFixture() - conConfig := testnet.NewNodeConfig( - flow.RoleConsensus, - testnet.WithID(suite.conID), - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, conConfig) - // execution node (unused) - exeConfig := testnet.NewNodeConfig( - flow.RoleExecution, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, exeConfig) + collectionConfigs := []func(*testnet.NodeConfig){ + testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), + testnet.WithAdditionalFlag("--block-rate-delay=100ms"), + testnet.WithLogLevel(zerolog.WarnLevel), + // TODO replace these with actual values + testnet.WithAdditionalFlag("--access-address=null"), + } - // verification node (unused) - verConfig := testnet.NewNodeConfig( - flow.RoleVerification, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, verConfig) + consensusConfigs := []func(config *testnet.NodeConfig){ + testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), + testnet.WithAdditionalFlag("--block-rate-delay=100ms"), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 1)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 1)), + testnet.WithLogLevel(zerolog.WarnLevel), + } - // collection node (unused) - collConfig := testnet.NewNodeConfig( - flow.RoleCollection, - testnet.AsGhost(), - testnet.WithLogLevel(zerolog.FatalLevel), - ) - nodeConfigs = append(nodeConfigs, collConfig) + net := []testnet.NodeConfig{ + testnet.NewNodeConfig(flow.RoleCollection, collectionConfigs...), + testnet.NewNodeConfig(flow.RoleCollection, collectionConfigs...), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.WarnLevel)), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.WarnLevel)), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.WarnLevel), testnet.WithDebugImage(false)), + stakedConfig, + } // consensus follower unstakedKey, err := unittest.NetworkingKey() @@ -95,22 +106,13 @@ func (suite *UnstakedAccessSuite) SetupTest() { // TODO: derive node id from the key suite.unstakedID = unittest.IdentifierFixture() - followerConfigs := []testnet.ConsensusFollowerConfig{ testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } - conf := testnet.NewNetworkConfig("unstaked_node_test", nodeConfigs, testnet.WithConsensusFollowers(followerConfigs...)) + conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) - - // start the network - suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.net.Start(suite.ctx) -} - -func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - go suite.follower.Run(suite.ctx) - // TODO: to be implemented later + suite.follower.AddOnBlockFinalizedConsumer(suite.OnBlockFinalizedConsumer) } From 5adf91e89ae2d87fa2cabb3f00eac8248763eeb5 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 18:30:12 -0700 Subject: [PATCH 238/291] adding default size for cache --- cmd/access/node_builder/unstaked_access_node_builder.go | 6 ++++++ cmd/node_builder.go | 3 +++ integration/tests/access/unstaked_node_test.go | 2 +- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index fb6a2b17492..23a305adfd6 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -66,6 +66,12 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() + //builder.EnqueueMetricsServerInit() + // + //builder.RegisterBadgerMetrics() + // + //builder.EnqueueTracer() + builder.PreInit(builder.initUnstakedLocal()) return builder diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 63f64381a2f..ae844cc1f2c 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -21,6 +21,7 @@ import ( "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/state/protocol/events" + bstorage "github.com/onflow/flow-go/storage/badger" ) const NotSet = "not set" @@ -170,5 +171,7 @@ func DefaultBaseConfig() *BaseConfig { profilerDuration: 10 * time.Second, tracerEnabled: false, metricsEnabled: true, + receiptsCacheSize: bstorage.DefaultCacheSize, + guaranteesCacheSize: bstorage.DefaultCacheSize, } } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 0a18d01617a..daddce23936 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -52,7 +52,7 @@ func (suite *UnstakedAccessSuite) SetupTest() { } func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - //go suite.follower.Run(suite.ctx) + go suite.follower.Run(suite.ctx) // TODO: to be implemented later time.Sleep(time.Second * 30) } From 4aa689256f8e5c34deb4f340d6976e2d80a763a4 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 19:42:07 -0700 Subject: [PATCH 239/291] moving builder.deriveBootstrapPeerIdentities to access_node_builder from unstaked_access_node_builder --- cmd/access/node_builder/access_node_builder.go | 10 ++++++++++ .../node_builder/unstaked_access_node_builder.go | 12 +----------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b4bdf3f6d36..1c53c4155d3 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -546,6 +546,8 @@ func (builder *FlowAccessNodeBuilder) ParseFlags() { builder.extraFlags() builder.ParseAndPrintFlags() + + builder.deriveBootstrapPeerIdentities() } func (builder *FlowAccessNodeBuilder) extraFlags() { @@ -581,6 +583,14 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { }) } +// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstrap peers from the parameters. +// These are the identities of the staked and unstaked ANs also acting as the DHT bootstrap server +func (builder *FlowAccessNodeBuilder) deriveBootstrapPeerIdentities() { + ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) + builder.MustNot(err) + builder.bootstrapIdentites = ids +} + // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. // The factory function is later passed into the initMiddleware function to eventually instantiate the p2p.LibP2PNode instance func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 23a305adfd6..25657415037 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -60,8 +60,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.InitIDProviders() - builder.deriveBootstrapPeerIdentities() - builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() @@ -83,14 +81,6 @@ func (builder *UnstakedAccessNodeBuilder) validateParams() { } } -// deriveBootstrapPeerIdentities derives the Flow Identity of the bootstreap peers from the parameters. -// These are the identity of the staked and unstaked AN also acting as the DHT bootstrap server -func (builder *UnstakedAccessNodeBuilder) deriveBootstrapPeerIdentities() { - ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) - builder.MustNot(err) - builder.bootstrapIdentites = ids -} - // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files @@ -170,7 +160,7 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("unstaked network", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil }) } From 3f2ac35ed9277d68a74987369a2914fa10c72e7e Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Mon, 23 Aug 2021 22:14:53 -0700 Subject: [PATCH 240/291] exposing libp2p port externally in integration test docker container for consensus follower to access --- cmd/access/node_builder/upstream_connector.go | 2 +- integration/testnet/network.go | 15 ++++++++------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/cmd/access/node_builder/upstream_connector.go b/cmd/access/node_builder/upstream_connector.go index fdc608cfadb..233614d1f87 100644 --- a/cmd/access/node_builder/upstream_connector.go +++ b/cmd/access/node_builder/upstream_connector.go @@ -40,7 +40,7 @@ func (connector *upstreamConnector) Ready() <-chan struct{} { defer close(resultChan) // a shorter context for the connection worker - workerCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + workerCtx, cancel := context.WithTimeout(ctx, 30*time.Second) defer cancel() // spawn a connect worker for each bootstrap node diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 98f5b660184..9b14f56bdc4 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -66,6 +66,8 @@ const ( AccessNodeAPIPort = "access-api-port" // AccessNodeAPIProxyPort is the name used for the access node API HTTP proxy port. AccessNodeAPIProxyPort = "access-api-http-proxy-port" + // AccessNodeExternalNetworkPort is the name used for the access node network port accessible from outside any docker container + AccessNodeExternalNetworkPort = "access-external-network-port" // GhostNodeAPIPort is the name used for the access node API port. GhostNodeAPIPort = "ghost-api-port" @@ -507,17 +509,13 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, } require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) - hostPort := strings.Split(stakedANContainer.Address, ":") - require.Len(t, hostPort, 2, "invalid address for staked AN %s", stakedANContainer.Address) - - host := hostPort[0] - portStr := hostPort[1] + portStr := net.AccessPorts[AccessNodeExternalNetworkPort] portU64, err := strconv.ParseUint(portStr, 10, 32) require.NoError(t, err) port := uint(portU64) bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ - Host: host, + Host: "localhost", Port: port, NetworkPublicKey: stakedANContainer.NetworkPubKey(), } @@ -667,7 +665,9 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont net.AccessPorts[AccessNodeAPIProxyPort] = hostHTTPProxyPort if nodeConf.SupportsUnstakedNodes { - // TODO: define this flag for Access node + hostExternalNetworkPort := testingdock.RandomPort(t) + nodeContainer.bindPort(hostExternalNetworkPort, fmt.Sprintf("%s/tcp", strconv.Itoa(DefaultFlowPort))) + net.AccessPorts[AccessNodeExternalNetworkPort] = hostExternalNetworkPort nodeContainer.addFlag("supports-unstaked-node", "true") } @@ -697,6 +697,7 @@ func (net *FlowNetwork) AddNode(t *testing.T, bootstrapDir string, nodeConf Cont // therefore the ghost node will deny incoming connections from all consensus // followers. A flag for the ghost node will need to be created to enable // overriding the default behavior. + return fmt.Errorf("currently ghost node for an access node which supports unstaked node is not implemented") } } From 0eec24f58a30d47fd0c1f3d84b6ea5daa743d028 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:23:29 -0700 Subject: [PATCH 241/291] update node id logic --- cmd/access/node_builder/access_node_builder.go | 8 ++++---- .../unstaked_access_node_builder.go | 18 +++++++++++------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 1c53c4155d3..b85c49e5bed 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -88,7 +88,7 @@ type AccessNodeConfig struct { staked bool bootstrapNodeAddresses []string bootstrapNodePublicKeys []string - bootstrapIdentites flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes + bootstrapIdentities flow.IdentityList // the identity list of bootstrap peers the node uses to discover other nodes NetworkKey crypto.PrivateKey // the networking key passed in by the caller when being used as a library supportsUnstakedFollower bool collectionGRPCPort uint @@ -491,7 +491,7 @@ type Option func(*AccessNodeConfig) func WithBootStrapPeers(bootstrapNodes ...*flow.Identity) Option { return func(config *AccessNodeConfig) { - config.bootstrapIdentites = bootstrapNodes + config.bootstrapIdentities = bootstrapNodes } } @@ -588,7 +588,7 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { func (builder *FlowAccessNodeBuilder) deriveBootstrapPeerIdentities() { ids, err := BootstrapIdentities(builder.bootstrapNodeAddresses, builder.bootstrapNodePublicKeys) builder.MustNot(err) - builder.bootstrapIdentites = ids + builder.bootstrapIdentities = ids } // initLibP2PFactory creates the LibP2P factory function for the given node ID and network key. @@ -602,7 +602,7 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, // if this is an unstaked access node, then seed the DHT with the boostrap identities if !builder.IsStaked() { - bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentites) + bootstrapPeersOpt, err := p2p.WithBootstrapPeers(builder.bootstrapIdentities) builder.MustNot(err) dhtOptions = append(dhtOptions, bootstrapPeersOpt) } diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 25657415037..7024a7d3003 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -3,6 +3,7 @@ package node_builder import ( "context" + "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -24,17 +25,20 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config networkingKey := fnb.AccessNodeConfig.NetworkKey - nodeID, err := flow.PublicKeyToID(networkingKey.PublicKey()) // TODO: verify this + pubKey, err := p2p.LibP2PPublicKeyFromFlow(networkingKey.PublicKey()) + fnb.MustNot(err) + peerID, err := peer.IDFromPublicKey(pubKey) + fnb.MustNot(err) + fnb.NodeID, err = fnb.IDTranslator.GetFlowID(peerID) fnb.MustNot(err) - fnb.NodeID = nodeID fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() + fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { return err @@ -53,13 +57,13 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() + builder.InitIDProviders() + // if a network key has been passed in the init node info here if builder.AccessNodeConfig.NetworkKey != nil { builder.initNodeInfo() } - builder.InitIDProviders() - builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() @@ -161,6 +165,6 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // of an explicit connect to the staked AN before the node attempts to subscribe to topics. func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentites, builder.LibP2PNode, builder.Logger), nil + return newUpstreamConnector(builder.bootstrapIdentities, builder.LibP2PNode, builder.Logger), nil }) } From c039b5b32e9cb5f8715785122ab582db3ff4089d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:31:44 -0700 Subject: [PATCH 242/291] Add NetworkingIdentifierProider to staked node --- cmd/access/node_builder/staked_access_node_builder.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 9e5ffb4cd01..8910c8da79d 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -49,9 +49,9 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) - // TODO: NetworkingIdentifierProvidzer should be the same as the one used in scaffold.go if this AN - // doesn't participate in unstaked network. - // If it does, then we can just use the default one (peerstoreProvider) + if !fnb.SupportsUnstakedNode() { + fnb.NetworkingIdentifierProvider = id.NewFilteredIdentifierProvider(p2p.NotEjectedFilter, idCache) + } return nil }) From 9b644265c5054d5936d84b2bd87ca780d1dd608a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:35:19 -0700 Subject: [PATCH 243/291] Use hierarchical translator for unstaked node --- .../node_builder/unstaked_access_node_builder.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 7024a7d3003..05fcbb6da85 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -29,15 +29,13 @@ func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { fnb.MustNot(err) peerID, err := peer.IDFromPublicKey(pubKey) fnb.MustNot(err) - fnb.NodeID, err = fnb.IDTranslator.GetFlowID(peerID) + fnb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) fnb.MustNot(err) fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.IDTranslator = p2p.NewUnstakedNetworkIDTranslator() - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) if err != nil { @@ -46,6 +44,8 @@ func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { fnb.IdentityProvider = idCache + fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) + return nil }) } @@ -57,13 +57,13 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.validateParams() - builder.InitIDProviders() - // if a network key has been passed in the init node info here if builder.AccessNodeConfig.NetworkKey != nil { builder.initNodeInfo() } + builder.InitIDProviders() + builder.enqueueUnstakedNetworkInit(ctx) builder.enqueueConnectWithStakedAN() From 68a906ee592017a698907bd7c381826902f5077a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 23 Aug 2021 23:38:46 -0700 Subject: [PATCH 244/291] Add identity delta for staked and unstaked AN's --- cmd/access/node_builder/staked_access_node_builder.go | 4 ++++ cmd/access/node_builder/unstaked_access_node_builder.go | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 8910c8da79d..4d00a44895c 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/topology" + "github.com/onflow/flow-go/state/protocol/events/gadgets" ) // StakedAccessNodeBuilder builds a staked access node. The staked access node can optionally participate in the @@ -132,6 +133,9 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C builder.Network = network builder.Middleware = middleware + idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) + builder.ProtocolEvents.AddConsumer(idEvents) + node.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) return builder.Network, err }) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 05fcbb6da85..31cda8ac094 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/state/protocol/events/gadgets" ) type UnstakedAccessNodeBuilder struct { @@ -153,6 +154,9 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) + builder.ProtocolEvents.AddConsumer(idEvents) + return builder.Network, err }) } From 5e6db19a9f680e044763994ea4e1524f381f0414 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 00:19:57 -0700 Subject: [PATCH 245/291] Implement proper message validators for unstaked network. --- .../node_builder/access_node_builder.go | 9 +++++- .../staked_access_node_builder.go | 2 +- .../unstaked_access_node_builder.go | 2 +- network/validator/anyValidator.go | 28 +++++++++++++++++++ network/validator/originValidator.go | 24 ++++++++++++++++ 5 files changed, 62 insertions(+), 3 deletions(-) create mode 100644 network/validator/anyValidator.go create mode 100644 network/validator/originValidator.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b85c49e5bed..9436f1fdb82 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -9,6 +9,7 @@ import ( dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/onflow/flow/protobuf/go/flow/access" + "github.com/rs/zerolog" "github.com/spf13/pflag" "google.golang.org/grpc" "google.golang.org/grpc/credentials" @@ -682,10 +683,16 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, return net, nil } -func unstakedNetworkMsgValidators(selfID flow.Identifier) []network.MessageValidator { +func unstakedNetworkMsgValidators(log zerolog.Logger, idProvider id.IdentityProvider, selfID flow.Identifier) []network.MessageValidator { return []network.MessageValidator{ // filter out messages sent by this node itself validator.ValidateNotSender(selfID), + validator.NewAnyValidator( + validator.NewOriginValidator( + id.NewFilteredIdentifierProvider(filter.IsValidCurrentEpochParticipant, idProvider), + ), + validator.ValidateTarget(log, selfID), + ), } } diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 4d00a44895c..827f056072e 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -112,7 +112,7 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C libP2PFactory, err := builder.initLibP2PFactory(ctx, builder.NodeID, builder.NodeConfig.NetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(builder.NodeID) + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, builder.NodeID) middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 31cda8ac094..ea50d7e93a7 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -137,7 +137,7 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) builder.MustNot(err) - msgValidators := unstakedNetworkMsgValidators(unstakedNodeID) + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) diff --git a/network/validator/anyValidator.go b/network/validator/anyValidator.go new file mode 100644 index 00000000000..0639de95bbb --- /dev/null +++ b/network/validator/anyValidator.go @@ -0,0 +1,28 @@ +package validator + +import ( + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" +) + +var _ network.MessageValidator = &AnyValidator{} + +// AnyValidator returns true if any of the given validators returns true +type AnyValidator struct { + validators []network.MessageValidator +} + +func NewAnyValidator(validators ...network.MessageValidator) network.MessageValidator { + return &AnyValidator{ + validators: validators, + } +} + +func (v AnyValidator) Validate(msg message.Message) bool { + for _, validator := range v.validators { + if validator.Validate(msg) { + return true + } + } + return false +} diff --git a/network/validator/originValidator.go b/network/validator/originValidator.go new file mode 100644 index 00000000000..250f464d6d5 --- /dev/null +++ b/network/validator/originValidator.go @@ -0,0 +1,24 @@ +package validator + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" +) + +var _ network.MessageValidator = &OriginValidator{} + +// OriginValidator returns true if the sender of the message is among the set of identifiers +// returned by the given IdentifierProvider +type OriginValidator struct { + idProvider id.IdentifierProvider +} + +func NewOriginValidator(provider id.IdentifierProvider) network.MessageValidator { + return &OriginValidator{provider} +} + +func (v OriginValidator) Validate(msg message.Message) bool { + return v.idProvider.Identifiers().Contains(flow.HashToID(msg.OriginID)) +} From e8938937f2a8bc339981660f8998e17415d3ac0d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 00:31:13 -0700 Subject: [PATCH 246/291] recover sync engine participants provider --- .../unstaked_access_node_builder.go | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index ea50d7e93a7..54313ce2b21 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -2,6 +2,7 @@ package node_builder import ( "context" + "errors" "github.com/libp2p/go-libp2p-core/peer" "github.com/onflow/flow-go/cmd" @@ -109,12 +110,17 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { - //anb. - // Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - // // use the default identifier provider - // node.SyncEngineIdentifierProvider = node.Middleware.IdentifierProvider() - // return nil - // }) + anb. + Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + middleware, ok := node.Middleware.(*p2p.Middleware) + if !ok { + return errors.New("middleware was of unexpected type") + } + // use the default identifier provider + anb.SyncEngineParticipantsProvider = middleware.IdentifierProvider() + return nil + }) + anb.FlowAccessNodeBuilder.BuildConsensusFollower() return anb } From 5047a726f86fe914352234c79283418a3c712365 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 11:02:50 -0700 Subject: [PATCH 247/291] using the unstaked networking key for the consesus follower --- .../unstaked_access_node_builder.go | 6 ------ .../tests/access/unstaked_node_test.go | 19 ++++++++++++++++--- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 54313ce2b21..1ce91142d40 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -70,12 +70,6 @@ func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { builder.enqueueConnectWithStakedAN() - //builder.EnqueueMetricsServerInit() - // - //builder.RegisterBadgerMetrics() - // - //builder.EnqueueTracer() - builder.PreInit(builder.initUnstakedLocal()) return builder diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index daddce23936..6bdac10eb30 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -2,6 +2,7 @@ package access import ( "context" + "crypto/rand" "fmt" "testing" "time" @@ -10,6 +11,8 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/onflow/flow-go/cmd/bootstrap/utils" + "github.com/onflow/flow-go/crypto" consensus_follower "github.com/onflow/flow-go/follower" "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" @@ -69,7 +72,7 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { flow.RoleAccess, testnet.WithID(suite.stakedID), testnet.SupportsUnstakedNodes(), - testnet.WithLogLevel(zerolog.InfoLevel), + testnet.WithLogLevel(zerolog.TraceLevel), ) collectionConfigs := []func(*testnet.NodeConfig){ @@ -100,8 +103,7 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { stakedConfig, } - // consensus follower - unstakedKey, err := unittest.NetworkingKey() + unstakedKey, err := UnstakedNetworkingKey() require.NoError(suite.T(), err) // TODO: derive node id from the key suite.unstakedID = unittest.IdentifierFixture() @@ -110,9 +112,20 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), } + // consensus follower conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) suite.follower = suite.net.ConsensusFollowerByID(suite.unstakedID) suite.follower.AddOnBlockFinalizedConsumer(suite.OnBlockFinalizedConsumer) } + +// TODO: Move this to unittest and resolve the circular dependency issue +func UnstakedNetworkingKey() (crypto.PrivateKey, error) { + seed := make([]byte, crypto.KeyGenSeedMinLenECDSASecp256k1) + n, err := rand.Read(seed) + if err != nil || n != crypto.KeyGenSeedMinLenECDSASecp256k1 { + return nil, err + } + return utils.GenerateUnstakedNetworkingKey(unittest.SeedFixture(n)) +} From 7ff21f90b096ae292d4872214857892e8e25f43a Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:06:10 -0700 Subject: [PATCH 248/291] chance consensus follower to use build function --- follower/consensus_follower.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index 33a52064bf6..f35ab03689e 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -96,7 +96,7 @@ func buildAccessNode(accessNodeOptions []access.Option) *access.UnstakedAccessNo nodeBuilder := access.NewUnstakedAccessNodeBuilder(anb) nodeBuilder.Initialize() - nodeBuilder.BuildConsensusFollower() + nodeBuilder.Build() return nodeBuilder } From 358eb5c463e8a88534a583b757c8b50f9fc8252d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:27:30 -0700 Subject: [PATCH 249/291] derive node id from key --- integration/testnet/network.go | 19 ++++++++++++------- .../tests/access/unstaked_node_test.go | 4 +--- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 9b14f56bdc4..2a4c4f34f92 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -37,6 +37,7 @@ import ( "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" + "github.com/onflow/flow-go/network/p2p" clusterstate "github.com/onflow/flow-go/state/cluster" "github.com/onflow/flow-go/state/protocol/inmem" "github.com/onflow/flow-go/utils/io" @@ -219,16 +220,20 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier + nodeID flow.Identifier networkingPrivKey crypto.PrivateKey - stakedNodeID flow.Identifier + stakedNodeID flow.Identifier } -func NewConsensusFollowerConfig(networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier, nodeID flow.Identifier) ConsensusFollowerConfig { +func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier) ConsensusFollowerConfig { + pid, err := p2p.ExtractPeerID(networkingPrivKey.PublicKey()) + assert.NoError(t, err) + nodeID, err := p2p.NewUnstakedNetworkIDTranslator().GetFlowID(pid) + assert.NoError(t, err) return ConsensusFollowerConfig{ networkingPrivKey: networkingPrivKey, - stakedNodeID: stakedNodeID, - nodeID: nodeID, // TODO: remove this and derive it from the key instead + stakedNodeID: stakedNodeID, + nodeID: nodeID, } } @@ -515,8 +520,8 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, port := uint(portU64) bootstrapNodeInfo := consensus_follower.BootstrapNodeInfo{ - Host: "localhost", - Port: port, + Host: "localhost", + Port: port, NetworkPublicKey: stakedANContainer.NetworkPubKey(), } diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 6bdac10eb30..c4ff35d242b 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -105,11 +105,9 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { unstakedKey, err := UnstakedNetworkingKey() require.NoError(suite.T(), err) - // TODO: derive node id from the key - suite.unstakedID = unittest.IdentifierFixture() followerConfigs := []testnet.ConsensusFollowerConfig{ - testnet.NewConsensusFollowerConfig(unstakedKey, suite.stakedID, suite.unstakedID), + testnet.NewConsensusFollowerConfig(suite.T(), unstakedKey, suite.stakedID), } // consensus follower From 693041603c38c5d74ca3f957a5d8f0217af97949 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:30:05 -0700 Subject: [PATCH 250/291] remove comment --- integration/testnet/network.go | 1 - 1 file changed, 1 deletion(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index 2a4c4f34f92..a27410ddef7 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -741,7 +741,6 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( - // TODO: Need to convert from network key here conf.nodeID, flow.RoleAccess, // use Access role "", // no address From f4bc94e7a0306f14257bd98a598b3986322789a1 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:42:09 -0700 Subject: [PATCH 251/291] remove topology --- .../node_builder/staked_access_node_builder.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 827f056072e..10068570bb9 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -117,17 +117,9 @@ func (builder *StakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.C middleware := builder.initMiddleware(builder.NodeID, node.Metrics.Network, libP2PFactory, msgValidators...) // topology returns empty list since peers are not known upfront - top, err := topology.NewTopicBasedTopology( - builder.NodeID, - builder.Logger, - builder.State, - ) - if err != nil { - return nil, fmt.Errorf("could not create topology: %w", err) - } - topologyCache := topology.NewCache(builder.Logger, top) + top := topology.EmptyListTopology{} - network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, topologyCache) + network, err := builder.initNetwork(builder.Me, node.Metrics.Network, middleware, top) builder.MustNot(err) builder.Network = network From ca472332664b47fd13356dac7083aa30dcb0e5f8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 24 Aug 2021 11:47:51 -0700 Subject: [PATCH 252/291] fix node ID --- integration/testnet/network.go | 24 +++++++++---------- .../tests/access/unstaked_node_test.go | 2 ++ 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/integration/testnet/network.go b/integration/testnet/network.go index a27410ddef7..26489f23982 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -220,9 +220,9 @@ func (net *FlowNetwork) ContainerByName(name string) *Container { } type ConsensusFollowerConfig struct { - nodeID flow.Identifier - networkingPrivKey crypto.PrivateKey - stakedNodeID flow.Identifier + NodeID flow.Identifier + NetworkingPrivKey crypto.PrivateKey + StakedNodeID flow.Identifier } func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKey, stakedNodeID flow.Identifier) ConsensusFollowerConfig { @@ -231,9 +231,9 @@ func NewConsensusFollowerConfig(t *testing.T, networkingPrivKey crypto.PrivateKe nodeID, err := p2p.NewUnstakedNetworkIDTranslator().GetFlowID(pid) assert.NoError(t, err) return ConsensusFollowerConfig{ - networkingPrivKey: networkingPrivKey, - stakedNodeID: stakedNodeID, - nodeID: nodeID, + NetworkingPrivKey: networkingPrivKey, + StakedNodeID: stakedNodeID, + NodeID: nodeID, } } @@ -507,12 +507,12 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, var stakedANContainer *ContainerConfig // find the upstream Access node container for this follower engine for _, cont := range containers { - if cont.NodeID == followerConf.stakedNodeID { + if cont.NodeID == followerConf.StakedNodeID { stakedANContainer = &cont break } } - require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.nodeID.String()) + require.NotNil(t, stakedANContainer, "unable to find staked AN for the follower engine %s", followerConf.NodeID.String()) portStr := net.AccessPorts[AccessNodeExternalNetworkPort] portU64, err := strconv.ParseUint(portStr, 10, 32) @@ -527,11 +527,11 @@ func (net *FlowNetwork) addConsensusFollower(t *testing.T, bootstrapDir string, // TODO: update consensus follower to just accept a networking key instead of a node ID // it should be able to figure out the rest on its own. - follower, err := consensus_follower.NewConsensusFollower(followerConf.networkingPrivKey, bindAddr, + follower, err := consensus_follower.NewConsensusFollower(followerConf.NetworkingPrivKey, bindAddr, []consensus_follower.BootstrapNodeInfo{bootstrapNodeInfo}, opts...) // TODO: convert key to node ID? or just store with the network key as map key - net.ConsensusFollowers[followerConf.nodeID] = follower + net.ConsensusFollowers[followerConf.NodeID] = follower } // AddNode creates a node container with the given config and adds it to the @@ -741,11 +741,11 @@ func followerNodeInfos(confs []ConsensusFollowerConfig) ([]bootstrap.NodeInfo, e for _, conf := range confs { info := bootstrap.NewPrivateNodeInfo( - conf.nodeID, + conf.NodeID, flow.RoleAccess, // use Access role "", // no address 0, // no stake - conf.networkingPrivKey, + conf.NetworkingPrivKey, dummyStakingKey, ) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index c4ff35d242b..b48db55734f 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -110,6 +110,8 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.NewConsensusFollowerConfig(suite.T(), unstakedKey, suite.stakedID), } + suite.unstakedID = followerConfigs[0].NodeID + // consensus follower conf := testnet.NewNetworkConfig("consensus follower test", net, testnet.WithConsensusFollowers(followerConfigs...)) suite.net = testnet.PrepareFlowNetwork(suite.T(), conf) From 8a8e0a179b8c158e652205d17a689f425c01ec85 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 11:53:34 -0700 Subject: [PATCH 253/291] removing unneeded flags --- integration/tests/access/unstaked_node_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index b48db55734f..03a9d1f8af6 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -79,15 +79,13 @@ func (suite *UnstakedAccessSuite) buildNetworkConfig() { testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), testnet.WithAdditionalFlag("--block-rate-delay=100ms"), testnet.WithLogLevel(zerolog.WarnLevel), - // TODO replace these with actual values - testnet.WithAdditionalFlag("--access-address=null"), } consensusConfigs := []func(config *testnet.NodeConfig){ testnet.WithAdditionalFlag("--hotstuff-timeout=12s"), testnet.WithAdditionalFlag("--block-rate-delay=100ms"), - testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 1)), - testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 1)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-verification-seal-approvals=%d", 0)), + testnet.WithAdditionalFlag(fmt.Sprintf("--required-construction-seal-approvals=%d", 0)), testnet.WithLogLevel(zerolog.WarnLevel), } From 14b19094303e94f1862f0dc7f00e667536640e92 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 13:09:35 -0700 Subject: [PATCH 254/291] initializing middleware before initiliazing the sync follower for the unstaked node --- .../unstaked_access_node_builder.go | 134 ++++++++++-------- 1 file changed, 73 insertions(+), 61 deletions(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 1ce91142d40..044c58564a0 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -5,6 +5,7 @@ import ( "errors" "github.com/libp2p/go-libp2p-core/peer" + "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -24,67 +25,69 @@ func NewUnstakedAccessNodeBuilder(anb *FlowAccessNodeBuilder) *UnstakedAccessNod } } -func (fnb *UnstakedAccessNodeBuilder) initNodeInfo() { +func (anb *UnstakedAccessNodeBuilder) initNodeInfo() { // use the networking key that has been passed in the config - networkingKey := fnb.AccessNodeConfig.NetworkKey + networkingKey := anb.AccessNodeConfig.NetworkKey pubKey, err := p2p.LibP2PPublicKeyFromFlow(networkingKey.PublicKey()) - fnb.MustNot(err) + anb.MustNot(err) peerID, err := peer.IDFromPublicKey(pubKey) - fnb.MustNot(err) - fnb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) - fnb.MustNot(err) - fnb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig - fnb.NodeConfig.StakingKey = nil // no staking key for the unstaked node + anb.MustNot(err) + anb.NodeID, err = p2p.NewUnstakedNetworkIDTranslator().GetFlowID(peerID) + anb.MustNot(err) + anb.NodeConfig.NetworkKey = networkingKey // copy the key to NodeConfig + anb.NodeConfig.StakingKey = nil // no staking key for the unstaked node } -func (fnb *UnstakedAccessNodeBuilder) InitIDProviders() { - fnb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, fnb.ProtocolEvents) +func (anb *UnstakedAccessNodeBuilder) InitIDProviders() { + anb.Module("id providers", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { + idCache, err := p2p.NewProtocolStateIDCache(node.Logger, node.State, anb.ProtocolEvents) if err != nil { return err } - fnb.IdentityProvider = idCache + anb.IdentityProvider = idCache - fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) + anb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) return nil }) } -func (builder *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { +func (anb *UnstakedAccessNodeBuilder) Initialize() cmd.NodeBuilder { ctx, cancel := context.WithCancel(context.Background()) - builder.Cancel = cancel + anb.Cancel = cancel - builder.validateParams() + anb.validateParams() // if a network key has been passed in the init node info here - if builder.AccessNodeConfig.NetworkKey != nil { - builder.initNodeInfo() + if anb.AccessNodeConfig.NetworkKey != nil { + anb.initNodeInfo() } - builder.InitIDProviders() + anb.InitIDProviders() + + anb.enqueueMiddleware(ctx) - builder.enqueueUnstakedNetworkInit(ctx) + anb.enqueueUnstakedNetworkInit(ctx) - builder.enqueueConnectWithStakedAN() + anb.enqueueConnectWithStakedAN() - builder.PreInit(builder.initUnstakedLocal()) + anb.PreInit(anb.initUnstakedLocal()) - return builder + return anb } -func (builder *UnstakedAccessNodeBuilder) validateParams() { - if len(builder.bootstrapNodeAddresses) != len(builder.bootstrapNodePublicKeys) { - builder.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") +func (anb *UnstakedAccessNodeBuilder) validateParams() { + if len(anb.bootstrapNodeAddresses) != len(anb.bootstrapNodePublicKeys) { + anb.Logger.Fatal().Msg("number of bootstrap node addresses and public keys should match") } } // initUnstakedLocal initializes the unstaked node ID, network key and network address // Currently, it reads a node-info.priv.json like any other node. // TODO: read the node ID from the special bootstrap files -func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.NodeBuilder, node *cmd.NodeConfig) { +func (anb *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.NodeBuilder, node *cmd.NodeConfig) { return func(_ cmd.NodeBuilder, node *cmd.NodeConfig) { // for an unstaked node, set the identity here explicitly since it will not be found in the protocol state self := &flow.Identity{ @@ -92,21 +95,48 @@ func (builder *UnstakedAccessNodeBuilder) initUnstakedLocal() func(builder cmd.N NetworkPubKey: node.NetworkKey.PublicKey(), StakingPubKey: nil, // no staking key needed for the unstaked node Role: flow.RoleAccess, // unstaked node can only run as an access node - Address: builder.BindAddr, + Address: anb.BindAddr, } me, err := local.New(self, nil) - builder.MustNot(err).Msg("could not initialize local") + anb.MustNot(err).Msg("could not initialize local") node.Me = me } } +// enqueueMiddleware enqueues the creation of the network middleware +// this needs to be done before sync engine participants module +func (anb *UnstakedAccessNodeBuilder) enqueueMiddleware(ctx context.Context) { + anb. + Module("network middleware", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) error { + + // NodeID for the unstaked node on the unstaked network + unstakedNodeID := node.NodeID + + // Networking key + unstakedNetworkKey := node.NetworkKey + + // Network Metrics + // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics + unstakedNetworkMetrics := metrics.NewNoopCollector() + + libP2PFactory, err := anb.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) + anb.MustNot(err) + + msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) + + anb.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) + + return nil + }) +} + // Build enqueues the sync engine and the follower engine for the unstaked access node. // Currently, the unstaked AN only runs the follower engine. func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. - Module("sync engine participants provider", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) error { - middleware, ok := node.Middleware.(*p2p.Middleware) + Module("sync engine participants provider", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) error { + middleware, ok := anb.Middleware.(*p2p.Middleware) if !ok { return errors.New("middleware was of unexpected type") } @@ -120,44 +150,26 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { } // enqueueUnstakedNetworkInit enqueues the unstaked network component initialized for the unstaked node -func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { - - builder.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { +func (anb *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Context) { - // NodeID for the unstaked node on the unstaked network - unstakedNodeID := node.NodeID - - // Networking key - unstakedNetworkKey := node.NetworkKey + anb.Component("unstaked network", func(_ cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { // Network Metrics // for now we use the empty metrics NoopCollector till we have defined the new unstaked network metrics unstakedNetworkMetrics := metrics.NewNoopCollector() - libP2PFactory, err := builder.initLibP2PFactory(ctx, unstakedNodeID, unstakedNetworkKey) - builder.MustNot(err) - - msgValidators := unstakedNetworkMsgValidators(node.Logger, node.IdentityProvider, unstakedNodeID) - - middleware := builder.initMiddleware(unstakedNodeID, unstakedNetworkMetrics, libP2PFactory, msgValidators...) - // topology is nil since its automatically managed by libp2p - network, err := builder.initNetwork(builder.Me, unstakedNetworkMetrics, middleware, nil) - builder.MustNot(err) - - builder.Network = network - builder.Middleware = middleware + network, err := anb.initNetwork(anb.Me, unstakedNetworkMetrics, anb.Middleware, nil) + anb.MustNot(err) - // for an unstaked node, the staked network and middleware is set to the same as the unstaked network and middlware - builder.Network = network - builder.Middleware = middleware + anb.Network = network - builder.Logger.Info().Msgf("network will run on address: %s", builder.BindAddr) + anb.Logger.Info().Msgf("network will run on address: %s", anb.BindAddr) - idEvents := gadgets.NewIdentityDeltas(builder.Middleware.UpdateNodeAddresses) - builder.ProtocolEvents.AddConsumer(idEvents) + idEvents := gadgets.NewIdentityDeltas(anb.Middleware.UpdateNodeAddresses) + anb.ProtocolEvents.AddConsumer(idEvents) - return builder.Network, err + return anb.Network, err }) } @@ -167,8 +179,8 @@ func (builder *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context // (https://github.com/libp2p/go-libp2p-pubsub/issues/442). This means that an unstaked AN could end up not being // discovered by other unstaked ANs if it subscribes to a topic before connecting to the staked AN. Hence, the need // of an explicit connect to the staked AN before the node attempts to subscribe to topics. -func (builder *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { - builder.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return newUpstreamConnector(builder.bootstrapIdentities, builder.LibP2PNode, builder.Logger), nil +func (anb *UnstakedAccessNodeBuilder) enqueueConnectWithStakedAN() { + anb.Component("upstream connector", func(_ cmd.NodeBuilder, _ *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return newUpstreamConnector(anb.bootstrapIdentities, anb.LibP2PNode, anb.Logger), nil }) } From d9f04e0264eb4f47e8cb884fc9f30f736660460a Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 13:59:55 -0700 Subject: [PATCH 255/291] using a factory method for the SyncEngineParticipantsProvider to wait for middleware to have started for the unstaked an --- .../node_builder/access_node_builder.go | 37 ++++++++++--------- .../staked_access_node_builder.go | 19 ++++++---- .../unstaked_access_node_builder.go | 3 +- 3 files changed, 32 insertions(+), 27 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 9436f1fdb82..08f6155acbd 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -151,23 +151,24 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - LibP2PNode *p2p.Node - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + LibP2PNode *p2p.Node + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower + SyncEngineParticipantsProviderFactory func() id.IdentifierProvider // engines IngestEng *ingestion.Engine @@ -320,7 +321,7 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { builder.FollowerEng, builder.SyncCore, builder.FinalizedHeader, - node.SyncEngineIdentifierProvider, + builder.SyncEngineParticipantsProviderFactory(), ) if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 10068570bb9..22c05ee0f7b 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -39,14 +39,17 @@ func (fnb *StakedAccessNodeBuilder) InitIDProviders() { } fnb.IdentityProvider = idCache - fnb.SyncEngineIdentifierProvider = id.NewFilteredIdentifierProvider( - filter.And( - filter.HasRole(flow.RoleConsensus), - filter.Not(filter.HasNodeID(node.Me.NodeID())), - p2p.NotEjectedFilter, - ), - idCache, - ) + // translator + fnb.SyncEngineParticipantsProviderFactory = func() id.IdentifierProvider { + return id.NewFilteredIdentifierProvider( + filter.And( + filter.HasRole(flow.RoleConsensus), + filter.Not(filter.HasNodeID(node.Me.NodeID())), + p2p.NotEjectedFilter, + ), + idCache, + ) + } fnb.IDTranslator = p2p.NewHierarchicalIDTranslator(idCache, p2p.NewUnstakedNetworkIDTranslator()) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index 044c58564a0..eddfcb7ca47 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -9,6 +9,7 @@ import ( "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" @@ -141,7 +142,7 @@ func (anb *UnstakedAccessNodeBuilder) Build() AccessNodeBuilder { return errors.New("middleware was of unexpected type") } // use the default identifier provider - anb.SyncEngineParticipantsProvider = middleware.IdentifierProvider() + anb.SyncEngineParticipantsProviderFactory = func() id.IdentifierProvider { return middleware.IdentifierProvider() } return nil }) From 0f7f6c802e35de4de9472414b8ceeaea3ce7ce73 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 14:06:43 -0700 Subject: [PATCH 256/291] adding comment --- .../node_builder/access_node_builder.go | 37 ++++++++++--------- 1 file changed, 20 insertions(+), 17 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 08f6155acbd..0c1392bfe80 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -151,23 +151,26 @@ type FlowAccessNodeBuilder struct { *AccessNodeConfig // components - LibP2PNode *p2p.Node - FollowerState protocol.MutableState - SyncCore *synchronization.Core - RpcEng *rpc.Engine - FinalizationDistributor *pubsub.FinalizationDistributor - FinalizedHeader *synceng.FinalizedHeaderCache - CollectionRPC access.AccessAPIClient - TransactionTimings *stdmap.TransactionTimings - CollectionsToMarkFinalized *stdmap.Times - CollectionsToMarkExecuted *stdmap.Times - BlocksToMarkExecuted *stdmap.Times - TransactionMetrics module.TransactionMetrics - PingMetrics module.PingMetrics - Committee hotstuff.Committee - Finalized *flow.Header - Pending []*flow.Header - FollowerCore module.HotStuffFollower + LibP2PNode *p2p.Node + FollowerState protocol.MutableState + SyncCore *synchronization.Core + RpcEng *rpc.Engine + FinalizationDistributor *pubsub.FinalizationDistributor + FinalizedHeader *synceng.FinalizedHeaderCache + CollectionRPC access.AccessAPIClient + TransactionTimings *stdmap.TransactionTimings + CollectionsToMarkFinalized *stdmap.Times + CollectionsToMarkExecuted *stdmap.Times + BlocksToMarkExecuted *stdmap.Times + TransactionMetrics module.TransactionMetrics + PingMetrics module.PingMetrics + Committee hotstuff.Committee + Finalized *flow.Header + Pending []*flow.Header + FollowerCore module.HotStuffFollower + // for the untsaked access node, the sync engine participants provider is the libp2p peer store which is not + // available until after the network has started. Hence, a factory function that needs to be called just before + // creating the sync engine SyncEngineParticipantsProviderFactory func() id.IdentifierProvider // engines From 7a25e93bc7b9ed4f6290d76703b6a931f8be2f56 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Tue, 24 Aug 2021 16:49:49 -0700 Subject: [PATCH 257/291] go mod tidy & lints --- .../node_builder/access_node_builder.go | 1 + integration/go.sum | 10 + module/mocks/network.go | 64 +++--- storage/mocks/storage.go | 208 +++++++++--------- 4 files changed, 145 insertions(+), 138 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 0c1392bfe80..a0a320778be 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -37,6 +37,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" + "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/signature" diff --git a/integration/go.sum b/integration/go.sum index 794f7786af0..0a0870f6690 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -88,6 +88,7 @@ github.com/VictoriaMetrics/fastcache v1.5.7/go.mod h1:ptDBkNMQI4RtmVo8VS/XwRY6Ro github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= +github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af h1:wVe6/Ea46ZMeNkQjjBW6xcqyQA/j5e0D6GytH95g0gQ= github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -127,6 +128,7 @@ github.com/blang/semver v3.1.0+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnweb github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40/go.mod h1:8rLXio+WjiTceGBHIoTvn60HIbs7Hm7bcHjyrSqYB9c= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= +github.com/bsipos/thist v1.0.0 h1:vZ3W5/ZnT54s4LHeonTCbnzCb20ERlJUnhiwXoGpsbY= github.com/bsipos/thist v1.0.0/go.mod h1:7i0xwRua1/bmUxcxi2xAxaFL895rLtOpKUwnw3NrT8I= github.com/btcsuite/btcd v0.0.0-20171128150713-2e60448ffcc6/go.mod h1:Dmm/EzmjnCiweXmzRIAiUWCInVmPgjkzgv5k4tVyXiQ= github.com/btcsuite/btcd v0.0.0-20190213025234-306aecffea32/go.mod h1:DrZx5ec/dmnfpw9KyYoQyYo7d0KEvTkk/5M/vbZjAr8= @@ -287,6 +289,7 @@ github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5/go.mod h1:VvhXpOYNQvB+ github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= github.com/flynn/noise v1.0.0 h1:DlTHqmzmvcEiKj+4RYo/imoswx/4r6iBlCMfVtrMXpQ= github.com/flynn/noise v1.0.0/go.mod h1:xbMo+0i6+IGbYdJhF31t2eR1BIU0CYc12+BNAKwUTag= +github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90 h1:WXb3TSNmHp2vHoCroCIB1foO/yQ36swABL8aOVeDpgg= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= @@ -375,6 +378,7 @@ github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXP github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 h1:DACJavvAHhabrF08vX0COfcOBJRhZ8lUbR+ZWIs0Y5g= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -643,6 +647,7 @@ github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfV github.com/julienschmidt/httprouter v1.1.1-0.20170430222011-975b5c4c7c21/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5 h1:PJr+ZMXIecYc1Ey2zucXdR73SMBtgjPgwa31099IMv0= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jwilder/encoding v0.0.0-20170811194829-b4e1701a28ef/go.mod h1:Ct9fl0F6iIOGgxJ5npU/IUOhOhqlVrGjyIZc8/MagT0= github.com/k0kubun/go-ansi v0.0.0-20180517002512-3bf9e2903213/go.mod h1:vNUNkEQ1e29fT/6vq2aBdFsgNPmy8qMdSay1npru+Sw= @@ -1081,6 +1086,7 @@ github.com/multiformats/go-varint v0.0.5/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXS github.com/multiformats/go-varint v0.0.6 h1:gk85QWKxh3TazbLxED/NlDVv8+q+ReFJk7Y2W/KhfNY= github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/naoina/go-stringutil v0.1.0/go.mod h1:XJ2SJL9jCtBh+P9q5btrd/Ylo8XwT/h1USek5+NqSA0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416/go.mod h1:NBIhNtsFMo3G2szEBne+bO4gS192HuIYRqfvOWb4i1E= @@ -1523,6 +1529,7 @@ golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6 h1:QE6XYQK6naiK1EPAe1g/ILLxN golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b h1:+qEpEAPhDZ1o0x3tHzZTQDArnOixOzGD9HUJfcg0mb4= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -1720,6 +1727,7 @@ golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744 h1:yhBbb4IRs2HS9PPlAg6DMC6mU golang.org/x/sys v0.0.0-20210511113859-b0526f3d8744/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d h1:SZxvLBoTP5yHO3Frd4z4vrF+DBX9vMVanchswa69toE= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1819,6 +1827,7 @@ gonum.org/v1/gonum v0.6.1/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0 h1:OE9mWmgKkjJyEmDAAtGMPjXu+YNeGvK9VTSHY6+Qihc= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= +gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b h1:Qh4dB5D/WpoUUp3lSod7qgoyEHbDGPUWjIbnqdqqe1k= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= @@ -1985,6 +1994,7 @@ k8s.io/kubernetes v1.13.0/go.mod h1:ocZa8+6APFNC2tX1DZASIbocyYT5jHzqFVsY5aoB7Jk= pgregory.net/rapid v0.4.7 h1:MTNRktPuv5FNqOO151TM9mDTa+XHcX6ypYeISDVD14g= pgregory.net/rapid v0.4.7/go.mod h1:UYpPVyjFHzYBGHIxLFoupi8vwk6rXNzRY9OMvVxFIOU= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/pdf v0.1.1 h1:k1MczvYDUvJBe93bYd7wrZLLUEcLZAuF824/I4e5Xr4= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/module/mocks/network.go b/module/mocks/network.go index 305cee9f346..38f65a552ce 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -5,40 +5,38 @@ package mocks import ( - reflect "reflect" - gomock "github.com/golang/mock/gomock" - crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" network "github.com/onflow/flow-go/network" + reflect "reflect" ) -// MockNetwork is a mock of Network interface. +// MockNetwork is a mock of Network interface type MockNetwork struct { ctrl *gomock.Controller recorder *MockNetworkMockRecorder } -// MockNetworkMockRecorder is the mock recorder for MockNetwork. +// MockNetworkMockRecorder is the mock recorder for MockNetwork type MockNetworkMockRecorder struct { mock *MockNetwork } -// NewMockNetwork creates a new mock instance. +// NewMockNetwork creates a new mock instance func NewMockNetwork(ctrl *gomock.Controller) *MockNetwork { mock := &MockNetwork{ctrl: ctrl} mock.recorder = &MockNetworkMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockNetwork) EXPECT() *MockNetworkMockRecorder { return m.recorder } -// Register mocks base method. +// Register mocks base method func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (network.Conduit, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Register", arg0, arg1) @@ -47,36 +45,36 @@ func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (netwo return ret0, ret1 } -// Register indicates an expected call of Register. +// Register indicates an expected call of Register func (mr *MockNetworkMockRecorder) Register(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Register", reflect.TypeOf((*MockNetwork)(nil).Register), arg0, arg1) } -// MockLocal is a mock of Local interface. +// MockLocal is a mock of Local interface type MockLocal struct { ctrl *gomock.Controller recorder *MockLocalMockRecorder } -// MockLocalMockRecorder is the mock recorder for MockLocal. +// MockLocalMockRecorder is the mock recorder for MockLocal type MockLocalMockRecorder struct { mock *MockLocal } -// NewMockLocal creates a new mock instance. +// NewMockLocal creates a new mock instance func NewMockLocal(ctrl *gomock.Controller) *MockLocal { mock := &MockLocal{ctrl: ctrl} mock.recorder = &MockLocalMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockLocal) EXPECT() *MockLocalMockRecorder { return m.recorder } -// Address mocks base method. +// Address mocks base method func (m *MockLocal) Address() string { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Address") @@ -84,13 +82,13 @@ func (m *MockLocal) Address() string { return ret0 } -// Address indicates an expected call of Address. +// Address indicates an expected call of Address func (mr *MockLocalMockRecorder) Address() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Address", reflect.TypeOf((*MockLocal)(nil).Address)) } -// NodeID mocks base method. +// NodeID mocks base method func (m *MockLocal) NodeID() flow.Identifier { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NodeID") @@ -98,13 +96,13 @@ func (m *MockLocal) NodeID() flow.Identifier { return ret0 } -// NodeID indicates an expected call of NodeID. +// NodeID indicates an expected call of NodeID func (mr *MockLocalMockRecorder) NodeID() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NodeID", reflect.TypeOf((*MockLocal)(nil).NodeID)) } -// NotMeFilter mocks base method. +// NotMeFilter mocks base method func (m *MockLocal) NotMeFilter() flow.IdentityFilter { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NotMeFilter") @@ -112,13 +110,13 @@ func (m *MockLocal) NotMeFilter() flow.IdentityFilter { return ret0 } -// NotMeFilter indicates an expected call of NotMeFilter. +// NotMeFilter indicates an expected call of NotMeFilter func (mr *MockLocalMockRecorder) NotMeFilter() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NotMeFilter", reflect.TypeOf((*MockLocal)(nil).NotMeFilter)) } -// Sign mocks base method. +// Sign mocks base method func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Sign", arg0, arg1) @@ -127,13 +125,13 @@ func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error return ret0, ret1 } -// Sign indicates an expected call of Sign. +// Sign indicates an expected call of Sign func (mr *MockLocalMockRecorder) Sign(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sign", reflect.TypeOf((*MockLocal)(nil).Sign), arg0, arg1) } -// SignFunc mocks base method. +// SignFunc mocks base method func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.PrivateKey, []byte, hash.Hasher) (crypto.Signature, error)) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SignFunc", arg0, arg1, arg2) @@ -142,66 +140,66 @@ func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.Pri return ret0, ret1 } -// SignFunc indicates an expected call of SignFunc. +// SignFunc indicates an expected call of SignFunc func (mr *MockLocalMockRecorder) SignFunc(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignFunc", reflect.TypeOf((*MockLocal)(nil).SignFunc), arg0, arg1, arg2) } -// MockRequester is a mock of Requester interface. +// MockRequester is a mock of Requester interface type MockRequester struct { ctrl *gomock.Controller recorder *MockRequesterMockRecorder } -// MockRequesterMockRecorder is the mock recorder for MockRequester. +// MockRequesterMockRecorder is the mock recorder for MockRequester type MockRequesterMockRecorder struct { mock *MockRequester } -// NewMockRequester creates a new mock instance. +// NewMockRequester creates a new mock instance func NewMockRequester(ctrl *gomock.Controller) *MockRequester { mock := &MockRequester{ctrl: ctrl} mock.recorder = &MockRequesterMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockRequester) EXPECT() *MockRequesterMockRecorder { return m.recorder } -// EntityByID mocks base method. +// EntityByID mocks base method func (m *MockRequester) EntityByID(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "EntityByID", arg0, arg1) } -// EntityByID indicates an expected call of EntityByID. +// EntityByID indicates an expected call of EntityByID func (mr *MockRequesterMockRecorder) EntityByID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EntityByID", reflect.TypeOf((*MockRequester)(nil).EntityByID), arg0, arg1) } -// Force mocks base method. +// Force mocks base method func (m *MockRequester) Force() { m.ctrl.T.Helper() m.ctrl.Call(m, "Force") } -// Force indicates an expected call of Force. +// Force indicates an expected call of Force func (mr *MockRequesterMockRecorder) Force() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Force", reflect.TypeOf((*MockRequester)(nil).Force)) } -// Query mocks base method. +// Query mocks base method func (m *MockRequester) Query(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Query", arg0, arg1) } -// Query indicates an expected call of Query. +// Query indicates an expected call of Query func (mr *MockRequesterMockRecorder) Query(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockRequester)(nil).Query), arg0, arg1) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 969044a53ee..2967a3e5cce 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -5,39 +5,37 @@ package mocks import ( - reflect "reflect" - gomock "github.com/golang/mock/gomock" - flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" + reflect "reflect" ) -// MockBlocks is a mock of Blocks interface. +// MockBlocks is a mock of Blocks interface type MockBlocks struct { ctrl *gomock.Controller recorder *MockBlocksMockRecorder } -// MockBlocksMockRecorder is the mock recorder for MockBlocks. +// MockBlocksMockRecorder is the mock recorder for MockBlocks type MockBlocksMockRecorder struct { mock *MockBlocks } -// NewMockBlocks creates a new mock instance. +// NewMockBlocks creates a new mock instance func NewMockBlocks(ctrl *gomock.Controller) *MockBlocks { mock := &MockBlocks{ctrl: ctrl} mock.recorder = &MockBlocksMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockBlocks) EXPECT() *MockBlocksMockRecorder { return m.recorder } -// ByCollectionID mocks base method. +// ByCollectionID mocks base method func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByCollectionID", arg0) @@ -46,13 +44,13 @@ func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByCollectionID indicates an expected call of ByCollectionID. +// ByCollectionID indicates an expected call of ByCollectionID func (mr *MockBlocksMockRecorder) ByCollectionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByCollectionID", reflect.TypeOf((*MockBlocks)(nil).ByCollectionID), arg0) } -// ByHeight mocks base method. +// ByHeight mocks base method func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -61,13 +59,13 @@ func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight. +// ByHeight indicates an expected call of ByHeight func (mr *MockBlocksMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockBlocks)(nil).ByHeight), arg0) } -// ByID mocks base method. +// ByID mocks base method func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -76,13 +74,13 @@ func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID. +// ByID indicates an expected call of ByID func (mr *MockBlocksMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockBlocks)(nil).ByID), arg0) } -// GetLastFullBlockHeight mocks base method. +// GetLastFullBlockHeight mocks base method func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetLastFullBlockHeight") @@ -91,13 +89,13 @@ func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { return ret0, ret1 } -// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight. +// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight func (mr *MockBlocksMockRecorder) GetLastFullBlockHeight() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).GetLastFullBlockHeight)) } -// IndexBlockForCollections mocks base method. +// IndexBlockForCollections mocks base method func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexBlockForCollections", arg0, arg1) @@ -105,13 +103,13 @@ func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow. return ret0 } -// IndexBlockForCollections indicates an expected call of IndexBlockForCollections. +// IndexBlockForCollections indicates an expected call of IndexBlockForCollections func (mr *MockBlocksMockRecorder) IndexBlockForCollections(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexBlockForCollections", reflect.TypeOf((*MockBlocks)(nil).IndexBlockForCollections), arg0, arg1) } -// Store mocks base method. +// Store mocks base method func (m *MockBlocks) Store(arg0 *flow.Block) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -119,13 +117,13 @@ func (m *MockBlocks) Store(arg0 *flow.Block) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockBlocksMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockBlocks)(nil).Store), arg0) } -// StoreTx mocks base method. +// StoreTx mocks base method func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreTx", arg0) @@ -133,13 +131,13 @@ func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { return ret0 } -// StoreTx indicates an expected call of StoreTx. +// StoreTx indicates an expected call of StoreTx func (mr *MockBlocksMockRecorder) StoreTx(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreTx", reflect.TypeOf((*MockBlocks)(nil).StoreTx), arg0) } -// UpdateLastFullBlockHeight mocks base method. +// UpdateLastFullBlockHeight mocks base method func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "UpdateLastFullBlockHeight", arg0) @@ -147,36 +145,36 @@ func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { return ret0 } -// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight. +// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight func (mr *MockBlocksMockRecorder) UpdateLastFullBlockHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).UpdateLastFullBlockHeight), arg0) } -// MockHeaders is a mock of Headers interface. +// MockHeaders is a mock of Headers interface type MockHeaders struct { ctrl *gomock.Controller recorder *MockHeadersMockRecorder } -// MockHeadersMockRecorder is the mock recorder for MockHeaders. +// MockHeadersMockRecorder is the mock recorder for MockHeaders type MockHeadersMockRecorder struct { mock *MockHeaders } -// NewMockHeaders creates a new mock instance. +// NewMockHeaders creates a new mock instance func NewMockHeaders(ctrl *gomock.Controller) *MockHeaders { mock := &MockHeaders{ctrl: ctrl} mock.recorder = &MockHeadersMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockHeaders) EXPECT() *MockHeadersMockRecorder { return m.recorder } -// BatchIndexByChunkID mocks base method. +// BatchIndexByChunkID mocks base method func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchIndexByChunkID", arg0, arg1, arg2) @@ -184,13 +182,13 @@ func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 stora return ret0 } -// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID. +// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID func (mr *MockHeadersMockRecorder) BatchIndexByChunkID(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchIndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).BatchIndexByChunkID), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -199,13 +197,13 @@ func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockHeadersMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockHeaders)(nil).ByBlockID), arg0) } -// ByHeight mocks base method. +// ByHeight mocks base method func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -214,13 +212,13 @@ func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight. +// ByHeight indicates an expected call of ByHeight func (mr *MockHeadersMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockHeaders)(nil).ByHeight), arg0) } -// ByParentID mocks base method. +// ByParentID mocks base method func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByParentID", arg0) @@ -229,13 +227,13 @@ func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { return ret0, ret1 } -// ByParentID indicates an expected call of ByParentID. +// ByParentID indicates an expected call of ByParentID func (mr *MockHeadersMockRecorder) ByParentID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByParentID", reflect.TypeOf((*MockHeaders)(nil).ByParentID), arg0) } -// IDByChunkID mocks base method. +// IDByChunkID mocks base method func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IDByChunkID", arg0) @@ -244,13 +242,13 @@ func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) return ret0, ret1 } -// IDByChunkID indicates an expected call of IDByChunkID. +// IDByChunkID indicates an expected call of IDByChunkID func (mr *MockHeadersMockRecorder) IDByChunkID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDByChunkID", reflect.TypeOf((*MockHeaders)(nil).IDByChunkID), arg0) } -// IndexByChunkID mocks base method. +// IndexByChunkID mocks base method func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexByChunkID", arg0, arg1) @@ -258,13 +256,13 @@ func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { return ret0 } -// IndexByChunkID indicates an expected call of IndexByChunkID. +// IndexByChunkID indicates an expected call of IndexByChunkID func (mr *MockHeadersMockRecorder) IndexByChunkID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).IndexByChunkID), arg0, arg1) } -// Store mocks base method. +// Store mocks base method func (m *MockHeaders) Store(arg0 *flow.Header) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -272,36 +270,36 @@ func (m *MockHeaders) Store(arg0 *flow.Header) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockHeadersMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockHeaders)(nil).Store), arg0) } -// MockPayloads is a mock of Payloads interface. +// MockPayloads is a mock of Payloads interface type MockPayloads struct { ctrl *gomock.Controller recorder *MockPayloadsMockRecorder } -// MockPayloadsMockRecorder is the mock recorder for MockPayloads. +// MockPayloadsMockRecorder is the mock recorder for MockPayloads type MockPayloadsMockRecorder struct { mock *MockPayloads } -// NewMockPayloads creates a new mock instance. +// NewMockPayloads creates a new mock instance func NewMockPayloads(ctrl *gomock.Controller) *MockPayloads { mock := &MockPayloads{ctrl: ctrl} mock.recorder = &MockPayloadsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockPayloads) EXPECT() *MockPayloadsMockRecorder { return m.recorder } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -310,13 +308,13 @@ func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockPayloadsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockPayloads)(nil).ByBlockID), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -324,36 +322,36 @@ func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockPayloadsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockPayloads)(nil).Store), arg0, arg1) } -// MockCollections is a mock of Collections interface. +// MockCollections is a mock of Collections interface type MockCollections struct { ctrl *gomock.Controller recorder *MockCollectionsMockRecorder } -// MockCollectionsMockRecorder is the mock recorder for MockCollections. +// MockCollectionsMockRecorder is the mock recorder for MockCollections type MockCollectionsMockRecorder struct { mock *MockCollections } -// NewMockCollections creates a new mock instance. +// NewMockCollections creates a new mock instance func NewMockCollections(ctrl *gomock.Controller) *MockCollections { mock := &MockCollections{ctrl: ctrl} mock.recorder = &MockCollectionsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockCollections) EXPECT() *MockCollectionsMockRecorder { return m.recorder } -// ByID mocks base method. +// ByID mocks base method func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -362,13 +360,13 @@ func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID. +// ByID indicates an expected call of ByID func (mr *MockCollectionsMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockCollections)(nil).ByID), arg0) } -// LightByID mocks base method. +// LightByID mocks base method func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByID", arg0) @@ -377,13 +375,13 @@ func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection return ret0, ret1 } -// LightByID indicates an expected call of LightByID. +// LightByID indicates an expected call of LightByID func (mr *MockCollectionsMockRecorder) LightByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByID", reflect.TypeOf((*MockCollections)(nil).LightByID), arg0) } -// LightByTransactionID mocks base method. +// LightByTransactionID mocks base method func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByTransactionID", arg0) @@ -392,13 +390,13 @@ func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.Ligh return ret0, ret1 } -// LightByTransactionID indicates an expected call of LightByTransactionID. +// LightByTransactionID indicates an expected call of LightByTransactionID func (mr *MockCollectionsMockRecorder) LightByTransactionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByTransactionID", reflect.TypeOf((*MockCollections)(nil).LightByTransactionID), arg0) } -// Remove mocks base method. +// Remove mocks base method func (m *MockCollections) Remove(arg0 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Remove", arg0) @@ -406,13 +404,13 @@ func (m *MockCollections) Remove(arg0 flow.Identifier) error { return ret0 } -// Remove indicates an expected call of Remove. +// Remove indicates an expected call of Remove func (mr *MockCollectionsMockRecorder) Remove(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockCollections)(nil).Remove), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockCollections) Store(arg0 *flow.Collection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -420,13 +418,13 @@ func (m *MockCollections) Store(arg0 *flow.Collection) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockCollectionsMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCollections)(nil).Store), arg0) } -// StoreLight mocks base method. +// StoreLight mocks base method func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLight", arg0) @@ -434,13 +432,13 @@ func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { return ret0 } -// StoreLight indicates an expected call of StoreLight. +// StoreLight indicates an expected call of StoreLight func (mr *MockCollectionsMockRecorder) StoreLight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLight", reflect.TypeOf((*MockCollections)(nil).StoreLight), arg0) } -// StoreLightAndIndexByTransaction mocks base method. +// StoreLightAndIndexByTransaction mocks base method func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLightAndIndexByTransaction", arg0) @@ -448,36 +446,36 @@ func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollec return ret0 } -// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction. +// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction func (mr *MockCollectionsMockRecorder) StoreLightAndIndexByTransaction(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLightAndIndexByTransaction", reflect.TypeOf((*MockCollections)(nil).StoreLightAndIndexByTransaction), arg0) } -// MockCommits is a mock of Commits interface. +// MockCommits is a mock of Commits interface type MockCommits struct { ctrl *gomock.Controller recorder *MockCommitsMockRecorder } -// MockCommitsMockRecorder is the mock recorder for MockCommits. +// MockCommitsMockRecorder is the mock recorder for MockCommits type MockCommitsMockRecorder struct { mock *MockCommits } -// NewMockCommits creates a new mock instance. +// NewMockCommits creates a new mock instance func NewMockCommits(ctrl *gomock.Controller) *MockCommits { mock := &MockCommits{ctrl: ctrl} mock.recorder = &MockCommitsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -485,13 +483,13 @@ func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockCommitsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockCommits)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -500,13 +498,13 @@ func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, err return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockCommitsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockCommits)(nil).ByBlockID), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -514,36 +512,36 @@ func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) err return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockCommitsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCommits)(nil).Store), arg0, arg1) } -// MockEvents is a mock of Events interface. +// MockEvents is a mock of Events interface type MockEvents struct { ctrl *gomock.Controller recorder *MockEventsMockRecorder } -// MockEventsMockRecorder is the mock recorder for MockEvents. +// MockEventsMockRecorder is the mock recorder for MockEvents type MockEventsMockRecorder struct { mock *MockEvents } -// NewMockEvents creates a new mock instance. +// NewMockEvents creates a new mock instance func NewMockEvents(ctrl *gomock.Controller) *MockEvents { mock := &MockEvents{ctrl: ctrl} mock.recorder = &MockEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockEvents) EXPECT() *MockEventsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -551,13 +549,13 @@ func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, ar return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -566,13 +564,13 @@ func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockEvents)(nil).ByBlockID), arg0) } -// ByBlockIDEventType mocks base method. +// ByBlockIDEventType mocks base method func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventType) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDEventType", arg0, arg1) @@ -581,13 +579,13 @@ func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventTyp return ret0, ret1 } -// ByBlockIDEventType indicates an expected call of ByBlockIDEventType. +// ByBlockIDEventType indicates an expected call of ByBlockIDEventType func (mr *MockEventsMockRecorder) ByBlockIDEventType(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDEventType", reflect.TypeOf((*MockEvents)(nil).ByBlockIDEventType), arg0, arg1) } -// ByBlockIDTransactionID mocks base method. +// ByBlockIDTransactionID mocks base method func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -596,36 +594,36 @@ func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow. return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID func (mr *MockEventsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockEvents)(nil).ByBlockIDTransactionID), arg0, arg1) } -// MockServiceEvents is a mock of ServiceEvents interface. +// MockServiceEvents is a mock of ServiceEvents interface type MockServiceEvents struct { ctrl *gomock.Controller recorder *MockServiceEventsMockRecorder } -// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents. +// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents type MockServiceEventsMockRecorder struct { mock *MockServiceEvents } -// NewMockServiceEvents creates a new mock instance. +// NewMockServiceEvents creates a new mock instance func NewMockServiceEvents(ctrl *gomock.Controller) *MockServiceEvents { mock := &MockServiceEvents{ctrl: ctrl} mock.recorder = &MockServiceEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -633,13 +631,13 @@ func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockServiceEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockServiceEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -648,36 +646,36 @@ func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockServiceEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockServiceEvents)(nil).ByBlockID), arg0) } -// MockTransactionResults is a mock of TransactionResults interface. +// MockTransactionResults is a mock of TransactionResults interface type MockTransactionResults struct { ctrl *gomock.Controller recorder *MockTransactionResultsMockRecorder } -// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults. +// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults type MockTransactionResultsMockRecorder struct { mock *MockTransactionResults } -// NewMockTransactionResults creates a new mock instance. +// NewMockTransactionResults creates a new mock instance func NewMockTransactionResults(ctrl *gomock.Controller) *MockTransactionResults { mock := &MockTransactionResults{ctrl: ctrl} mock.recorder = &MockTransactionResultsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -685,13 +683,13 @@ func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.Tr return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockTransactionResultsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockTransactionResults)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockIDTransactionID mocks base method. +// ByBlockIDTransactionID mocks base method func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) (*flow.TransactionResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -700,7 +698,7 @@ func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifi return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID func (mr *MockTransactionResultsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockTransactionResults)(nil).ByBlockIDTransactionID), arg0, arg1) From 9689cbb2f885aa4267e32de430c3d99e7ece5719 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Wed, 25 Aug 2021 15:09:07 -0400 Subject: [PATCH 258/291] fix integration tests compilation --- integration/localnet/bootstrap.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration/localnet/bootstrap.go b/integration/localnet/bootstrap.go index bb042e0972c..b55c2bf8d7e 100644 --- a/integration/localnet/bootstrap.go +++ b/integration/localnet/bootstrap.go @@ -208,7 +208,7 @@ func prepareNodes() []testnet.NodeConfig { for i := 0; i < unstakedAccessCount; i++ { nodes = append(nodes, testnet.NewNodeConfig(flow.RoleAccess, func(cfg *testnet.NodeConfig) { - cfg.Unstaked = true + cfg.SupportsUnstakedNodes = true })) } From a6c7043353f06decb75340860206197512f65a14 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Wed, 25 Aug 2021 17:46:44 -0700 Subject: [PATCH 259/291] fixing scaffold --- cmd/scaffold.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 91f6a13e6cd..689490ab3a7 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -21,6 +21,7 @@ import ( "github.com/onflow/flow-go/fvm" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/lifecycle" From c3f87e403681a1d0367e0c5f57ff8214156f7629 Mon Sep 17 00:00:00 2001 From: vishal <1117327+vishalchangrani@users.noreply.github.com> Date: Wed, 25 Aug 2021 22:34:48 -0700 Subject: [PATCH 260/291] adding assertions to the TestReceiveBlocks consumer follower test --- follower/consensus_follower.go | 6 +- .../tests/access/unstaked_node_test.go | 67 ++++++++++++++++--- 2 files changed, 61 insertions(+), 12 deletions(-) diff --git a/follower/consensus_follower.go b/follower/consensus_follower.go index f35ab03689e..eace310cb3e 100644 --- a/follower/consensus_follower.go +++ b/follower/consensus_follower.go @@ -102,7 +102,7 @@ func buildAccessNode(accessNodeOptions []access.Option) *access.UnstakedAccessNo } type ConsensusFollowerImpl struct { - nodeBuilder *access.UnstakedAccessNodeBuilder + NodeBuilder *access.UnstakedAccessNodeBuilder consumersMu sync.RWMutex consumers []pubsub.OnBlockFinalizedConsumer } @@ -127,7 +127,7 @@ func NewConsensusFollower( accessNodeOptions := getAccessNodeOptions(config) anb := buildAccessNode(accessNodeOptions) - consensusFollower := &ConsensusFollowerImpl{nodeBuilder: anb} + consensusFollower := &ConsensusFollowerImpl{NodeBuilder: anb} anb.FinalizationDistributor.AddOnBlockFinalizedConsumer(consensusFollower.onBlockFinalized) @@ -154,7 +154,7 @@ func (cf *ConsensusFollowerImpl) AddOnBlockFinalizedConsumer(consumer pubsub.OnB // Run starts the consensus follower. func (cf *ConsensusFollowerImpl) Run(ctx context.Context) { - runAccessNode(ctx, cf.nodeBuilder) + runAccessNode(ctx, cf.NodeBuilder) } func runAccessNode(ctx context.Context, anb *access.UnstakedAccessNodeBuilder) { diff --git a/integration/tests/access/unstaked_node_test.go b/integration/tests/access/unstaked_node_test.go index 03a9d1f8af6..4d6ec35c48d 100644 --- a/integration/tests/access/unstaked_node_test.go +++ b/integration/tests/access/unstaked_node_test.go @@ -19,6 +19,8 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +const blockCount = 5 // number of finalized blocks to wait for + type UnstakedAccessSuite struct { suite.Suite @@ -26,11 +28,12 @@ type UnstakedAccessSuite struct { ctx context.Context cancel context.CancelFunc - net *testnet.FlowNetwork - stakedID flow.Identifier - unstakedID flow.Identifier - conID flow.Identifier - follower consensus_follower.ConsensusFollower + net *testnet.FlowNetwork + stakedID flow.Identifier + unstakedID flow.Identifier + conID flow.Identifier + follower consensus_follower.ConsensusFollower + finalizedBlockIDsChan chan flow.Identifier } func TestUnstakedAccessSuite(t *testing.T) { @@ -48,20 +51,66 @@ func (suite *UnstakedAccessSuite) TearDownTest() { } func (suite *UnstakedAccessSuite) SetupTest() { + suite.finalizedBlockIDsChan = make(chan flow.Identifier, blockCount) suite.buildNetworkConfig() // start the network suite.ctx, suite.cancel = context.WithCancel(context.Background()) suite.net.Start(suite.ctx) } +// TestReceiveBlocks tests that consensus follower follows the chain and persists blocks in storage func (suite *UnstakedAccessSuite) TestReceiveBlocks() { - go suite.follower.Run(suite.ctx) - // TODO: to be implemented later - time.Sleep(time.Second * 30) + ctx, cancel := context.WithCancel(suite.ctx) + defer cancel() + // kick off the follower + go suite.follower.Run(ctx) + + followerImpl, ok := suite.follower.(*consensus_follower.ConsensusFollowerImpl) + if !ok { + suite.Fail("unexpected consensus follower implementation") + return + } + + // get the underlying node builder + node := followerImpl.NodeBuilder + // wait for the follower to have completely started + unittest.RequireCloseBefore(suite.T(), node.Ready(), 10*time.Second, + "timed out while waiting for consensus follower to start") + + // get the underlying storage that the follower is using + storage := node.Storage + require.NotNil(suite.T(), storage) + blocks := storage.Blocks + require.NotNil(suite.T(), blocks) + + rcvdBlockCnt := 0 + var err error + receiveBlocks := func() { + for ; rcvdBlockCnt < blockCount; rcvdBlockCnt++ { + select { + case blockID := <-suite.finalizedBlockIDsChan: + _, err = blocks.ByID(blockID) + if err != nil { + return + } + } + } + } + + // wait for finalized blocks + unittest.AssertReturnsBefore(suite.T(), receiveBlocks, 1*time.Minute) // waiting 1 minute for 5 blocks + + // all blocks were found in the storage + require.NoError(suite.T(), err, "finalized block not found in storage") + + // assert that blockCount number of blocks were received + require.Equal(suite.T(), blockCount, rcvdBlockCnt) + } func (suite *UnstakedAccessSuite) OnBlockFinalizedConsumer(finalizedBlockID flow.Identifier) { - fmt.Println(finalizedBlockID.String()) + // push the finalized block ID to the finalizedBlockIDsChan channel + suite.finalizedBlockIDsChan <- finalizedBlockID } func (suite *UnstakedAccessSuite) buildNetworkConfig() { From 8d050bc7fa9049042f1433a19516b876e51ce2b5 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 26 Aug 2021 15:00:52 -0700 Subject: [PATCH 261/291] Add subscription filter test --- cmd/scaffold.go | 2 +- engine/channels.go | 6 +- network/p2p/libp2pNode.go | 21 +++---- network/p2p/libp2pUtils.go | 8 +-- network/p2p/middleware.go | 4 +- network/p2p/subscription_filter.go | 38 +++++++---- network/p2p/subscription_filter_test.go | 84 +++++++++++++++++++++++++ network/test/testUtil.go | 2 +- 8 files changed, 131 insertions(+), 34 deletions(-) create mode 100644 network/p2p/subscription_filter_test.go diff --git a/cmd/scaffold.go b/cmd/scaffold.go index fcafc9bf65c..ae631f10b11 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -160,7 +160,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Me.NodeID(), myAddr, fnb.NetworkKey, - fnb.RootBlock.ID().String(), + fnb.RootBlock.ID(), p2p.DefaultMaxPubSubMsgSize, fnb.Metrics.Network, pingProvider, diff --git a/engine/channels.go b/engine/channels.go index 1088ccc35f8..2f25931a537 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -197,6 +197,8 @@ func initializeChannelRoleMap() { channelRoleMap[syncClusterPrefix] = flow.RoleList{flow.RoleCollection} channelRoleMap[consensusClusterPrefix] = flow.RoleList{flow.RoleCollection} + + channelRoleMap[UnstakedSyncCommittee] = flow.RoleList{flow.RoleAccess} } // ClusterChannel returns true if channel is cluster-based. @@ -217,13 +219,13 @@ func ClusterChannel(channel network.Channel) (network.Channel, bool) { // TopicFromChannel returns the unique LibP2P topic form the channel. // The channel is made up of name string suffixed with root block id. // The root block id is used to prevent cross talks between nodes on different sporks. -func TopicFromChannel(channel network.Channel, rootBlockID string) network.Topic { +func TopicFromChannel(channel network.Channel, rootBlockID flow.Identifier) network.Topic { // skip root block suffix, if this is a cluster specific channel. A cluster specific channel is inherently // unique for each epoch if strings.HasPrefix(channel.String(), syncClusterPrefix.String()) || strings.HasPrefix(string(channel), consensusClusterPrefix.String()) { return network.Topic(channel) } - return network.Topic(fmt.Sprintf("%s/%s", string(channel), rootBlockID)) + return network.Topic(fmt.Sprintf("%s/%s", string(channel), rootBlockID.String())) } // ChannelConsensusCluster returns a dynamic cluster consensus channel based on diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 102eb5971f8..4ad6819cebe 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -54,7 +54,7 @@ type LibP2PFactoryFunc func() (*Node, error) // DefaultLibP2PNodeFactory returns a LibP2PFactoryFunc which generates the libp2p host initialized with the // default options for the host, the pubsub and the ping service. -func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.Identifier, address string, flowKey fcrypto.PrivateKey, rootBlockID string, +func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.Identifier, address string, flowKey fcrypto.PrivateKey, rootBlockID flow.Identifier, maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider, idProvider id.IdentityProvider, idTranslator IDTranslator) (LibP2PFactoryFunc, error) { connManager := NewConnManager(log, idProvider, metrics) @@ -66,7 +66,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize, idProvider, idTranslator)...). + SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize, idProvider, idTranslator, rootBlockID)...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). Build(ctx) @@ -74,7 +74,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I } type NodeBuilder interface { - SetRootBlockID(string) NodeBuilder + SetRootBlockID(flow.Identifier) NodeBuilder SetConnectionManager(TagLessConnManager) NodeBuilder SetConnectionGater(*ConnGater) NodeBuilder SetPubsubOptions(...PubsubOption) NodeBuilder @@ -86,7 +86,7 @@ type NodeBuilder interface { type DefaultLibP2PNodeBuilder struct { id flow.Identifier - rootBlockID string + rootBlockID flow.Identifier logger zerolog.Logger connGater *ConnGater connMngr TagLessConnManager @@ -114,7 +114,7 @@ func (builder *DefaultLibP2PNodeBuilder) SetDHTOptions(opts ...dht.Option) NodeB return builder } -func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId string) NodeBuilder { +func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId flow.Identifier) NodeBuilder { builder.rootBlockID = rootBlockId return builder } @@ -160,7 +160,7 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro return nil, errors.New("unable to create libp2p pubsub: factory function not provided") } - if builder.rootBlockID == "" { + if builder.rootBlockID == flow.ZeroID { return nil, errors.New("root block ID must be provided") } node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) @@ -177,11 +177,6 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.connMgr = builder.connMngr } - if builder.rootBlockID == "" { - return nil, errors.New("root block ID must be provided") - } - node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) - if builder.pingInfoProvider != nil { opts = append(opts, libp2p.Ping(true)) } @@ -644,7 +639,7 @@ func DefaultPubSub(ctx context.Context, host host.Host, psOption ...pubsub.Optio // PubsubOption generates a libp2p pubsub.Option from the given context and host type PubsubOption func(ctx context.Context, host host.Host) (pubsub.Option, error) -func DefaultPubsubOptions(maxPubSubMsgSize int, idProvider id.IdentityProvider, idTranslator IDTranslator) []PubsubOption { +func DefaultPubsubOptions(maxPubSubMsgSize int, idProvider id.IdentityProvider, idTranslator IDTranslator, rootBlockID flow.Identifier) []PubsubOption { pubSubOptionFunc := func(option pubsub.Option) PubsubOption { return func(_ context.Context, _ host.Host) (pubsub.Option, error) { return option, nil @@ -661,7 +656,7 @@ func DefaultPubsubOptions(maxPubSubMsgSize int, idProvider id.IdentityProvider, // subscription filter func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), idProvider, idTranslator)), nil + return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), rootBlockID, idProvider, idTranslator)), nil }, } } diff --git a/network/p2p/libp2pUtils.go b/network/p2p/libp2pUtils.go index e37b4460b1b..373fc11c11b 100644 --- a/network/p2p/libp2pUtils.go +++ b/network/p2p/libp2pUtils.go @@ -162,12 +162,12 @@ func IPPortFromMultiAddress(addrs ...multiaddr.Multiaddr) (string, string, error return "", "", fmt.Errorf("ip address or hostname not found") } -func generateFlowProtocolID(rootBlockID string) protocol.ID { - return protocol.ID(FlowLibP2POneToOneProtocolIDPrefix + rootBlockID) +func generateFlowProtocolID(rootBlockID flow.Identifier) protocol.ID { + return protocol.ID(FlowLibP2POneToOneProtocolIDPrefix + rootBlockID.String()) } -func generatePingProtcolID(rootBlockID string) protocol.ID { - return protocol.ID(FlowLibP2PPingProtocolPrefix + rootBlockID) +func generatePingProtcolID(rootBlockID flow.Identifier) protocol.ID { + return protocol.ID(FlowLibP2PPingProtocolPrefix + rootBlockID.String()) } // PeerAddressInfo generates the libp2p peer.AddrInfo for the given Flow.Identity. diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 8236931a863..19398a0ac0b 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -73,7 +73,7 @@ type Middleware struct { libP2PNodeFactory LibP2PFactoryFunc me flow.Identifier metrics module.NetworkMetrics - rootBlockID string + rootBlockID flow.Identifier validators []network.MessageValidator peerManager *PeerManager peerUpdateInterval time.Duration @@ -113,7 +113,7 @@ func NewMiddleware( libP2PNodeFactory LibP2PFactoryFunc, flowID flow.Identifier, metrics module.NetworkMetrics, - rootBlockID string, + rootBlockID flow.Identifier, peerUpdateInterval time.Duration, unicastMessageTimeout time.Duration, connectionGating bool, diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index 6e1c0d8cb38..0ad3a5b55e6 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -15,13 +15,15 @@ type Filter struct { idProvider id.IdentityProvider idTranslator IDTranslator myPeerID peer.ID + rootBlockID flow.Identifier } -func NewSubscriptionFilter(pid peer.ID, idProvider id.IdentityProvider, idTranslator IDTranslator) *Filter { +func NewSubscriptionFilter(pid peer.ID, rootBlockID flow.Identifier, idProvider id.IdentityProvider, idTranslator IDTranslator) *Filter { return &Filter{ idProvider, idTranslator, pid, + rootBlockID, } } @@ -40,31 +42,45 @@ func (f *Filter) getIdentity(pid peer.ID) *flow.Identity { return identities[0] } -func (f *Filter) allowedChannels(pid peer.ID) network.ChannelList { +func (f *Filter) allowedTopics(pid peer.ID) []network.Topic { id := f.getIdentity(pid) + var channels network.ChannelList + if id == nil { - return engine.UnstakedChannels() + channels = engine.UnstakedChannels() + } else { + channels = engine.ChannelsByRole(id.Role) + } + + var topics []network.Topic + + for _, ch := range channels { + topics = append(topics, engine.TopicFromChannel(ch, f.rootBlockID)) } - return engine.ChannelsByRole(id.Role) + return topics } func (f *Filter) CanSubscribe(topic string) bool { - // TODO: check if this is correct - channel := network.Channel(topic) + for _, allowedTopic := range f.allowedTopics(f.myPeerID) { + if topic == allowedTopic.String() { + return true + } + } - return f.allowedChannels(f.myPeerID).Contains(channel) + return false } func (f *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { - allowedChannels := f.allowedChannels(from) + allowedTopics := f.allowedTopics(from) var filtered []*pb.RPC_SubOpts for _, opt := range opts { - channel := network.Channel(*opt.Topicid) - if allowedChannels.Contains(channel) { - filtered = append(filtered, opt) + for _, allowedTopic := range allowedTopics { + if *opt.Topicid == allowedTopic.String() { + filtered = append(filtered, opt) + } } } diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go new file mode 100644 index 00000000000..867c6a21828 --- /dev/null +++ b/network/p2p/subscription_filter_test.go @@ -0,0 +1,84 @@ +package p2p + +import ( + "context" + "fmt" + "testing" + + "github.com/libp2p/go-libp2p-core/host" + pubsub "github.com/libp2p/go-libp2p-pubsub" + "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/utils/unittest" + "github.com/stretchr/testify/require" +) + +func TestCanSubscribe(t *testing.T) { + identity, privateKey := createID(t) + rootBlockID := unittest.IdentifierFixture() + + node := createNode(t, rootBlockID, identity.NodeID, privateKey, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity}, rootBlockID)) + + badTopic := getDisallowedTopic(t, identity, rootBlockID) + _, err := node.pubSub.Join(badTopic.String()) + + fmt.Println(err) + require.Error(t, err) +} + +func getDisallowedTopic(t *testing.T, id *flow.Identity, rootBlockID flow.Identifier) network.Topic { + allowedChannels := engine.UnstakedChannels() + if id != nil { + allowedChannels = engine.ChannelsByRole(id.Role) + } + + for _, ch := range engine.Channels() { + if !allowedChannels.Contains(ch) { + return engine.TopicFromChannel(ch, rootBlockID) + } + } + + require.FailNow(t, "could not find disallowed topic for role %s", id.Role) + + return "" +} + +func createSubscriptionFilterPubsubOption(t *testing.T, ids flow.IdentityList, rootBlockID flow.Identifier) PubsubOption { + idTranslator, err := NewFixedTableIdentityTranslator(ids) + require.NoError(t, err) + + idProvider := id.NewFixedIdentityProvider(ids) + + return func(_ context.Context, h host.Host) (pubsub.Option, error) { + return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), rootBlockID, idProvider, idTranslator)), nil + } +} + +func createID(t *testing.T) (*flow.Identity, crypto.PrivateKey) { + networkKey, err := unittest.NetworkingKey() + require.NoError(t, err) + id := unittest.IdentityFixture( + unittest.WithRole(flow.RoleAccess), + unittest.WithNetworkingKey(networkKey.PublicKey()), + ) + return id, networkKey +} + +func createNode( + t *testing.T, + rootBlockID flow.Identifier, + nodeID flow.Identifier, + networkKey crypto.PrivateKey, + psOpts ...PubsubOption, +) *Node { + libp2pNode, err := NewDefaultLibP2PNodeBuilder(nodeID, "0.0.0.0:0", networkKey). + SetRootBlockID(rootBlockID). + SetPubsubOptions(psOpts...). + Build(context.TODO()) + require.NoError(t, err) + + return libp2pNode +} diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 9fe6dfe2447..1ec68143e24 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -35,7 +35,7 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) -var rootBlockID = unittest.IdentifierFixture().String() +var rootBlockID = unittest.IdentifierFixture() const DryRun = true From 588c262c9d7b87e87f06c3831f3ea73722155588 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 26 Aug 2021 15:07:37 -0700 Subject: [PATCH 262/291] fix connectionmanager compilation errors --- network/test/testUtil.go | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 9fe6dfe2447..2a125e65167 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -82,8 +82,8 @@ func (cwcm *TagWatchingConnManager) Unprotect(id peer.ID, tag string) bool { return res } -func NewTagWatchingConnManager(log zerolog.Logger, metrics module.NetworkMetrics) *TagWatchingConnManager { - cm := p2p.NewConnManager(log, metrics) +func NewTagWatchingConnManager(log zerolog.Logger, idProvider idModule.IdentityProvider, metrics module.NetworkMetrics) *TagWatchingConnManager { + cm := p2p.NewConnManager(log, idProvider, metrics) return &TagWatchingConnManager{ ConnManager: cm, observers: make(map[observable.Observer]struct{}), @@ -100,6 +100,8 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op identities := unittest.IdentityListFixture(n, opts...) + idProvider := id.NewFixedIdentityProvider(identities) + // generates keys and address for the node for i, id := range identities { // generate key @@ -108,7 +110,7 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, dryRunMode bool, op port := "0" if !dryRunMode { - libP2PNodes[i], tagObservables[i] = generateLibP2PNode(t, logger, *id, key) + libP2PNodes[i], tagObservables[i] = generateLibP2PNode(t, logger, *id, key, idProvider) _, port, err = libP2PNodes[i].GetIPPort() require.NoError(t, err) @@ -258,7 +260,9 @@ func GenerateEngines(t *testing.T, nets []*p2p.Network) []*MeshEngine { func generateLibP2PNode(t *testing.T, logger zerolog.Logger, id flow.Identity, - key crypto.PrivateKey) (*p2p.Node, observable.Observable) { + key crypto.PrivateKey, + idProvider idModule.IdentityProvider, +) (*p2p.Node, observable.Observable) { noopMetrics := metrics.NewNoopCollector() @@ -269,7 +273,7 @@ func generateLibP2PNode(t *testing.T, ctx := context.Background() connGater := p2p.NewConnGater(logger) // Inject some logic to be able to observe connections of this node - connManager := NewTagWatchingConnManager(logger, noopMetrics) + connManager := NewTagWatchingConnManager(logger, idProvider, noopMetrics) libP2PNode, err := p2p.NewDefaultLibP2PNodeBuilder(id.NodeID, "0.0.0.0:0", key). SetRootBlockID(rootBlockID). From 36090428ad59a5fbcef3bdfb41b63d8ae27b0aa8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 26 Aug 2021 15:26:12 -0700 Subject: [PATCH 263/291] Remove unneeded files --- network/converter/subscriptionManager.go | 47 -------- network/p2p/subscriptionFilter_test.go | 145 ----------------------- network/p2p/topicValidator_test.go | 139 ---------------------- 3 files changed, 331 deletions(-) delete mode 100644 network/converter/subscriptionManager.go delete mode 100644 network/p2p/subscriptionFilter_test.go delete mode 100644 network/p2p/topicValidator_test.go diff --git a/network/converter/subscriptionManager.go b/network/converter/subscriptionManager.go deleted file mode 100644 index 169ecb47036..00000000000 --- a/network/converter/subscriptionManager.go +++ /dev/null @@ -1,47 +0,0 @@ -package converter - -import "github.com/onflow/flow-go/network" - -type SubscriptionManager struct { - subMngr network.SubscriptionManager - from network.Channel - to network.Channel -} - -func NewSubscriptionManager(subMngr network.SubscriptionManager, from network.Channel, to network.Channel) *SubscriptionManager { - return &SubscriptionManager{subMngr, from, to} -} - -func (sm *SubscriptionManager) convert(channel network.Channel) network.Channel { - if channel == sm.from { - return sm.to - } - return channel -} - -func (sm *SubscriptionManager) reverse(channel network.Channel) network.Channel { - if channel == sm.to { - return sm.from - } - return channel -} - -func (sm *SubscriptionManager) Register(channel network.Channel, engine network.Engine) error { - return sm.subMngr.Register(sm.convert(channel), engine) -} - -func (sm *SubscriptionManager) Unregister(channel network.Channel) error { - return sm.subMngr.Unregister(sm.convert(channel)) -} - -func (sm *SubscriptionManager) GetEngine(channel network.Channel) (network.Engine, error) { - return sm.subMngr.GetEngine(sm.convert(channel)) -} - -func (sm *SubscriptionManager) Channels() network.ChannelList { - var channels network.ChannelList - for _, ch := range sm.subMngr.Channels() { - channels = append(channels, sm.reverse(ch)) - } - return channels -} diff --git a/network/p2p/subscriptionFilter_test.go b/network/p2p/subscriptionFilter_test.go deleted file mode 100644 index 2ccf022b87a..00000000000 --- a/network/p2p/subscriptionFilter_test.go +++ /dev/null @@ -1,145 +0,0 @@ -package p2p - -import ( - "context" - "fmt" - "testing" - "time" - - golog "github.com/ipfs/go-log" - "github.com/libp2p/go-libp2p" - "github.com/libp2p/go-libp2p-core/host" - "github.com/libp2p/go-libp2p-core/peer" - pubsub "github.com/libp2p/go-libp2p-pubsub" - pb "github.com/libp2p/go-libp2p-pubsub/pb" - "github.com/stretchr/testify/require" -) - -func TestBasicSubscriptionFilter(t *testing.T) { - golog.SetAllLoggers(golog.LevelDebug) - ctx := context.Background() - host1, err := libp2p.New(ctx) - require.NoError(t, err) - host2, err := libp2p.New(ctx) - require.NoError(t, err) - host3, err := libp2p.New(ctx) - require.NoError(t, err) - - require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host2))) - require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host3))) - - topicname1 := "testtopic1" - topicname2 := "testtopic2" - - filter := &Filter{ - allowedIDs: make(map[peer.ID]struct{}), - topic: topicname2, - } - filter.allowedIDs[host1.ID()] = struct{}{} - filter.allowedIDs[host2.ID()] = struct{}{} - - ps1, err := pubsub.NewGossipSub(ctx, host1, pubsub.WithSubscriptionFilter(filter)) - require.NoError(t, err) - ps2, err := pubsub.NewGossipSub(ctx, host2, pubsub.WithSubscriptionFilter(filter)) - require.NoError(t, err) - ps3, err := pubsub.NewGossipSub(ctx, host3) - require.NoError(t, err) - - topic1, err := ps1.Join(topicname1) - require.NoError(t, err) - _, err = topic1.Subscribe() - require.NoError(t, err) - topic1, err = ps1.Join(topicname2) - require.NoError(t, err) - subscriberHost1Topic2, err := topic1.Subscribe() - require.NoError(t, err) - - topic2, err := ps2.Join(topicname1) - require.NoError(t, err) - _, err = topic2.Subscribe() - require.NoError(t, err) - topic2Host2, err := ps2.Join(topicname2) - require.NoError(t, err) - subscriberHost2Topic2, err := topic2.Subscribe() - require.NoError(t, err) - - topic3, err := ps3.Join(topicname1) - require.NoError(t, err) - _, err = topic3.Subscribe() - require.NoError(t, err) - wrongTopic, err := ps3.Join(topicname2) - require.NoError(t, err) - _, err = wrongTopic.Subscribe() - require.NoError(t, err) - - time.Sleep(2 * time.Second) - - fmt.Printf("host1: %s, host2: %s, host3 :%s\n", host1.ID(), host2.ID(), host3.ID()) - //fmt.Print(host1.Peerstore().Peers()) - //fmt.Print(host2.Peerstore().Peers()) - fmt.Print("host 1 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps1.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps1.ListPeers(topicname2)) - - fmt.Print("host 2 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps2.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps2.ListPeers(topicname2)) - - fmt.Print("host 3 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps3.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps3.ListPeers(topicname2)) - - err = topic2Host2.Publish(ctx, []byte("hello")) - require.NoError(t, err) - - time.Sleep(2 * time.Second) - - msg, err := subscriberHost2Topic2.Next(ctx) - require.NoError(t, err) - fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) - fmt.Println(msg.Data) - - msg, err = subscriberHost1Topic2.Next(ctx) - require.NoError(t, err) - fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) - fmt.Println(msg) - - fmt.Print("host 2 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps2.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps2.ListPeers(topicname2)) - t.Fail() -} - -var _ pubsub.SubscriptionFilter = (*Filter)(nil) - -type Filter struct { - allowedIDs map[peer.ID]struct{} - topic string -} - -func (filter *Filter) CanSubscribe(topic string) bool { - return true -} - -func (filter *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { - if _, found := filter.allowedIDs[from]; !found { - var newopts []*pb.RPC_SubOpts - for _, opt := range opts { - if *opt.Topicid != filter.topic { - newopts = append(newopts, opt) - } else { - return nil, fmt.Errorf(">>>>>> message received on a topic on which peer %s should not publish", from.String()) - } - } - return newopts, nil - } - return opts, nil -} diff --git a/network/p2p/topicValidator_test.go b/network/p2p/topicValidator_test.go deleted file mode 100644 index 1282306e159..00000000000 --- a/network/p2p/topicValidator_test.go +++ /dev/null @@ -1,139 +0,0 @@ -package p2p - -import ( - "context" - "fmt" - "testing" - "time" - - golog "github.com/ipfs/go-log" - "github.com/libp2p/go-libp2p" - "github.com/libp2p/go-libp2p-core/host" - "github.com/libp2p/go-libp2p-core/peer" - pubsub "github.com/libp2p/go-libp2p-pubsub" - "github.com/stretchr/testify/require" -) - -func TestTopicValidator(t *testing.T) { - golog.SetAllLoggers(golog.LevelDebug) - ctx := context.Background() - host1, err := libp2p.New(ctx) - require.NoError(t, err) - host2, err := libp2p.New(ctx) - require.NoError(t, err) - host3, err := libp2p.New(ctx) - require.NoError(t, err) - - require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host2))) - require.NoError(t, host1.Connect(ctx, *host.InfoFromHost(host3))) - - - topicname1 := "testtopic1" - topicname2 := "testtopic2" - - - ps1, err := pubsub.NewGossipSub(ctx, host1) - require.NoError(t, err) - - ps2, err := pubsub.NewGossipSub(ctx, host2) - require.NoError(t, err) - - ps3, err := pubsub.NewGossipSub(ctx, host3) - require.NoError(t, err) - - validator := &topicValidator{ - allowedIDs: make(map[peer.ID]struct{}), - blackListFunc: ps1.BlacklistPeer, - } - validator.allowedIDs[host1.ID()] = struct{}{} - validator.allowedIDs[host2.ID()] = struct{}{} - ps1.RegisterTopicValidator(topicname2, validator.validate) - - - topic1, err := ps1.Join(topicname1) - require.NoError(t, err) - _, err = topic1.Subscribe() - require.NoError(t, err) - topic1, err = ps1.Join(topicname2) - require.NoError(t, err) - subscriberHost1Topic2, err := topic1.Subscribe() - require.NoError(t, err) - - topic2, err := ps2.Join(topicname1) - require.NoError(t, err) - _, err = topic2.Subscribe() - require.NoError(t, err) - topic2, err = ps2.Join(topicname2) - require.NoError(t, err) - subscriberHost2Topic2, err := topic2.Subscribe() - require.NoError(t, err) - - topic3, err := ps3.Join(topicname1) - require.NoError(t, err) - _, err = topic3.Subscribe() - require.NoError(t, err) - wrongTopic, err := ps3.Join(topicname2) - require.NoError(t, err) - _, err = wrongTopic.Subscribe() - require.NoError(t, err) - - time.Sleep(2 * time.Second) - - fmt.Printf("host1: %s, host2: %s, host3 :%s\n", host1.ID(), host2.ID(), host3.ID()) - //fmt.Print(host1.Peerstore().Peers()) - //fmt.Print(host2.Peerstore().Peers()) - fmt.Print("host 1 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps1.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps1.ListPeers(topicname2)) - - fmt.Print("host 2 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps2.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps2.ListPeers(topicname2)) - - fmt.Print("host 3 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps3.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps3.ListPeers(topicname2)) - - err = wrongTopic.Publish(ctx, []byte("hello")) - require.NoError(t, err) - - time.Sleep(2 * time.Second) - - - fmt.Print("host 1 peers\n") - fmt.Printf("\t For %s", topicname1) - fmt.Println(ps1.ListPeers(topicname1)) - fmt.Printf("\t For %s", topicname2) - fmt.Println(ps1.ListPeers(topicname2)) - fmt.Println(host1.Peerstore().Peers()) - - msg, err := subscriberHost2Topic2.Next(ctx) - require.NoError(t, err) - fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) - fmt.Println(msg) - - msg, err = subscriberHost1Topic2.Next(ctx) - require.NoError(t, err) - fmt.Printf(" message recvd on topic %s from peer %s\n", *msg.Topic, msg.ReceivedFrom.String()) - fmt.Println(msg) - - -} - -type topicValidator struct { - allowedIDs map[peer.ID]struct{} - blackListFunc func(id peer.ID) -} -func(validator *topicValidator) validate(ctx context.Context, from peer.ID, _ *pubsub.Message) bool { - _, found := validator.allowedIDs[from] - if !found { - validator.blackListFunc(from) - } - return found -} From 2c455787b988842907a2c9d92c06ad9a5285a2f2 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 26 Aug 2021 16:06:43 -0700 Subject: [PATCH 264/291] Add todos, remove string --- .../node_builder/access_node_builder.go | 5 +++-- .../staked_access_node_builder.go | 2 +- network/p2p/libp2pNode.go | 22 +++++++++---------- 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index d638c01d7d0..e3aaed3da4f 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -620,10 +620,11 @@ func (builder *FlowAccessNodeBuilder) initLibP2PFactory(ctx context.Context, return func() (*p2p.Node, error) { libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). - SetRootBlockID(builder.RootBlock.ID().String()). + SetRootBlockID(builder.RootBlock.ID()). // unlike the staked side of the network where currently all the node addresses are known upfront, // for the unstaked side of the network, the nodes need to discover each other using DHT Discovery. SetDHTOptions(dhtOptions...). + // TODO: set pubsub options with subscription filter SetLogger(builder.Logger). Build(ctx) if err != nil { @@ -645,7 +646,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, factoryFunc, nodeID, networkMetrics, - builder.RootBlock.ID().String(), + builder.RootBlock.ID(), time.Hour, // TODO: this is pretty meaningless since there is no peermanager in play. p2p.DefaultUnicastTimeout, false, // no connection gating for the unstaked network diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 251ee6ea4d3..0e0c4f6b06b 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -194,7 +194,7 @@ func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, return func() (*p2p.Node, error) { libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). - SetRootBlockID(builder.RootBlock.ID().String()). + SetRootBlockID(builder.RootBlock.ID()). // no connection gater SetConnectionManager(connManager). // act as a DHT server diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 7bf8de2b1ba..e048d37cabb 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -66,17 +66,22 @@ func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.I connGater := NewConnGater(log) // TODO: uncomment following lines to activate dns caching - //resolver, err := dns.NewResolver(metrics) - //if err != nil { - // return nil, fmt.Errorf("could not create dns resolver: %w", err) - //} + // resolver, err := dns.NewResolver(metrics) + // if err != nil { + // return nil, fmt.Errorf("could not create dns resolver: %w", err) + // } + + psOpts := DefaultPubsubOptions(maxPubSubMsgSize) + psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { + return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), rootBlockID, idProvider, idTranslator)), nil + }) return func() (*Node, error) { return NewDefaultLibP2PNodeBuilder(me, address, flowKey). SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize, idProvider, idTranslator, rootBlockID)...). + SetPubsubOptions(psOpts...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). // SetResolver(resolver). @@ -667,7 +672,7 @@ func DefaultPubSub(ctx context.Context, host host.Host, psOption ...pubsub.Optio // PubsubOption generates a libp2p pubsub.Option from the given context and host type PubsubOption func(ctx context.Context, host host.Host) (pubsub.Option, error) -func DefaultPubsubOptions(maxPubSubMsgSize int, idProvider id.IdentityProvider, idTranslator IDTranslator, rootBlockID flow.Identifier) []PubsubOption { +func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { pubSubOptionFunc := func(option pubsub.Option) PubsubOption { return func(_ context.Context, _ host.Host) (pubsub.Option, error) { return option, nil @@ -681,11 +686,6 @@ func DefaultPubsubOptions(maxPubSubMsgSize int, idProvider id.IdentityProvider, // set max message size limit for 1-k PubSub messaging pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), // no discovery - - // subscription filter - func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), rootBlockID, idProvider, idTranslator)), nil - }, } } From 4b698ca551887d94293418a61274fcf742e17496 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 26 Aug 2021 16:26:51 -0700 Subject: [PATCH 265/291] Update connmanager --- .../staked_access_node_builder.go | 2 +- cmd/scaffold.go | 1 - network/p2p/connManager.go | 38 +++++++++++++------ network/p2p/dht_test.go | 5 --- network/p2p/libp2pNode.go | 5 +-- network/p2p/libp2pNode_test.go | 4 -- network/test/testUtil.go | 2 +- 7 files changed, 31 insertions(+), 26 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index ea7b5aa834d..3b742b0349b 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -191,7 +191,7 @@ func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, myAddr = builder.BaseConfig.BindAddr } - connManager := p2p.NewConnManager(builder.Logger, builder.IdentityProvider, builder.Metrics.Network) + connManager := p2p.NewConnManager(builder.Logger, builder.Metrics.Network, p2p.TrackUnstakedConnections(builder.IdentityProvider)) return func() (*p2p.Node, error) { libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). diff --git a/cmd/scaffold.go b/cmd/scaffold.go index cefc7fb6cb4..d95ae22cc96 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -165,7 +165,6 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { p2p.DefaultMaxPubSubMsgSize, fnb.Metrics.Network, pingProvider, - node.IdentityProvider, ) if err != nil { return nil, fmt.Errorf("could not generate libp2p node factory: %w", err) diff --git a/network/p2p/connManager.go b/network/p2p/connManager.go index ed5b0f8c176..5e6115f3eae 100644 --- a/network/p2p/connManager.go +++ b/network/p2p/connManager.go @@ -42,19 +42,31 @@ type ConnManager struct { idProvider id.IdentityProvider } -func NewConnManager(log zerolog.Logger, idProvider id.IdentityProvider, metrics module.NetworkMetrics) *ConnManager { +type ConnManagerOption func(*ConnManager) + +func TrackUnstakedConnections(idProvider id.IdentityProvider) ConnManagerOption { + return func(cm *ConnManager) { + cm.idProvider = idProvider + } +} + +func NewConnManager(log zerolog.Logger, metrics module.NetworkMetrics, opts ...ConnManagerOption) *ConnManager { cn := &ConnManager{ log: log, NullConnMgr: connmgr.NullConnMgr{}, metrics: metrics, streamSetupInProgressCnt: make(map[peer.ID]int), - idProvider: idProvider, } n := &network.NotifyBundle{ListenCloseF: cn.ListenCloseNotifee, ListenF: cn.ListenNotifee, ConnectedF: cn.Connected, DisconnectedF: cn.Disconnected} cn.n = n + + for _, opt := range opts { + opt(cn) + } + return cn } @@ -93,12 +105,14 @@ func (c *ConnManager) updateConnectionMetric(n network.Network) { var totalOutbound uint = 0 stakedPeers := make(map[peer.ID]struct{}) - for _, id := range c.idProvider.Identities(NotEjectedFilter) { - pid, err := ExtractPeerID(id.NetworkPubKey) - if err != nil { - c.log.Fatal().Err(err).Msg("failed to convert network public key of staked node to peer ID") + if c.idProvider != nil { + for _, id := range c.idProvider.Identities(NotEjectedFilter) { + pid, err := ExtractPeerID(id.NetworkPubKey) + if err != nil { + c.log.Fatal().Err(err).Msg("failed to convert network public key of staked node to peer ID") + } + stakedPeers[pid] = struct{}{} } - stakedPeers[pid] = struct{}{} } for _, conn := range n.Conns() { _, isStaked := stakedPeers[conn.RemotePeer()] @@ -116,10 +130,12 @@ func (c *ConnManager) updateConnectionMetric(n network.Network) { } } - c.metrics.InboundConnections(stakedInbound) - c.metrics.OutboundConnections(stakedOutbound) - c.metrics.UnstakedInboundConnections(totalInbound - stakedInbound) - c.metrics.UnstakedOutboundConnections(totalOutbound - stakedOutbound) + c.metrics.InboundConnections(totalInbound) + c.metrics.OutboundConnections(totalOutbound) + if c.idProvider != nil { + c.metrics.UnstakedInboundConnections(totalInbound - stakedInbound) + c.metrics.UnstakedOutboundConnections(totalOutbound - stakedOutbound) + } } func (c *ConnManager) logConnectionUpdate(n network.Network, con network.Conn, logMsg string) { diff --git a/network/p2p/dht_test.go b/network/p2p/dht_test.go index 4d2cebcb00e..d4dc481ea98 100644 --- a/network/p2p/dht_test.go +++ b/network/p2p/dht_test.go @@ -18,7 +18,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/metrics" flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/utils/unittest" ) @@ -229,15 +228,11 @@ func (suite *DHTTestSuite) CreateNodes(count int, dhtServer bool) (nodes []*Node // creating nodes for i := 1; i <= count; i++ { key := generateNetworkingKey(suite.T()) - noopMetrics := metrics.NewNoopCollector() pingInfoProvider, _, _ := MockPingInfoProvider() - connManager := NewConnManager(logger, noopMetrics) - n, err := NewDefaultLibP2PNodeBuilder(flow.Identifier{}, "0.0.0.0:0", key). SetRootBlockID(rootBlockID). - SetConnectionManager(connManager). SetDHTOptions(AsServer(dhtServer)). SetPingInfoProvider(pingInfoProvider). SetLogger(logger). diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 3bd17fe8eec..f0bd7d37260 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -31,7 +31,6 @@ import ( fcrypto "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/id" flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/logging" @@ -59,9 +58,9 @@ type LibP2PFactoryFunc func() (*Node, error) // DefaultLibP2PNodeFactory returns a LibP2PFactoryFunc which generates the libp2p host initialized with the // default options for the host, the pubsub and the ping service. func DefaultLibP2PNodeFactory(ctx context.Context, log zerolog.Logger, me flow.Identifier, address string, flowKey fcrypto.PrivateKey, rootBlockID string, - maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider, idProvider id.IdentityProvider) (LibP2PFactoryFunc, error) { + maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider) (LibP2PFactoryFunc, error) { - connManager := NewConnManager(log, idProvider, metrics) + connManager := NewConnManager(log, metrics) connGater := NewConnGater(log) diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index 8d80914f9d2..fe2bae9ed18 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -680,12 +680,8 @@ func NodeFixture(t *testing.T, log zerolog.Logger, key fcrypto.PrivateKey, rootI resolver, err := dns.NewResolver(metrics.NewNoopCollector()) require.NoError(t, err) - noopMetrics := metrics.NewNoopCollector() - connManager := NewConnManager(log, noopMetrics) - builder := NewDefaultLibP2PNodeBuilder(identity.NodeID, address, key). SetRootBlockID(rootID). - SetConnectionManager(connManager). SetPingInfoProvider(pingInfoProvider). SetResolver(resolver). SetLogger(log) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index f35ce8f7e60..55f75ddfae9 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -84,7 +84,7 @@ func (cwcm *TagWatchingConnManager) Unprotect(id peer.ID, tag string) bool { } func NewTagWatchingConnManager(log zerolog.Logger, idProvider idModule.IdentityProvider, metrics module.NetworkMetrics) *TagWatchingConnManager { - cm := p2p.NewConnManager(log, idProvider, metrics) + cm := p2p.NewConnManager(log, metrics) return &TagWatchingConnManager{ ConnManager: cm, observers: make(map[observable.Observer]struct{}), From 332979d7ac75baae0312307585e2425608264884 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 31 Aug 2021 17:33:42 -0400 Subject: [PATCH 266/291] do things --- engine/channels.go | 2 +- network/p2p/libp2pNode_test.go | 4 +- network/p2p/sporking_test.go | 8 +-- network/p2p/subscription_filter.go | 32 +++++---- network/p2p/subscription_filter_test.go | 95 +++++++++++++++++++++++-- 5 files changed, 113 insertions(+), 28 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index 2f25931a537..10dc48a51c8 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -161,7 +161,7 @@ func initializeChannelRoleMap() { channelRoleMap[ConsensusCommittee] = flow.RoleList{flow.RoleConsensus} // Channels for protocols actively synchronizing state across nodes - channelRoleMap[SyncCommittee] = flow.RoleList{flow.RoleConsensus} + channelRoleMap[SyncCommittee] = flow.Roles() channelRoleMap[SyncExecution] = flow.RoleList{flow.RoleExecution} // Channels for DKG communication diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index fe2bae9ed18..974374b6eab 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -41,7 +41,7 @@ const tickForAssertEventually = 100 * time.Millisecond // "0.0.0.0: const defaultAddress = "0.0.0.0:0" -var rootBlockID = unittest.IdentifierFixture().String() +var rootBlockID = unittest.IdentifierFixture() type LibP2PNodeTestSuite struct { suite.Suite @@ -661,7 +661,7 @@ func (suite *LibP2PNodeTestSuite) NodesFixture(count int, handler func(t *testin // NodeFixture creates a single LibP2PNodes with the given key, root block id, and callback function for stream handling. // It returns the nodes and their identities. -func NodeFixture(t *testing.T, log zerolog.Logger, key fcrypto.PrivateKey, rootID string, handler func(t *testing.T) network.StreamHandler, allowList bool, address string) (*Node, flow.Identity) { +func NodeFixture(t *testing.T, log zerolog.Logger, key fcrypto.PrivateKey, rootID flow.Identifier, handler func(t *testing.T) network.StreamHandler, allowList bool, address string) (*Node, flow.Identity) { identity := unittest.IdentityFixture(unittest.WithNetworkingKey(key.PublicKey()), unittest.WithAddress(address)) diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index b3462e05ccb..32cdc456dc3 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -82,7 +82,7 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { // root id before spork - rootID1 := unittest.BlockFixture().ID().String() + rootID1 := unittest.BlockFixture().ID() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -104,7 +104,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { StopNode(suite.T(), node2) // update the flow root id for node 2. node1 is still listening on the old protocol - rootID2 := unittest.BlockFixture().ID().String() + rootID2 := unittest.BlockFixture().ID() // start node2 with the same address and root key but different root block id node2, id2New := NodeFixture(suite.T(), suite.logger, node2key, rootID2, nil, false, id2.Address) @@ -123,7 +123,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { // root id before spork - rootIDBeforeSpork := unittest.BlockFixture().ID().String() + rootIDBeforeSpork := unittest.BlockFixture().ID() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -160,7 +160,7 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { testOneToKMessagingSucceeds(ctx, suite.T(), node1, sub2, topicBeforeSpork) // new root id after spork - rootIDAfterSpork := unittest.BlockFixture().ID().String() + rootIDAfterSpork := unittest.BlockFixture().ID() // topic after the spork topicAfterSpork := engine.TopicFromChannel(engine.TestNetwork, rootIDAfterSpork) diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index 0ad3a5b55e6..1ed005653cc 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -1,6 +1,8 @@ package p2p import ( + "fmt" + "github.com/libp2p/go-libp2p-core/peer" pb "github.com/libp2p/go-libp2p-pubsub/pb" @@ -42,34 +44,34 @@ func (f *Filter) getIdentity(pid peer.ID) *flow.Identity { return identities[0] } -func (f *Filter) allowedTopics(pid peer.ID) []network.Topic { +func (f *Filter) allowedTopics(pid peer.ID) map[network.Topic]struct{} { id := f.getIdentity(pid) var channels network.ChannelList if id == nil { + fmt.Println("Unstaked channels allowed by " + f.myPeerID.String() + " for " + pid.String()) channels = engine.UnstakedChannels() } else { + fmt.Println("Staked channels allowed by " + f.myPeerID.String() + " for " + pid.String()) channels = engine.ChannelsByRole(id.Role) } - var topics []network.Topic + topics := make(map[network.Topic]struct{}) for _, ch := range channels { - topics = append(topics, engine.TopicFromChannel(ch, f.rootBlockID)) + // TODO: we will probably have problems here with cluster channels + // We probably need special checking for this + // Add a unit test for cluster channels + topics[engine.TopicFromChannel(ch, f.rootBlockID)] = struct{}{} } return topics } func (f *Filter) CanSubscribe(topic string) bool { - for _, allowedTopic := range f.allowedTopics(f.myPeerID) { - if topic == allowedTopic.String() { - return true - } - } - - return false + _, allowed := f.allowedTopics(f.myPeerID)[network.Topic(topic)] + return allowed } func (f *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { @@ -77,10 +79,12 @@ func (f *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpt var filtered []*pb.RPC_SubOpts for _, opt := range opts { - for _, allowedTopic := range allowedTopics { - if *opt.Topicid == allowedTopic.String() { - filtered = append(filtered, opt) - } + fmt.Println("Request " + *opt.Topicid + "to") + if _, allowed := allowedTopics[network.Topic(opt.GetTopicid())]; allowed { + + filtered = append(filtered, opt) + } else { + fmt.Println("Blocked") } } diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index 867c6a21828..c054f3b9cd0 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -3,7 +3,9 @@ package p2p import ( "context" "fmt" + "sync" "testing" + "time" "github.com/libp2p/go-libp2p-core/host" pubsub "github.com/libp2p/go-libp2p-pubsub" @@ -16,20 +18,100 @@ import ( "github.com/stretchr/testify/require" ) +func TestFilterSubscribe(t *testing.T) { + identity1, privateKey1 := createID(t) + identity2, privateKey2 := createID(t) + ids := flow.IdentityList{identity1, identity2} + + node1 := createNode(t, identity1.NodeID, privateKey1, createSubscriptionFilterPubsubOption(t, ids)) + node2 := createNode(t, identity2.NodeID, privateKey2, createSubscriptionFilterPubsubOption(t, ids)) + + unstakedKey, err := unittest.NetworkingKey() + require.NoError(t, err) + unstakedNode := createNode(t, flow.ZeroID, unstakedKey) + + fmt.Println(node1.host.ID()) + fmt.Println(node2.host.ID()) + fmt.Println(unstakedNode.host.ID()) + + require.NoError(t, node1.AddPeer(context.TODO(), *host.InfoFromHost(node2.Host()))) + require.NoError(t, node1.AddPeer(context.TODO(), *host.InfoFromHost(unstakedNode.Host()))) + + badTopic := engine.TopicFromChannel(engine.SyncCommittee, rootBlockID) + + sub1, err := node1.Subscribe(context.TODO(), badTopic) + require.NoError(t, err) + time.Sleep(300 * time.Millisecond) + sub2, err := node2.Subscribe(context.TODO(), badTopic) + require.NoError(t, err) + time.Sleep(300 * time.Millisecond) + unstakedSub, err := unstakedNode.Subscribe(context.TODO(), badTopic) + require.NoError(t, err) + time.Sleep(300 * time.Millisecond) + + require.Eventually(t, func() bool { + return len(node1.pubSub.ListPeers(badTopic.String())) > 0 && + len(node2.pubSub.ListPeers(badTopic.String())) > 0 && + len(unstakedNode.pubSub.ListPeers(badTopic.String())) > 0 + }, 1*time.Second, 100*time.Millisecond) + + require.Never(t, func() bool { + for _, pid := range node1.pubSub.ListPeers(badTopic.String()) { + if pid == unstakedNode.Host().ID() { + return true + } + } + return false + }, 1*time.Second, 100*time.Millisecond) + + var wg sync.WaitGroup + wg.Add(2) + + testPublish := func(wg *sync.WaitGroup, from *Node, sub *pubsub.Subscription) { + data := []byte("hello") + + from.Publish(context.TODO(), badTopic, data) + + fmt.Println(from.pubSub.ListPeers(badTopic.String())) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + msg, err := sub.Next(ctx) + cancel() + require.NoError(t, err) + require.Equal(t, msg.Data, data) + + ctx, cancel = context.WithTimeout(context.Background(), time.Second) + msg, err = unstakedSub.Next(ctx) + fmt.Println(msg) + cancel() + require.ErrorIs(t, err, context.DeadlineExceeded) + + wg.Done() + } + + // publish a message from node 1 and check that only node2 receives + testPublish(&wg, node1, sub2) + + // publish a message from node 2 and check that only node1 receives + testPublish(&wg, node2, sub1) + + fmt.Println(sub2) + + wg.Wait() +} + func TestCanSubscribe(t *testing.T) { identity, privateKey := createID(t) - rootBlockID := unittest.IdentifierFixture() - node := createNode(t, rootBlockID, identity.NodeID, privateKey, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity}, rootBlockID)) + node := createNode(t, identity.NodeID, privateKey, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity})) - badTopic := getDisallowedTopic(t, identity, rootBlockID) + badTopic := getDisallowedTopic(t, identity) _, err := node.pubSub.Join(badTopic.String()) - fmt.Println(err) require.Error(t, err) } -func getDisallowedTopic(t *testing.T, id *flow.Identity, rootBlockID flow.Identifier) network.Topic { +func getDisallowedTopic(t *testing.T, id *flow.Identity) network.Topic { allowedChannels := engine.UnstakedChannels() if id != nil { allowedChannels = engine.ChannelsByRole(id.Role) @@ -46,7 +128,7 @@ func getDisallowedTopic(t *testing.T, id *flow.Identity, rootBlockID flow.Identi return "" } -func createSubscriptionFilterPubsubOption(t *testing.T, ids flow.IdentityList, rootBlockID flow.Identifier) PubsubOption { +func createSubscriptionFilterPubsubOption(t *testing.T, ids flow.IdentityList) PubsubOption { idTranslator, err := NewFixedTableIdentityTranslator(ids) require.NoError(t, err) @@ -69,7 +151,6 @@ func createID(t *testing.T) (*flow.Identity, crypto.PrivateKey) { func createNode( t *testing.T, - rootBlockID flow.Identifier, nodeID flow.Identifier, networkKey crypto.PrivateKey, psOpts ...PubsubOption, From d3d45e7163d391019f0be9678e5356a6ad5f45a7 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 9 Sep 2021 22:47:42 -0400 Subject: [PATCH 267/291] bleh --- module/id/updatable_provider.go | 39 ------ module/metrics/network.go | 8 -- module/metrics/noop.go | 2 - module/mocks/network.go | 64 +++++----- network/p2p/sporking_test.go | 8 +- storage/mocks/storage.go | 208 ++++++++++++++++---------------- 6 files changed, 138 insertions(+), 191 deletions(-) delete mode 100644 module/id/updatable_provider.go diff --git a/module/id/updatable_provider.go b/module/id/updatable_provider.go deleted file mode 100644 index d35a3da0566..00000000000 --- a/module/id/updatable_provider.go +++ /dev/null @@ -1,39 +0,0 @@ -package id - -import ( - "sync" - - "github.com/onflow/flow-go/model/flow" -) - -// UpdatableIDProvider implements an IdentityProvider which can be manually updated by setting -// the IdentityList to a new value. -// It also implements an IdentifierProvider which provides the identifiers of the IdentityList. -// This is mainly used to simulate epoch transitions in tests. -type UpdatableIDProvider struct { - mu sync.RWMutex - identities flow.IdentityList -} - -func NewUpdatableIDProvider(identities flow.IdentityList) *UpdatableIDProvider { - return &UpdatableIDProvider{identities: identities} -} - -// SetIdentities updates the IdentityList returned by this provider. -func (p *UpdatableIDProvider) SetIdentities(identities flow.IdentityList) { - p.mu.Lock() - defer p.mu.Unlock() - p.identities = identities -} - -func (p *UpdatableIDProvider) Identities(filter flow.IdentityFilter) flow.IdentityList { - p.mu.RLock() - defer p.mu.RUnlock() - return p.identities.Filter(filter) -} - -func (p *UpdatableIDProvider) Identifiers() flow.IdentifierList { - p.mu.RLock() - defer p.mu.RUnlock() - return p.identities.NodeIDs() -} diff --git a/module/metrics/network.go b/module/metrics/network.go index 9cae7309886..b6cedeb9601 100644 --- a/module/metrics/network.go +++ b/module/metrics/network.go @@ -186,14 +186,6 @@ func (nc *NetworkCollector) InboundConnections(connectionCount uint) { nc.inboundConnectionCount.Set(float64(connectionCount)) } -func (nc *NetworkCollector) UnstakedOutboundConnections(connectionCount uint) { - nc.unstakedOutboundConnectionCount.Set(float64(connectionCount)) -} - -func (nc *NetworkCollector) UnstakedInboundConnections(connectionCount uint) { - nc.unstakedInboundConnectionCount.Set(float64(connectionCount)) -} - // DNSLookupDuration tracks the time spent to resolve a DNS address. func (nc *NetworkCollector) DNSLookupDuration(duration time.Duration) { nc.dnsLookupDuration.Observe(float64(duration.Milliseconds())) diff --git a/module/metrics/noop.go b/module/metrics/noop.go index 242131568e7..de5005c8392 100644 --- a/module/metrics/noop.go +++ b/module/metrics/noop.go @@ -27,8 +27,6 @@ func (nc *NoopCollector) MessageReceived(engine string, message string) func (nc *NoopCollector) MessageHandled(engine string, message string) {} func (nc *NoopCollector) OutboundConnections(_ uint) {} func (nc *NoopCollector) InboundConnections(_ uint) {} -func (nc *NoopCollector) UnstakedOutboundConnections(_ uint) {} -func (nc *NoopCollector) UnstakedInboundConnections(_ uint) {} func (nc *NoopCollector) DNSLookupDuration(duration time.Duration) {} func (nc *NoopCollector) OnDNSCacheMiss() {} func (nc *NoopCollector) OnDNSCacheInvalidated() {} diff --git a/module/mocks/network.go b/module/mocks/network.go index 305cee9f346..38f65a552ce 100644 --- a/module/mocks/network.go +++ b/module/mocks/network.go @@ -5,40 +5,38 @@ package mocks import ( - reflect "reflect" - gomock "github.com/golang/mock/gomock" - crypto "github.com/onflow/flow-go/crypto" hash "github.com/onflow/flow-go/crypto/hash" flow "github.com/onflow/flow-go/model/flow" network "github.com/onflow/flow-go/network" + reflect "reflect" ) -// MockNetwork is a mock of Network interface. +// MockNetwork is a mock of Network interface type MockNetwork struct { ctrl *gomock.Controller recorder *MockNetworkMockRecorder } -// MockNetworkMockRecorder is the mock recorder for MockNetwork. +// MockNetworkMockRecorder is the mock recorder for MockNetwork type MockNetworkMockRecorder struct { mock *MockNetwork } -// NewMockNetwork creates a new mock instance. +// NewMockNetwork creates a new mock instance func NewMockNetwork(ctrl *gomock.Controller) *MockNetwork { mock := &MockNetwork{ctrl: ctrl} mock.recorder = &MockNetworkMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockNetwork) EXPECT() *MockNetworkMockRecorder { return m.recorder } -// Register mocks base method. +// Register mocks base method func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (network.Conduit, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Register", arg0, arg1) @@ -47,36 +45,36 @@ func (m *MockNetwork) Register(arg0 network.Channel, arg1 network.Engine) (netwo return ret0, ret1 } -// Register indicates an expected call of Register. +// Register indicates an expected call of Register func (mr *MockNetworkMockRecorder) Register(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Register", reflect.TypeOf((*MockNetwork)(nil).Register), arg0, arg1) } -// MockLocal is a mock of Local interface. +// MockLocal is a mock of Local interface type MockLocal struct { ctrl *gomock.Controller recorder *MockLocalMockRecorder } -// MockLocalMockRecorder is the mock recorder for MockLocal. +// MockLocalMockRecorder is the mock recorder for MockLocal type MockLocalMockRecorder struct { mock *MockLocal } -// NewMockLocal creates a new mock instance. +// NewMockLocal creates a new mock instance func NewMockLocal(ctrl *gomock.Controller) *MockLocal { mock := &MockLocal{ctrl: ctrl} mock.recorder = &MockLocalMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockLocal) EXPECT() *MockLocalMockRecorder { return m.recorder } -// Address mocks base method. +// Address mocks base method func (m *MockLocal) Address() string { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Address") @@ -84,13 +82,13 @@ func (m *MockLocal) Address() string { return ret0 } -// Address indicates an expected call of Address. +// Address indicates an expected call of Address func (mr *MockLocalMockRecorder) Address() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Address", reflect.TypeOf((*MockLocal)(nil).Address)) } -// NodeID mocks base method. +// NodeID mocks base method func (m *MockLocal) NodeID() flow.Identifier { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NodeID") @@ -98,13 +96,13 @@ func (m *MockLocal) NodeID() flow.Identifier { return ret0 } -// NodeID indicates an expected call of NodeID. +// NodeID indicates an expected call of NodeID func (mr *MockLocalMockRecorder) NodeID() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NodeID", reflect.TypeOf((*MockLocal)(nil).NodeID)) } -// NotMeFilter mocks base method. +// NotMeFilter mocks base method func (m *MockLocal) NotMeFilter() flow.IdentityFilter { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NotMeFilter") @@ -112,13 +110,13 @@ func (m *MockLocal) NotMeFilter() flow.IdentityFilter { return ret0 } -// NotMeFilter indicates an expected call of NotMeFilter. +// NotMeFilter indicates an expected call of NotMeFilter func (mr *MockLocalMockRecorder) NotMeFilter() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NotMeFilter", reflect.TypeOf((*MockLocal)(nil).NotMeFilter)) } -// Sign mocks base method. +// Sign mocks base method func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Sign", arg0, arg1) @@ -127,13 +125,13 @@ func (m *MockLocal) Sign(arg0 []byte, arg1 hash.Hasher) (crypto.Signature, error return ret0, ret1 } -// Sign indicates an expected call of Sign. +// Sign indicates an expected call of Sign func (mr *MockLocalMockRecorder) Sign(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sign", reflect.TypeOf((*MockLocal)(nil).Sign), arg0, arg1) } -// SignFunc mocks base method. +// SignFunc mocks base method func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.PrivateKey, []byte, hash.Hasher) (crypto.Signature, error)) (crypto.Signature, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SignFunc", arg0, arg1, arg2) @@ -142,66 +140,66 @@ func (m *MockLocal) SignFunc(arg0 []byte, arg1 hash.Hasher, arg2 func(crypto.Pri return ret0, ret1 } -// SignFunc indicates an expected call of SignFunc. +// SignFunc indicates an expected call of SignFunc func (mr *MockLocalMockRecorder) SignFunc(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignFunc", reflect.TypeOf((*MockLocal)(nil).SignFunc), arg0, arg1, arg2) } -// MockRequester is a mock of Requester interface. +// MockRequester is a mock of Requester interface type MockRequester struct { ctrl *gomock.Controller recorder *MockRequesterMockRecorder } -// MockRequesterMockRecorder is the mock recorder for MockRequester. +// MockRequesterMockRecorder is the mock recorder for MockRequester type MockRequesterMockRecorder struct { mock *MockRequester } -// NewMockRequester creates a new mock instance. +// NewMockRequester creates a new mock instance func NewMockRequester(ctrl *gomock.Controller) *MockRequester { mock := &MockRequester{ctrl: ctrl} mock.recorder = &MockRequesterMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockRequester) EXPECT() *MockRequesterMockRecorder { return m.recorder } -// EntityByID mocks base method. +// EntityByID mocks base method func (m *MockRequester) EntityByID(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "EntityByID", arg0, arg1) } -// EntityByID indicates an expected call of EntityByID. +// EntityByID indicates an expected call of EntityByID func (mr *MockRequesterMockRecorder) EntityByID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EntityByID", reflect.TypeOf((*MockRequester)(nil).EntityByID), arg0, arg1) } -// Force mocks base method. +// Force mocks base method func (m *MockRequester) Force() { m.ctrl.T.Helper() m.ctrl.Call(m, "Force") } -// Force indicates an expected call of Force. +// Force indicates an expected call of Force func (mr *MockRequesterMockRecorder) Force() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Force", reflect.TypeOf((*MockRequester)(nil).Force)) } -// Query mocks base method. +// Query mocks base method func (m *MockRequester) Query(arg0 flow.Identifier, arg1 flow.IdentityFilter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Query", arg0, arg1) } -// Query indicates an expected call of Query. +// Query indicates an expected call of Query func (mr *MockRequesterMockRecorder) Query(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockRequester)(nil).Query), arg0, arg1) diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 32cdc456dc3..b3462e05ccb 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -82,7 +82,7 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { // root id before spork - rootID1 := unittest.BlockFixture().ID() + rootID1 := unittest.BlockFixture().ID().String() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -104,7 +104,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { StopNode(suite.T(), node2) // update the flow root id for node 2. node1 is still listening on the old protocol - rootID2 := unittest.BlockFixture().ID() + rootID2 := unittest.BlockFixture().ID().String() // start node2 with the same address and root key but different root block id node2, id2New := NodeFixture(suite.T(), suite.logger, node2key, rootID2, nil, false, id2.Address) @@ -123,7 +123,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { // root id before spork - rootIDBeforeSpork := unittest.BlockFixture().ID() + rootIDBeforeSpork := unittest.BlockFixture().ID().String() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -160,7 +160,7 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { testOneToKMessagingSucceeds(ctx, suite.T(), node1, sub2, topicBeforeSpork) // new root id after spork - rootIDAfterSpork := unittest.BlockFixture().ID() + rootIDAfterSpork := unittest.BlockFixture().ID().String() // topic after the spork topicAfterSpork := engine.TopicFromChannel(engine.TestNetwork, rootIDAfterSpork) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index 969044a53ee..2967a3e5cce 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -5,39 +5,37 @@ package mocks import ( - reflect "reflect" - gomock "github.com/golang/mock/gomock" - flow "github.com/onflow/flow-go/model/flow" storage "github.com/onflow/flow-go/storage" transaction "github.com/onflow/flow-go/storage/badger/transaction" + reflect "reflect" ) -// MockBlocks is a mock of Blocks interface. +// MockBlocks is a mock of Blocks interface type MockBlocks struct { ctrl *gomock.Controller recorder *MockBlocksMockRecorder } -// MockBlocksMockRecorder is the mock recorder for MockBlocks. +// MockBlocksMockRecorder is the mock recorder for MockBlocks type MockBlocksMockRecorder struct { mock *MockBlocks } -// NewMockBlocks creates a new mock instance. +// NewMockBlocks creates a new mock instance func NewMockBlocks(ctrl *gomock.Controller) *MockBlocks { mock := &MockBlocks{ctrl: ctrl} mock.recorder = &MockBlocksMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockBlocks) EXPECT() *MockBlocksMockRecorder { return m.recorder } -// ByCollectionID mocks base method. +// ByCollectionID mocks base method func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByCollectionID", arg0) @@ -46,13 +44,13 @@ func (m *MockBlocks) ByCollectionID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByCollectionID indicates an expected call of ByCollectionID. +// ByCollectionID indicates an expected call of ByCollectionID func (mr *MockBlocksMockRecorder) ByCollectionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByCollectionID", reflect.TypeOf((*MockBlocks)(nil).ByCollectionID), arg0) } -// ByHeight mocks base method. +// ByHeight mocks base method func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -61,13 +59,13 @@ func (m *MockBlocks) ByHeight(arg0 uint64) (*flow.Block, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight. +// ByHeight indicates an expected call of ByHeight func (mr *MockBlocksMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockBlocks)(nil).ByHeight), arg0) } -// ByID mocks base method. +// ByID mocks base method func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -76,13 +74,13 @@ func (m *MockBlocks) ByID(arg0 flow.Identifier) (*flow.Block, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID. +// ByID indicates an expected call of ByID func (mr *MockBlocksMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockBlocks)(nil).ByID), arg0) } -// GetLastFullBlockHeight mocks base method. +// GetLastFullBlockHeight mocks base method func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetLastFullBlockHeight") @@ -91,13 +89,13 @@ func (m *MockBlocks) GetLastFullBlockHeight() (uint64, error) { return ret0, ret1 } -// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight. +// GetLastFullBlockHeight indicates an expected call of GetLastFullBlockHeight func (mr *MockBlocksMockRecorder) GetLastFullBlockHeight() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).GetLastFullBlockHeight)) } -// IndexBlockForCollections mocks base method. +// IndexBlockForCollections mocks base method func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexBlockForCollections", arg0, arg1) @@ -105,13 +103,13 @@ func (m *MockBlocks) IndexBlockForCollections(arg0 flow.Identifier, arg1 []flow. return ret0 } -// IndexBlockForCollections indicates an expected call of IndexBlockForCollections. +// IndexBlockForCollections indicates an expected call of IndexBlockForCollections func (mr *MockBlocksMockRecorder) IndexBlockForCollections(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexBlockForCollections", reflect.TypeOf((*MockBlocks)(nil).IndexBlockForCollections), arg0, arg1) } -// Store mocks base method. +// Store mocks base method func (m *MockBlocks) Store(arg0 *flow.Block) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -119,13 +117,13 @@ func (m *MockBlocks) Store(arg0 *flow.Block) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockBlocksMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockBlocks)(nil).Store), arg0) } -// StoreTx mocks base method. +// StoreTx mocks base method func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreTx", arg0) @@ -133,13 +131,13 @@ func (m *MockBlocks) StoreTx(arg0 *flow.Block) func(*transaction.Tx) error { return ret0 } -// StoreTx indicates an expected call of StoreTx. +// StoreTx indicates an expected call of StoreTx func (mr *MockBlocksMockRecorder) StoreTx(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreTx", reflect.TypeOf((*MockBlocks)(nil).StoreTx), arg0) } -// UpdateLastFullBlockHeight mocks base method. +// UpdateLastFullBlockHeight mocks base method func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "UpdateLastFullBlockHeight", arg0) @@ -147,36 +145,36 @@ func (m *MockBlocks) UpdateLastFullBlockHeight(arg0 uint64) error { return ret0 } -// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight. +// UpdateLastFullBlockHeight indicates an expected call of UpdateLastFullBlockHeight func (mr *MockBlocksMockRecorder) UpdateLastFullBlockHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateLastFullBlockHeight", reflect.TypeOf((*MockBlocks)(nil).UpdateLastFullBlockHeight), arg0) } -// MockHeaders is a mock of Headers interface. +// MockHeaders is a mock of Headers interface type MockHeaders struct { ctrl *gomock.Controller recorder *MockHeadersMockRecorder } -// MockHeadersMockRecorder is the mock recorder for MockHeaders. +// MockHeadersMockRecorder is the mock recorder for MockHeaders type MockHeadersMockRecorder struct { mock *MockHeaders } -// NewMockHeaders creates a new mock instance. +// NewMockHeaders creates a new mock instance func NewMockHeaders(ctrl *gomock.Controller) *MockHeaders { mock := &MockHeaders{ctrl: ctrl} mock.recorder = &MockHeadersMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockHeaders) EXPECT() *MockHeadersMockRecorder { return m.recorder } -// BatchIndexByChunkID mocks base method. +// BatchIndexByChunkID mocks base method func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchIndexByChunkID", arg0, arg1, arg2) @@ -184,13 +182,13 @@ func (m *MockHeaders) BatchIndexByChunkID(arg0, arg1 flow.Identifier, arg2 stora return ret0 } -// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID. +// BatchIndexByChunkID indicates an expected call of BatchIndexByChunkID func (mr *MockHeadersMockRecorder) BatchIndexByChunkID(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchIndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).BatchIndexByChunkID), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -199,13 +197,13 @@ func (m *MockHeaders) ByBlockID(arg0 flow.Identifier) (*flow.Header, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockHeadersMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockHeaders)(nil).ByBlockID), arg0) } -// ByHeight mocks base method. +// ByHeight mocks base method func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByHeight", arg0) @@ -214,13 +212,13 @@ func (m *MockHeaders) ByHeight(arg0 uint64) (*flow.Header, error) { return ret0, ret1 } -// ByHeight indicates an expected call of ByHeight. +// ByHeight indicates an expected call of ByHeight func (mr *MockHeadersMockRecorder) ByHeight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByHeight", reflect.TypeOf((*MockHeaders)(nil).ByHeight), arg0) } -// ByParentID mocks base method. +// ByParentID mocks base method func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByParentID", arg0) @@ -229,13 +227,13 @@ func (m *MockHeaders) ByParentID(arg0 flow.Identifier) ([]*flow.Header, error) { return ret0, ret1 } -// ByParentID indicates an expected call of ByParentID. +// ByParentID indicates an expected call of ByParentID func (mr *MockHeadersMockRecorder) ByParentID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByParentID", reflect.TypeOf((*MockHeaders)(nil).ByParentID), arg0) } -// IDByChunkID mocks base method. +// IDByChunkID mocks base method func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IDByChunkID", arg0) @@ -244,13 +242,13 @@ func (m *MockHeaders) IDByChunkID(arg0 flow.Identifier) (flow.Identifier, error) return ret0, ret1 } -// IDByChunkID indicates an expected call of IDByChunkID. +// IDByChunkID indicates an expected call of IDByChunkID func (mr *MockHeadersMockRecorder) IDByChunkID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDByChunkID", reflect.TypeOf((*MockHeaders)(nil).IDByChunkID), arg0) } -// IndexByChunkID mocks base method. +// IndexByChunkID mocks base method func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IndexByChunkID", arg0, arg1) @@ -258,13 +256,13 @@ func (m *MockHeaders) IndexByChunkID(arg0, arg1 flow.Identifier) error { return ret0 } -// IndexByChunkID indicates an expected call of IndexByChunkID. +// IndexByChunkID indicates an expected call of IndexByChunkID func (mr *MockHeadersMockRecorder) IndexByChunkID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexByChunkID", reflect.TypeOf((*MockHeaders)(nil).IndexByChunkID), arg0, arg1) } -// Store mocks base method. +// Store mocks base method func (m *MockHeaders) Store(arg0 *flow.Header) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -272,36 +270,36 @@ func (m *MockHeaders) Store(arg0 *flow.Header) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockHeadersMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockHeaders)(nil).Store), arg0) } -// MockPayloads is a mock of Payloads interface. +// MockPayloads is a mock of Payloads interface type MockPayloads struct { ctrl *gomock.Controller recorder *MockPayloadsMockRecorder } -// MockPayloadsMockRecorder is the mock recorder for MockPayloads. +// MockPayloadsMockRecorder is the mock recorder for MockPayloads type MockPayloadsMockRecorder struct { mock *MockPayloads } -// NewMockPayloads creates a new mock instance. +// NewMockPayloads creates a new mock instance func NewMockPayloads(ctrl *gomock.Controller) *MockPayloads { mock := &MockPayloads{ctrl: ctrl} mock.recorder = &MockPayloadsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockPayloads) EXPECT() *MockPayloadsMockRecorder { return m.recorder } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -310,13 +308,13 @@ func (m *MockPayloads) ByBlockID(arg0 flow.Identifier) (*flow.Payload, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockPayloadsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockPayloads)(nil).ByBlockID), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -324,36 +322,36 @@ func (m *MockPayloads) Store(arg0 flow.Identifier, arg1 *flow.Payload) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockPayloadsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockPayloads)(nil).Store), arg0, arg1) } -// MockCollections is a mock of Collections interface. +// MockCollections is a mock of Collections interface type MockCollections struct { ctrl *gomock.Controller recorder *MockCollectionsMockRecorder } -// MockCollectionsMockRecorder is the mock recorder for MockCollections. +// MockCollectionsMockRecorder is the mock recorder for MockCollections type MockCollectionsMockRecorder struct { mock *MockCollections } -// NewMockCollections creates a new mock instance. +// NewMockCollections creates a new mock instance func NewMockCollections(ctrl *gomock.Controller) *MockCollections { mock := &MockCollections{ctrl: ctrl} mock.recorder = &MockCollectionsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockCollections) EXPECT() *MockCollectionsMockRecorder { return m.recorder } -// ByID mocks base method. +// ByID mocks base method func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByID", arg0) @@ -362,13 +360,13 @@ func (m *MockCollections) ByID(arg0 flow.Identifier) (*flow.Collection, error) { return ret0, ret1 } -// ByID indicates an expected call of ByID. +// ByID indicates an expected call of ByID func (mr *MockCollectionsMockRecorder) ByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByID", reflect.TypeOf((*MockCollections)(nil).ByID), arg0) } -// LightByID mocks base method. +// LightByID mocks base method func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByID", arg0) @@ -377,13 +375,13 @@ func (m *MockCollections) LightByID(arg0 flow.Identifier) (*flow.LightCollection return ret0, ret1 } -// LightByID indicates an expected call of LightByID. +// LightByID indicates an expected call of LightByID func (mr *MockCollectionsMockRecorder) LightByID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByID", reflect.TypeOf((*MockCollections)(nil).LightByID), arg0) } -// LightByTransactionID mocks base method. +// LightByTransactionID mocks base method func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.LightCollection, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LightByTransactionID", arg0) @@ -392,13 +390,13 @@ func (m *MockCollections) LightByTransactionID(arg0 flow.Identifier) (*flow.Ligh return ret0, ret1 } -// LightByTransactionID indicates an expected call of LightByTransactionID. +// LightByTransactionID indicates an expected call of LightByTransactionID func (mr *MockCollectionsMockRecorder) LightByTransactionID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LightByTransactionID", reflect.TypeOf((*MockCollections)(nil).LightByTransactionID), arg0) } -// Remove mocks base method. +// Remove mocks base method func (m *MockCollections) Remove(arg0 flow.Identifier) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Remove", arg0) @@ -406,13 +404,13 @@ func (m *MockCollections) Remove(arg0 flow.Identifier) error { return ret0 } -// Remove indicates an expected call of Remove. +// Remove indicates an expected call of Remove func (mr *MockCollectionsMockRecorder) Remove(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockCollections)(nil).Remove), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockCollections) Store(arg0 *flow.Collection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0) @@ -420,13 +418,13 @@ func (m *MockCollections) Store(arg0 *flow.Collection) error { return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockCollectionsMockRecorder) Store(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCollections)(nil).Store), arg0) } -// StoreLight mocks base method. +// StoreLight mocks base method func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLight", arg0) @@ -434,13 +432,13 @@ func (m *MockCollections) StoreLight(arg0 *flow.LightCollection) error { return ret0 } -// StoreLight indicates an expected call of StoreLight. +// StoreLight indicates an expected call of StoreLight func (mr *MockCollectionsMockRecorder) StoreLight(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLight", reflect.TypeOf((*MockCollections)(nil).StoreLight), arg0) } -// StoreLightAndIndexByTransaction mocks base method. +// StoreLightAndIndexByTransaction mocks base method func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollection) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "StoreLightAndIndexByTransaction", arg0) @@ -448,36 +446,36 @@ func (m *MockCollections) StoreLightAndIndexByTransaction(arg0 *flow.LightCollec return ret0 } -// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction. +// StoreLightAndIndexByTransaction indicates an expected call of StoreLightAndIndexByTransaction func (mr *MockCollectionsMockRecorder) StoreLightAndIndexByTransaction(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreLightAndIndexByTransaction", reflect.TypeOf((*MockCollections)(nil).StoreLightAndIndexByTransaction), arg0) } -// MockCommits is a mock of Commits interface. +// MockCommits is a mock of Commits interface type MockCommits struct { ctrl *gomock.Controller recorder *MockCommitsMockRecorder } -// MockCommitsMockRecorder is the mock recorder for MockCommits. +// MockCommitsMockRecorder is the mock recorder for MockCommits type MockCommitsMockRecorder struct { mock *MockCommits } -// NewMockCommits creates a new mock instance. +// NewMockCommits creates a new mock instance func NewMockCommits(ctrl *gomock.Controller) *MockCommits { mock := &MockCommits{ctrl: ctrl} mock.recorder = &MockCommitsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -485,13 +483,13 @@ func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockCommitsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockCommits)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -500,13 +498,13 @@ func (m *MockCommits) ByBlockID(arg0 flow.Identifier) (flow.StateCommitment, err return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockCommitsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockCommits)(nil).ByBlockID), arg0) } -// Store mocks base method. +// Store mocks base method func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Store", arg0, arg1) @@ -514,36 +512,36 @@ func (m *MockCommits) Store(arg0 flow.Identifier, arg1 flow.StateCommitment) err return ret0 } -// Store indicates an expected call of Store. +// Store indicates an expected call of Store func (mr *MockCommitsMockRecorder) Store(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Store", reflect.TypeOf((*MockCommits)(nil).Store), arg0, arg1) } -// MockEvents is a mock of Events interface. +// MockEvents is a mock of Events interface type MockEvents struct { ctrl *gomock.Controller recorder *MockEventsMockRecorder } -// MockEventsMockRecorder is the mock recorder for MockEvents. +// MockEventsMockRecorder is the mock recorder for MockEvents type MockEventsMockRecorder struct { mock *MockEvents } -// NewMockEvents creates a new mock instance. +// NewMockEvents creates a new mock instance func NewMockEvents(ctrl *gomock.Controller) *MockEvents { mock := &MockEvents{ctrl: ctrl} mock.recorder = &MockEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockEvents) EXPECT() *MockEventsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -551,13 +549,13 @@ func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, ar return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -566,13 +564,13 @@ func (m *MockEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockEvents)(nil).ByBlockID), arg0) } -// ByBlockIDEventType mocks base method. +// ByBlockIDEventType mocks base method func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventType) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDEventType", arg0, arg1) @@ -581,13 +579,13 @@ func (m *MockEvents) ByBlockIDEventType(arg0 flow.Identifier, arg1 flow.EventTyp return ret0, ret1 } -// ByBlockIDEventType indicates an expected call of ByBlockIDEventType. +// ByBlockIDEventType indicates an expected call of ByBlockIDEventType func (mr *MockEventsMockRecorder) ByBlockIDEventType(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDEventType", reflect.TypeOf((*MockEvents)(nil).ByBlockIDEventType), arg0, arg1) } -// ByBlockIDTransactionID mocks base method. +// ByBlockIDTransactionID mocks base method func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -596,36 +594,36 @@ func (m *MockEvents) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) ([]flow. return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID func (mr *MockEventsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockEvents)(nil).ByBlockIDTransactionID), arg0, arg1) } -// MockServiceEvents is a mock of ServiceEvents interface. +// MockServiceEvents is a mock of ServiceEvents interface type MockServiceEvents struct { ctrl *gomock.Controller recorder *MockServiceEventsMockRecorder } -// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents. +// MockServiceEventsMockRecorder is the mock recorder for MockServiceEvents type MockServiceEventsMockRecorder struct { mock *MockServiceEvents } -// NewMockServiceEvents creates a new mock instance. +// NewMockServiceEvents creates a new mock instance func NewMockServiceEvents(ctrl *gomock.Controller) *MockServiceEvents { mock := &MockServiceEvents{ctrl: ctrl} mock.recorder = &MockServiceEventsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -633,13 +631,13 @@ func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockServiceEventsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockServiceEvents)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockID mocks base method. +// ByBlockID mocks base method func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockID", arg0) @@ -648,36 +646,36 @@ func (m *MockServiceEvents) ByBlockID(arg0 flow.Identifier) ([]flow.Event, error return ret0, ret1 } -// ByBlockID indicates an expected call of ByBlockID. +// ByBlockID indicates an expected call of ByBlockID func (mr *MockServiceEventsMockRecorder) ByBlockID(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockID", reflect.TypeOf((*MockServiceEvents)(nil).ByBlockID), arg0) } -// MockTransactionResults is a mock of TransactionResults interface. +// MockTransactionResults is a mock of TransactionResults interface type MockTransactionResults struct { ctrl *gomock.Controller recorder *MockTransactionResultsMockRecorder } -// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults. +// MockTransactionResultsMockRecorder is the mock recorder for MockTransactionResults type MockTransactionResultsMockRecorder struct { mock *MockTransactionResults } -// NewMockTransactionResults creates a new mock instance. +// NewMockTransactionResults creates a new mock instance func NewMockTransactionResults(ctrl *gomock.Controller) *MockTransactionResults { mock := &MockTransactionResults{ctrl: ctrl} mock.recorder = &MockTransactionResultsMockRecorder{mock} return mock } -// EXPECT returns an object that allows the caller to indicate expected use. +// EXPECT returns an object that allows the caller to indicate expected use func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } -// BatchStore mocks base method. +// BatchStore mocks base method func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) @@ -685,13 +683,13 @@ func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.Tr return ret0 } -// BatchStore indicates an expected call of BatchStore. +// BatchStore indicates an expected call of BatchStore func (mr *MockTransactionResultsMockRecorder) BatchStore(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchStore", reflect.TypeOf((*MockTransactionResults)(nil).BatchStore), arg0, arg1, arg2) } -// ByBlockIDTransactionID mocks base method. +// ByBlockIDTransactionID mocks base method func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifier) (*flow.TransactionResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ByBlockIDTransactionID", arg0, arg1) @@ -700,7 +698,7 @@ func (m *MockTransactionResults) ByBlockIDTransactionID(arg0, arg1 flow.Identifi return ret0, ret1 } -// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID. +// ByBlockIDTransactionID indicates an expected call of ByBlockIDTransactionID func (mr *MockTransactionResultsMockRecorder) ByBlockIDTransactionID(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ByBlockIDTransactionID", reflect.TypeOf((*MockTransactionResults)(nil).ByBlockIDTransactionID), arg0, arg1) From 414d36c9a6bb5e407bb49df9f675f9e8dbf4448d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 10 Sep 2021 19:06:16 -0400 Subject: [PATCH 268/291] Tests --- .../staked_access_node_builder.go | 14 ++- .../unstaked_access_node_builder.go | 4 +- cmd/scaffold.go | 6 +- engine/channels.go | 6 +- network/mocknetwork/overlay.go | 34 ++++---- network/p2p/libp2pNode.go | 28 +++--- network/p2p/libp2pNode_test.go | 4 +- network/p2p/libp2pUtils.go | 8 +- network/p2p/middleware.go | 4 +- network/p2p/sporking_test.go | 8 +- network/p2p/subscription_filter.go | 57 +++++------- network/p2p/subscription_filter_test.go | 87 +++++-------------- network/p2p/topic_validator_test.go | 18 ++-- network/test/testUtil.go | 2 +- 14 files changed, 118 insertions(+), 162 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 8ee01190b9e..5840e3fe86b 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -4,7 +4,9 @@ import ( "context" "fmt" + "github.com/libp2p/go-libp2p-core/host" dht "github.com/libp2p/go-libp2p-kad-dht" + pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/crypto" @@ -210,13 +212,19 @@ func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, } return func() (*p2p.Node, error) { + psOpts := p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize) + psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { + return pubsub.WithSubscriptionFilter(p2p.NewSubscriptionFilter( + h.ID(), builder.RootBlock.ID(), builder.RootChainID, builder.IdentityProvider, + )), nil + }) libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). - SetRootBlockID(builder.RootBlock.ID().String()). + SetRootBlockID(builder.RootBlock.ID()). // no connection gater SetConnectionManager(connManager). // act as a DHT server SetDHTOptions(dhtOptions...). - SetPubsubOptions(p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize)...). + SetPubsubOptions(psOpts...). SetLogger(builder.Logger). SetResolver(resolver). Build(ctx) @@ -243,7 +251,7 @@ func (builder *StakedAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, factoryFunc, nodeID, networkMetrics, - builder.RootBlock.ID().String(), + builder.RootBlock.ID(), p2p.DefaultUnicastTimeout, false, // no connection gating to allow unstaked nodes to connect builder.IDTranslator, diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index f5468901bd1..5a13d5992b6 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -157,7 +157,7 @@ func (builder *UnstakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, return func() (*p2p.Node, error) { libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, builder.BaseConfig.BindAddr, networkKey). - SetRootBlockID(builder.RootBlock.ID().String()). + SetRootBlockID(builder.RootBlock.ID()). SetConnectionManager(connManager). // unlike the staked side of the network where currently all the node addresses are known upfront, // for the unstaked side of the network, the nodes need to discover each other using DHT Discovery. @@ -275,7 +275,7 @@ func (anb *UnstakedAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, factoryFunc, nodeID, networkMetrics, - anb.RootBlock.ID().String(), + anb.RootBlock.ID(), p2p.DefaultUnicastTimeout, false, // no connection gating for the unstaked nodes anb.IDTranslator, diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 889d6fa396e..c6ce1aa5c84 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -165,7 +165,9 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { fnb.Me.NodeID(), myAddr, fnb.NetworkKey, - fnb.RootBlock.ID().String(), + fnb.RootBlock.ID(), + fnb.RootChainID, + fnb.IdentityProvider, p2p.DefaultMaxPubSubMsgSize, fnb.Metrics.Network, pingProvider, @@ -191,7 +193,7 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit(ctx context.Context) { libP2PNodeFactory, fnb.Me.NodeID(), fnb.Metrics.Network, - fnb.RootBlock.ID().String(), + fnb.RootBlock.ID(), fnb.BaseConfig.UnicastMessageTimeout, true, fnb.IDTranslator, diff --git a/engine/channels.go b/engine/channels.go index 18c0f8f1053..099a23d13b7 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -217,13 +217,13 @@ func ClusterChannel(channel network.Channel) (network.Channel, bool) { // TopicFromChannel returns the unique LibP2P topic form the channel. // The channel is made up of name string suffixed with root block id. // The root block id is used to prevent cross talks between nodes on different sporks. -func TopicFromChannel(channel network.Channel, rootBlockID string) network.Topic { +func TopicFromChannel(channel network.Channel, rootBlockID flow.Identifier) network.Topic { // skip root block suffix, if this is a cluster specific channel. A cluster specific channel is inherently // unique for each epoch - if strings.HasPrefix(channel.String(), syncClusterPrefix.String()) || strings.HasPrefix(string(channel), consensusClusterPrefix.String()) { + if _, isClusterChannel := ClusterChannel(channel); isClusterChannel { return network.Topic(channel) } - return network.Topic(fmt.Sprintf("%s/%s", string(channel), rootBlockID)) + return network.Topic(fmt.Sprintf("%s/%s", string(channel), rootBlockID.String())) } // ChannelConsensusCluster returns a dynamic cluster consensus channel based on diff --git a/network/mocknetwork/overlay.go b/network/mocknetwork/overlay.go index dd6baaede58..5bc08593a9b 100644 --- a/network/mocknetwork/overlay.go +++ b/network/mocknetwork/overlay.go @@ -16,7 +16,23 @@ type Overlay struct { mock.Mock } -// GetIdentity provides a mock function with given fields: _a0 +// Identities provides a mock function with given fields: +func (_m *Overlay) Identities() flow.IdentityList { + ret := _m.Called() + + var r0 flow.IdentityList + if rf, ok := ret.Get(0).(func() flow.IdentityList); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(flow.IdentityList) + } + } + + return r0 +} + +// Identity provides a mock function with given fields: _a0 func (_m *Overlay) Identity(_a0 peer.ID) (*flow.Identity, bool) { ret := _m.Called(_a0) @@ -39,22 +55,6 @@ func (_m *Overlay) Identity(_a0 peer.ID) (*flow.Identity, bool) { return r0, r1 } -// Identities provides a mock function with given fields: -func (_m *Overlay) Identities() flow.IdentityList { - ret := _m.Called() - - var r0 flow.IdentityList - if rf, ok := ret.Get(0).(func() flow.IdentityList); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(flow.IdentityList) - } - } - - return r0 -} - // Receive provides a mock function with given fields: nodeID, msg func (_m *Overlay) Receive(nodeID flow.Identifier, msg *message.Message) error { ret := _m.Called(nodeID, msg) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 09b25205cd4..aa2cb76d1c8 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -31,6 +31,7 @@ import ( fcrypto "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/id" flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p/dns" @@ -64,7 +65,9 @@ func DefaultLibP2PNodeFactory(ctx context.Context, me flow.Identifier, address string, flowKey fcrypto.PrivateKey, - rootBlockID string, + rootBlockID flow.Identifier, + chainID flow.ChainID, + idProvider id.IdentityProvider, maxPubSubMsgSize int, metrics module.NetworkMetrics, pingInfoProvider PingInfoProvider, @@ -79,12 +82,19 @@ func DefaultLibP2PNodeFactory(ctx context.Context, return nil, fmt.Errorf("could not create dns resolver: %w", err) } + psOpts := DefaultPubsubOptions(maxPubSubMsgSize) + psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { + return pubsub.WithSubscriptionFilter(NewSubscriptionFilter( + h.ID(), rootBlockID, chainID, idProvider, + )), nil + }) + return func() (*Node, error) { return NewDefaultLibP2PNodeBuilder(me, address, flowKey). SetRootBlockID(rootBlockID). SetConnectionGater(connGater). SetConnectionManager(connManager). - SetPubsubOptions(DefaultPubsubOptions(maxPubSubMsgSize)...). + SetPubsubOptions(psOpts...). SetPingInfoProvider(pingInfoProvider). SetLogger(log). SetResolver(resolver). @@ -93,7 +103,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, } type NodeBuilder interface { - SetRootBlockID(string) NodeBuilder + SetRootBlockID(flow.Identifier) NodeBuilder SetConnectionManager(TagLessConnManager) NodeBuilder SetConnectionGater(*ConnGater) NodeBuilder SetPubsubOptions(...PubsubOption) NodeBuilder @@ -106,7 +116,7 @@ type NodeBuilder interface { type DefaultLibP2PNodeBuilder struct { id flow.Identifier - rootBlockID string + rootBlockID flow.Identifier logger zerolog.Logger connGater *ConnGater connMngr TagLessConnManager @@ -135,7 +145,7 @@ func (builder *DefaultLibP2PNodeBuilder) SetDHTOptions(opts ...dht.Option) NodeB return builder } -func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId string) NodeBuilder { +func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId flow.Identifier) NodeBuilder { builder.rootBlockID = rootBlockId return builder } @@ -186,7 +196,7 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro return nil, errors.New("unable to create libp2p pubsub: factory function not provided") } - if builder.rootBlockID == "" { + if builder.rootBlockID == flow.ZeroID { return nil, errors.New("root block ID must be provided") } node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) @@ -203,11 +213,6 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro node.connMgr = builder.connMngr } - if builder.rootBlockID == "" { - return nil, errors.New("root block ID must be provided") - } - node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) - if builder.pingInfoProvider != nil { opts = append(opts, libp2p.Ping(true)) } @@ -718,7 +723,6 @@ func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { pubSubOptionFunc(pubsub.WithStrictSignatureVerification(true)), // set max message size limit for 1-k PubSub messaging pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), - // no discovery } } diff --git a/network/p2p/libp2pNode_test.go b/network/p2p/libp2pNode_test.go index 65f8d4e526b..eebd5da07c4 100644 --- a/network/p2p/libp2pNode_test.go +++ b/network/p2p/libp2pNode_test.go @@ -41,7 +41,7 @@ const tickForAssertEventually = 100 * time.Millisecond // "0.0.0.0: const defaultAddress = "0.0.0.0:0" -var rootBlockID = unittest.IdentifierFixture().String() +var rootBlockID = unittest.IdentifierFixture() type LibP2PNodeTestSuite struct { suite.Suite @@ -685,7 +685,7 @@ func (suite *LibP2PNodeTestSuite) NodesFixture(count int, handler func(t *testin // NodeFixture creates a single LibP2PNodes with the given key, root block id, and callback function for stream handling. // It returns the nodes and their identities. -func NodeFixture(t *testing.T, log zerolog.Logger, key fcrypto.PrivateKey, rootID string, handler func(t *testing.T) network.StreamHandler, allowList bool, address string) (*Node, flow.Identity) { +func NodeFixture(t *testing.T, log zerolog.Logger, key fcrypto.PrivateKey, rootID flow.Identifier, handler func(t *testing.T) network.StreamHandler, allowList bool, address string) (*Node, flow.Identity) { identity := unittest.IdentityFixture(unittest.WithNetworkingKey(key.PublicKey()), unittest.WithAddress(address)) diff --git a/network/p2p/libp2pUtils.go b/network/p2p/libp2pUtils.go index 734dcbd2ef8..9a6d0ebc29b 100644 --- a/network/p2p/libp2pUtils.go +++ b/network/p2p/libp2pUtils.go @@ -163,12 +163,12 @@ func IPPortFromMultiAddress(addrs ...multiaddr.Multiaddr) (string, string, error return "", "", fmt.Errorf("ip address or hostname not found") } -func generateFlowProtocolID(rootBlockID string) protocol.ID { - return protocol.ID(FlowLibP2POneToOneProtocolIDPrefix + rootBlockID) +func generateFlowProtocolID(rootBlockID flow.Identifier) protocol.ID { + return protocol.ID(FlowLibP2POneToOneProtocolIDPrefix + rootBlockID.String()) } -func generatePingProtcolID(rootBlockID string) protocol.ID { - return protocol.ID(FlowLibP2PPingProtocolPrefix + rootBlockID) +func generatePingProtcolID(rootBlockID flow.Identifier) protocol.ID { + return protocol.ID(FlowLibP2PPingProtocolPrefix + rootBlockID.String()) } // PeerAddressInfo generates the libp2p peer.AddrInfo for the given Flow.Identity. diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index 0aaa9877928..c18f01ca2ea 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -74,7 +74,7 @@ type Middleware struct { libP2PNodeFactory LibP2PFactoryFunc me flow.Identifier metrics module.NetworkMetrics - rootBlockID string + rootBlockID flow.Identifier validators []network.MessageValidator peerManagerFactory PeerManagerFactoryFunc peerManager *PeerManager @@ -120,7 +120,7 @@ func NewMiddleware( libP2PNodeFactory LibP2PFactoryFunc, flowID flow.Identifier, metrics module.NetworkMetrics, - rootBlockID string, + rootBlockID flow.Identifier, unicastMessageTimeout time.Duration, connectionGating bool, idTranslator IDTranslator, diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index b3462e05ccb..32cdc456dc3 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -82,7 +82,7 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { // root id before spork - rootID1 := unittest.BlockFixture().ID().String() + rootID1 := unittest.BlockFixture().ID() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -104,7 +104,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { StopNode(suite.T(), node2) // update the flow root id for node 2. node1 is still listening on the old protocol - rootID2 := unittest.BlockFixture().ID().String() + rootID2 := unittest.BlockFixture().ID() // start node2 with the same address and root key but different root block id node2, id2New := NodeFixture(suite.T(), suite.logger, node2key, rootID2, nil, false, id2.Address) @@ -123,7 +123,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { // root id before spork - rootIDBeforeSpork := unittest.BlockFixture().ID().String() + rootIDBeforeSpork := unittest.BlockFixture().ID() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -160,7 +160,7 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { testOneToKMessagingSucceeds(ctx, suite.T(), node1, sub2, topicBeforeSpork) // new root id after spork - rootIDAfterSpork := unittest.BlockFixture().ID().String() + rootIDAfterSpork := unittest.BlockFixture().ID() // topic after the spork topicAfterSpork := engine.TopicFromChannel(engine.TestNetwork, rootIDAfterSpork) diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index 1ed005653cc..74477584afd 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -1,68 +1,59 @@ package p2p import ( - "fmt" + "strings" "github.com/libp2p/go-libp2p-core/peer" pb "github.com/libp2p/go-libp2p-pubsub/pb" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/network" ) type Filter struct { - idProvider id.IdentityProvider - idTranslator IDTranslator - myPeerID peer.ID - rootBlockID flow.Identifier + idProvider id.IdentityProvider + myPeerID peer.ID + rootBlockID flow.Identifier + chainID flow.ChainID } -func NewSubscriptionFilter(pid peer.ID, rootBlockID flow.Identifier, idProvider id.IdentityProvider, idTranslator IDTranslator) *Filter { +func NewSubscriptionFilter(pid peer.ID, rootBlockID flow.Identifier, chainID flow.ChainID, idProvider id.IdentityProvider) *Filter { return &Filter{ idProvider, - idTranslator, pid, rootBlockID, + chainID, } } -func (f *Filter) getIdentity(pid peer.ID) *flow.Identity { - fid, err := f.idTranslator.GetFlowID(pid) - if err != nil { - // translation should always succeed for staked nodes - return nil - } - - identities := f.idProvider.Identities(filter.HasNodeID(fid)) - if len(identities) == 0 { - return nil - } - - return identities[0] -} - func (f *Filter) allowedTopics(pid peer.ID) map[network.Topic]struct{} { - id := f.getIdentity(pid) - + id, found := f.idProvider.ByPeerID(pid) var channels network.ChannelList - if id == nil { - fmt.Println("Unstaked channels allowed by " + f.myPeerID.String() + " for " + pid.String()) + if !found { channels = engine.UnstakedChannels() + // TODO: eventually we should have block proposals relayed on a separate + // channel on the public network. For now, we need to make sure that + // unstaked nodes can subscribe to the block proposal channel. + channels = append(channels, engine.ReceiveBlocks) } else { - fmt.Println("Staked channels allowed by " + f.myPeerID.String() + " for " + pid.String()) channels = engine.ChannelsByRole(id.Role) } topics := make(map[network.Topic]struct{}) for _, ch := range channels { - // TODO: we will probably have problems here with cluster channels - // We probably need special checking for this - // Add a unit test for cluster channels + consensusCluster := engine.ChannelConsensusCluster(f.chainID) + syncCluster := engine.ChannelSyncCluster(f.chainID) + + if strings.HasPrefix(consensusCluster.String(), ch.String()) { + ch = consensusCluster + } else if strings.HasPrefix(syncCluster.String(), ch.String()) { + ch = syncCluster + } + topics[engine.TopicFromChannel(ch, f.rootBlockID)] = struct{}{} } @@ -79,12 +70,8 @@ func (f *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpt var filtered []*pb.RPC_SubOpts for _, opt := range opts { - fmt.Println("Request " + *opt.Topicid + "to") if _, allowed := allowedTopics[network.Topic(opt.GetTopicid())]; allowed { - filtered = append(filtered, opt) - } else { - fmt.Println("Blocked") } } diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index c054f3b9cd0..130dd2238d5 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -2,25 +2,26 @@ package p2p import ( "context" - "fmt" "sync" "testing" "time" "github.com/libp2p/go-libp2p-core/host" pubsub "github.com/libp2p/go-libp2p-pubsub" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/id" - "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/utils/unittest" "github.com/stretchr/testify/require" ) func TestFilterSubscribe(t *testing.T) { - identity1, privateKey1 := createID(t) - identity2, privateKey2 := createID(t) + // skip for now due to bug in libp2p gossipsub implementation: + // https://github.com/libp2p/go-libp2p-pubsub/issues/449 + t.Skip() + + identity1, privateKey1 := createID(t, unittest.WithRole(flow.RoleAccess)) + identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) ids := flow.IdentityList{identity1, identity2} node1 := createNode(t, identity1.NodeID, privateKey1, createSubscriptionFilterPubsubOption(t, ids)) @@ -30,10 +31,6 @@ func TestFilterSubscribe(t *testing.T) { require.NoError(t, err) unstakedNode := createNode(t, flow.ZeroID, unstakedKey) - fmt.Println(node1.host.ID()) - fmt.Println(node2.host.ID()) - fmt.Println(unstakedNode.host.ID()) - require.NoError(t, node1.AddPeer(context.TODO(), *host.InfoFromHost(node2.Host()))) require.NoError(t, node1.AddPeer(context.TODO(), *host.InfoFromHost(unstakedNode.Host()))) @@ -41,13 +38,12 @@ func TestFilterSubscribe(t *testing.T) { sub1, err := node1.Subscribe(context.TODO(), badTopic) require.NoError(t, err) - time.Sleep(300 * time.Millisecond) + sub2, err := node2.Subscribe(context.TODO(), badTopic) require.NoError(t, err) - time.Sleep(300 * time.Millisecond) + unstakedSub, err := unstakedNode.Subscribe(context.TODO(), badTopic) require.NoError(t, err) - time.Sleep(300 * time.Millisecond) require.Eventually(t, func() bool { return len(node1.pubSub.ListPeers(badTopic.String())) > 0 && @@ -72,8 +68,6 @@ func TestFilterSubscribe(t *testing.T) { from.Publish(context.TODO(), badTopic, data) - fmt.Println(from.pubSub.ListPeers(badTopic.String())) - ctx, cancel := context.WithTimeout(context.Background(), time.Second) msg, err := sub.Next(ctx) cancel() @@ -82,7 +76,6 @@ func TestFilterSubscribe(t *testing.T) { ctx, cancel = context.WithTimeout(context.Background(), time.Second) msg, err = unstakedSub.Next(ctx) - fmt.Println(msg) cancel() require.ErrorIs(t, err, context.DeadlineExceeded) @@ -95,71 +88,31 @@ func TestFilterSubscribe(t *testing.T) { // publish a message from node 2 and check that only node1 receives testPublish(&wg, node2, sub1) - fmt.Println(sub2) - wg.Wait() } func TestCanSubscribe(t *testing.T) { - identity, privateKey := createID(t) + identity, privateKey := createID(t, unittest.WithRole(flow.RoleCollection)) - node := createNode(t, identity.NodeID, privateKey, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity})) + collectionNode := createNode(t, identity.NodeID, privateKey, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity})) + defer collectionNode.Stop() - badTopic := getDisallowedTopic(t, identity) - _, err := node.pubSub.Join(badTopic.String()) + goodTopic := engine.TopicFromChannel(engine.ProvideCollections, rootBlockID) + _, err := collectionNode.pubSub.Join(goodTopic.String()) + require.NoError(t, err) + badTopic := engine.TopicFromChannel(engine.ProvideReceiptsByBlockID, rootBlockID) + _, err = collectionNode.pubSub.Join(badTopic.String()) require.Error(t, err) -} - -func getDisallowedTopic(t *testing.T, id *flow.Identity) network.Topic { - allowedChannels := engine.UnstakedChannels() - if id != nil { - allowedChannels = engine.ChannelsByRole(id.Role) - } - - for _, ch := range engine.Channels() { - if !allowedChannels.Contains(ch) { - return engine.TopicFromChannel(ch, rootBlockID) - } - } - require.FailNow(t, "could not find disallowed topic for role %s", id.Role) - - return "" + clusterTopic := engine.TopicFromChannel(engine.ChannelSyncCluster(unittest.ClusterBlockFixture().Header.ChainID), rootBlockID) + _, err = collectionNode.pubSub.Join(clusterTopic.String()) + require.NoError(t, err) } func createSubscriptionFilterPubsubOption(t *testing.T, ids flow.IdentityList) PubsubOption { - idTranslator, err := NewFixedTableIdentityTranslator(ids) - require.NoError(t, err) - idProvider := id.NewFixedIdentityProvider(ids) - return func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), rootBlockID, idProvider, idTranslator)), nil + return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), rootBlockID, unittest.BlockFixture().Header.ChainID, idProvider)), nil } } - -func createID(t *testing.T) (*flow.Identity, crypto.PrivateKey) { - networkKey, err := unittest.NetworkingKey() - require.NoError(t, err) - id := unittest.IdentityFixture( - unittest.WithRole(flow.RoleAccess), - unittest.WithNetworkingKey(networkKey.PublicKey()), - ) - return id, networkKey -} - -func createNode( - t *testing.T, - nodeID flow.Identifier, - networkKey crypto.PrivateKey, - psOpts ...PubsubOption, -) *Node { - libp2pNode, err := NewDefaultLibP2PNodeBuilder(nodeID, "0.0.0.0:0", networkKey). - SetRootBlockID(rootBlockID). - SetPubsubOptions(psOpts...). - Build(context.TODO()) - require.NoError(t, err) - - return libp2pNode -} diff --git a/network/p2p/topic_validator_test.go b/network/p2p/topic_validator_test.go index 974125b46d1..cd7d5ff9715 100644 --- a/network/p2p/topic_validator_test.go +++ b/network/p2p/topic_validator_test.go @@ -20,10 +20,10 @@ import ( func TestTopicValidator(t *testing.T) { // create two staked nodes - node1 and node2 - identity1, privateKey1 := createID(t) + identity1, privateKey1 := createID(t, unittest.WithRole(flow.RoleAccess)) node1 := createNode(t, identity1.NodeID, privateKey1) - identity2, privateKey2 := createID(t) + identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) node2 := createNode(t, identity2.NodeID, privateKey2) badTopic := engine.TopicFromChannel(engine.SyncCommittee, rootBlockID) @@ -127,13 +127,11 @@ func TestTopicValidator(t *testing.T) { wg.Wait() } -func createID(t *testing.T) (*flow.Identity, crypto.PrivateKey) { +func createID(t *testing.T, opts ...func(*flow.Identity)) (*flow.Identity, crypto.PrivateKey) { networkKey, err := unittest.NetworkingKey() require.NoError(t, err) - id := unittest.IdentityFixture( - unittest.WithRole(flow.RoleAccess), - unittest.WithNetworkingKey(networkKey.PublicKey()), - ) + opts = append(opts, unittest.WithNetworkingKey(networkKey.PublicKey())) + id := unittest.IdentityFixture(opts...) return id, networkKey } @@ -141,10 +139,14 @@ func createNode( t *testing.T, nodeID flow.Identifier, networkKey crypto.PrivateKey, + psOpts ...PubsubOption, ) *Node { + if len(psOpts) == 0 { + psOpts = DefaultPubsubOptions(DefaultMaxPubSubMsgSize) + } libp2pNode, err := NewDefaultLibP2PNodeBuilder(nodeID, "0.0.0.0:0", networkKey). SetRootBlockID(rootBlockID). - SetPubsubOptions(DefaultPubsubOptions(DefaultMaxPubSubMsgSize)...). + SetPubsubOptions(psOpts...). Build(context.TODO()) require.NoError(t, err) diff --git a/network/test/testUtil.go b/network/test/testUtil.go index 296e8bde54f..aa3c1132b38 100644 --- a/network/test/testUtil.go +++ b/network/test/testUtil.go @@ -35,7 +35,7 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) -var rootBlockID = unittest.IdentifierFixture().String() +var rootBlockID = unittest.IdentifierFixture() const DryRun = true From 647d753bc3654028755e2e130ac61bc9f85c6fc8 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 10 Sep 2021 19:17:27 -0400 Subject: [PATCH 269/291] Update subscription_filter_test.go --- network/p2p/subscription_filter_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index 130dd2238d5..0d7c991d2d2 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -8,11 +8,12 @@ import ( "github.com/libp2p/go-libp2p-core/host" pubsub "github.com/libp2p/go-libp2p-pubsub" + "github.com/stretchr/testify/require" + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/utils/unittest" - "github.com/stretchr/testify/require" ) func TestFilterSubscribe(t *testing.T) { From 9d542c45f640fa2015cc811267a00240d568d5cb Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 16 Sep 2021 22:24:53 -0700 Subject: [PATCH 270/291] address comments --- .../staked_access_node_builder.go | 2 +- engine/channels.go | 14 +++++------ network/p2p/libp2pNode.go | 2 +- network/p2p/middleware.go | 2 +- network/p2p/subscription_filter.go | 20 ++++++++-------- network/p2p/subscription_filter_test.go | 23 ++++++++++++++----- 6 files changed, 37 insertions(+), 26 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 5840e3fe86b..65a222708d7 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -214,7 +214,7 @@ func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, return func() (*p2p.Node, error) { psOpts := p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize) psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(p2p.NewSubscriptionFilter( + return pubsub.WithSubscriptionFilter(p2p.NewRoleBasedFilter( h.ID(), builder.RootBlock.ID(), builder.RootChainID, builder.IdentityProvider, )), nil }) diff --git a/engine/channels.go b/engine/channels.go index 099a23d13b7..bfb09440123 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -34,7 +34,7 @@ func RolesByChannel(channel network.Channel) (flow.RoleList, bool) { // to channelRoleMap as a constant channel type manually. func Exists(channel network.Channel) bool { _, exists := RolesByChannel(channel) - return exists || UnstakedChannels().Contains(channel) + return exists || PublicChannels().Contains(channel) } // ChannelsByRole returns a list of all channels the role subscribes to. @@ -90,10 +90,10 @@ func Channels() network.ChannelList { return channels } -// UnstakedChannels returns all channels that unstaked nodes can send messages on. -func UnstakedChannels() network.ChannelList { +// PublicChannels returns all channels that on the public network. +func PublicChannels() network.ChannelList { return network.ChannelList{ - UnstakedSyncCommittee, + PublicSyncCommittee, } } @@ -141,8 +141,8 @@ const ( ProvideReceiptsByBlockID = RequestReceiptsByBlockID ProvideApprovalsByChunk = RequestApprovalsByChunk - // Unstaked network channels - UnstakedSyncCommittee = network.Channel("unstaked-sync-committee") + // Public network channels + PublicSyncCommittee = network.Channel("public-sync-committee") ) // initializeChannelRoleMap initializes an instance of channelRoleMap and populates it with the channels and their @@ -161,7 +161,7 @@ func initializeChannelRoleMap() { channelRoleMap[ConsensusCommittee] = flow.RoleList{flow.RoleConsensus} // Channels for protocols actively synchronizing state across nodes - channelRoleMap[SyncCommittee] = flow.RoleList{flow.RoleConsensus} + channelRoleMap[SyncCommittee] = flow.Roles() channelRoleMap[SyncExecution] = flow.RoleList{flow.RoleExecution} // Channels for DKG communication diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index aa2cb76d1c8..097f6474b58 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -84,7 +84,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, psOpts := DefaultPubsubOptions(maxPubSubMsgSize) psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewSubscriptionFilter( + return pubsub.WithSubscriptionFilter(NewRoleBasedFilter( h.ID(), rootBlockID, chainID, idProvider, )), nil }) diff --git a/network/p2p/middleware.go b/network/p2p/middleware.go index c18f01ca2ea..9036d85bb4c 100644 --- a/network/p2p/middleware.go +++ b/network/p2p/middleware.go @@ -391,7 +391,7 @@ func (m *Middleware) Subscribe(channel network.Channel) error { topic := engine.TopicFromChannel(channel, m.rootBlockID) var validators []pubsub.ValidatorEx - if !engine.UnstakedChannels().Contains(channel) { + if !engine.PublicChannels().Contains(channel) { // for channels used by the staked nodes, add the topic validator to filter out messages from non-staked nodes validators = append(validators, m.stakedTopicValidator.Validate) } diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index 74477584afd..cd837ae968d 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -12,15 +12,16 @@ import ( "github.com/onflow/flow-go/network" ) -type Filter struct { +// RoleBasedFilter implements a subscription filter that filters subscriptions based on a node's role. +type RoleBasedFilter struct { idProvider id.IdentityProvider myPeerID peer.ID rootBlockID flow.Identifier chainID flow.ChainID } -func NewSubscriptionFilter(pid peer.ID, rootBlockID flow.Identifier, chainID flow.ChainID, idProvider id.IdentityProvider) *Filter { - return &Filter{ +func NewRoleBasedFilter(pid peer.ID, rootBlockID flow.Identifier, chainID flow.ChainID, idProvider id.IdentityProvider) *RoleBasedFilter { + return &RoleBasedFilter{ idProvider, pid, rootBlockID, @@ -28,18 +29,17 @@ func NewSubscriptionFilter(pid peer.ID, rootBlockID flow.Identifier, chainID flo } } -func (f *Filter) allowedTopics(pid peer.ID) map[network.Topic]struct{} { +func (f *RoleBasedFilter) allowedTopics(pid peer.ID) map[network.Topic]struct{} { id, found := f.idProvider.ByPeerID(pid) - var channels network.ChannelList + channels := engine.PublicChannels() if !found { - channels = engine.UnstakedChannels() // TODO: eventually we should have block proposals relayed on a separate // channel on the public network. For now, we need to make sure that - // unstaked nodes can subscribe to the block proposal channel. + // full observer nodes can subscribe to the block proposal channel. channels = append(channels, engine.ReceiveBlocks) } else { - channels = engine.ChannelsByRole(id.Role) + channels = append(channels, engine.ChannelsByRole(id.Role)...) } topics := make(map[network.Topic]struct{}) @@ -60,12 +60,12 @@ func (f *Filter) allowedTopics(pid peer.ID) map[network.Topic]struct{} { return topics } -func (f *Filter) CanSubscribe(topic string) bool { +func (f *RoleBasedFilter) CanSubscribe(topic string) bool { _, allowed := f.allowedTopics(f.myPeerID)[network.Topic(topic)] return allowed } -func (f *Filter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { +func (f *RoleBasedFilter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { allowedTopics := f.allowedTopics(from) var filtered []*pb.RPC_SubOpts diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index 0d7c991d2d2..ad9a209596a 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -13,14 +13,11 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/id" + "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/utils/unittest" ) func TestFilterSubscribe(t *testing.T) { - // skip for now due to bug in libp2p gossipsub implementation: - // https://github.com/libp2p/go-libp2p-pubsub/issues/449 - t.Skip() - identity1, privateKey1 := createID(t, unittest.WithRole(flow.RoleAccess)) identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) ids := flow.IdentityList{identity1, identity2} @@ -61,6 +58,10 @@ func TestFilterSubscribe(t *testing.T) { return false }, 1*time.Second, 100*time.Millisecond) + // skip for now due to bug in libp2p gossipsub implementation: + // https://github.com/libp2p/go-libp2p-pubsub/issues/449 + t.Skip() + var wg sync.WaitGroup wg.Add(2) @@ -102,7 +103,17 @@ func TestCanSubscribe(t *testing.T) { _, err := collectionNode.pubSub.Join(goodTopic.String()) require.NoError(t, err) - badTopic := engine.TopicFromChannel(engine.ProvideReceiptsByBlockID, rootBlockID) + var badTopic network.Topic + allowedChannels := make(map[network.Channel]struct{}) + for _, ch := range engine.ChannelsByRole(flow.RoleCollection) { + allowedChannels[ch] = struct{}{} + } + for _, ch := range engine.Channels() { + if _, ok := allowedChannels[ch]; !ok { + badTopic = engine.TopicFromChannel(ch, rootBlockID) + break + } + } _, err = collectionNode.pubSub.Join(badTopic.String()) require.Error(t, err) @@ -114,6 +125,6 @@ func TestCanSubscribe(t *testing.T) { func createSubscriptionFilterPubsubOption(t *testing.T, ids flow.IdentityList) PubsubOption { idProvider := id.NewFixedIdentityProvider(ids) return func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewSubscriptionFilter(h.ID(), rootBlockID, unittest.BlockFixture().Header.ChainID, idProvider)), nil + return pubsub.WithSubscriptionFilter(NewRoleBasedFilter(h.ID(), rootBlockID, unittest.BlockFixture().Header.ChainID, idProvider)), nil } } From 25968e028db626797b98034c280b2d827a2d3f74 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 16 Sep 2021 22:31:39 -0700 Subject: [PATCH 271/291] Disable for ghost nodes --- network/p2p/libp2pNode.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 097f6474b58..a8b148f5a2f 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -83,11 +83,13 @@ func DefaultLibP2PNodeFactory(ctx context.Context, } psOpts := DefaultPubsubOptions(maxPubSubMsgSize) - psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewRoleBasedFilter( - h.ID(), rootBlockID, chainID, idProvider, - )), nil - }) + if chainID == flow.Mainnet { + psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { + return pubsub.WithSubscriptionFilter(NewRoleBasedFilter( + h.ID(), rootBlockID, chainID, idProvider, + )), nil + }) + } return func() (*Node, error) { return NewDefaultLibP2PNodeBuilder(me, address, flowKey). From ac49207614372eaee8b2ad91d8f970c2b0625409 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Thu, 16 Sep 2021 22:45:25 -0700 Subject: [PATCH 272/291] address comment --- network/p2p/testUtils.go | 38 +++++++++++++++++++++++++++++ network/p2p/topic_validator_test.go | 27 -------------------- 2 files changed, 38 insertions(+), 27 deletions(-) create mode 100644 network/p2p/testUtils.go diff --git a/network/p2p/testUtils.go b/network/p2p/testUtils.go new file mode 100644 index 00000000000..f96942ddc9b --- /dev/null +++ b/network/p2p/testUtils.go @@ -0,0 +1,38 @@ +package p2p + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" +) + +func createID(t *testing.T, opts ...func(*flow.Identity)) (*flow.Identity, crypto.PrivateKey) { + networkKey, err := unittest.NetworkingKey() + require.NoError(t, err) + opts = append(opts, unittest.WithNetworkingKey(networkKey.PublicKey())) + id := unittest.IdentityFixture(opts...) + return id, networkKey +} + +func createNode( + t *testing.T, + nodeID flow.Identifier, + networkKey crypto.PrivateKey, + psOpts ...PubsubOption, +) *Node { + if len(psOpts) == 0 { + psOpts = DefaultPubsubOptions(DefaultMaxPubSubMsgSize) + } + libp2pNode, err := NewDefaultLibP2PNodeBuilder(nodeID, "0.0.0.0:0", networkKey). + SetRootBlockID(rootBlockID). + SetPubsubOptions(psOpts...). + Build(context.TODO()) + require.NoError(t, err) + + return libp2pNode +} diff --git a/network/p2p/topic_validator_test.go b/network/p2p/topic_validator_test.go index cd7d5ff9715..1a1e0986e45 100644 --- a/network/p2p/topic_validator_test.go +++ b/network/p2p/topic_validator_test.go @@ -10,7 +10,6 @@ import ( "github.com/libp2p/go-libp2p-core/peer" "github.com/stretchr/testify/require" - "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/utils/unittest" @@ -126,29 +125,3 @@ func TestTopicValidator(t *testing.T) { wg.Wait() } - -func createID(t *testing.T, opts ...func(*flow.Identity)) (*flow.Identity, crypto.PrivateKey) { - networkKey, err := unittest.NetworkingKey() - require.NoError(t, err) - opts = append(opts, unittest.WithNetworkingKey(networkKey.PublicKey())) - id := unittest.IdentityFixture(opts...) - return id, networkKey -} - -func createNode( - t *testing.T, - nodeID flow.Identifier, - networkKey crypto.PrivateKey, - psOpts ...PubsubOption, -) *Node { - if len(psOpts) == 0 { - psOpts = DefaultPubsubOptions(DefaultMaxPubSubMsgSize) - } - libp2pNode, err := NewDefaultLibP2PNodeBuilder(nodeID, "0.0.0.0:0", networkKey). - SetRootBlockID(rootBlockID). - SetPubsubOptions(psOpts...). - Build(context.TODO()) - require.NoError(t, err) - - return libp2pNode -} From fac03b318464f85076bf9b238b9caba83b6e1eb6 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 01:26:06 -0700 Subject: [PATCH 273/291] Update subscription_filter_test.go --- network/p2p/subscription_filter_test.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index ad9a209596a..5c6bbaca5ba 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -68,7 +68,8 @@ func TestFilterSubscribe(t *testing.T) { testPublish := func(wg *sync.WaitGroup, from *Node, sub *pubsub.Subscription) { data := []byte("hello") - from.Publish(context.TODO(), badTopic, data) + err := from.Publish(context.TODO(), badTopic, data) + require.NoError(t, err) ctx, cancel := context.WithTimeout(context.Background(), time.Second) msg, err := sub.Next(ctx) @@ -97,7 +98,11 @@ func TestCanSubscribe(t *testing.T) { identity, privateKey := createID(t, unittest.WithRole(flow.RoleCollection)) collectionNode := createNode(t, identity.NodeID, privateKey, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity})) - defer collectionNode.Stop() + defer func() { + done, err := collectionNode.Stop() + require.NoError(t, err) + <-done + }() goodTopic := engine.TopicFromChannel(engine.ProvideCollections, rootBlockID) _, err := collectionNode.pubSub.Join(goodTopic.String()) From 4454c21c823195ab9beff125f11db5f7e9104ed4 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 01:44:42 -0700 Subject: [PATCH 274/291] renamne --- network/p2p/{testUtils.go => test_utils.go} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename network/p2p/{testUtils.go => test_utils.go} (100%) diff --git a/network/p2p/testUtils.go b/network/p2p/test_utils.go similarity index 100% rename from network/p2p/testUtils.go rename to network/p2p/test_utils.go From a1bf678bff7a29cca272ee972969ff8995eb2a9d Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 02:06:31 -0700 Subject: [PATCH 275/291] Update subscription_filter_test.go --- network/p2p/subscription_filter_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index 5c6bbaca5ba..228dcce2eb0 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -18,6 +18,10 @@ import ( ) func TestFilterSubscribe(t *testing.T) { + // skip for now due to bug in libp2p gossipsub implementation: + // https://github.com/libp2p/go-libp2p-pubsub/issues/449 + t.Skip() + identity1, privateKey1 := createID(t, unittest.WithRole(flow.RoleAccess)) identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) ids := flow.IdentityList{identity1, identity2} @@ -58,10 +62,6 @@ func TestFilterSubscribe(t *testing.T) { return false }, 1*time.Second, 100*time.Millisecond) - // skip for now due to bug in libp2p gossipsub implementation: - // https://github.com/libp2p/go-libp2p-pubsub/issues/449 - t.Skip() - var wg sync.WaitGroup wg.Add(2) @@ -78,7 +78,7 @@ func TestFilterSubscribe(t *testing.T) { require.Equal(t, msg.Data, data) ctx, cancel = context.WithTimeout(context.Background(), time.Second) - msg, err = unstakedSub.Next(ctx) + _, err = unstakedSub.Next(ctx) cancel() require.ErrorIs(t, err, context.DeadlineExceeded) From a8e0f9e0d39cc7f9a06cb433c4b20a16bb1ccc26 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 07:29:35 -0700 Subject: [PATCH 276/291] Update staked_access_node_builder.go --- cmd/access/node_builder/staked_access_node_builder.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 0f782a152e5..9f52d428c92 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -102,11 +102,11 @@ func (anb *StakedAccessNodeBuilder) Build() AccessNodeBuilder { anb. Component("unstaked sync request proxy", func(builder cmd.NodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - proxyEngine = splitter.New(node.Logger, engine.UnstakedSyncCommittee) + proxyEngine = splitter.New(node.Logger, engine.PublicSyncCommittee) // register the proxy engine with the unstaked network var err error - unstakedNetworkConduit, err = node.Network.Register(engine.UnstakedSyncCommittee, proxyEngine) + unstakedNetworkConduit, err = node.Network.Register(engine.PublicSyncCommittee, proxyEngine) if err != nil { return nil, fmt.Errorf("could not register unstaked sync request proxy: %w", err) } From bd78b018d927c7f424ef5c8ed8f8667bfbdd4dbc Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 07:30:40 -0700 Subject: [PATCH 277/291] Update unstaked_access_node_builder.go --- cmd/access/node_builder/unstaked_access_node_builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/access/node_builder/unstaked_access_node_builder.go b/cmd/access/node_builder/unstaked_access_node_builder.go index f69514ddb4d..2c0b8ab78da 100644 --- a/cmd/access/node_builder/unstaked_access_node_builder.go +++ b/cmd/access/node_builder/unstaked_access_node_builder.go @@ -249,7 +249,7 @@ func (anb *UnstakedAccessNodeBuilder) enqueueUnstakedNetworkInit(ctx context.Con return nil, err } - anb.Network = converter.NewNetwork(network, engine.SyncCommittee, engine.UnstakedSyncCommittee) + anb.Network = converter.NewNetwork(network, engine.SyncCommittee, engine.PublicSyncCommittee) anb.Logger.Info().Msgf("network will run on address: %s", anb.BindAddr) From b6ccebcc3e5c957081f9915915b43fbf80625b21 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 07:42:45 -0700 Subject: [PATCH 278/291] fix bug --- model/flow/role.go | 2 +- network/p2p/subscription_filter_test.go | 8 ++++---- network/p2p/test_utils.go | 1 + network/p2p/topic_validator_test.go | 6 +++--- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/model/flow/role.go b/model/flow/role.go index 3c9db592093..f138a185d75 100644 --- a/model/flow/role.go +++ b/model/flow/role.go @@ -71,7 +71,7 @@ func (r *Role) UnmarshalText(text []byte) error { return err } -func Roles() []Role { +func Roles() RoleList { return []Role{RoleCollection, RoleConsensus, RoleExecution, RoleVerification, RoleAccess} } diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index 228dcce2eb0..8daa8aafe7e 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -26,12 +26,12 @@ func TestFilterSubscribe(t *testing.T) { identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) ids := flow.IdentityList{identity1, identity2} - node1 := createNode(t, identity1.NodeID, privateKey1, createSubscriptionFilterPubsubOption(t, ids)) - node2 := createNode(t, identity2.NodeID, privateKey2, createSubscriptionFilterPubsubOption(t, ids)) + node1 := createNode(t, identity1.NodeID, privateKey1, rootBlockID, createSubscriptionFilterPubsubOption(t, ids)) + node2 := createNode(t, identity2.NodeID, privateKey2, rootBlockID, createSubscriptionFilterPubsubOption(t, ids)) unstakedKey, err := unittest.NetworkingKey() require.NoError(t, err) - unstakedNode := createNode(t, flow.ZeroID, unstakedKey) + unstakedNode := createNode(t, flow.ZeroID, unstakedKey, rootBlockID) require.NoError(t, node1.AddPeer(context.TODO(), *host.InfoFromHost(node2.Host()))) require.NoError(t, node1.AddPeer(context.TODO(), *host.InfoFromHost(unstakedNode.Host()))) @@ -97,7 +97,7 @@ func TestFilterSubscribe(t *testing.T) { func TestCanSubscribe(t *testing.T) { identity, privateKey := createID(t, unittest.WithRole(flow.RoleCollection)) - collectionNode := createNode(t, identity.NodeID, privateKey, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity})) + collectionNode := createNode(t, identity.NodeID, privateKey, rootBlockID, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity})) defer func() { done, err := collectionNode.Stop() require.NoError(t, err) diff --git a/network/p2p/test_utils.go b/network/p2p/test_utils.go index f96942ddc9b..e7130ea598b 100644 --- a/network/p2p/test_utils.go +++ b/network/p2p/test_utils.go @@ -23,6 +23,7 @@ func createNode( t *testing.T, nodeID flow.Identifier, networkKey crypto.PrivateKey, + rootBlockID flow.Identifier, psOpts ...PubsubOption, ) *Node { if len(psOpts) == 0 { diff --git a/network/p2p/topic_validator_test.go b/network/p2p/topic_validator_test.go index 1a1e0986e45..0069f7057e8 100644 --- a/network/p2p/topic_validator_test.go +++ b/network/p2p/topic_validator_test.go @@ -20,10 +20,10 @@ func TestTopicValidator(t *testing.T) { // create two staked nodes - node1 and node2 identity1, privateKey1 := createID(t, unittest.WithRole(flow.RoleAccess)) - node1 := createNode(t, identity1.NodeID, privateKey1) + node1 := createNode(t, identity1.NodeID, privateKey1, rootBlockID) identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) - node2 := createNode(t, identity2.NodeID, privateKey2) + node2 := createNode(t, identity2.NodeID, privateKey2, rootBlockID) badTopic := engine.TopicFromChannel(engine.SyncCommittee, rootBlockID) @@ -42,7 +42,7 @@ func TestTopicValidator(t *testing.T) { unstakedKey, err := unittest.NetworkingKey() require.NoError(t, err) // create one unstaked node - unstakedNode := createNode(t, flow.ZeroID, unstakedKey) + unstakedNode := createNode(t, flow.ZeroID, unstakedKey, rootBlockID) require.NoError(t, err) // node1 is connected to node2, and the unstaked node is connected to node1 From 92328e6538d7a75fa33e9971a02071e5e9f09724 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 08:28:46 -0700 Subject: [PATCH 279/291] Update channels_test.go --- engine/channels_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/engine/channels_test.go b/engine/channels_test.go index a05792183f6..010e51e9f4b 100644 --- a/engine/channels_test.go +++ b/engine/channels_test.go @@ -67,6 +67,7 @@ func TestGetChannelByRole(t *testing.T) { assert.Contains(t, topics, RequestChunks) assert.Contains(t, topics, TestMetrics) assert.Contains(t, topics, TestNetwork) + assert.Contains(t, topics, SyncCommittee) } // TestIsClusterChannel verifies the correctness of ClusterChannel method From 8f3a0d970ed7c3654178dd672eace79186f7840b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 10:23:57 -0700 Subject: [PATCH 280/291] Update channels_test.go --- engine/channels_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/channels_test.go b/engine/channels_test.go index 010e51e9f4b..80638268c29 100644 --- a/engine/channels_test.go +++ b/engine/channels_test.go @@ -59,7 +59,7 @@ func TestGetChannelByRole(t *testing.T) { // - TestMetric // the roles list should contain collection and consensus roles topics := ChannelsByRole(flow.RoleVerification) - assert.Len(t, topics, 7) + assert.Len(t, topics, 8) assert.Contains(t, topics, PushBlocks) assert.Contains(t, topics, PushReceipts) assert.Contains(t, topics, PushApprovals) From 39a17122845acff9dc1d7e52117e8ebccdc36b40 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 17 Sep 2021 10:25:27 -0700 Subject: [PATCH 281/291] Update channels.go --- engine/channels.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/channels.go b/engine/channels.go index bfb09440123..956adbf9195 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -177,7 +177,7 @@ func initializeChannelRoleMap() { channelRoleMap[PushApprovals] = flow.RoleList{flow.RoleConsensus, flow.RoleVerification} // Channels for actively requesting missing entities - channelRoleMap[RequestCollections] = flow.RoleList{flow.RoleCollection, flow.RoleExecution} + channelRoleMap[RequestCollections] = flow.RoleList{flow.RoleCollection, flow.RoleExecution, flow.RoleAccess} channelRoleMap[RequestChunks] = flow.RoleList{flow.RoleExecution, flow.RoleVerification} channelRoleMap[RequestReceiptsByBlockID] = flow.RoleList{flow.RoleConsensus, flow.RoleExecution} channelRoleMap[RequestApprovalsByChunk] = flow.RoleList{flow.RoleConsensus, flow.RoleVerification} From feb5c01a3ed0a28d6b805c3dd19dea6fd99092b6 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 20 Sep 2021 12:36:58 -0700 Subject: [PATCH 282/291] Address comments --- engine/channels.go | 6 ++++-- network/p2p/subscription_filter.go | 18 +++++------------- 2 files changed, 9 insertions(+), 15 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index 956adbf9195..98597e0c390 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -194,9 +194,11 @@ func initializeChannelRoleMap() { channelRoleMap[ProvideChunks] = flow.RoleList{flow.RoleExecution, flow.RoleVerification} channelRoleMap[ProvideReceiptsByBlockID] = flow.RoleList{flow.RoleConsensus, flow.RoleExecution} channelRoleMap[ProvideApprovalsByChunk] = flow.RoleList{flow.RoleConsensus, flow.RoleVerification} +} - channelRoleMap[syncClusterPrefix] = flow.RoleList{flow.RoleCollection} - channelRoleMap[consensusClusterPrefix] = flow.RoleList{flow.RoleCollection} +// ClusterChannelRoles returns the list of roles that are involved in cluster-based channels. +func ClusterChannelRoles() flow.RoleList { + return flow.RoleList{flow.RoleCollection} } // ClusterChannel returns true if channel is cluster-based. diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index cd837ae968d..3b72b2bbbd2 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -1,8 +1,6 @@ package p2p import ( - "strings" - "github.com/libp2p/go-libp2p-core/peer" pb "github.com/libp2p/go-libp2p-pubsub/pb" @@ -32,6 +30,7 @@ func NewRoleBasedFilter(pid peer.ID, rootBlockID flow.Identifier, chainID flow.C func (f *RoleBasedFilter) allowedTopics(pid peer.ID) map[network.Topic]struct{} { id, found := f.idProvider.ByPeerID(pid) channels := engine.PublicChannels() + topics := make(map[network.Topic]struct{}) if !found { // TODO: eventually we should have block proposals relayed on a separate @@ -40,20 +39,13 @@ func (f *RoleBasedFilter) allowedTopics(pid peer.ID) map[network.Topic]struct{} channels = append(channels, engine.ReceiveBlocks) } else { channels = append(channels, engine.ChannelsByRole(id.Role)...) - } - - topics := make(map[network.Topic]struct{}) - for _, ch := range channels { - consensusCluster := engine.ChannelConsensusCluster(f.chainID) - syncCluster := engine.ChannelSyncCluster(f.chainID) - - if strings.HasPrefix(consensusCluster.String(), ch.String()) { - ch = consensusCluster - } else if strings.HasPrefix(syncCluster.String(), ch.String()) { - ch = syncCluster + if engine.ClusterChannelRoles().Contains(id.Role) { + channels = append(channels, engine.ChannelConsensusCluster(f.chainID), engine.ChannelSyncCluster(f.chainID)) } + } + for _, ch := range channels { topics[engine.TopicFromChannel(ch, f.rootBlockID)] = struct{}{} } From 9e81a81319b0a14e39cb03eeb0c6fa373b58ade7 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 20 Sep 2021 16:45:50 -0700 Subject: [PATCH 283/291] update channels --- engine/channels.go | 13 ++++++++----- network/topology/randomizedTopology_test.go | 2 +- network/topology/topicBasedTopology_test.go | 2 +- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index 98597e0c390..d294c7409c1 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -29,12 +29,15 @@ func RolesByChannel(channel network.Channel) (flow.RoleList, bool) { return roles, ok } -// Exists returns true if channel exists in channelRoleMap. -// At the current state, any developer-defined channel should be added -// to channelRoleMap as a constant channel type manually. +// Exists returns true if the channel exists. func Exists(channel network.Channel) bool { - _, exists := RolesByChannel(channel) - return exists || PublicChannels().Contains(channel) + if _, ok := RolesByChannel(channel); !ok { + if _, isClusterChannel := ClusterChannel(channel); !isClusterChannel { + return PublicChannels().Contains(channel) + } + } + + return true } // ChannelsByRole returns a list of all channels the role subscribes to. diff --git a/network/topology/randomizedTopology_test.go b/network/topology/randomizedTopology_test.go index 926090dfbf6..9159500e7a0 100644 --- a/network/topology/randomizedTopology_test.go +++ b/network/topology/randomizedTopology_test.go @@ -151,7 +151,7 @@ func (suite *RandomizedTopologyTestSuite) TestConnectedness_NonClusterChannel() // cluster channel are individually connected. func (suite *RandomizedTopologyTestSuite) TestConnectedness_ClusterChannel() { // picks one cluster channel as sample - channel := clusterChannels(suite.T())[0] + channel := engine.ChannelSyncCluster(unittest.BlockFixture().Header.ChainID) // adjacency map keeps graph component of a single channel channelAdjMap := make(map[flow.Identifier]flow.IdentityList) diff --git a/network/topology/topicBasedTopology_test.go b/network/topology/topicBasedTopology_test.go index 602214fc315..e731490dfc3 100644 --- a/network/topology/topicBasedTopology_test.go +++ b/network/topology/topicBasedTopology_test.go @@ -196,7 +196,7 @@ func (suite *TopicAwareTopologyTestSuite) TestConnectedness_NonClusterChannel() // cluster channel are individually connected. func (suite *TopicAwareTopologyTestSuite) TestConnectedness_ClusterChannel() { // picks one cluster channel as sample - channel := clusterChannels(suite.T())[0] + channel := engine.ChannelSyncCluster(unittest.BlockFixture().Header.ChainID) // adjacency map keeps graph component of a single channel channelAdjMap := make(map[flow.Identifier]flow.IdentityList) From 9e2e5bd9650c0f425feb16d484425af70b82404b Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Mon, 20 Sep 2021 23:18:57 -0700 Subject: [PATCH 284/291] fix bug --- engine/channels.go | 7 +++---- network/topology/helper.go | 16 ---------------- 2 files changed, 3 insertions(+), 20 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index d294c7409c1..542f482e550 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -21,9 +21,8 @@ var channelRoleMap map[network.Channel]flow.RoleList // RolesByChannel returns list of flow roles involved in the channel. func RolesByChannel(channel network.Channel) (flow.RoleList, bool) { - if clusterChannel, isCluster := ClusterChannel(channel); isCluster { - // replaces channel with the stripped-off prefix - channel = clusterChannel + if _, isCluster := ClusterChannel(channel); isCluster { + return ClusterChannelRoles(), true } roles, ok := channelRoleMap[channel] return roles, ok @@ -193,7 +192,7 @@ func initializeChannelRoleMap() { flow.RoleAccess} channelRoleMap[ReceiveApprovals] = flow.RoleList{flow.RoleConsensus, flow.RoleVerification} - channelRoleMap[ProvideCollections] = flow.RoleList{flow.RoleCollection, flow.RoleExecution} + channelRoleMap[ProvideCollections] = flow.RoleList{flow.RoleCollection, flow.RoleExecution, flow.RoleAccess} channelRoleMap[ProvideChunks] = flow.RoleList{flow.RoleExecution, flow.RoleVerification} channelRoleMap[ProvideReceiptsByBlockID] = flow.RoleList{flow.RoleConsensus, flow.RoleExecution} channelRoleMap[ProvideApprovalsByChunk] = flow.RoleList{flow.RoleConsensus, flow.RoleVerification} diff --git a/network/topology/helper.go b/network/topology/helper.go index c5d62301576..c0ccfbfeac7 100644 --- a/network/topology/helper.go +++ b/network/topology/helper.go @@ -148,22 +148,6 @@ func uniquenessCheck(t *testing.T, ids flow.IdentityList) { } } -// clusterChannels is a test helper method that returns all cluster-based channel. -func clusterChannels(t *testing.T) network.ChannelList { - channels := make(network.ChannelList, 0) - for _, channel := range engine.Channels() { - if _, ok := engine.ClusterChannel(channel); !ok { - // skips non-cluster channels - continue - } - - channels = append(channels, channel) - } - - require.NotEmpty(t, channels, "empty cluster-based channels") - return channels -} - // connectedByCluster is a test helper that checks `all` nodes belong to a cluster are connected. func connectedByCluster(t *testing.T, adjMap map[flow.Identifier]flow.IdentityList, all flow.IdentityList, cluster flow.IdentityList) { Connected(t, adjMap, all, filter.In(cluster)) From 9ad681c5c4b2df653c25c2e746cc139702616854 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Tue, 21 Sep 2021 23:21:25 -0700 Subject: [PATCH 285/291] Update unique channels test --- engine/channels.go | 3 +-- engine/channels_test.go | 9 ++++++--- network/p2p/subscription_filter_test.go | 4 ++-- network/topology/randomizedTopology_test.go | 2 +- network/topology/topicBasedTopology_test.go | 2 +- 5 files changed, 11 insertions(+), 9 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index 542f482e550..f88a33f5930 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -65,10 +65,9 @@ func UniqueChannels(channels network.ChannelList) network.ChannelList { // has already been added to uniques. // We use identifier of RoleList to determine its uniqueness. for _, channel := range channels { - id := channelRoleMap[channel].ID() - // non-cluster channel deduplicated based identifier of role list if _, cluster := ClusterChannel(channel); !cluster { + id := channelRoleMap[channel].ID() if _, ok := added[id]; ok { // a channel with same RoleList already added, hence skips continue diff --git a/engine/channels_test.go b/engine/channels_test.go index 80638268c29..55dec9fdb58 100644 --- a/engine/channels_test.go +++ b/engine/channels_test.go @@ -122,10 +122,13 @@ func TestUniqueChannels_Uniqueness(t *testing.T) { // We use the identifier of RoleList to determine their uniqueness. func TestUniqueChannels_ClusterChannels(t *testing.T) { channels := ChannelsByRole(flow.RoleCollection) + consensusCluster := ChannelConsensusCluster(flow.Emulator) + syncCluster := ChannelSyncCluster(flow.Emulator) + channels = append(channels, consensusCluster, syncCluster) uniques := UniqueChannels(channels) // collection role has two cluster and one non-cluster channels all with the same RoleList. // Hence all of them should be returned as unique channels. - require.Contains(t, uniques, syncClusterPrefix) // cluster channel - require.Contains(t, uniques, consensusClusterPrefix) // cluster channel - require.Contains(t, uniques, PushTransactions) // non-cluster channel + require.Contains(t, uniques, syncCluster) // cluster channel + require.Contains(t, uniques, consensusCluster) // cluster channel + require.Contains(t, uniques, PushTransactions) // non-cluster channel } diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index 8daa8aafe7e..a6c98bc1f46 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -122,7 +122,7 @@ func TestCanSubscribe(t *testing.T) { _, err = collectionNode.pubSub.Join(badTopic.String()) require.Error(t, err) - clusterTopic := engine.TopicFromChannel(engine.ChannelSyncCluster(unittest.ClusterBlockFixture().Header.ChainID), rootBlockID) + clusterTopic := engine.TopicFromChannel(engine.ChannelSyncCluster(flow.Emulator), rootBlockID) _, err = collectionNode.pubSub.Join(clusterTopic.String()) require.NoError(t, err) } @@ -130,6 +130,6 @@ func TestCanSubscribe(t *testing.T) { func createSubscriptionFilterPubsubOption(t *testing.T, ids flow.IdentityList) PubsubOption { idProvider := id.NewFixedIdentityProvider(ids) return func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewRoleBasedFilter(h.ID(), rootBlockID, unittest.BlockFixture().Header.ChainID, idProvider)), nil + return pubsub.WithSubscriptionFilter(NewRoleBasedFilter(h.ID(), rootBlockID, flow.Emulator, idProvider)), nil } } diff --git a/network/topology/randomizedTopology_test.go b/network/topology/randomizedTopology_test.go index 9159500e7a0..4685ed10fd0 100644 --- a/network/topology/randomizedTopology_test.go +++ b/network/topology/randomizedTopology_test.go @@ -151,7 +151,7 @@ func (suite *RandomizedTopologyTestSuite) TestConnectedness_NonClusterChannel() // cluster channel are individually connected. func (suite *RandomizedTopologyTestSuite) TestConnectedness_ClusterChannel() { // picks one cluster channel as sample - channel := engine.ChannelSyncCluster(unittest.BlockFixture().Header.ChainID) + channel := engine.ChannelSyncCluster(flow.Emulator) // adjacency map keeps graph component of a single channel channelAdjMap := make(map[flow.Identifier]flow.IdentityList) diff --git a/network/topology/topicBasedTopology_test.go b/network/topology/topicBasedTopology_test.go index e731490dfc3..e2edd523662 100644 --- a/network/topology/topicBasedTopology_test.go +++ b/network/topology/topicBasedTopology_test.go @@ -196,7 +196,7 @@ func (suite *TopicAwareTopologyTestSuite) TestConnectedness_NonClusterChannel() // cluster channel are individually connected. func (suite *TopicAwareTopologyTestSuite) TestConnectedness_ClusterChannel() { // picks one cluster channel as sample - channel := engine.ChannelSyncCluster(unittest.BlockFixture().Header.ChainID) + channel := engine.ChannelSyncCluster(flow.Emulator) // adjacency map keeps graph component of a single channel channelAdjMap := make(map[flow.Identifier]flow.IdentityList) From f93c5080ebbc25b4e521dfebda04cbe8d0b22603 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 24 Sep 2021 01:51:39 -0700 Subject: [PATCH 286/291] address comments --- engine/channels.go | 14 ++++++++------ network/p2p/libp2pNode.go | 11 ++++++----- network/p2p/sporking_test.go | 8 ++++---- network/p2p/subscription_filter_test.go | 11 ++++++++--- network/p2p/test_utils.go | 9 --------- network/p2p/topic_validator_test.go | 4 ++-- utils/unittest/fixtures.go | 11 +++++++++++ 7 files changed, 39 insertions(+), 29 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index f88a33f5930..9aff513fa2e 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -30,13 +30,15 @@ func RolesByChannel(channel network.Channel) (flow.RoleList, bool) { // Exists returns true if the channel exists. func Exists(channel network.Channel) bool { - if _, ok := RolesByChannel(channel); !ok { - if _, isClusterChannel := ClusterChannel(channel); !isClusterChannel { - return PublicChannels().Contains(channel) - } + if _, ok := RolesByChannel(channel); ok { + return true + } + + if _, ok := ClusterChannel(channel); ok { + return true } - return true + return PublicChannels().Contains(channel) } // ChannelsByRole returns a list of all channels the role subscribes to. @@ -91,7 +93,7 @@ func Channels() network.ChannelList { return channels } -// PublicChannels returns all channels that on the public network. +// PublicChannels returns all channels that are used on the public network. func PublicChannels() network.ChannelList { return network.ChannelList{ PublicSyncCommittee, diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index dfa397524a3..4e383ae2979 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -120,7 +120,7 @@ type NodeBuilder interface { type DefaultLibP2PNodeBuilder struct { id flow.Identifier - rootBlockID flow.Identifier + rootBlockID *flow.Identifier logger zerolog.Logger connGater *ConnGater connMngr TagLessConnManager @@ -157,7 +157,7 @@ func (builder *DefaultLibP2PNodeBuilder) SetTopicValidation(enabled bool) NodeBu } func (builder *DefaultLibP2PNodeBuilder) SetRootBlockID(rootBlockId flow.Identifier) NodeBuilder { - builder.rootBlockID = rootBlockId + builder.rootBlockID = &rootBlockId return builder } @@ -208,10 +208,10 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro return nil, errors.New("unable to create libp2p pubsub: factory function not provided") } - if builder.rootBlockID == flow.ZeroID { + if builder.rootBlockID == nil { return nil, errors.New("root block ID must be provided") } - node.flowLibP2PProtocolID = generateFlowProtocolID(builder.rootBlockID) + node.flowLibP2PProtocolID = generateFlowProtocolID(*builder.rootBlockID) var opts []config.Option @@ -249,7 +249,7 @@ func (builder *DefaultLibP2PNodeBuilder) Build(ctx context.Context) (*Node, erro } if builder.pingInfoProvider != nil { - pingLibP2PProtocolID := generatePingProtcolID(builder.rootBlockID) + pingLibP2PProtocolID := generatePingProtcolID(*builder.rootBlockID) pingService := NewPingService(libp2pHost, pingLibP2PProtocolID, builder.pingInfoProvider, node.logger) node.pingService = pingService } @@ -736,6 +736,7 @@ func DefaultPubsubOptions(maxPubSubMsgSize int) []PubsubOption { pubSubOptionFunc(pubsub.WithStrictSignatureVerification(true)), // set max message size limit for 1-k PubSub messaging pubSubOptionFunc(pubsub.WithMaxMessageSize(maxPubSubMsgSize)), + // no discovery } } diff --git a/network/p2p/sporking_test.go b/network/p2p/sporking_test.go index 32cdc456dc3..e6f5f5d3802 100644 --- a/network/p2p/sporking_test.go +++ b/network/p2p/sporking_test.go @@ -82,7 +82,7 @@ func (suite *SporkingTestSuite) TestCrosstalkPreventionOnNetworkKeyChange() { func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { // root id before spork - rootID1 := unittest.BlockFixture().ID() + rootID1 := unittest.IdentifierFixture() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -104,7 +104,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { StopNode(suite.T(), node2) // update the flow root id for node 2. node1 is still listening on the old protocol - rootID2 := unittest.BlockFixture().ID() + rootID2 := unittest.IdentifierFixture() // start node2 with the same address and root key but different root block id node2, id2New := NodeFixture(suite.T(), suite.logger, node2key, rootID2, nil, false, id2.Address) @@ -123,7 +123,7 @@ func (suite *SporkingTestSuite) TestOneToOneCrosstalkPrevention() { func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { // root id before spork - rootIDBeforeSpork := unittest.BlockFixture().ID() + rootIDBeforeSpork := unittest.IdentifierFixture() // create and start node 1 on localhost and random port node1key := generateNetworkingKey(suite.T()) @@ -160,7 +160,7 @@ func (suite *SporkingTestSuite) TestOneToKCrosstalkPrevention() { testOneToKMessagingSucceeds(ctx, suite.T(), node1, sub2, topicBeforeSpork) // new root id after spork - rootIDAfterSpork := unittest.BlockFixture().ID() + rootIDAfterSpork := unittest.IdentifierFixture() // topic after the spork topicAfterSpork := engine.TopicFromChannel(engine.TestNetwork, rootIDAfterSpork) diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index a6c98bc1f46..c8628ae9b60 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -17,13 +17,15 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +// TestFilterSubscribe tests that if node X is filtered out on a specific channel by node Y's subscription +// filter, then node Y will never propagate any of node X's messages on that channel func TestFilterSubscribe(t *testing.T) { // skip for now due to bug in libp2p gossipsub implementation: // https://github.com/libp2p/go-libp2p-pubsub/issues/449 t.Skip() - identity1, privateKey1 := createID(t, unittest.WithRole(flow.RoleAccess)) - identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) + identity1, privateKey1 := unittest.IdentityWithNetworkingKeyFixture(unittest.WithRole(flow.RoleAccess)) + identity2, privateKey2 := unittest.IdentityWithNetworkingKeyFixture(unittest.WithRole(flow.RoleAccess)) ids := flow.IdentityList{identity1, identity2} node1 := createNode(t, identity1.NodeID, privateKey1, rootBlockID, createSubscriptionFilterPubsubOption(t, ids)) @@ -53,6 +55,7 @@ func TestFilterSubscribe(t *testing.T) { len(unstakedNode.pubSub.ListPeers(badTopic.String())) > 0 }, 1*time.Second, 100*time.Millisecond) + // check that node1 and node2 don't accept unstakedNode as a peer require.Never(t, func() bool { for _, pid := range node1.pubSub.ListPeers(badTopic.String()) { if pid == unstakedNode.Host().ID() { @@ -94,8 +97,10 @@ func TestFilterSubscribe(t *testing.T) { wg.Wait() } +// TestCanSubscribe tests that the subscription filter blocks a node from subscribing +// to channels that its role shouldn't subscribe to func TestCanSubscribe(t *testing.T) { - identity, privateKey := createID(t, unittest.WithRole(flow.RoleCollection)) + identity, privateKey := unittest.IdentityWithNetworkingKeyFixture(unittest.WithRole(flow.RoleCollection)) collectionNode := createNode(t, identity.NodeID, privateKey, rootBlockID, createSubscriptionFilterPubsubOption(t, flow.IdentityList{identity})) defer func() { diff --git a/network/p2p/test_utils.go b/network/p2p/test_utils.go index e7130ea598b..135859fb9df 100644 --- a/network/p2p/test_utils.go +++ b/network/p2p/test_utils.go @@ -8,17 +8,8 @@ import ( "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/utils/unittest" ) -func createID(t *testing.T, opts ...func(*flow.Identity)) (*flow.Identity, crypto.PrivateKey) { - networkKey, err := unittest.NetworkingKey() - require.NoError(t, err) - opts = append(opts, unittest.WithNetworkingKey(networkKey.PublicKey())) - id := unittest.IdentityFixture(opts...) - return id, networkKey -} - func createNode( t *testing.T, nodeID flow.Identifier, diff --git a/network/p2p/topic_validator_test.go b/network/p2p/topic_validator_test.go index 9d759a06b58..f7ce2f489d6 100644 --- a/network/p2p/topic_validator_test.go +++ b/network/p2p/topic_validator_test.go @@ -21,10 +21,10 @@ import ( func TestTopicValidator(t *testing.T) { // create two staked nodes - node1 and node2 - identity1, privateKey1 := createID(t, unittest.WithRole(flow.RoleAccess)) + identity1, privateKey1 := unittest.IdentityWithNetworkingKeyFixture(unittest.WithRole(flow.RoleAccess)) node1 := createNode(t, identity1.NodeID, privateKey1, rootBlockID) - identity2, privateKey2 := createID(t, unittest.WithRole(flow.RoleAccess)) + identity2, privateKey2 := unittest.IdentityWithNetworkingKeyFixture(unittest.WithRole(flow.RoleAccess)) node2 := createNode(t, identity2.NodeID, privateKey2, rootBlockID) badTopic := engine.TopicFromChannel(engine.SyncCommittee, rootBlockID) diff --git a/utils/unittest/fixtures.go b/utils/unittest/fixtures.go index e27ee0fe0fe..119d3b01a75 100644 --- a/utils/unittest/fixtures.go +++ b/utils/unittest/fixtures.go @@ -875,6 +875,17 @@ func IdentityFixture(opts ...func(*flow.Identity)) *flow.Identity { return &identity } +// IdentityFixture returns a node identity and networking private key +func IdentityWithNetworkingKeyFixture(opts ...func(*flow.Identity)) (*flow.Identity, crypto.PrivateKey) { + networkKey, err := NetworkingKey() + if err != nil { + panic(err) + } + opts = append(opts, WithNetworkingKey(networkKey.PublicKey())) + id := IdentityFixture(opts...) + return id, networkKey +} + func WithKeys(identity *flow.Identity) { staking, err := StakingKey() if err != nil { From 6c9813f9cc8e4ee7fe31ba8b4c2489b1d1407ce9 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 24 Sep 2021 13:51:16 -0700 Subject: [PATCH 287/291] Refactor channels code and subscription filter --- engine/channels.go | 76 ++++++++++++++++++++---------- network/p2p/subscription_filter.go | 53 +++++++++++---------- 2 files changed, 77 insertions(+), 52 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index 9aff513fa2e..50aa31c3ad5 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -11,7 +11,7 @@ import ( ) // init is called first time this package is imported. -// It creates and initializes the channelRoleMap map. +// It creates and initializes channelRoleMap and clusterChannelPrefixRoleMap. func init() { initializeChannelRoleMap() } @@ -19,10 +19,18 @@ func init() { // channelRoleMap keeps a map between channels and the list of flow roles involved in them. var channelRoleMap map[network.Channel]flow.RoleList +// clusterChannelPrefixRoleMap keeps a map between cluster channel prefixes and the list of flow roles involved in them. +var clusterChannelPrefixRoleMap map[string]flow.RoleList + // RolesByChannel returns list of flow roles involved in the channel. +// If the given channel is a public channel, the returned list will +// contain all roles. func RolesByChannel(channel network.Channel) (flow.RoleList, bool) { - if _, isCluster := ClusterChannel(channel); isCluster { - return ClusterChannelRoles(), true + if IsClusterChannel(channel) { + return ClusterChannelRoles(channel), true + } + if PublicChannels().Contains(channel) { + return flow.Roles(), true } roles, ok := channelRoleMap[channel] return roles, ok @@ -34,14 +42,10 @@ func Exists(channel network.Channel) bool { return true } - if _, ok := ClusterChannel(channel); ok { - return true - } - - return PublicChannels().Contains(channel) + return false } -// ChannelsByRole returns a list of all channels the role subscribes to. +// ChannelsByRole returns a list of all channels the role subscribes to (except cluster-based channels and public channels). func ChannelsByRole(role flow.Role) network.ChannelList { channels := make(network.ChannelList, 0) for channel, roles := range channelRoleMap { @@ -68,7 +72,7 @@ func UniqueChannels(channels network.ChannelList) network.ChannelList { // We use identifier of RoleList to determine its uniqueness. for _, channel := range channels { // non-cluster channel deduplicated based identifier of role list - if _, cluster := ClusterChannel(channel); !cluster { + if !IsClusterChannel(channel) { id := channelRoleMap[channel].ID() if _, ok := added[id]; ok { // a channel with same RoleList already added, hence skips @@ -83,12 +87,13 @@ func UniqueChannels(channels network.ChannelList) network.ChannelList { return uniques } -// Channels returns all channels that nodes of any role have subscribed to. +// Channels returns all channels that nodes of any role have subscribed to (except cluster-based channels). func Channels() network.ChannelList { channels := make(network.ChannelList, 0) for channel := range channelRoleMap { channels = append(channels, channel) } + channels = append(channels, PublicChannels()...) return channels } @@ -109,11 +114,11 @@ const ( // Channels for consensus protocols ConsensusCommittee = network.Channel("consensus-committee") - consensusClusterPrefix = network.Channel("consensus-cluster") // dynamic channel, use ChannelConsensusCluster function + consensusClusterPrefix = "consensus-cluster" // dynamic channel, use ChannelConsensusCluster function // Channels for protocols actively synchronizing state across nodes SyncCommittee = network.Channel("sync-committee") - syncClusterPrefix = network.Channel("sync-cluster") // dynamic channel, use ChannelSyncCluster function + syncClusterPrefix = "sync-cluster" // dynamic channel, use ChannelSyncCluster function SyncExecution = network.Channel("sync-execution") // Channels for dkg communication @@ -197,40 +202,59 @@ func initializeChannelRoleMap() { channelRoleMap[ProvideChunks] = flow.RoleList{flow.RoleExecution, flow.RoleVerification} channelRoleMap[ProvideReceiptsByBlockID] = flow.RoleList{flow.RoleConsensus, flow.RoleExecution} channelRoleMap[ProvideApprovalsByChunk] = flow.RoleList{flow.RoleConsensus, flow.RoleVerification} -} -// ClusterChannelRoles returns the list of roles that are involved in cluster-based channels. -func ClusterChannelRoles() flow.RoleList { - return flow.RoleList{flow.RoleCollection} + clusterChannelPrefixRoleMap = make(map[string]flow.RoleList) + + clusterChannelPrefixRoleMap[syncClusterPrefix] = flow.RoleList{flow.RoleCollection} + clusterChannelPrefixRoleMap[consensusClusterPrefix] = flow.RoleList{flow.RoleCollection} } -// ClusterChannel returns true if channel is cluster-based. -// At the current implementation, only collection nodes are involved in a cluster-based channels. -// If the channel is a cluster-based one, this method also strips off the channel prefix and returns it. -func ClusterChannel(channel network.Channel) (network.Channel, bool) { - if strings.HasPrefix(channel.String(), syncClusterPrefix.String()) { - return syncClusterPrefix, true +// ClusterChannelRoles returns the list of roles that are involved in the given cluster-based channel. +func ClusterChannelRoles(clusterChannel network.Channel) flow.RoleList { + if prefix, ok := clusterChannelPrefix(clusterChannel); ok { + return clusterChannelPrefixRoleMap[prefix] } - if strings.HasPrefix(channel.String(), consensusClusterPrefix.String()) { - return consensusClusterPrefix, true + return flow.RoleList{} +} + +func clusterChannelPrefix(clusterChannel network.Channel) (string, bool) { + for prefix := range clusterChannelPrefixRoleMap { + if strings.HasPrefix(clusterChannel.String(), prefix) { + return prefix, true + } } return "", false } +// IsClusterChannel returns true if channel is cluster-based. +// Currently, only collection nodes are involved in a cluster-based channels. +func IsClusterChannel(channel network.Channel) bool { + _, ok := clusterChannelPrefix(channel) + return ok +} + // TopicFromChannel returns the unique LibP2P topic form the channel. // The channel is made up of name string suffixed with root block id. // The root block id is used to prevent cross talks between nodes on different sporks. func TopicFromChannel(channel network.Channel, rootBlockID flow.Identifier) network.Topic { // skip root block suffix, if this is a cluster specific channel. A cluster specific channel is inherently // unique for each epoch - if _, isClusterChannel := ClusterChannel(channel); isClusterChannel { + if IsClusterChannel(channel) { return network.Topic(channel) } return network.Topic(fmt.Sprintf("%s/%s", string(channel), rootBlockID.String())) } +func ChannelFromTopic(topic network.Topic) network.Channel { + if IsClusterChannel(network.Channel(topic)) { + return network.Channel(topic) + } + + return network.Channel(topic[strings.LastIndex(topic.String(), "/")+1:]) +} + // ChannelConsensusCluster returns a dynamic cluster consensus channel based on // the chain ID of the cluster in question. func ChannelConsensusCluster(clusterID flow.ChainID) network.Channel { diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index 3b72b2bbbd2..9c837b19672 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -14,55 +14,56 @@ import ( type RoleBasedFilter struct { idProvider id.IdentityProvider myPeerID peer.ID + myRole *flow.Role rootBlockID flow.Identifier - chainID flow.ChainID } -func NewRoleBasedFilter(pid peer.ID, rootBlockID flow.Identifier, chainID flow.ChainID, idProvider id.IdentityProvider) *RoleBasedFilter { - return &RoleBasedFilter{ - idProvider, - pid, - rootBlockID, - chainID, +func NewRoleBasedFilter(pid peer.ID, rootBlockID flow.Identifier, idProvider id.IdentityProvider) *RoleBasedFilter { + filter := &RoleBasedFilter{ + idProvider: idProvider, + myPeerID: pid, + rootBlockID: rootBlockID, } + filter.myRole = filter.getRole(pid) + + return filter +} + +func (f *RoleBasedFilter) getRole(pid peer.ID) *flow.Role { + if id, ok := f.idProvider.ByPeerID(pid); ok { + return &id.Role + } + + return nil } -func (f *RoleBasedFilter) allowedTopics(pid peer.ID) map[network.Topic]struct{} { - id, found := f.idProvider.ByPeerID(pid) - channels := engine.PublicChannels() - topics := make(map[network.Topic]struct{}) +func (f *RoleBasedFilter) allowed(role *flow.Role, topic string) bool { + channel := engine.ChannelFromTopic(network.Topic(topic)) - if !found { + if role == nil { // TODO: eventually we should have block proposals relayed on a separate // channel on the public network. For now, we need to make sure that // full observer nodes can subscribe to the block proposal channel. - channels = append(channels, engine.ReceiveBlocks) + return append(engine.PublicChannels(), engine.ReceiveBlocks).Contains(channel) } else { - channels = append(channels, engine.ChannelsByRole(id.Role)...) - - if engine.ClusterChannelRoles().Contains(id.Role) { - channels = append(channels, engine.ChannelConsensusCluster(f.chainID), engine.ChannelSyncCluster(f.chainID)) + if roles, ok := engine.RolesByChannel(channel); ok { + return roles.Contains(*role) } - } - for _, ch := range channels { - topics[engine.TopicFromChannel(ch, f.rootBlockID)] = struct{}{} + return false } - - return topics } func (f *RoleBasedFilter) CanSubscribe(topic string) bool { - _, allowed := f.allowedTopics(f.myPeerID)[network.Topic(topic)] - return allowed + return f.allowed(f.myRole, topic) } func (f *RoleBasedFilter) FilterIncomingSubscriptions(from peer.ID, opts []*pb.RPC_SubOpts) ([]*pb.RPC_SubOpts, error) { - allowedTopics := f.allowedTopics(from) + role := f.getRole(from) var filtered []*pb.RPC_SubOpts for _, opt := range opts { - if _, allowed := allowedTopics[network.Topic(opt.GetTopicid())]; allowed { + if f.allowed(role, opt.GetTopicid()) { filtered = append(filtered, opt) } } From 568a63e25ea197618c4a6a7bc495f79b528a04b2 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 24 Sep 2021 13:58:51 -0700 Subject: [PATCH 288/291] fix build errors --- network/p2p/libp2pNode.go | 4 +++- network/topology/randomizedTopology.go | 4 ++-- network/topology/topicBasedTopology.go | 4 ++-- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index cf767a17975..980894a15f4 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -81,10 +81,12 @@ func DefaultLibP2PNodeFactory(ctx context.Context, resolver := dns.NewResolver(metrics, dns.WithTTL(dnsResolverTTL)) psOpts := DefaultPubsubOptions(maxPubSubMsgSize) + + // only use subscription filters on Mainnet to allow ghost nodes to run on all other networks if chainID == flow.Mainnet { psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { return pubsub.WithSubscriptionFilter(NewRoleBasedFilter( - h.ID(), rootBlockID, chainID, idProvider, + h.ID(), rootBlockID, idProvider, )), nil }) } diff --git a/network/topology/randomizedTopology.go b/network/topology/randomizedTopology.go index c1bc959ed9e..98b7e367275 100644 --- a/network/topology/randomizedTopology.go +++ b/network/topology/randomizedTopology.go @@ -98,7 +98,7 @@ func (r RandomizedTopology) subsetChannel(ids flow.IdentityList, channel network sampleSpace := ids.Filter(filter.Not(filter.HasNodeID(r.myNodeID))) // samples a random graph based on whether channel is cluster-based or not. - if _, ok := engine.ClusterChannel(channel); ok { + if engine.IsClusterChannel(channel) { return r.clusterChannelHandler(sampleSpace) } return r.nonClusterChannelHandler(sampleSpace, channel) @@ -149,7 +149,7 @@ func (r RandomizedTopology) clusterChannelHandler(ids flow.IdentityList) (flow.I // clusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. func (r RandomizedTopology) nonClusterChannelHandler(ids flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { - if _, ok := engine.ClusterChannel(channel); ok { + if engine.IsClusterChannel(channel) { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } diff --git a/network/topology/topicBasedTopology.go b/network/topology/topicBasedTopology.go index 9d14e34049b..4baee18c957 100644 --- a/network/topology/topicBasedTopology.go +++ b/network/topology/topicBasedTopology.go @@ -103,7 +103,7 @@ func (t TopicBasedTopology) GenerateFanout(ids flow.IdentityList, channels netwo // Returned identities should all subscribed to the specified `channel`. // Note: this method should not include identity of its executor. func (t *TopicBasedTopology) subsetChannel(ids flow.IdentityList, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { - if _, ok := engine.ClusterChannel(channel); ok { + if engine.IsClusterChannel(channel) { return t.clusterChannelHandler(ids, shouldHave) } return t.nonClusterChannelHandler(ids, shouldHave, channel) @@ -198,7 +198,7 @@ func (t TopicBasedTopology) clusterChannelHandler(ids, shouldHave flow.IdentityL // nonClusterChannelHandler returns a connected graph fanout of peers from `ids` that subscribed to `channel`. // The returned sample contains `shouldHave` ones that also subscribed to `channel`. func (t TopicBasedTopology) nonClusterChannelHandler(ids, shouldHave flow.IdentityList, channel network.Channel) (flow.IdentityList, error) { - if _, ok := engine.ClusterChannel(channel); ok { + if engine.IsClusterChannel(channel) { return nil, fmt.Errorf("could not handle cluster channel: %s", channel) } From d0eebc8d7ebaecf7f56b955e803cb3cd912f5080 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 24 Sep 2021 14:01:46 -0700 Subject: [PATCH 289/291] fix build error --- cmd/access/node_builder/staked_access_node_builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/access/node_builder/staked_access_node_builder.go b/cmd/access/node_builder/staked_access_node_builder.go index 295482c57e5..1d0ea2f55af 100644 --- a/cmd/access/node_builder/staked_access_node_builder.go +++ b/cmd/access/node_builder/staked_access_node_builder.go @@ -218,7 +218,7 @@ func (builder *StakedAccessNodeBuilder) initLibP2PFactory(ctx context.Context, psOpts := p2p.DefaultPubsubOptions(p2p.DefaultMaxPubSubMsgSize) psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { return pubsub.WithSubscriptionFilter(p2p.NewRoleBasedFilter( - h.ID(), builder.RootBlock.ID(), builder.RootChainID, builder.IdentityProvider, + h.ID(), builder.RootBlock.ID(), builder.IdentityProvider, )), nil }) libp2pNode, err := p2p.NewDefaultLibP2PNodeBuilder(nodeID, myAddr, networkKey). From e31a0c2fe75df701abb7acc0b8c916b38243c8e7 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 24 Sep 2021 14:14:02 -0700 Subject: [PATCH 290/291] fix tests --- engine/channels.go | 10 +++++++--- engine/channels_test.go | 11 ++++------- network/p2p/subscription_filter.go | 5 ++++- network/p2p/subscription_filter_test.go | 2 +- 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/engine/channels.go b/engine/channels.go index 50aa31c3ad5..46bbcd03c42 100644 --- a/engine/channels.go +++ b/engine/channels.go @@ -247,12 +247,16 @@ func TopicFromChannel(channel network.Channel, rootBlockID flow.Identifier) netw return network.Topic(fmt.Sprintf("%s/%s", string(channel), rootBlockID.String())) } -func ChannelFromTopic(topic network.Topic) network.Channel { +func ChannelFromTopic(topic network.Topic) (network.Channel, bool) { if IsClusterChannel(network.Channel(topic)) { - return network.Channel(topic) + return network.Channel(topic), true } - return network.Channel(topic[strings.LastIndex(topic.String(), "/")+1:]) + if index := strings.LastIndex(topic.String(), "/"); index != -1 { + return network.Channel(topic[:index]), true + } + + return "", false } // ChannelConsensusCluster returns a dynamic cluster consensus channel based on diff --git a/engine/channels_test.go b/engine/channels_test.go index 55dec9fdb58..b3c0af6c09a 100644 --- a/engine/channels_test.go +++ b/engine/channels_test.go @@ -75,20 +75,17 @@ func TestGetChannelByRole(t *testing.T) { func TestIsClusterChannel(t *testing.T) { // creates a consensus cluster channel and verifies it conClusterChannel := ChannelConsensusCluster("some-consensus-cluster-id") - clusterChannel, ok := ClusterChannel(conClusterChannel) + ok := IsClusterChannel(conClusterChannel) require.True(t, ok) - require.Equal(t, clusterChannel, consensusClusterPrefix) // creates a sync cluster channel and verifies it syncClusterChannel := ChannelSyncCluster("some-sync-cluster-id") - clusterChannel, ok = ClusterChannel(syncClusterChannel) + ok = IsClusterChannel(syncClusterChannel) require.True(t, ok) - require.Equal(t, clusterChannel, syncClusterPrefix) // non-cluster channel should not be verified - clusterChannel, ok = ClusterChannel("non-cluster-channel-id") + ok = IsClusterChannel("non-cluster-channel-id") require.False(t, ok) - require.Empty(t, clusterChannel) } // TestUniqueChannels_Uniqueness verifies that non-cluster channels returned by @@ -101,7 +98,7 @@ func TestUniqueChannels_Uniqueness(t *testing.T) { visited := make(map[flow.Identifier]struct{}) for _, channel := range uniques { - if _, ok := ClusterChannel(channel); ok { + if IsClusterChannel(channel) { continue //only considering non-cluster channel in this test case } diff --git a/network/p2p/subscription_filter.go b/network/p2p/subscription_filter.go index 9c837b19672..d6910452d22 100644 --- a/network/p2p/subscription_filter.go +++ b/network/p2p/subscription_filter.go @@ -38,7 +38,10 @@ func (f *RoleBasedFilter) getRole(pid peer.ID) *flow.Role { } func (f *RoleBasedFilter) allowed(role *flow.Role, topic string) bool { - channel := engine.ChannelFromTopic(network.Topic(topic)) + channel, ok := engine.ChannelFromTopic(network.Topic(topic)) + if !ok { + return false + } if role == nil { // TODO: eventually we should have block proposals relayed on a separate diff --git a/network/p2p/subscription_filter_test.go b/network/p2p/subscription_filter_test.go index c8628ae9b60..ae55589ff0a 100644 --- a/network/p2p/subscription_filter_test.go +++ b/network/p2p/subscription_filter_test.go @@ -135,6 +135,6 @@ func TestCanSubscribe(t *testing.T) { func createSubscriptionFilterPubsubOption(t *testing.T, ids flow.IdentityList) PubsubOption { idProvider := id.NewFixedIdentityProvider(ids) return func(_ context.Context, h host.Host) (pubsub.Option, error) { - return pubsub.WithSubscriptionFilter(NewRoleBasedFilter(h.ID(), rootBlockID, flow.Emulator, idProvider)), nil + return pubsub.WithSubscriptionFilter(NewRoleBasedFilter(h.ID(), rootBlockID, idProvider)), nil } } From 3dab281c1dbd68faa0f636a1eeb80f119b103357 Mon Sep 17 00:00:00 2001 From: Simon Zhu Date: Fri, 24 Sep 2021 16:02:30 -0700 Subject: [PATCH 291/291] Update libp2pNode.go --- network/p2p/libp2pNode.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 980894a15f4..6843791e8e8 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -82,8 +82,7 @@ func DefaultLibP2PNodeFactory(ctx context.Context, psOpts := DefaultPubsubOptions(maxPubSubMsgSize) - // only use subscription filters on Mainnet to allow ghost nodes to run on all other networks - if chainID == flow.Mainnet { + if chainID != flow.Localnet { psOpts = append(psOpts, func(_ context.Context, h host.Host) (pubsub.Option, error) { return pubsub.WithSubscriptionFilter(NewRoleBasedFilter( h.ID(), rootBlockID, idProvider,