From 76b6dccb1bd9769f0c7a580b7345030aafd23a3d Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 6 Jun 2023 11:22:36 -0400 Subject: [PATCH 01/78] bloom filter gossip --- go.mod | 2 + go.sum | 4 +- gossip/bloom.go | 58 +++++ gossip/bloom_test.go | 62 ++++++ gossip/gossip.go | 120 ++++++++++ gossip/gossip_test.go | 29 +++ gossip/handler.go | 70 ++++++ gossip/mempool.go | 29 +++ gossip/message.go | 12 + gossip/test_gossip.go | 81 +++++++ peer/network.go | 45 ++-- peer/network_test.go | 26 ++- plugin/evm/codec.go | 1 + plugin/evm/gossip_mempool.go | 217 +++++++++++++++++++ plugin/evm/gossip_mempool_test.go | 194 +++++++++++++++++ plugin/evm/gossiper.go | 2 + plugin/evm/gossiper_atomic_gossiping_test.go | 24 +- plugin/evm/mempool.go | 40 +++- plugin/evm/mempool_atomic_gossiping_test.go | 20 +- plugin/evm/message/codec.go | 3 + plugin/evm/tx_test.go | 4 +- plugin/evm/vm.go | 106 +++++++-- plugin/evm/vm_test.go | 10 +- 23 files changed, 1089 insertions(+), 70 deletions(-) create mode 100644 gossip/bloom.go create mode 100644 gossip/bloom_test.go create mode 100644 gossip/gossip.go create mode 100644 gossip/gossip_test.go create mode 100644 gossip/handler.go create mode 100644 gossip/mempool.go create mode 100644 gossip/message.go create mode 100644 gossip/test_gossip.go create mode 100644 plugin/evm/gossip_mempool.go create mode 100644 plugin/evm/gossip_mempool_test.go diff --git a/go.mod b/go.mod index aba0d3ab0f..4e3c04aee4 100644 --- a/go.mod +++ b/go.mod @@ -141,3 +141,5 @@ require ( gopkg.in/yaml.v3 v3.0.1 // indirect rsc.io/tmplfunc v0.0.3 // indirect ) + +replace github.com/ava-labs/avalanchego => github.com/joshua-kim/avalanchego v1.4.9-0.20230727193239-84cd2070ba32 diff --git a/go.sum b/go.sum index 5fcaf4625a..3b689cddd6 100644 --- a/go.sum +++ b/go.sum @@ -55,8 +55,6 @@ github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/ava-labs/avalanchego v1.10.5 h1:opYyroLzhJPTJw9LlSRks8ItcezerwuGAT0MkVSotBs= -github.com/ava-labs/avalanchego v1.10.5/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -372,6 +370,8 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/joshua-kim/avalanchego v1.4.9-0.20230727193239-84cd2070ba32 h1:F3wR1cDsTjXW8M0mUJVbP+v/PEM1ZShTpwptPcVMu1c= +github.com/joshua-kim/avalanchego v1.4.9-0.20230727193239-84cd2070ba32/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= diff --git a/gossip/bloom.go b/gossip/bloom.go new file mode 100644 index 0000000000..aa8e27581b --- /dev/null +++ b/gossip/bloom.go @@ -0,0 +1,58 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "encoding/binary" + "hash" + + bloomfilter "github.com/holiman/bloomfilter/v2" + + "github.com/ava-labs/avalanchego/ids" +) + +const ( + DefaultBloomM = 8 * 1024 // 8 KiB + DefaultBloomK = 4 + // DefaultBloomMaxFilledRatio is the max ratio of filled slots in the bloom + // filter before we reset it to avoid too many collisions. + DefaultBloomMaxFilledRatio = 0.75 +) + +var _ hash.Hash64 = (*hasher)(nil) + +// ResetBloomFilterIfNeeded resets a bloom filter if it breaches a ratio of +// filled elements. Returns true if the bloom filter was reset. +func ResetBloomFilterIfNeeded( + bloomFilter **bloomfilter.Filter, + maxFilledRatio float64, +) bool { + if (*bloomFilter).PreciseFilledRatio() < maxFilledRatio { + return false + } + + // it's not possible for this to error assuming that the original + // bloom filter's parameters were valid + fresh, _ := bloomfilter.New((*bloomFilter).M(), (*bloomFilter).K()) + *bloomFilter = fresh + + return true +} + +func NewHasher(id ids.ID) hash.Hash64 { + return hasher{ID: id} +} + +type hasher struct { + hash.Hash64 + ID ids.ID +} + +func (h hasher) Sum64() uint64 { + return binary.BigEndian.Uint64(h.ID[:]) +} + +func (h hasher) Size() int { + return 8 +} diff --git a/gossip/bloom_test.go b/gossip/bloom_test.go new file mode 100644 index 0000000000..6f6dbce200 --- /dev/null +++ b/gossip/bloom_test.go @@ -0,0 +1,62 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "testing" + + "github.com/ava-labs/avalanchego/ids" + bloomfilter "github.com/holiman/bloomfilter/v2" + "github.com/stretchr/testify/require" +) + +func TestBloomFilterRefresh(t *testing.T) { + tests := []struct { + name string + refreshRatio float64 + add []ids.ID + expected []ids.ID + }{ + { + name: "no refresh", + refreshRatio: 1, + add: []ids.ID{ + {0}, + }, + expected: []ids.ID{ + {0}, + }, + }, + { + name: "refresh", + refreshRatio: 0.1, + add: []ids.ID{ + {0}, + {1}, + }, + expected: []ids.ID{ + {1}, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + require := require.New(t) + b, err := bloomfilter.New(10, 1) + require.NoError(err) + + for _, item := range tt.add { + ResetBloomFilterIfNeeded(&b, tt.refreshRatio) + b.Add(hasher{ID: item}) + } + + require.Equal(uint64(len(tt.expected)), b.N()) + + for _, expected := range tt.expected { + require.True(b.Contains(hasher{ID: expected})) + } + }) + } +} diff --git a/gossip/gossip.go b/gossip/gossip.go new file mode 100644 index 0000000000..00ce76820c --- /dev/null +++ b/gossip/gossip.go @@ -0,0 +1,120 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "context" + "sync" + "time" + + "github.com/ethereum/go-ethereum/log" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/x/sdk/p2p" +) + +type TxConstraint[T any] interface { + *T + Tx +} + +func NewGossiper[T any, U TxConstraint[T]]( + mempool Mempool[U], + client *p2p.Client, + codec codec.Manager, + codecVersion uint16, + gossipSize int, + frequency time.Duration, +) *Gossiper[T, U] { + return &Gossiper[T, U]{ + mempool: mempool, + client: client, + codec: codec, + codecVersion: codecVersion, + gossipSize: gossipSize, + frequency: frequency, + } +} + +type Gossiper[T any, U TxConstraint[T]] struct { + mempool Mempool[U] + client *p2p.Client + codec codec.Manager + codecVersion uint16 + gossipSize int + frequency time.Duration +} + +func (g *Gossiper[T, U]) Pull( + shutdownChan chan struct{}, + shutdownWg *sync.WaitGroup, +) { + gossipTicker := time.NewTicker(g.frequency) + defer func() { + gossipTicker.Stop() + shutdownWg.Done() + }() + + for { + select { + case <-gossipTicker.C: + bloom, err := g.mempool.GetBloomFilter() + if err != nil { + log.Warn("failed to marshal bloom filter", "error", err) + continue + } + + request := PullTxsRequest{ + BloomFilter: bloom, + } + msgBytes, err := g.codec.Marshal(g.codecVersion, request) + if err != nil { + log.Warn("failed to marshal gossip message", "error", err) + continue + } + + onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { + if err != nil { + log.Warn("failed to pull txs", "nodeID", nodeID, "error", err) + return + } + + response := PullTxsResponse{} + if _, err := g.codec.Unmarshal(responseBytes, &response); err != nil { + log.Warn("failed to unmarshal txs", "error", err) + return + } + + for _, txBytes := range response.Txs { + tx := U(new(T)) + if err := tx.Unmarshal(txBytes); err != nil { + log.Debug("failed to unmarshal transaction", "error", err, "nodeID", nodeID) + continue + } + + ok, err := g.mempool.AddTx(tx, true) + if err != nil { + log.Debug("failed to add transaction to the mempool", "error", err, "nodeID", nodeID, "id", tx.ID()) + continue + } + if !ok { + log.Debug("failed to add transaction to the mempool", "error", err, "nodeID", nodeID, "id", tx.ID()) + continue + } + } + } + + for i := 0; i < g.gossipSize; i++ { + if err := g.client.AppRequestAny(context.TODO(), msgBytes, onResponse); err != nil { + log.Warn("failed to gossip txs", "error", err) + continue + } + } + case <-shutdownChan: + log.Debug("shutting down tx gossip") + return + } + } +} diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go new file mode 100644 index 0000000000..51667872ff --- /dev/null +++ b/gossip/gossip_test.go @@ -0,0 +1,29 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "sync" + "testing" + "time" +) + +func TestPullGossiperShutdown(t *testing.T) { + puller := NewGossiper[testTx, *testTx]( + nil, + nil, + nil, + 0, + 0, + time.Hour, + ) + done := make(chan struct{}) + wg := &sync.WaitGroup{} + + wg.Add(1) + go puller.Pull(done, wg) + + close(done) + wg.Wait() +} diff --git a/gossip/handler.go b/gossip/handler.go new file mode 100644 index 0000000000..91c7f44ca0 --- /dev/null +++ b/gossip/handler.go @@ -0,0 +1,70 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "context" + "time" + + bloomfilter "github.com/holiman/bloomfilter/v2" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/x/sdk/p2p" +) + +func NewHandler[T any, U TxConstraint[T]](mempool Mempool[U], codec codec.Manager, codecVersion uint16) p2p.Handler { + return &Handler[T, U]{ + mempool: mempool, + codec: codec, + codecVersion: codecVersion, + } +} + +type Handler[T any, U TxConstraint[T]] struct { + mempool Mempool[U] + codec codec.Manager + codecVersion uint16 +} + +func (h Handler[T, U]) AppGossip(context.Context, ids.NodeID, []byte) error { + return nil +} + +func (h Handler[T, U]) AppRequest(_ context.Context, nodeID ids.NodeID, _ uint32, _ time.Time, requestBytes []byte) ([]byte, error) { + request := PullTxsRequest{} + if _, err := h.codec.Unmarshal(requestBytes, &request); err != nil { + return nil, err + } + peerFilter := &bloomfilter.Filter{} + if err := peerFilter.UnmarshalBinary(request.BloomFilter); err != nil { + return nil, err + } + + unknownTxs := h.mempool.GetTxs(func(tx U) bool { + return !peerFilter.Contains(NewHasher(tx.ID())) + }) + txs := make([][]byte, 0, len(unknownTxs)) + for _, tx := range unknownTxs { + bytes, err := tx.Marshal() + if err != nil { + return nil, err + } + txs = append(txs, bytes) + } + + response := PullTxsResponse{ + Txs: txs, + } + responseBytes, err := h.codec.Marshal(h.codecVersion, response) + if err != nil { + return nil, err + } + + return responseBytes, nil +} + +func (Handler[T, U]) CrossChainAppRequest(context.Context, ids.ID, uint32, time.Time, []byte) ([]byte, error) { + return nil, nil +} diff --git a/gossip/mempool.go b/gossip/mempool.go new file mode 100644 index 0000000000..d13b4747b4 --- /dev/null +++ b/gossip/mempool.go @@ -0,0 +1,29 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "github.com/ava-labs/avalanchego/ids" +) + +// Tx is a transaction that can be gossiped across the network. +type Tx interface { + // ID returns the unique id of this transaction + ID() ids.ID + // Marshal returns the byte representation of this transaction + Marshal() ([]byte, error) + // Unmarshal deserializes the provided bytes in-place + Unmarshal(b []byte) error +} + +// Mempool holds pending transactions +type Mempool[T Tx] interface { + // AddTx adds a transaction to the mempool + AddTx(tx T, local bool) (bool, error) + // GetTxs returns transactions that match the provided filter function + GetTxs(filter func(tx T) bool) []T + // GetBloomFilter returns a bloom filter representing the transactions in + // the mempool + GetBloomFilter() ([]byte, error) +} diff --git a/gossip/message.go b/gossip/message.go new file mode 100644 index 0000000000..8883ce9417 --- /dev/null +++ b/gossip/message.go @@ -0,0 +1,12 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +type PullTxsRequest struct { + BloomFilter []byte `serialize:"true"` +} + +type PullTxsResponse struct { + Txs [][]byte `serialize:"true"` +} diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go new file mode 100644 index 0000000000..374da68b35 --- /dev/null +++ b/gossip/test_gossip.go @@ -0,0 +1,81 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "sync" + + bloomfilter "github.com/holiman/bloomfilter/v2" + + "github.com/ava-labs/avalanchego/ids" +) + +var ( + _ Tx = (*testTx)(nil) + _ Mempool[*testTx] = (*testMempool)(nil) +) + +type testTx struct { + id ids.ID +} + +func (t *testTx) ID() ids.ID { + return t.id +} + +func (t *testTx) Marshal() ([]byte, error) { + return t.id[:], nil +} + +func (t *testTx) Unmarshal(b []byte) error { + for i := 0; i < 32 || i < len(b); i++ { + t.id[i] = b[i] + } + + return nil +} + +type testMempool struct { + mempool []*testTx + lock sync.Mutex +} + +func (t *testMempool) AddTx(tx *testTx, _ bool) (bool, error) { + t.lock.Lock() + defer t.lock.Unlock() + + t.mempool = append(t.mempool, tx) + return true, nil +} + +func (t *testMempool) GetTxs(filter func(tx *testTx) bool) []*testTx { + t.lock.Lock() + defer t.lock.Unlock() + + result := make([]*testTx, 0) + for _, tx := range t.mempool { + if !filter(tx) { + continue + } + result = append(result, tx) + } + + return result +} + +func (t *testMempool) GetBloomFilter() ([]byte, error) { + t.lock.Lock() + defer t.lock.Unlock() + + bloom, err := bloomfilter.New(DefaultBloomM, DefaultBloomK) + if err != nil { + return nil, err + } + + for _, tx := range t.mempool { + bloom.Add(hasher{ID: tx.ID()}) + } + + return bloom.MarshalBinary() +} diff --git a/peer/network.go b/peer/network.go index 8c658a402f..0311b77521 100644 --- a/peer/network.go +++ b/peer/network.go @@ -21,6 +21,7 @@ import ( "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/avalanchego/version" + "github.com/ava-labs/avalanchego/x/sdk/p2p" "github.com/ava-labs/coreth/peer/stats" "github.com/ava-labs/coreth/plugin/evm/message" @@ -87,23 +88,25 @@ type network struct { outstandingRequestHandlers map[uint32]message.ResponseHandler // maps avalanchego requestID => message.ResponseHandler activeAppRequests *semaphore.Weighted // controls maximum number of active outbound requests activeCrossChainRequests *semaphore.Weighted // controls maximum number of active outbound cross chain requests - appSender common.AppSender // avalanchego AppSender for sending messages - codec codec.Manager // Codec used for parsing messages - crossChainCodec codec.Manager // Codec used for parsing cross chain messages - appRequestHandler message.RequestHandler // maps request type => handler - crossChainRequestHandler message.CrossChainRequestHandler // maps cross chain request type => handler - gossipHandler message.GossipHandler // maps gossip type => handler - peers *peerTracker // tracking of peers & bandwidth - appStats stats.RequestHandlerStats // Provide request handler metrics - crossChainStats stats.RequestHandlerStats // Provide cross chain request handler metrics + router *p2p.Router + appSender common.AppSender // avalanchego AppSender for sending messages + codec codec.Manager // Codec used for parsing messages + crossChainCodec codec.Manager // Codec used for parsing cross chain messages + appRequestHandler message.RequestHandler // maps request type => handler + crossChainRequestHandler message.CrossChainRequestHandler // maps cross chain request type => handler + gossipHandler message.GossipHandler // maps gossip type => handler + peers *peerTracker // tracking of peers & bandwidth + appStats stats.RequestHandlerStats // Provide request handler metrics + crossChainStats stats.RequestHandlerStats // Provide cross chain request handler metrics // Set to true when Shutdown is called, after which all operations on this // struct are no-ops. closed utils.Atomic[bool] } -func NewNetwork(appSender common.AppSender, codec codec.Manager, crossChainCodec codec.Manager, self ids.NodeID, maxActiveAppRequests int64, maxActiveCrossChainRequests int64) Network { +func NewNetwork(router *p2p.Router, appSender common.AppSender, codec codec.Manager, crossChainCodec codec.Manager, self ids.NodeID, maxActiveAppRequests int64, maxActiveCrossChainRequests int64) Network { return &network{ + router: router, appSender: appSender, codec: codec, crossChainCodec: crossChainCodec, @@ -333,6 +336,11 @@ func (n *network) AppRequest(ctx context.Context, nodeID ids.NodeID, requestID u log.Debug("received AppRequest from node", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request)) + if err := n.router.AppRequest(ctx, nodeID, requestID, deadline, request); err == nil { + return nil + } + // if we errored, it's possible that this is a non-sdk request + var req message.Request if _, err := n.codec.Unmarshal(request, &req); err != nil { log.Debug("failed to unmarshal app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) @@ -366,7 +374,7 @@ func (n *network) AppRequest(ctx context.Context, nodeID ids.NodeID, requestID u // Error returned by this function is expected to be treated as fatal by the engine // If [requestID] is not known, this function will emit a log and return a nil error. // If the response handler returns an error it is propagated as a fatal error. -func (n *network) AppResponse(_ context.Context, nodeID ids.NodeID, requestID uint32, response []byte) error { +func (n *network) AppResponse(ctx context.Context, nodeID ids.NodeID, requestID uint32, response []byte) error { n.lock.Lock() defer n.lock.Unlock() @@ -376,6 +384,11 @@ func (n *network) AppResponse(_ context.Context, nodeID ids.NodeID, requestID ui log.Debug("received AppResponse from peer", "nodeID", nodeID, "requestID", requestID) + if err := n.router.AppResponse(ctx, nodeID, requestID, response); err == nil { + return nil + } + // if we errored, it's possible that this is a non-sdk response + handler, exists := n.markRequestFulfilled(requestID) if !exists { // Should never happen since the engine should be managing outstanding requests @@ -482,7 +495,7 @@ func (n *network) AppGossip(_ context.Context, nodeID ids.NodeID, gossipBytes [] } // Connected adds the given nodeID to the peer list so that it can receive messages -func (n *network) Connected(_ context.Context, nodeID ids.NodeID, nodeVersion *version.Application) error { +func (n *network) Connected(ctx context.Context, nodeID ids.NodeID, nodeVersion *version.Application) error { log.Debug("adding new peer", "nodeID", nodeID) n.lock.Lock() @@ -497,12 +510,15 @@ func (n *network) Connected(_ context.Context, nodeID ids.NodeID, nodeVersion *v return nil } + if err := n.router.Connected(ctx, nodeID, nodeVersion); err != nil { + return err + } n.peers.Connected(nodeID, nodeVersion) return nil } // Disconnected removes given [nodeID] from the peer list -func (n *network) Disconnected(_ context.Context, nodeID ids.NodeID) error { +func (n *network) Disconnected(ctx context.Context, nodeID ids.NodeID) error { log.Debug("disconnecting peer", "nodeID", nodeID) n.lock.Lock() defer n.lock.Unlock() @@ -511,6 +527,9 @@ func (n *network) Disconnected(_ context.Context, nodeID ids.NodeID) error { return nil } + if err := n.router.Disconnected(ctx, nodeID); err != nil { + return err + } n.peers.Disconnected(nodeID) return nil } diff --git a/peer/network_test.go b/peer/network_test.go index 3e1c32f492..023218f894 100644 --- a/peer/network_test.go +++ b/peer/network_test.go @@ -14,6 +14,8 @@ import ( "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/avalanchego/x/sdk/p2p" + ethcommon "github.com/ethereum/go-ethereum/common" "github.com/ava-labs/coreth/plugin/evm/message" @@ -53,7 +55,7 @@ var ( func TestNetworkDoesNotConnectToItself(t *testing.T) { selfNodeID := ids.GenerateTestNodeID() - n := NewNetwork(nil, nil, nil, selfNodeID, 1, 1) + n := NewNetwork(p2p.NewRouter(), nil, nil, nil, selfNodeID, 1, 1) assert.NoError(t, n.Connected(context.Background(), selfNodeID, defaultPeerVersion)) assert.EqualValues(t, 0, n.Size()) } @@ -89,7 +91,7 @@ func TestRequestAnyRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) net.SetRequestHandler(&HelloGreetingRequestHandler{codec: codecManager}) client := NewNetworkClient(net) nodeID := ids.GenerateTestNodeID() @@ -164,7 +166,7 @@ func TestRequestRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) net.SetRequestHandler(&HelloGreetingRequestHandler{codec: codecManager}) client := NewNetworkClient(net) @@ -244,7 +246,7 @@ func TestAppRequestOnShutdown(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) client := NewNetworkClient(net) nodeID := ids.GenerateTestNodeID() require.NoError(t, net.Connected(context.Background(), nodeID, defaultPeerVersion)) @@ -293,7 +295,7 @@ func TestRequestMinVersion(t *testing.T) { } // passing nil as codec works because the net.AppRequest is never called - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 16) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 16) client := NewNetworkClient(net) requestMessage := TestMessage{Message: "this is a request"} requestBytes, err := message.RequestToBytes(codecManager, requestMessage) @@ -356,7 +358,7 @@ func TestOnRequestHonoursDeadline(t *testing.T) { processingDuration: 500 * time.Millisecond, } - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetRequestHandler(requestHandler) nodeID := ids.GenerateTestNodeID() @@ -396,7 +398,7 @@ func TestGossip(t *testing.T) { } gossipHandler := &testGossipHandler{} - clientNetwork = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(gossipHandler) assert.NoError(t, clientNetwork.Connected(context.Background(), nodeID, defaultPeerVersion)) @@ -423,7 +425,7 @@ func TestHandleInvalidMessages(t *testing.T) { requestID := uint32(1) sender := testAppSender{} - clientNetwork := NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork := NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) clientNetwork.SetRequestHandler(&testRequestHandler{}) @@ -473,7 +475,7 @@ func TestNetworkPropagatesRequestHandlerError(t *testing.T) { requestID := uint32(1) sender := testAppSender{} - clientNetwork := NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork := NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) clientNetwork.SetRequestHandler(&testRequestHandler{err: errors.New("fail")}) // Return an error from the request handler @@ -513,7 +515,7 @@ func TestCrossChainAppRequest(t *testing.T) { }, } - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetCrossChainRequestHandler(&testCrossChainHandler{codec: crossChainCodecManager}) client := NewNetworkClient(net) @@ -568,7 +570,7 @@ func TestCrossChainRequestRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, TestMessage{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetCrossChainRequestHandler(&testCrossChainHandler{codec: crossChainCodecManager}) client := NewNetworkClient(net) @@ -628,7 +630,7 @@ func TestCrossChainRequestOnShutdown(t *testing.T) { } codecManager := buildCodec(t, TestMessage{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) client := NewNetworkClient(net) exampleCrossChainRequest := ExampleCrossChainRequest{ diff --git a/plugin/evm/codec.go b/plugin/evm/codec.go index b9f8de449f..e097abfeb4 100644 --- a/plugin/evm/codec.go +++ b/plugin/evm/codec.go @@ -21,6 +21,7 @@ func init() { errs := wrappers.Errs{} errs.Add( + c.RegisterType(&TestUnsignedTx{}), c.RegisterType(&UnsignedImportTx{}), c.RegisterType(&UnsignedExportTx{}), ) diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go new file mode 100644 index 0000000000..2176ed86a4 --- /dev/null +++ b/plugin/evm/gossip_mempool.go @@ -0,0 +1,217 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "fmt" + "sync" + + "github.com/ethereum/go-ethereum/log" + + "github.com/ava-labs/coreth/gossip" + "github.com/ava-labs/coreth/plugin/evm/message" + + bloomfilter "github.com/holiman/bloomfilter/v2" + + "github.com/ava-labs/avalanchego/ids" + + "github.com/ava-labs/coreth/core" + "github.com/ava-labs/coreth/core/txpool" + "github.com/ava-labs/coreth/core/types" +) + +var ( + _ gossip.Mempool[*GossipAtomicTx] = (*GossipAtomicMempool)(nil) + _ gossip.Tx = (*GossipAtomicTx)(nil) + + _ gossip.Mempool[*GossipEthTx] = (*GossipEthTxPool)(nil) + _ gossip.Tx = (*GossipEthTx)(nil) +) + +func NewGossipAtomicMempool(Mempool *Mempool) (*GossipAtomicMempool, error) { + bloom, err := bloomfilter.New(gossip.DefaultBloomM, gossip.DefaultBloomK) + if err != nil { + return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) + } + + return &GossipAtomicMempool{ + mempool: Mempool, + bloom: bloom, + }, nil +} + +type GossipAtomicMempool struct { + mempool *Mempool + bloom *bloomfilter.Filter + lock sync.RWMutex +} + +func (g *GossipAtomicMempool) AddTx(tx *GossipAtomicTx, local bool) (bool, error) { + ok, err := g.mempool.AddTx(tx.Tx) + if err != nil { + if !local { + // unlike local txs, invalid remote txs are recorded as discarded + // so that they won't be requested again + txID := tx.ID() + g.mempool.discardedTxs.Put(tx.ID(), tx.Tx) + log.Debug("failed to issue remote tx to mempool", + "txID", txID, + "err", err, + ) + } + return false, err + } + + if !ok { + return false, nil + } + + g.lock.Lock() + defer g.lock.Unlock() + + g.bloom.Add(gossip.NewHasher(tx.ID())) + gossip.ResetBloomFilterIfNeeded(&g.bloom, gossip.DefaultBloomMaxFilledRatio) + + return true, nil +} + +func (g *GossipAtomicMempool) GetTxs(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { + f := func(tx *Tx) bool { + return filter(&GossipAtomicTx{ + Tx: tx, + }) + } + txs := g.mempool.GetTxs(f) + gossipTxs := make([]*GossipAtomicTx, 0, len(txs)) + for _, tx := range txs { + gossipTxs = append(gossipTxs, &GossipAtomicTx{ + Tx: tx, + }) + } + + return gossipTxs +} + +func (g *GossipAtomicMempool) GetBloomFilter() ([]byte, error) { + g.lock.RLock() + defer g.lock.RUnlock() + + return g.bloom.MarshalBinary() +} + +type GossipAtomicTx struct { + Tx *Tx +} + +func (tx *GossipAtomicTx) ID() ids.ID { + return tx.Tx.ID() +} + +func (tx *GossipAtomicTx) Marshal() ([]byte, error) { + return Codec.Marshal(message.Version, tx.Tx) +} + +func (tx *GossipAtomicTx) Unmarshal(bytes []byte) error { + tx.Tx = &Tx{} + _, err := Codec.Unmarshal(bytes, tx.Tx) + + return err +} + +func NewGossipEthTxPool(mempool *txpool.TxPool) (*GossipEthTxPool, error) { + bloom, err := bloomfilter.New(gossip.DefaultBloomM, gossip.DefaultBloomK) + if err != nil { + return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) + } + + g := &GossipEthTxPool{ + mempool: mempool, + pendingTxs: make(chan core.NewTxsEvent), + bloom: bloom, + } + return g, nil +} + +type GossipEthTxPool struct { + mempool *txpool.TxPool + pendingTxs chan core.NewTxsEvent + + bloom *bloomfilter.Filter + lock sync.RWMutex +} + +func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync.WaitGroup) { + defer shutdownWg.Done() + g.mempool.SubscribeNewTxsEvent(g.pendingTxs) + + for { + select { + case <-shutdownChan: + log.Debug("shutting down subscription") + return + case tx := <-g.pendingTxs: + g.lock.Lock() + for _, tx := range tx.Txs { + g.bloom.Add(gossip.NewHasher(ids.ID(tx.Hash()))) + gossip.ResetBloomFilterIfNeeded(&g.bloom, gossip.DefaultBloomMaxFilledRatio) + } + g.lock.Unlock() + } + } +} + +// AddTx enqueues the transaction to the mempool. Subscribe should be called +// to receive an event if tx is actually added to the mempool or not. +func (g *GossipEthTxPool) AddTx(tx *GossipEthTx, _ bool) (bool, error) { + err := g.mempool.AddRemotes([]*types.Transaction{tx.Tx})[0] + if err != nil { + return false, err + } + + return true, nil +} + +func (g *GossipEthTxPool) GetTxs(filter func(tx *GossipEthTx) bool) []*GossipEthTx { + pending, _ := g.mempool.Content() + result := make([]*GossipEthTx, 0) + + for _, txs := range pending { + for _, tx := range txs { + gossipTx := &GossipEthTx{Tx: tx} + if !filter(gossipTx) { + continue + } + + result = append(result, gossipTx) + } + } + + return result +} + +func (g *GossipEthTxPool) GetBloomFilter() ([]byte, error) { + g.lock.RLock() + defer g.lock.RUnlock() + + return g.bloom.MarshalBinary() +} + +type GossipEthTx struct { + Tx *types.Transaction +} + +func (tx *GossipEthTx) ID() ids.ID { + return ids.ID(tx.Tx.Hash()) +} + +func (tx *GossipEthTx) Marshal() ([]byte, error) { + return tx.Tx.MarshalBinary() +} + +func (tx *GossipEthTx) Unmarshal(bytes []byte) error { + tx.Tx = &types.Transaction{} + err := tx.Tx.UnmarshalBinary(bytes) + + return err +} diff --git a/plugin/evm/gossip_mempool_test.go b/plugin/evm/gossip_mempool_test.go new file mode 100644 index 0000000000..eac1267b56 --- /dev/null +++ b/plugin/evm/gossip_mempool_test.go @@ -0,0 +1,194 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/ava-labs/avalanchego/ids" +) + +func TestAtomicMempoolAddTx(t *testing.T) { + txs := []*GossipAtomicTx{ + { + Tx: &Tx{ + UnsignedAtomicTx: &TestUnsignedTx{ + IDV: ids.GenerateTestID(), + }, + }, + }, + { + Tx: &Tx{ + UnsignedAtomicTx: &TestUnsignedTx{ + IDV: ids.GenerateTestID(), + }, + }, + }, + } + + tests := []struct { + name string + add []*GossipAtomicTx + filter func(tx *GossipAtomicTx) bool + expected []*GossipAtomicTx + }{ + { + name: "empty", + }, + { + name: "filter matches nothing", + add: txs, + filter: func(*GossipAtomicTx) bool { + return false + }, + expected: nil, + }, + { + name: "filter matches all", + add: txs, + filter: func(*GossipAtomicTx) bool { + return true + }, + expected: txs, + }, + { + name: "filter matches subset", + add: txs, + filter: func(tx *GossipAtomicTx) bool { + return tx.Tx == txs[0].Tx + }, + expected: txs[:1], + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + require := require.New(t) + + m := NewMempool(ids.Empty, 10) + mempool, err := NewGossipAtomicMempool(m) + require.NoError(err) + + for _, add := range tt.add { + ok, err := mempool.AddTx(add, false) + require.True(ok) + require.NoError(err) + } + + txs := mempool.GetTxs(tt.filter) + require.Len(txs, len(tt.expected)) + + for _, expected := range tt.expected { + require.Contains(txs, expected) + } + }) + } +} + +// +// func TestEthTxPoolAddTx(t *testing.T) { +// key, err := crypto.GenerateKey() +// require.NoError(t, err) +// +// txs := []*types.Transaction{ +// types.NewTx(&types.AccessListTx{ +// ChainID: params.TestChainConfig.ChainID, +// Nonce: 0, +// GasPrice: big.NewInt(1), +// Gas: 100_000, +// To: &common.Address{}, +// Value: big.NewInt(0), +// Data: []byte{}, +// V: big.NewInt(32), +// R: big.NewInt(10), +// S: big.NewInt(11), +// }), +// types.NewTx(&types.AccessListTx{ +// ChainID: params.TestChainConfig.ChainID, +// Nonce: 1, +// GasPrice: big.NewInt(1), +// Gas: 100_000, +// To: &common.Address{}, +// Value: big.NewInt(0), +// Data: []byte{}, +// V: big.NewInt(32), +// R: big.NewInt(10), +// S: big.NewInt(11), +// }), +// } +// +// signedTxs := make([]*GossipEthTx, 0, len(txs)) +// for _, tx := range txs { +// signedTx, err := types.SignTx(tx, types.LatestSigner(params.TestChainConfig), key) +// require.NoError(t, err) +// signedTxs = append(signedTxs, &GossipEthTx{ +// Tx: signedTx, +// }) +// } +// +// tests := []struct { +// name string +// add []*GossipEthTx +// filter func(tx *GossipEthTx) bool +// expected []*GossipEthTx +// }{ +// { +// name: "empty", +// }, +// { +// name: "filter matches nothing", +// add: signedTxs, +// filter: func(tx *GossipEthTx) bool { +// return false +// }, +// expected: nil, +// }, +// { +// name: "filter matches all", +// add: signedTxs, +// filter: func(*GossipEthTx) bool { +// return true +// }, +// expected: signedTxs, +// }, +// { +// name: "filter matches subset", +// add: signedTxs, +// filter: func(tx *GossipEthTx) bool { +// return tx.Tx == signedTxs[0].Tx +// }, +// expected: signedTxs[:1], +// }, +// } +// +// for _, tt := range tests { +// t.Run(tt.name, func(t *testing.T) { +// require := require.New(t) +// +// config := txpool.DefaultConfig +// config.Journal = "" +// m := txpool.NewTestTxPool(config, params.TestChainConfig, 100_000_000, key.PublicKey) +// pending := make(chan core.NewTxsEvent) +// m.SubscribeNewTxsEvent(pending) +// mempool, err := NewGossipEthTxPool(m) +// require.NoError(err) +// +// for _, add := range tt.add { +// ok, err := mempool.AddTx(add, false) +// require.True(ok) +// require.NoError(err) +// <-pending +// } +// +// txs := mempool.GetTxs(tt.filter) +// require.Len(txs, len(tt.expected)) +// +// for _, expected := range tt.expected { +// require.Contains(txs, expected) +// } +// }) +// } +// } diff --git a/plugin/evm/gossiper.go b/plugin/evm/gossiper.go index 2ced1de529..a541ddb195 100644 --- a/plugin/evm/gossiper.go +++ b/plugin/evm/gossiper.go @@ -43,6 +43,8 @@ const ( minGossipBatchInterval = 50 * time.Millisecond ) +var _ message.GossipHandler = (*GossipHandler)(nil) + // Gossiper handles outgoing gossip of transactions type Gossiper interface { // GossipAtomicTxs sends AppGossip message containing the given [txs] diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index 73e3ce17b3..b7ce36872e 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -10,8 +10,12 @@ import ( "testing" "time" + bloomfilter "github.com/holiman/bloomfilter/v2" + "github.com/stretchr/testify/require" + "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/coreth/gossip" "github.com/stretchr/testify/assert" @@ -26,6 +30,7 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { defer func() { assert.NoError(vm.Shutdown(context.Background())) }() + assert.NoError(vm.Connected(context.Background(), ids.GenerateTestNodeID(), nil)) // Create conflicting transactions importTxs := createImportTxOptions(t, vm, sharedMemory) @@ -56,12 +61,29 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { return nil } + addedToBloomFilter := false + sender.SendAppRequestF = func(ctx context.Context, _ set.Set[ids.NodeID], _ uint32, bytes []byte) error { + bytes = bytes[1:] // first byte is an identifier + msg := gossip.PullTxsRequest{} + _, err := vm.networkCodec.Unmarshal(bytes, &msg) + require.NoError(t, err) + + bloom := &bloomfilter.Filter{} + require.NoError(t, bloom.UnmarshalBinary(msg.BloomFilter)) + if !bloom.Contains(gossip.NewHasher(tx.ID())) { + return nil + } + addedToBloomFilter = true + return nil + } + // Optimistically gossip raw tx assert.NoError(vm.issueTx(tx, true /*=local*/)) - time.Sleep(500 * time.Millisecond) + time.Sleep(1 * time.Second) gossipedLock.Lock() assert.Equal(1, gossiped) gossipedLock.Unlock() + assert.True(addedToBloomFilter) // Test hash on retry assert.NoError(vm.gossiper.GossipAtomicTxs([]*Tx{tx})) diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index 25b67298f4..6cdc95585c 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -8,10 +8,11 @@ import ( "fmt" "sync" + "github.com/ethereum/go-ethereum/log" + "github.com/ava-labs/avalanchego/cache" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/coreth/metrics" - "github.com/ethereum/go-ethereum/log" ) const ( @@ -127,7 +128,7 @@ func (m *Mempool) atomicTxGasPrice(tx *Tx) (uint64, error) { // Add attempts to add [tx] to the mempool and returns an error if // it could not be addeed to the mempool. -func (m *Mempool) AddTx(tx *Tx) error { +func (m *Mempool) AddTx(tx *Tx) (bool, error) { m.lock.Lock() defer m.lock.Unlock() @@ -135,7 +136,7 @@ func (m *Mempool) AddTx(tx *Tx) error { } // forceAddTx forcibly adds a *Tx to the mempool and bypasses all verification. -func (m *Mempool) ForceAddTx(tx *Tx) error { +func (m *Mempool) ForceAddTx(tx *Tx) (bool, error) { m.lock.Lock() defer m.lock.Unlock() @@ -176,31 +177,31 @@ func (m *Mempool) checkConflictTx(tx *Tx) (uint64, ids.ID, []*Tx, error) { // addTx adds [tx] to the mempool. Assumes [m.lock] is held. // If [force], skips conflict checks within the mempool. -func (m *Mempool) addTx(tx *Tx, force bool) error { +func (m *Mempool) addTx(tx *Tx, force bool) (bool, error) { txID := tx.ID() // If [txID] has already been issued or is in the currentTxs map // there's no need to add it. if _, exists := m.issuedTxs[txID]; exists { - return nil + return false, nil } if _, exists := m.currentTxs[txID]; exists { - return nil + return false, nil } if _, exists := m.txHeap.Get(txID); exists { - return nil + return false, nil } utxoSet := tx.InputUTXOs() gasPrice, _ := m.atomicTxGasPrice(tx) highestGasPrice, highestGasPriceConflictTxID, conflictingTxs, err := m.checkConflictTx(tx) if err != nil { - return err + return false, err } if len(conflictingTxs) != 0 && !force { // If [tx] does not have a higher fee than all of its conflicts, // we refuse to issue it to the mempool. if highestGasPrice >= gasPrice { - return fmt.Errorf( + return false, fmt.Errorf( "%w: issued tx (%s) gas price %d <= conflict tx (%s) gas price %d (%d total conflicts in mempool)", errConflictingAtomicTx, txID, @@ -225,7 +226,7 @@ func (m *Mempool) addTx(tx *Tx, force bool) error { // submitted item, discard the submitted item (we prefer items // already in the mempool). if minGasPrice >= gasPrice { - return fmt.Errorf( + return false, fmt.Errorf( "%w currentMin=%d provided=%d", errInsufficientAtomicTxFee, minGasPrice, @@ -237,7 +238,7 @@ func (m *Mempool) addTx(tx *Tx, force bool) error { } else { // This could occur if we have used our entire size allowance on // transactions that are currently processing. - return errTooManyAtomicTx + return false, errTooManyAtomicTx } } @@ -266,7 +267,7 @@ func (m *Mempool) addTx(tx *Tx, force bool) error { // and CancelCurrentTx. m.newTxs = append(m.newTxs, tx) m.addPending() - return nil + return true, nil } // NextTx returns a transaction to be issued from the mempool. @@ -297,6 +298,21 @@ func (m *Mempool) GetPendingTx(txID ids.ID) (*Tx, bool) { return m.txHeap.Get(txID) } +func (m *Mempool) GetTxs(filter func(tx *Tx) bool) []*Tx { + m.lock.RLock() + defer m.lock.RUnlock() + + result := make([]*Tx, 0, len(m.txHeap.maxHeap.items)) + for _, item := range m.txHeap.maxHeap.items { + if !filter(item.tx) { + continue + } + result = append(result, item.tx) + } + + return result +} + // GetTx returns the transaction [txID] if it was issued // by this node and returns whether it was dropped and whether // it exists. diff --git a/plugin/evm/mempool_atomic_gossiping_test.go b/plugin/evm/mempool_atomic_gossiping_test.go index a0f82a8c01..c9094ef374 100644 --- a/plugin/evm/mempool_atomic_gossiping_test.go +++ b/plugin/evm/mempool_atomic_gossiping_test.go @@ -108,13 +108,17 @@ func TestMempoolMaxMempoolSizeHandling(t *testing.T) { // shortcut to simulated almost filled mempool mempool.maxSize = 0 - assert.ErrorIs(mempool.AddTx(tx), errTooManyAtomicTx) + ok, err := mempool.AddTx(tx) + assert.False(ok) + assert.ErrorIs(err, errTooManyAtomicTx) assert.False(mempool.has(tx.ID())) // shortcut to simulated empty mempool mempool.maxSize = defaultMempoolSize - assert.NoError(mempool.AddTx(tx)) + ok, err = mempool.AddTx(tx) + assert.True(ok) + assert.NoError(err) assert.True(mempool.has(tx.ID())) } @@ -193,14 +197,20 @@ func TestMempoolPriorityDrop(t *testing.T) { mempool.maxSize = 1 tx1 := createImportTx(t, vm, ids.ID{1}, params.AvalancheAtomicTxFee) - assert.NoError(mempool.AddTx(tx1)) + ok, err := mempool.AddTx(tx1) + assert.True(ok) + assert.NoError(err) assert.True(mempool.has(tx1.ID())) tx2 := createImportTx(t, vm, ids.ID{2}, params.AvalancheAtomicTxFee) - assert.ErrorIs(mempool.AddTx(tx2), errInsufficientAtomicTxFee) + ok, err = mempool.AddTx(tx2) + assert.False(ok) + assert.ErrorIs(err, errInsufficientAtomicTxFee) assert.True(mempool.has(tx1.ID())) assert.False(mempool.has(tx2.ID())) tx3 := createImportTx(t, vm, ids.ID{3}, 2*params.AvalancheAtomicTxFee) - assert.NoError(mempool.AddTx(tx3)) + ok, err = mempool.AddTx(tx3) + assert.True(ok) + assert.NoError(err) assert.False(mempool.has(tx1.ID())) assert.False(mempool.has(tx2.ID())) assert.True(mempool.has(tx3.ID())) diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index a698e82124..c39a302d18 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -8,6 +8,7 @@ import ( "github.com/ava-labs/avalanchego/codec/linearcodec" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/utils/wrappers" + "github.com/ava-labs/coreth/gossip" ) const ( @@ -29,6 +30,8 @@ func init() { // Gossip types c.RegisterType(AtomicTxGossip{}), c.RegisterType(EthTxsGossip{}), + c.RegisterType(gossip.PullTxsRequest{}), + c.RegisterType(gossip.PullTxsResponse{}), // Types for state sync frontier consensus c.RegisterType(SyncSummary{}), diff --git a/plugin/evm/tx_test.go b/plugin/evm/tx_test.go index 3438f16296..769690d272 100644 --- a/plugin/evm/tx_test.go +++ b/plugin/evm/tx_test.go @@ -150,7 +150,9 @@ func executeTxTest(t *testing.T, test atomicTxTest) { // If this test simulates processing txs during bootstrapping (where some verification is skipped), // initialize the block building goroutines normally initialized in SetState(snow.NormalOps). // This ensures that the VM can build a block correctly during the test. - vm.initBlockBuilding() + if err := vm.initBlockBuilding(); err != nil { + t.Fatal(err) + } } if err := vm.issueTx(tx, true /*=local*/); err != nil { diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 7005537de6..cc3cfc3cf6 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -17,6 +17,7 @@ import ( "time" avalanchegoMetrics "github.com/ava-labs/avalanchego/api/metrics" + "github.com/ava-labs/avalanchego/x/sdk/p2p" "github.com/ava-labs/coreth/consensus/dummy" corethConstants "github.com/ava-labs/coreth/constants" @@ -28,6 +29,7 @@ import ( "github.com/ava-labs/coreth/eth" "github.com/ava-labs/coreth/eth/ethconfig" "github.com/ava-labs/coreth/ethdb" + "github.com/ava-labs/coreth/gossip" corethPrometheus "github.com/ava-labs/coreth/metrics/prometheus" "github.com/ava-labs/coreth/miner" "github.com/ava-labs/coreth/node" @@ -127,6 +129,10 @@ const ( unverifiedCacheSize = 50 targetAtomicTxsSize = 40 * units.KiB + + // How frequently we should attempt to poll other nodes for new transactions + pullTxsFrequency = 500 * time.Millisecond + pullTxsGossipSize = 10 ) // Define the API endpoints for the VM @@ -256,7 +262,10 @@ type VM struct { builder *blockBuilder - gossiper Gossiper + gossiper Gossiper + ethTxGossiper *gossip.Gossiper[GossipEthTx, *GossipEthTx] + atomicTxGossiper *gossip.Gossiper[GossipAtomicTx, *GossipAtomicTx] + gossipAtomicTxPool gossip.Mempool[*GossipAtomicTx] baseCodec codec.Registry codec codec.Manager @@ -274,8 +283,13 @@ type VM struct { peer.Network client peer.NetworkClient + appSender commonEng.AppSender networkCodec codec.Manager + router *p2p.Router + ethTxGossipClient *p2p.Client + atomicTxGossipClient *p2p.Client + // Metrics multiGatherer avalanchegoMetrics.MultiGatherer @@ -498,9 +512,11 @@ func (vm *VM) Initialize( } // initialize peer network + vm.router = p2p.NewRouter() vm.networkCodec = message.Codec - vm.Network = peer.NewNetwork(appSender, vm.networkCodec, message.CrossChainCodec, chainCtx.NodeID, vm.config.MaxOutboundActiveRequests, vm.config.MaxOutboundActiveCrossChainRequests) + vm.Network = peer.NewNetwork(vm.router, appSender, vm.networkCodec, message.CrossChainCodec, chainCtx.NodeID, vm.config.MaxOutboundActiveRequests, vm.config.MaxOutboundActiveCrossChainRequests) vm.client = peer.NewNetworkClient(vm.Network) + vm.appSender = appSender if err := vm.initializeChain(lastAcceptedHash); err != nil { return err @@ -924,7 +940,9 @@ func (vm *VM) SetState(_ context.Context, state snow.State) error { return vm.fx.Bootstrapping() case snow.NormalOp: // Initialize goroutines related to block building once we enter normal operation as there is no need to handle mempool gossip before this point. - vm.initBlockBuilding() + if err := vm.initBlockBuilding(); err != nil { + return fmt.Errorf("failed to initialize block building: %w", err) + } vm.bootstrapped = true return vm.fx.Bootstrapped() default: @@ -933,13 +951,70 @@ func (vm *VM) SetState(_ context.Context, state snow.State) error { } // initBlockBuilding starts goroutines to manage block building -func (vm *VM) initBlockBuilding() { +func (vm *VM) initBlockBuilding() error { // NOTE: gossip network must be initialized first otherwise ETH tx gossip will not work. gossipStats := NewGossipStats() vm.gossiper = vm.createGossiper(gossipStats) vm.builder = vm.NewBlockBuilder(vm.toEngine) vm.builder.awaitSubmittedTxs() - vm.Network.SetGossipHandler(NewGossipHandler(vm, gossipStats)) + + ethTxPool, err := NewGossipEthTxPool(vm.txPool) + if err != nil { + return err + } + vm.shutdownWg.Add(1) + go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) + + atomicMempool, err := NewGossipAtomicMempool(vm.mempool) + if err != nil { + return err + } + + vm.Network.SetGossipHandler(NewGossipHandler( + vm, + gossipStats, + )) + + // TODO needed? + gossipEthTxPool := gossip.Mempool[*GossipEthTx](ethTxPool) + ethTxGossipHandler := gossip.NewHandler[GossipEthTx, *GossipEthTx](gossipEthTxPool, vm.codec, message.Version) + ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler, vm.appSender) + if err != nil { + return err + } + vm.ethTxGossipClient = ethTxGossipClient + + vm.gossipAtomicTxPool = gossip.Mempool[*GossipAtomicTx](atomicMempool) + atomicTxGossipHandler := gossip.NewHandler[GossipAtomicTx, *GossipAtomicTx](vm.gossipAtomicTxPool, vm.codec, message.Version) + atomicTxGossipClient, err := vm.router.RegisterAppProtocol(0x1, atomicTxGossipHandler, vm.appSender) + if err != nil { + return err + } + vm.atomicTxGossipClient = atomicTxGossipClient + + vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx, *GossipEthTx]( + gossipEthTxPool, + vm.ethTxGossipClient, + vm.networkCodec, + message.Version, + pullTxsGossipSize, + pullTxsFrequency, + ) + vm.shutdownWg.Add(1) + go vm.ethTxGossiper.Pull(vm.shutdownChan, &vm.shutdownWg) + + vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( + vm.gossipAtomicTxPool, + vm.atomicTxGossipClient, + vm.networkCodec, + message.Version, + pullTxsGossipSize, + pullTxsFrequency, + ) + vm.shutdownWg.Add(1) + go vm.atomicTxGossiper.Pull(vm.shutdownChan, &vm.shutdownWg) + + return nil } // setAppRequestHandlers sets the request handlers for the VM to serve state sync @@ -1309,22 +1384,11 @@ func (vm *VM) issueTx(tx *Tx, local bool) error { } return err } - // add to mempool and possibly re-gossip - if err := vm.mempool.AddTx(tx); err != nil { - if !local { - // unlike local txs, invalid remote txs are recorded as discarded - // so that they won't be requested again - txID := tx.ID() - vm.mempool.discardedTxs.Put(tx.ID(), tx) - log.Debug("failed to issue remote tx to mempool", - "txID", txID, - "err", err, - ) - return nil - } + + if _, err := vm.gossipAtomicTxPool.AddTx(&GossipAtomicTx{tx}, local); err != nil { return err } - // NOTE: Gossiping of the issued [Tx] is handled in [AddTx] + return nil } @@ -1762,3 +1826,7 @@ func (vm *VM) stateSyncEnabled(lastAcceptedHeight uint64) bool { // enable state sync by default if the chain is empty. return lastAcceptedHeight == 0 } + +func (vm *VM) IssueTx(tx *Tx, b bool) error { + return vm.issueTx(tx, b) +} diff --git a/plugin/evm/vm_test.go b/plugin/evm/vm_test.go index bd3792df26..272197b08e 100644 --- a/plugin/evm/vm_test.go +++ b/plugin/evm/vm_test.go @@ -1043,7 +1043,7 @@ func testConflictingImportTxs(t *testing.T, genesis string) { t.Fatal("Expected issueTx to fail due to conflicting transaction") } // Force issue transaction directly to the mempool - if err := vm.mempool.ForceAddTx(tx); err != nil { + if _, err := vm.mempool.ForceAddTx(tx); err != nil { t.Fatal(err) } <-issuer @@ -1659,7 +1659,7 @@ func TestConflictingTransitiveAncestryWithGap(t *testing.T) { t.Fatalf("Should not have been able to issue import tx with conflict") } // Force issue transaction directly into the mempool - if err := vm.mempool.ForceAddTx(importTx0B); err != nil { + if _, err := vm.mempool.ForceAddTx(importTx0B); err != nil { t.Fatal(err) } <-issuer @@ -3278,12 +3278,12 @@ func TestAtomicTxFailsEVMStateTransferBuildBlock(t *testing.T) { t.Fatal("Should have failed to issue due to an invalid export tx") } - if err := vm.mempool.AddTx(exportTx2); err == nil { + if _, err := vm.mempool.AddTx(exportTx2); err == nil { t.Fatal("Should have failed to add because conflicting") } // Manually add transaction to mempool to bypass validation - if err := vm.mempool.ForceAddTx(exportTx2); err != nil { + if _, err := vm.mempool.ForceAddTx(exportTx2); err != nil { t.Fatal(err) } <-issuer @@ -3341,7 +3341,7 @@ func TestBuildInvalidBlockHead(t *testing.T) { t.Fatal("Should have failed to issue invalid transaction") } // Force issue the transaction directly to the mempool - if err := vm.mempool.AddTx(tx); err != nil { + if _, err := vm.mempool.AddTx(tx); err != nil { t.Fatal(err) } From 2df71994b6e5fc355b424b2b3ab48f50d1f72af3 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 27 Jul 2023 16:50:54 -0400 Subject: [PATCH 02/78] nit --- gossip/handler.go | 7 ++++--- plugin/evm/vm.go | 22 ++++++++++------------ 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/gossip/handler.go b/gossip/handler.go index 91c7f44ca0..3e1e8dfc15 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -11,10 +11,11 @@ import ( "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/x/sdk/p2p" ) -func NewHandler[T any, U TxConstraint[T]](mempool Mempool[U], codec codec.Manager, codecVersion uint16) p2p.Handler { +// var _ p2p.Handler = &Handler[any, *any]{} + +func NewHandler[T any, U TxConstraint[T]](mempool Mempool[U], codec codec.Manager, codecVersion uint16) *Handler[T, U] { return &Handler[T, U]{ mempool: mempool, codec: codec, @@ -32,7 +33,7 @@ func (h Handler[T, U]) AppGossip(context.Context, ids.NodeID, []byte) error { return nil } -func (h Handler[T, U]) AppRequest(_ context.Context, nodeID ids.NodeID, _ uint32, _ time.Time, requestBytes []byte) ([]byte, error) { +func (h Handler[T, U]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ time.Time, requestBytes []byte) ([]byte, error) { request := PullTxsRequest{} if _, err := h.codec.Unmarshal(requestBytes, &request); err != nil { return nil, err diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index cc3cfc3cf6..80ff4db516 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -262,10 +262,10 @@ type VM struct { builder *blockBuilder - gossiper Gossiper - ethTxGossiper *gossip.Gossiper[GossipEthTx, *GossipEthTx] - atomicTxGossiper *gossip.Gossiper[GossipAtomicTx, *GossipAtomicTx] - gossipAtomicTxPool gossip.Mempool[*GossipAtomicTx] + gossiper Gossiper + ethTxGossiper *gossip.Gossiper[GossipEthTx, *GossipEthTx] + atomicTxGossiper *gossip.Gossiper[GossipAtomicTx, *GossipAtomicTx] + atomicMempool gossip.Mempool[*GossipAtomicTx] baseCodec codec.Registry codec codec.Manager @@ -969,23 +969,21 @@ func (vm *VM) initBlockBuilding() error { if err != nil { return err } + vm.atomicMempool = atomicMempool vm.Network.SetGossipHandler(NewGossipHandler( vm, gossipStats, )) - // TODO needed? - gossipEthTxPool := gossip.Mempool[*GossipEthTx](ethTxPool) - ethTxGossipHandler := gossip.NewHandler[GossipEthTx, *GossipEthTx](gossipEthTxPool, vm.codec, message.Version) + ethTxGossipHandler := gossip.NewHandler[GossipEthTx, *GossipEthTx](ethTxPool, vm.codec, message.Version) ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler, vm.appSender) if err != nil { return err } vm.ethTxGossipClient = ethTxGossipClient - vm.gossipAtomicTxPool = gossip.Mempool[*GossipAtomicTx](atomicMempool) - atomicTxGossipHandler := gossip.NewHandler[GossipAtomicTx, *GossipAtomicTx](vm.gossipAtomicTxPool, vm.codec, message.Version) + atomicTxGossipHandler := gossip.NewHandler[GossipAtomicTx, *GossipAtomicTx](atomicMempool, vm.codec, message.Version) atomicTxGossipClient, err := vm.router.RegisterAppProtocol(0x1, atomicTxGossipHandler, vm.appSender) if err != nil { return err @@ -993,7 +991,7 @@ func (vm *VM) initBlockBuilding() error { vm.atomicTxGossipClient = atomicTxGossipClient vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx, *GossipEthTx]( - gossipEthTxPool, + ethTxPool, vm.ethTxGossipClient, vm.networkCodec, message.Version, @@ -1004,7 +1002,7 @@ func (vm *VM) initBlockBuilding() error { go vm.ethTxGossiper.Pull(vm.shutdownChan, &vm.shutdownWg) vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( - vm.gossipAtomicTxPool, + atomicMempool, vm.atomicTxGossipClient, vm.networkCodec, message.Version, @@ -1385,7 +1383,7 @@ func (vm *VM) issueTx(tx *Tx, local bool) error { return err } - if _, err := vm.gossipAtomicTxPool.AddTx(&GossipAtomicTx{tx}, local); err != nil { + if _, err := vm.atomicMempool.AddTx(&GossipAtomicTx{tx}, local); err != nil { return err } From 7b138e38493b5bb22494b3a1013b9378c42df8e7 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 01:43:01 -0400 Subject: [PATCH 03/78] nit --- gossip/gossip.go | 2 +- gossip/mempool.go | 2 +- gossip/test_gossip.go | 2 +- plugin/evm/gossip_mempool.go | 9 +++++---- plugin/evm/gossip_mempool_test.go | 2 +- plugin/evm/vm.go | 3 ++- 6 files changed, 11 insertions(+), 9 deletions(-) diff --git a/gossip/gossip.go b/gossip/gossip.go index 00ce76820c..b886397aa4 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -94,7 +94,7 @@ func (g *Gossiper[T, U]) Pull( continue } - ok, err := g.mempool.AddTx(tx, true) + ok, err := g.mempool.AddTx(tx) if err != nil { log.Debug("failed to add transaction to the mempool", "error", err, "nodeID", nodeID, "id", tx.ID()) continue diff --git a/gossip/mempool.go b/gossip/mempool.go index d13b4747b4..47b757c936 100644 --- a/gossip/mempool.go +++ b/gossip/mempool.go @@ -20,7 +20,7 @@ type Tx interface { // Mempool holds pending transactions type Mempool[T Tx] interface { // AddTx adds a transaction to the mempool - AddTx(tx T, local bool) (bool, error) + AddTx(tx T) (bool, error) // GetTxs returns transactions that match the provided filter function GetTxs(filter func(tx T) bool) []T // GetBloomFilter returns a bloom filter representing the transactions in diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index 374da68b35..b2754641c9 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -41,7 +41,7 @@ type testMempool struct { lock sync.Mutex } -func (t *testMempool) AddTx(tx *testTx, _ bool) (bool, error) { +func (t *testMempool) AddTx(tx *testTx) (bool, error) { t.lock.Lock() defer t.lock.Unlock() diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 2176ed86a4..11a3be6c38 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -47,10 +47,10 @@ type GossipAtomicMempool struct { lock sync.RWMutex } -func (g *GossipAtomicMempool) AddTx(tx *GossipAtomicTx, local bool) (bool, error) { +func (g *GossipAtomicMempool) AddTx(tx *GossipAtomicTx) (bool, error) { ok, err := g.mempool.AddTx(tx.Tx) if err != nil { - if !local { + if !tx.Local { // unlike local txs, invalid remote txs are recorded as discarded // so that they won't be requested again txID := tx.ID() @@ -101,7 +101,8 @@ func (g *GossipAtomicMempool) GetBloomFilter() ([]byte, error) { } type GossipAtomicTx struct { - Tx *Tx + Tx *Tx + Local bool } func (tx *GossipAtomicTx) ID() ids.ID { @@ -163,7 +164,7 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync // AddTx enqueues the transaction to the mempool. Subscribe should be called // to receive an event if tx is actually added to the mempool or not. -func (g *GossipEthTxPool) AddTx(tx *GossipEthTx, _ bool) (bool, error) { +func (g *GossipEthTxPool) AddTx(tx *GossipEthTx) (bool, error) { err := g.mempool.AddRemotes([]*types.Transaction{tx.Tx})[0] if err != nil { return false, err diff --git a/plugin/evm/gossip_mempool_test.go b/plugin/evm/gossip_mempool_test.go index eac1267b56..2ae569ed43 100644 --- a/plugin/evm/gossip_mempool_test.go +++ b/plugin/evm/gossip_mempool_test.go @@ -73,7 +73,7 @@ func TestAtomicMempoolAddTx(t *testing.T) { require.NoError(err) for _, add := range tt.add { - ok, err := mempool.AddTx(add, false) + ok, err := mempool.AddTx(add) require.True(ok) require.NoError(err) } diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 80ff4db516..622d673950 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -1383,7 +1383,8 @@ func (vm *VM) issueTx(tx *Tx, local bool) error { return err } - if _, err := vm.atomicMempool.AddTx(&GossipAtomicTx{tx}, local); err != nil { + gossipTx := &GossipAtomicTx{Tx: tx, Local: local} + if _, err := vm.atomicMempool.AddTx(gossipTx); err != nil { return err } From 74c6d878cd034db1816b07545cd5406aa834c57f Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 18:54:30 -0400 Subject: [PATCH 04/78] nits --- gossip/bloom.go | 6 +-- gossip/bloom_test.go | 4 +- gossip/gossip.go | 40 ++++++++++---------- gossip/handler.go | 27 ++++++------- gossip/mempool.go | 29 +++++++------- gossip/message.go | 6 +-- gossip/test_gossip.go | 12 +++--- plugin/evm/codec.go | 1 - plugin/evm/gossip_mempool.go | 29 +++++++------- plugin/evm/gossip_mempool_test.go | 6 +-- plugin/evm/gossiper.go | 2 - plugin/evm/gossiper_atomic_gossiping_test.go | 3 +- plugin/evm/message/codec.go | 5 ++- plugin/evm/vm.go | 14 ++----- 14 files changed, 86 insertions(+), 98 deletions(-) diff --git a/gossip/bloom.go b/gossip/bloom.go index aa8e27581b..232bbe07b8 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -41,16 +41,16 @@ func ResetBloomFilterIfNeeded( } func NewHasher(id ids.ID) hash.Hash64 { - return hasher{ID: id} + return hasher{id: id} } type hasher struct { hash.Hash64 - ID ids.ID + id ids.ID } func (h hasher) Sum64() uint64 { - return binary.BigEndian.Uint64(h.ID[:]) + return binary.BigEndian.Uint64(h.id[:]) } func (h hasher) Size() int { diff --git a/gossip/bloom_test.go b/gossip/bloom_test.go index 6f6dbce200..33f7c8b5eb 100644 --- a/gossip/bloom_test.go +++ b/gossip/bloom_test.go @@ -49,13 +49,13 @@ func TestBloomFilterRefresh(t *testing.T) { for _, item := range tt.add { ResetBloomFilterIfNeeded(&b, tt.refreshRatio) - b.Add(hasher{ID: item}) + b.Add(NewHasher(item)) } require.Equal(uint64(len(tt.expected)), b.N()) for _, expected := range tt.expected { - require.True(b.Contains(hasher{ID: expected})) + require.True(b.Contains(NewHasher(expected))) } }) } diff --git a/gossip/gossip.go b/gossip/gossip.go index b886397aa4..d1db216b96 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -15,13 +15,13 @@ import ( "github.com/ava-labs/avalanchego/x/sdk/p2p" ) -type TxConstraint[T any] interface { +type GossipableAny[T any] interface { *T - Tx + Gossipable } -func NewGossiper[T any, U TxConstraint[T]]( - mempool Mempool[U], +func NewGossiper[T any, U GossipableAny[T]]( + set Set[U], client *p2p.Client, codec codec.Manager, codecVersion uint16, @@ -29,7 +29,7 @@ func NewGossiper[T any, U TxConstraint[T]]( frequency time.Duration, ) *Gossiper[T, U] { return &Gossiper[T, U]{ - mempool: mempool, + set: set, client: client, codec: codec, codecVersion: codecVersion, @@ -38,8 +38,8 @@ func NewGossiper[T any, U TxConstraint[T]]( } } -type Gossiper[T any, U TxConstraint[T]] struct { - mempool Mempool[U] +type Gossiper[T any, U GossipableAny[T]] struct { + set Set[U] client *p2p.Client codec codec.Manager codecVersion uint16 @@ -60,13 +60,13 @@ func (g *Gossiper[T, U]) Pull( for { select { case <-gossipTicker.C: - bloom, err := g.mempool.GetBloomFilter() + bloom, err := g.set.GetBloomFilter() if err != nil { log.Warn("failed to marshal bloom filter", "error", err) continue } - request := PullTxsRequest{ + request := PullGossipRequest{ BloomFilter: bloom, } msgBytes, err := g.codec.Marshal(g.codecVersion, request) @@ -77,30 +77,30 @@ func (g *Gossiper[T, U]) Pull( onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { if err != nil { - log.Warn("failed to pull txs", "nodeID", nodeID, "error", err) + log.Warn("failed gossip request", "nodeID", nodeID, "error", err) return } - response := PullTxsResponse{} + response := PullGossipResponse{} if _, err := g.codec.Unmarshal(responseBytes, &response); err != nil { - log.Warn("failed to unmarshal txs", "error", err) + log.Warn("failed to unmarshal gossip", "error", err) return } - for _, txBytes := range response.Txs { - tx := U(new(T)) - if err := tx.Unmarshal(txBytes); err != nil { + for _, gossipBytes := range response.GossipBytes { + gossipable := U(new(T)) + if err := gossipable.Unmarshal(gossipBytes); err != nil { log.Debug("failed to unmarshal transaction", "error", err, "nodeID", nodeID) continue } - ok, err := g.mempool.AddTx(tx) + ok, err := g.set.Add(gossipable) if err != nil { - log.Debug("failed to add transaction to the mempool", "error", err, "nodeID", nodeID, "id", tx.ID()) + log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetID()) continue } if !ok { - log.Debug("failed to add transaction to the mempool", "error", err, "nodeID", nodeID, "id", tx.ID()) + log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetID()) continue } } @@ -108,12 +108,12 @@ func (g *Gossiper[T, U]) Pull( for i := 0; i < g.gossipSize; i++ { if err := g.client.AppRequestAny(context.TODO(), msgBytes, onResponse); err != nil { - log.Warn("failed to gossip txs", "error", err) + log.Warn("failed to gossip", "error", err) continue } } case <-shutdownChan: - log.Debug("shutting down tx gossip") + log.Debug("shutting down gossip") return } } diff --git a/gossip/handler.go b/gossip/handler.go index 3e1e8dfc15..dc3af3a9ab 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -15,16 +15,16 @@ import ( // var _ p2p.Handler = &Handler[any, *any]{} -func NewHandler[T any, U TxConstraint[T]](mempool Mempool[U], codec codec.Manager, codecVersion uint16) *Handler[T, U] { +func NewHandler[T any, U GossipableAny[T]](set Set[U], codec codec.Manager, codecVersion uint16) *Handler[T, U] { return &Handler[T, U]{ - mempool: mempool, + set: set, codec: codec, codecVersion: codecVersion, } } -type Handler[T any, U TxConstraint[T]] struct { - mempool Mempool[U] +type Handler[T any, U GossipableAny[T]] struct { + set Set[U] codec codec.Manager codecVersion uint16 } @@ -34,7 +34,7 @@ func (h Handler[T, U]) AppGossip(context.Context, ids.NodeID, []byte) error { } func (h Handler[T, U]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ time.Time, requestBytes []byte) ([]byte, error) { - request := PullTxsRequest{} + request := PullGossipRequest{} if _, err := h.codec.Unmarshal(requestBytes, &request); err != nil { return nil, err } @@ -43,20 +43,21 @@ func (h Handler[T, U]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ t return nil, err } - unknownTxs := h.mempool.GetTxs(func(tx U) bool { - return !peerFilter.Contains(NewHasher(tx.ID())) + // filter out what the requesting peer already knows about + unknown := h.set.Get(func(gossipable U) bool { + return !peerFilter.Contains(NewHasher(gossipable.GetID())) }) - txs := make([][]byte, 0, len(unknownTxs)) - for _, tx := range unknownTxs { - bytes, err := tx.Marshal() + gossipBytes := make([][]byte, 0, len(unknown)) + for _, gossipable := range unknown { + bytes, err := gossipable.Marshal() if err != nil { return nil, err } - txs = append(txs, bytes) + gossipBytes = append(gossipBytes, bytes) } - response := PullTxsResponse{ - Txs: txs, + response := PullGossipResponse{ + GossipBytes: gossipBytes, } responseBytes, err := h.codec.Marshal(h.codecVersion, response) if err != nil { diff --git a/gossip/mempool.go b/gossip/mempool.go index 47b757c936..aabaf9cebd 100644 --- a/gossip/mempool.go +++ b/gossip/mempool.go @@ -3,27 +3,24 @@ package gossip -import ( - "github.com/ava-labs/avalanchego/ids" -) +import "github.com/ava-labs/avalanchego/ids" -// Tx is a transaction that can be gossiped across the network. -type Tx interface { - // ID returns the unique id of this transaction - ID() ids.ID - // Marshal returns the byte representation of this transaction +// Gossipable is an item that can be gossiped across the network +type Gossipable interface { + // GetID represents the unique id of this item + GetID() ids.ID + // Marshal returns the byte representation of this item Marshal() ([]byte, error) // Unmarshal deserializes the provided bytes in-place Unmarshal(b []byte) error } -// Mempool holds pending transactions -type Mempool[T Tx] interface { - // AddTx adds a transaction to the mempool - AddTx(tx T) (bool, error) - // GetTxs returns transactions that match the provided filter function - GetTxs(filter func(tx T) bool) []T - // GetBloomFilter returns a bloom filter representing the transactions in - // the mempool +// Set holds a set of known Gossipable items +type Set[T Gossipable] interface { + // Add adds a Gossipable to the set + Add(gossipable T) (bool, error) + // Get returns elements that match the provided filter function + Get(filter func(gossipable T) bool) []T + // GetBloomFilter returns a bloom filter representing the items in Set. GetBloomFilter() ([]byte, error) } diff --git a/gossip/message.go b/gossip/message.go index 8883ce9417..606903028a 100644 --- a/gossip/message.go +++ b/gossip/message.go @@ -3,10 +3,10 @@ package gossip -type PullTxsRequest struct { +type PullGossipRequest struct { BloomFilter []byte `serialize:"true"` } -type PullTxsResponse struct { - Txs [][]byte `serialize:"true"` +type PullGossipResponse struct { + GossipBytes [][]byte `serialize:"true"` } diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index b2754641c9..c955945f45 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -12,15 +12,15 @@ import ( ) var ( - _ Tx = (*testTx)(nil) - _ Mempool[*testTx] = (*testMempool)(nil) + _ Gossipable = (*testTx)(nil) + _ Set[*testTx] = (*testMempool)(nil) ) type testTx struct { id ids.ID } -func (t *testTx) ID() ids.ID { +func (t *testTx) GetID() ids.ID { return t.id } @@ -41,7 +41,7 @@ type testMempool struct { lock sync.Mutex } -func (t *testMempool) AddTx(tx *testTx) (bool, error) { +func (t *testMempool) Add(tx *testTx) (bool, error) { t.lock.Lock() defer t.lock.Unlock() @@ -49,7 +49,7 @@ func (t *testMempool) AddTx(tx *testTx) (bool, error) { return true, nil } -func (t *testMempool) GetTxs(filter func(tx *testTx) bool) []*testTx { +func (t *testMempool) Get(filter func(tx *testTx) bool) []*testTx { t.lock.Lock() defer t.lock.Unlock() @@ -74,7 +74,7 @@ func (t *testMempool) GetBloomFilter() ([]byte, error) { } for _, tx := range t.mempool { - bloom.Add(hasher{ID: tx.ID()}) + bloom.Add(NewHasher(tx.GetID())) } return bloom.MarshalBinary() diff --git a/plugin/evm/codec.go b/plugin/evm/codec.go index e097abfeb4..b9f8de449f 100644 --- a/plugin/evm/codec.go +++ b/plugin/evm/codec.go @@ -21,7 +21,6 @@ func init() { errs := wrappers.Errs{} errs.Add( - c.RegisterType(&TestUnsignedTx{}), c.RegisterType(&UnsignedImportTx{}), c.RegisterType(&UnsignedExportTx{}), ) diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 11a3be6c38..1d2d43fdf4 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -7,6 +7,7 @@ import ( "fmt" "sync" + "github.com/ava-labs/avalanchego/ids" "github.com/ethereum/go-ethereum/log" "github.com/ava-labs/coreth/gossip" @@ -14,19 +15,17 @@ import ( bloomfilter "github.com/holiman/bloomfilter/v2" - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/txpool" "github.com/ava-labs/coreth/core/types" ) var ( - _ gossip.Mempool[*GossipAtomicTx] = (*GossipAtomicMempool)(nil) - _ gossip.Tx = (*GossipAtomicTx)(nil) + _ gossip.Set[*GossipAtomicTx] = (*GossipAtomicMempool)(nil) + _ gossip.Gossipable = (*GossipAtomicTx)(nil) - _ gossip.Mempool[*GossipEthTx] = (*GossipEthTxPool)(nil) - _ gossip.Tx = (*GossipEthTx)(nil) + _ gossip.Set[*GossipEthTx] = (*GossipEthTxPool)(nil) + _ gossip.Gossipable = (*GossipEthTx)(nil) ) func NewGossipAtomicMempool(Mempool *Mempool) (*GossipAtomicMempool, error) { @@ -47,14 +46,14 @@ type GossipAtomicMempool struct { lock sync.RWMutex } -func (g *GossipAtomicMempool) AddTx(tx *GossipAtomicTx) (bool, error) { +func (g *GossipAtomicMempool) Add(tx *GossipAtomicTx) (bool, error) { ok, err := g.mempool.AddTx(tx.Tx) if err != nil { if !tx.Local { // unlike local txs, invalid remote txs are recorded as discarded // so that they won't be requested again - txID := tx.ID() - g.mempool.discardedTxs.Put(tx.ID(), tx.Tx) + txID := tx.Tx.ID() + g.mempool.discardedTxs.Put(txID, tx.Tx) log.Debug("failed to issue remote tx to mempool", "txID", txID, "err", err, @@ -70,13 +69,13 @@ func (g *GossipAtomicMempool) AddTx(tx *GossipAtomicTx) (bool, error) { g.lock.Lock() defer g.lock.Unlock() - g.bloom.Add(gossip.NewHasher(tx.ID())) + g.bloom.Add(gossip.NewHasher(tx.GetID())) gossip.ResetBloomFilterIfNeeded(&g.bloom, gossip.DefaultBloomMaxFilledRatio) return true, nil } -func (g *GossipAtomicMempool) GetTxs(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { +func (g *GossipAtomicMempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { f := func(tx *Tx) bool { return filter(&GossipAtomicTx{ Tx: tx, @@ -105,7 +104,7 @@ type GossipAtomicTx struct { Local bool } -func (tx *GossipAtomicTx) ID() ids.ID { +func (tx *GossipAtomicTx) GetID() ids.ID { return tx.Tx.ID() } @@ -164,7 +163,7 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync // AddTx enqueues the transaction to the mempool. Subscribe should be called // to receive an event if tx is actually added to the mempool or not. -func (g *GossipEthTxPool) AddTx(tx *GossipEthTx) (bool, error) { +func (g *GossipEthTxPool) Add(tx *GossipEthTx) (bool, error) { err := g.mempool.AddRemotes([]*types.Transaction{tx.Tx})[0] if err != nil { return false, err @@ -173,7 +172,7 @@ func (g *GossipEthTxPool) AddTx(tx *GossipEthTx) (bool, error) { return true, nil } -func (g *GossipEthTxPool) GetTxs(filter func(tx *GossipEthTx) bool) []*GossipEthTx { +func (g *GossipEthTxPool) Get(filter func(tx *GossipEthTx) bool) []*GossipEthTx { pending, _ := g.mempool.Content() result := make([]*GossipEthTx, 0) @@ -202,7 +201,7 @@ type GossipEthTx struct { Tx *types.Transaction } -func (tx *GossipEthTx) ID() ids.ID { +func (tx *GossipEthTx) GetID() ids.ID { return ids.ID(tx.Tx.Hash()) } diff --git a/plugin/evm/gossip_mempool_test.go b/plugin/evm/gossip_mempool_test.go index 2ae569ed43..4b475c9ef2 100644 --- a/plugin/evm/gossip_mempool_test.go +++ b/plugin/evm/gossip_mempool_test.go @@ -73,12 +73,12 @@ func TestAtomicMempoolAddTx(t *testing.T) { require.NoError(err) for _, add := range tt.add { - ok, err := mempool.AddTx(add) + ok, err := mempool.Add(add) require.True(ok) require.NoError(err) } - txs := mempool.GetTxs(tt.filter) + txs := mempool.Get(tt.filter) require.Len(txs, len(tt.expected)) for _, expected := range tt.expected { @@ -183,7 +183,7 @@ func TestAtomicMempoolAddTx(t *testing.T) { // <-pending // } // -// txs := mempool.GetTxs(tt.filter) +// txs := mempool.Get(tt.filter) // require.Len(txs, len(tt.expected)) // // for _, expected := range tt.expected { diff --git a/plugin/evm/gossiper.go b/plugin/evm/gossiper.go index a541ddb195..2ced1de529 100644 --- a/plugin/evm/gossiper.go +++ b/plugin/evm/gossiper.go @@ -43,8 +43,6 @@ const ( minGossipBatchInterval = 50 * time.Millisecond ) -var _ message.GossipHandler = (*GossipHandler)(nil) - // Gossiper handles outgoing gossip of transactions type Gossiper interface { // GossipAtomicTxs sends AppGossip message containing the given [txs] diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index b7ce36872e..2386d34b71 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -15,6 +15,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/coreth/gossip" "github.com/stretchr/testify/assert" @@ -64,7 +65,7 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { addedToBloomFilter := false sender.SendAppRequestF = func(ctx context.Context, _ set.Set[ids.NodeID], _ uint32, bytes []byte) error { bytes = bytes[1:] // first byte is an identifier - msg := gossip.PullTxsRequest{} + msg := gossip.PullGossipRequest{} _, err := vm.networkCodec.Unmarshal(bytes, &msg) require.NoError(t, err) diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index c39a302d18..fb0a17e22d 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -8,6 +8,7 @@ import ( "github.com/ava-labs/avalanchego/codec/linearcodec" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/utils/wrappers" + "github.com/ava-labs/coreth/gossip" ) @@ -30,8 +31,8 @@ func init() { // Gossip types c.RegisterType(AtomicTxGossip{}), c.RegisterType(EthTxsGossip{}), - c.RegisterType(gossip.PullTxsRequest{}), - c.RegisterType(gossip.PullTxsResponse{}), + c.RegisterType(gossip.PullGossipRequest{}), + c.RegisterType(gossip.PullGossipResponse{}), // Types for state sync frontier consensus c.RegisterType(SyncSummary{}), diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 622d673950..18217d6f87 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -265,7 +265,7 @@ type VM struct { gossiper Gossiper ethTxGossiper *gossip.Gossiper[GossipEthTx, *GossipEthTx] atomicTxGossiper *gossip.Gossiper[GossipAtomicTx, *GossipAtomicTx] - atomicMempool gossip.Mempool[*GossipAtomicTx] + atomicMempool gossip.Set[*GossipAtomicTx] baseCodec codec.Registry codec codec.Manager @@ -957,6 +957,7 @@ func (vm *VM) initBlockBuilding() error { vm.gossiper = vm.createGossiper(gossipStats) vm.builder = vm.NewBlockBuilder(vm.toEngine) vm.builder.awaitSubmittedTxs() + vm.Network.SetGossipHandler(NewGossipHandler(vm, gossipStats)) ethTxPool, err := NewGossipEthTxPool(vm.txPool) if err != nil { @@ -971,11 +972,6 @@ func (vm *VM) initBlockBuilding() error { } vm.atomicMempool = atomicMempool - vm.Network.SetGossipHandler(NewGossipHandler( - vm, - gossipStats, - )) - ethTxGossipHandler := gossip.NewHandler[GossipEthTx, *GossipEthTx](ethTxPool, vm.codec, message.Version) ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler, vm.appSender) if err != nil { @@ -1384,7 +1380,7 @@ func (vm *VM) issueTx(tx *Tx, local bool) error { } gossipTx := &GossipAtomicTx{Tx: tx, Local: local} - if _, err := vm.atomicMempool.AddTx(gossipTx); err != nil { + if _, err := vm.atomicMempool.Add(gossipTx); err != nil { return err } @@ -1825,7 +1821,3 @@ func (vm *VM) stateSyncEnabled(lastAcceptedHeight uint64) bool { // enable state sync by default if the chain is empty. return lastAcceptedHeight == 0 } - -func (vm *VM) IssueTx(tx *Tx, b bool) error { - return vm.issueTx(tx, b) -} From 88c2d8ea6ebc9f866a090c9ccbcd4f880a37e0a8 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 20:29:43 -0400 Subject: [PATCH 05/78] nits --- gossip/{mempool.go => gossipable.go} | 0 plugin/evm/gossiper_atomic_gossiping_test.go | 3 +++ plugin/evm/message/codec.go | 6 ++++-- 3 files changed, 7 insertions(+), 2 deletions(-) rename gossip/{mempool.go => gossipable.go} (100%) diff --git a/gossip/mempool.go b/gossip/gossipable.go similarity index 100% rename from gossip/mempool.go rename to gossip/gossipable.go diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index 2386d34b71..3cf5824db4 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -64,6 +64,9 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { addedToBloomFilter := false sender.SendAppRequestF = func(ctx context.Context, _ set.Set[ids.NodeID], _ uint32, bytes []byte) error { + gossipedLock.Lock() + defer gossipedLock.Unlock() + bytes = bytes[1:] // first byte is an identifier msg := gossip.PullGossipRequest{} _, err := vm.networkCodec.Unmarshal(bytes, &msg) diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index fb0a17e22d..2ebcc1afce 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -31,8 +31,6 @@ func init() { // Gossip types c.RegisterType(AtomicTxGossip{}), c.RegisterType(EthTxsGossip{}), - c.RegisterType(gossip.PullGossipRequest{}), - c.RegisterType(gossip.PullGossipResponse{}), // Types for state sync frontier consensus c.RegisterType(SyncSummary{}), @@ -45,6 +43,10 @@ func init() { c.RegisterType(CodeRequest{}), c.RegisterType(CodeResponse{}), + // p2p sdk gossip types + c.RegisterType(gossip.PullGossipRequest{}), + c.RegisterType(gossip.PullGossipResponse{}), + Codec.RegisterCodec(Version, c), ) From 2d0aee3ea1651da2ea03831caf470d1673f61efb Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 21:51:09 -0400 Subject: [PATCH 06/78] nits --- go.mod | 4 +--- go.sum | 2 ++ gossip/gossip.go | 2 +- peer/network.go | 2 +- peer/network_test.go | 26 +++++++++++++------------- plugin/evm/vm.go | 10 ++++------ 6 files changed, 22 insertions(+), 24 deletions(-) diff --git a/go.mod b/go.mod index 4e3c04aee4..380220c389 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,7 @@ go 1.19 require ( github.com/VictoriaMetrics/fastcache v1.10.0 - github.com/ava-labs/avalanchego v1.10.5 + github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 github.com/cespare/cp v0.1.0 github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 github.com/davecgh/go-spew v1.1.1 @@ -141,5 +141,3 @@ require ( gopkg.in/yaml.v3 v3.0.1 // indirect rsc.io/tmplfunc v0.0.3 // indirect ) - -replace github.com/ava-labs/avalanchego => github.com/joshua-kim/avalanchego v1.4.9-0.20230727193239-84cd2070ba32 diff --git a/go.sum b/go.sum index 3b689cddd6..bf48bfea5d 100644 --- a/go.sum +++ b/go.sum @@ -55,6 +55,8 @@ github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 h1:kuz7rZrL3dm2ZhK6Vg0N8No1yM3Ur0HPPHUxGiZxbjU= +github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= diff --git a/gossip/gossip.go b/gossip/gossip.go index d1db216b96..0d632c752a 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -8,11 +8,11 @@ import ( "sync" "time" + "github.com/ava-labs/avalanchego/x/p2p" "github.com/ethereum/go-ethereum/log" "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/x/sdk/p2p" ) type GossipableAny[T any] interface { diff --git a/peer/network.go b/peer/network.go index 0311b77521..b08e86a3bf 100644 --- a/peer/network.go +++ b/peer/network.go @@ -10,6 +10,7 @@ import ( "sync" "time" + "github.com/ava-labs/avalanchego/x/p2p" "golang.org/x/sync/semaphore" "github.com/ethereum/go-ethereum/log" @@ -21,7 +22,6 @@ import ( "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/avalanchego/version" - "github.com/ava-labs/avalanchego/x/sdk/p2p" "github.com/ava-labs/coreth/peer/stats" "github.com/ava-labs/coreth/plugin/evm/message" diff --git a/peer/network_test.go b/peer/network_test.go index 023218f894..87384ea9be 100644 --- a/peer/network_test.go +++ b/peer/network_test.go @@ -14,7 +14,7 @@ import ( "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/x/sdk/p2p" + "github.com/ava-labs/avalanchego/x/p2p" ethcommon "github.com/ethereum/go-ethereum/common" @@ -55,7 +55,7 @@ var ( func TestNetworkDoesNotConnectToItself(t *testing.T) { selfNodeID := ids.GenerateTestNodeID() - n := NewNetwork(p2p.NewRouter(), nil, nil, nil, selfNodeID, 1, 1) + n := NewNetwork(p2p.NewRouter(nil), nil, nil, nil, selfNodeID, 1, 1) assert.NoError(t, n.Connected(context.Background(), selfNodeID, defaultPeerVersion)) assert.EqualValues(t, 0, n.Size()) } @@ -91,7 +91,7 @@ func TestRequestAnyRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) net.SetRequestHandler(&HelloGreetingRequestHandler{codec: codecManager}) client := NewNetworkClient(net) nodeID := ids.GenerateTestNodeID() @@ -166,7 +166,7 @@ func TestRequestRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) net.SetRequestHandler(&HelloGreetingRequestHandler{codec: codecManager}) client := NewNetworkClient(net) @@ -246,7 +246,7 @@ func TestAppRequestOnShutdown(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) client := NewNetworkClient(net) nodeID := ids.GenerateTestNodeID() require.NoError(t, net.Connected(context.Background(), nodeID, defaultPeerVersion)) @@ -295,7 +295,7 @@ func TestRequestMinVersion(t *testing.T) { } // passing nil as codec works because the net.AppRequest is never called - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 16) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 16) client := NewNetworkClient(net) requestMessage := TestMessage{Message: "this is a request"} requestBytes, err := message.RequestToBytes(codecManager, requestMessage) @@ -358,7 +358,7 @@ func TestOnRequestHonoursDeadline(t *testing.T) { processingDuration: 500 * time.Millisecond, } - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetRequestHandler(requestHandler) nodeID := ids.GenerateTestNodeID() @@ -398,7 +398,7 @@ func TestGossip(t *testing.T) { } gossipHandler := &testGossipHandler{} - clientNetwork = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(gossipHandler) assert.NoError(t, clientNetwork.Connected(context.Background(), nodeID, defaultPeerVersion)) @@ -425,7 +425,7 @@ func TestHandleInvalidMessages(t *testing.T) { requestID := uint32(1) sender := testAppSender{} - clientNetwork := NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork := NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) clientNetwork.SetRequestHandler(&testRequestHandler{}) @@ -475,7 +475,7 @@ func TestNetworkPropagatesRequestHandlerError(t *testing.T) { requestID := uint32(1) sender := testAppSender{} - clientNetwork := NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork := NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) clientNetwork.SetRequestHandler(&testRequestHandler{err: errors.New("fail")}) // Return an error from the request handler @@ -515,7 +515,7 @@ func TestCrossChainAppRequest(t *testing.T) { }, } - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetCrossChainRequestHandler(&testCrossChainHandler{codec: crossChainCodecManager}) client := NewNetworkClient(net) @@ -570,7 +570,7 @@ func TestCrossChainRequestRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, TestMessage{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetCrossChainRequestHandler(&testCrossChainHandler{codec: crossChainCodecManager}) client := NewNetworkClient(net) @@ -630,7 +630,7 @@ func TestCrossChainRequestOnShutdown(t *testing.T) { } codecManager := buildCodec(t, TestMessage{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) client := NewNetworkClient(net) exampleCrossChainRequest := ExampleCrossChainRequest{ diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 18217d6f87..9c239b4b44 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -17,7 +17,7 @@ import ( "time" avalanchegoMetrics "github.com/ava-labs/avalanchego/api/metrics" - "github.com/ava-labs/avalanchego/x/sdk/p2p" + "github.com/ava-labs/avalanchego/x/p2p" "github.com/ava-labs/coreth/consensus/dummy" corethConstants "github.com/ava-labs/coreth/constants" @@ -283,7 +283,6 @@ type VM struct { peer.Network client peer.NetworkClient - appSender commonEng.AppSender networkCodec codec.Manager router *p2p.Router @@ -512,11 +511,10 @@ func (vm *VM) Initialize( } // initialize peer network - vm.router = p2p.NewRouter() + vm.router = p2p.NewRouter(appSender) vm.networkCodec = message.Codec vm.Network = peer.NewNetwork(vm.router, appSender, vm.networkCodec, message.CrossChainCodec, chainCtx.NodeID, vm.config.MaxOutboundActiveRequests, vm.config.MaxOutboundActiveCrossChainRequests) vm.client = peer.NewNetworkClient(vm.Network) - vm.appSender = appSender if err := vm.initializeChain(lastAcceptedHash); err != nil { return err @@ -973,14 +971,14 @@ func (vm *VM) initBlockBuilding() error { vm.atomicMempool = atomicMempool ethTxGossipHandler := gossip.NewHandler[GossipEthTx, *GossipEthTx](ethTxPool, vm.codec, message.Version) - ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler, vm.appSender) + ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler) if err != nil { return err } vm.ethTxGossipClient = ethTxGossipClient atomicTxGossipHandler := gossip.NewHandler[GossipAtomicTx, *GossipAtomicTx](atomicMempool, vm.codec, message.Version) - atomicTxGossipClient, err := vm.router.RegisterAppProtocol(0x1, atomicTxGossipHandler, vm.appSender) + atomicTxGossipClient, err := vm.router.RegisterAppProtocol(0x1, atomicTxGossipHandler) if err != nil { return err } From 5926bcac1d4b57e986a1e0c0665fbbe213a5259e Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 21:52:34 -0400 Subject: [PATCH 07/78] nit --- gossip/gossip.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/gossip/gossip.go b/gossip/gossip.go index 0d632c752a..7632de0bb5 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -60,14 +60,14 @@ func (g *Gossiper[T, U]) Pull( for { select { case <-gossipTicker.C: - bloom, err := g.set.GetBloomFilter() + bloomBytes, err := g.set.GetBloomFilter() if err != nil { log.Warn("failed to marshal bloom filter", "error", err) continue } request := PullGossipRequest{ - BloomFilter: bloom, + BloomFilter: bloomBytes, } msgBytes, err := g.codec.Marshal(g.codecVersion, request) if err != nil { From bc5224bb3da349953a40cb2844946e8bb545590a Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 21:58:51 -0400 Subject: [PATCH 08/78] nit --- go.mod | 4 +++- go.sum | 2 -- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 380220c389..b74542ea03 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,7 @@ go 1.19 require ( github.com/VictoriaMetrics/fastcache v1.10.0 - github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 + github.com/ava-labs/avalanchego v1.10.5 github.com/cespare/cp v0.1.0 github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 github.com/davecgh/go-spew v1.1.1 @@ -141,3 +141,5 @@ require ( gopkg.in/yaml.v3 v3.0.1 // indirect rsc.io/tmplfunc v0.0.3 // indirect ) + +replace github.com/ava-labs/avalanchego => github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 diff --git a/go.sum b/go.sum index bf48bfea5d..a7aacd71fd 100644 --- a/go.sum +++ b/go.sum @@ -372,8 +372,6 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/joshua-kim/avalanchego v1.4.9-0.20230727193239-84cd2070ba32 h1:F3wR1cDsTjXW8M0mUJVbP+v/PEM1ZShTpwptPcVMu1c= -github.com/joshua-kim/avalanchego v1.4.9-0.20230727193239-84cd2070ba32/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= From 642d03cb9a6b666d98d29b5594d6c8d731529024 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 23:28:10 -0400 Subject: [PATCH 09/78] nit --- plugin/evm/gossip_mempool_test.go | 105 ------------------------------ 1 file changed, 105 deletions(-) diff --git a/plugin/evm/gossip_mempool_test.go b/plugin/evm/gossip_mempool_test.go index 4b475c9ef2..ec71eaa7fe 100644 --- a/plugin/evm/gossip_mempool_test.go +++ b/plugin/evm/gossip_mempool_test.go @@ -87,108 +87,3 @@ func TestAtomicMempoolAddTx(t *testing.T) { }) } } - -// -// func TestEthTxPoolAddTx(t *testing.T) { -// key, err := crypto.GenerateKey() -// require.NoError(t, err) -// -// txs := []*types.Transaction{ -// types.NewTx(&types.AccessListTx{ -// ChainID: params.TestChainConfig.ChainID, -// Nonce: 0, -// GasPrice: big.NewInt(1), -// Gas: 100_000, -// To: &common.Address{}, -// Value: big.NewInt(0), -// Data: []byte{}, -// V: big.NewInt(32), -// R: big.NewInt(10), -// S: big.NewInt(11), -// }), -// types.NewTx(&types.AccessListTx{ -// ChainID: params.TestChainConfig.ChainID, -// Nonce: 1, -// GasPrice: big.NewInt(1), -// Gas: 100_000, -// To: &common.Address{}, -// Value: big.NewInt(0), -// Data: []byte{}, -// V: big.NewInt(32), -// R: big.NewInt(10), -// S: big.NewInt(11), -// }), -// } -// -// signedTxs := make([]*GossipEthTx, 0, len(txs)) -// for _, tx := range txs { -// signedTx, err := types.SignTx(tx, types.LatestSigner(params.TestChainConfig), key) -// require.NoError(t, err) -// signedTxs = append(signedTxs, &GossipEthTx{ -// Tx: signedTx, -// }) -// } -// -// tests := []struct { -// name string -// add []*GossipEthTx -// filter func(tx *GossipEthTx) bool -// expected []*GossipEthTx -// }{ -// { -// name: "empty", -// }, -// { -// name: "filter matches nothing", -// add: signedTxs, -// filter: func(tx *GossipEthTx) bool { -// return false -// }, -// expected: nil, -// }, -// { -// name: "filter matches all", -// add: signedTxs, -// filter: func(*GossipEthTx) bool { -// return true -// }, -// expected: signedTxs, -// }, -// { -// name: "filter matches subset", -// add: signedTxs, -// filter: func(tx *GossipEthTx) bool { -// return tx.Tx == signedTxs[0].Tx -// }, -// expected: signedTxs[:1], -// }, -// } -// -// for _, tt := range tests { -// t.Run(tt.name, func(t *testing.T) { -// require := require.New(t) -// -// config := txpool.DefaultConfig -// config.Journal = "" -// m := txpool.NewTestTxPool(config, params.TestChainConfig, 100_000_000, key.PublicKey) -// pending := make(chan core.NewTxsEvent) -// m.SubscribeNewTxsEvent(pending) -// mempool, err := NewGossipEthTxPool(m) -// require.NoError(err) -// -// for _, add := range tt.add { -// ok, err := mempool.AddTx(add, false) -// require.True(ok) -// require.NoError(err) -// <-pending -// } -// -// txs := mempool.Get(tt.filter) -// require.Len(txs, len(tt.expected)) -// -// for _, expected := range tt.expected { -// require.Contains(txs, expected) -// } -// }) -// } -// } From 9fc69eb34692a5fb9760852f0ce29b497f160fb1 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 31 Jul 2023 23:36:53 -0400 Subject: [PATCH 10/78] nit --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index b74542ea03..5aae2083f6 100644 --- a/go.mod +++ b/go.mod @@ -142,4 +142,4 @@ require ( rsc.io/tmplfunc v0.0.3 // indirect ) -replace github.com/ava-labs/avalanchego => github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 +replace github.com/ava-labs/avalanchego => github.com/joshua-kim/avalanchego v1.0.0-sdk-pr diff --git a/go.sum b/go.sum index a7aacd71fd..342ec2413b 100644 --- a/go.sum +++ b/go.sum @@ -55,8 +55,6 @@ github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 h1:kuz7rZrL3dm2ZhK6Vg0N8No1yM3Ur0HPPHUxGiZxbjU= -github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -372,6 +370,8 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/joshua-kim/avalanchego v1.0.0-sdk-pr h1:8fHKc5gB0yczGIKzvLh0d2pGPR0gnGx0y4t2M3hpUTU= +github.com/joshua-kim/avalanchego v1.0.0-sdk-pr/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= From da10671017c6837bf1d8f0702da32e1205298be4 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 1 Aug 2023 00:12:18 -0400 Subject: [PATCH 11/78] public tag --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 5aae2083f6..b74542ea03 100644 --- a/go.mod +++ b/go.mod @@ -142,4 +142,4 @@ require ( rsc.io/tmplfunc v0.0.3 // indirect ) -replace github.com/ava-labs/avalanchego => github.com/joshua-kim/avalanchego v1.0.0-sdk-pr +replace github.com/ava-labs/avalanchego => github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 diff --git a/go.sum b/go.sum index 342ec2413b..a7aacd71fd 100644 --- a/go.sum +++ b/go.sum @@ -55,6 +55,8 @@ github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 h1:kuz7rZrL3dm2ZhK6Vg0N8No1yM3Ur0HPPHUxGiZxbjU= +github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -370,8 +372,6 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/joshua-kim/avalanchego v1.0.0-sdk-pr h1:8fHKc5gB0yczGIKzvLh0d2pGPR0gnGx0y4t2M3hpUTU= -github.com/joshua-kim/avalanchego v1.0.0-sdk-pr/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= From bd2cb7418376c4ba4dec894ff72b6f52a381082f Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 1 Aug 2023 02:32:27 -0400 Subject: [PATCH 12/78] remove double pointer --- gossip/bloom.go | 12 +++++------- gossip/bloom_test.go | 2 +- plugin/evm/gossip_mempool.go | 4 ++-- 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/gossip/bloom.go b/gossip/bloom.go index 232bbe07b8..26222c0d90 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -25,19 +25,17 @@ var _ hash.Hash64 = (*hasher)(nil) // ResetBloomFilterIfNeeded resets a bloom filter if it breaches a ratio of // filled elements. Returns true if the bloom filter was reset. func ResetBloomFilterIfNeeded( - bloomFilter **bloomfilter.Filter, + bloomFilter *bloomfilter.Filter, maxFilledRatio float64, -) bool { - if (*bloomFilter).PreciseFilledRatio() < maxFilledRatio { - return false +) (*bloomfilter.Filter, bool) { + if bloomFilter.PreciseFilledRatio() < maxFilledRatio { + return bloomFilter, false } // it's not possible for this to error assuming that the original // bloom filter's parameters were valid fresh, _ := bloomfilter.New((*bloomFilter).M(), (*bloomFilter).K()) - *bloomFilter = fresh - - return true + return fresh, true } func NewHasher(id ids.ID) hash.Hash64 { diff --git a/gossip/bloom_test.go b/gossip/bloom_test.go index 33f7c8b5eb..0c27f557a2 100644 --- a/gossip/bloom_test.go +++ b/gossip/bloom_test.go @@ -48,7 +48,7 @@ func TestBloomFilterRefresh(t *testing.T) { require.NoError(err) for _, item := range tt.add { - ResetBloomFilterIfNeeded(&b, tt.refreshRatio) + b, _ = ResetBloomFilterIfNeeded(b, tt.refreshRatio) b.Add(NewHasher(item)) } diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 1d2d43fdf4..511c81c13b 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -70,7 +70,7 @@ func (g *GossipAtomicMempool) Add(tx *GossipAtomicTx) (bool, error) { defer g.lock.Unlock() g.bloom.Add(gossip.NewHasher(tx.GetID())) - gossip.ResetBloomFilterIfNeeded(&g.bloom, gossip.DefaultBloomMaxFilledRatio) + g.bloom, _ = gossip.ResetBloomFilterIfNeeded(g.bloom, gossip.DefaultBloomMaxFilledRatio) return true, nil } @@ -154,7 +154,7 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync g.lock.Lock() for _, tx := range tx.Txs { g.bloom.Add(gossip.NewHasher(ids.ID(tx.Hash()))) - gossip.ResetBloomFilterIfNeeded(&g.bloom, gossip.DefaultBloomMaxFilledRatio) + g.bloom, _ = gossip.ResetBloomFilterIfNeeded(g.bloom, gossip.DefaultBloomMaxFilledRatio) } g.lock.Unlock() } From 7a9101437f505e005ffed986933280d3927a56ce Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 1 Aug 2023 02:37:06 -0400 Subject: [PATCH 13/78] nits --- gossip/gossip.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/gossip/gossip.go b/gossip/gossip.go index 7632de0bb5..62d3f67458 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -71,26 +71,26 @@ func (g *Gossiper[T, U]) Pull( } msgBytes, err := g.codec.Marshal(g.codecVersion, request) if err != nil { - log.Warn("failed to marshal gossip message", "error", err) + log.Warn("failed to marshal gossip request", "error", err) continue } onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { if err != nil { - log.Warn("failed gossip request", "nodeID", nodeID, "error", err) + log.Debug("failed gossip request", "nodeID", nodeID, "error", err) return } response := PullGossipResponse{} if _, err := g.codec.Unmarshal(responseBytes, &response); err != nil { - log.Warn("failed to unmarshal gossip", "error", err) + log.Debug("failed to unmarshal gossip response", "error", err) return } for _, gossipBytes := range response.GossipBytes { gossipable := U(new(T)) if err := gossipable.Unmarshal(gossipBytes); err != nil { - log.Debug("failed to unmarshal transaction", "error", err, "nodeID", nodeID) + log.Debug("failed to unmarshal gossip", "error", err, "nodeID", nodeID) continue } From d31e12d44ed871f97bfc1a69b394022fc33e5a73 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 1 Aug 2023 12:43:10 -0400 Subject: [PATCH 14/78] nit --- gossip/handler.go | 18 ++++++++---------- plugin/evm/gossip_mempool.go | 2 +- plugin/evm/vm.go | 4 ++-- 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/gossip/handler.go b/gossip/handler.go index dc3af3a9ab..68be420e4a 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -13,27 +13,25 @@ import ( "github.com/ava-labs/avalanchego/ids" ) -// var _ p2p.Handler = &Handler[any, *any]{} - -func NewHandler[T any, U GossipableAny[T]](set Set[U], codec codec.Manager, codecVersion uint16) *Handler[T, U] { - return &Handler[T, U]{ +func NewHandler[T Gossipable](set Set[T], codec codec.Manager, codecVersion uint16) *Handler[T] { + return &Handler[T]{ set: set, codec: codec, codecVersion: codecVersion, } } -type Handler[T any, U GossipableAny[T]] struct { - set Set[U] +type Handler[T Gossipable] struct { + set Set[T] codec codec.Manager codecVersion uint16 } -func (h Handler[T, U]) AppGossip(context.Context, ids.NodeID, []byte) error { +func (h Handler[T]) AppGossip(context.Context, ids.NodeID, []byte) error { return nil } -func (h Handler[T, U]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ time.Time, requestBytes []byte) ([]byte, error) { +func (h Handler[T]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ time.Time, requestBytes []byte) ([]byte, error) { request := PullGossipRequest{} if _, err := h.codec.Unmarshal(requestBytes, &request); err != nil { return nil, err @@ -44,7 +42,7 @@ func (h Handler[T, U]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ t } // filter out what the requesting peer already knows about - unknown := h.set.Get(func(gossipable U) bool { + unknown := h.set.Get(func(gossipable T) bool { return !peerFilter.Contains(NewHasher(gossipable.GetID())) }) gossipBytes := make([][]byte, 0, len(unknown)) @@ -67,6 +65,6 @@ func (h Handler[T, U]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ t return responseBytes, nil } -func (Handler[T, U]) CrossChainAppRequest(context.Context, ids.ID, uint32, time.Time, []byte) ([]byte, error) { +func (Handler[T]) CrossChainAppRequest(context.Context, ids.ID, uint32, time.Time, []byte) ([]byte, error) { return nil, nil } diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 511c81c13b..ba6c746d15 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -161,7 +161,7 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync } } -// AddTx enqueues the transaction to the mempool. Subscribe should be called +// Add enqueues the transaction to the mempool. Subscribe should be called // to receive an event if tx is actually added to the mempool or not. func (g *GossipEthTxPool) Add(tx *GossipEthTx) (bool, error) { err := g.mempool.AddRemotes([]*types.Transaction{tx.Tx})[0] diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 9c239b4b44..62cd5ce92a 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -970,14 +970,14 @@ func (vm *VM) initBlockBuilding() error { } vm.atomicMempool = atomicMempool - ethTxGossipHandler := gossip.NewHandler[GossipEthTx, *GossipEthTx](ethTxPool, vm.codec, message.Version) + ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, vm.codec, message.Version) ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler) if err != nil { return err } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := gossip.NewHandler[GossipAtomicTx, *GossipAtomicTx](atomicMempool, vm.codec, message.Version) + atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](atomicMempool, vm.codec, message.Version) atomicTxGossipClient, err := vm.router.RegisterAppProtocol(0x1, atomicTxGossipHandler) if err != nil { return err From e73eb43ed727ec5a0ebf3635c26bc783439cddc4 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 1 Aug 2023 12:43:37 -0400 Subject: [PATCH 15/78] nit --- gossip/handler.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gossip/handler.go b/gossip/handler.go index 68be420e4a..93f777c17c 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -27,7 +27,7 @@ type Handler[T Gossipable] struct { codecVersion uint16 } -func (h Handler[T]) AppGossip(context.Context, ids.NodeID, []byte) error { +func (Handler[T]) AppGossip(context.Context, ids.NodeID, []byte) error { return nil } From 46126c7a5eb2383f0b310f628fc8cf473bf71098 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 1 Aug 2023 12:44:46 -0400 Subject: [PATCH 16/78] nit --- gossip/gossip.go | 1 + 1 file changed, 1 insertion(+) diff --git a/gossip/gossip.go b/gossip/gossip.go index 62d3f67458..e14a715933 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -15,6 +15,7 @@ import ( "github.com/ava-labs/avalanchego/ids" ) +// GossipableAny exists to help create non-nil pointers to a concrete Gossipable type GossipableAny[T any] interface { *T Gossipable From 3c45c3fd18e66c9acb2cb8be6dcebd1abca0e64d Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 2 Aug 2023 11:25:15 -0400 Subject: [PATCH 17/78] Update gossip/test_gossip.go Co-authored-by: aaronbuchwald Signed-off-by: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> --- gossip/test_gossip.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index c955945f45..f842a683e3 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -29,10 +29,7 @@ func (t *testTx) Marshal() ([]byte, error) { } func (t *testTx) Unmarshal(b []byte) error { - for i := 0; i < 32 || i < len(b); i++ { - t.id[i] = b[i] - } - + copy(t.id[:], b) return nil } From 9720e3c8982ce85152a3f1758fe90432c8f05036 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 1 Aug 2023 17:35:08 -0400 Subject: [PATCH 18/78] nit --- go.mod | 4 +- go.sum | 13 ++++ gossip/gossip.go | 61 +++++++-------- gossip/gossipable.go | 2 +- gossip/handler.go | 22 ++++-- gossip/test_gossip.go | 4 +- peer/network.go | 24 +++--- peer/network_test.go | 27 +++---- plugin/evm/gossip_mempool.go | 86 ++------------------- plugin/evm/gossip_mempool_test.go | 9 +-- plugin/evm/mempool.go | 68 ++++++++++++---- plugin/evm/mempool_atomic_gossiping_test.go | 15 ++-- plugin/evm/vm.go | 35 +++++---- plugin/evm/vm_test.go | 10 +-- 14 files changed, 179 insertions(+), 201 deletions(-) diff --git a/go.mod b/go.mod index b74542ea03..742b59b9c3 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,7 @@ go 1.19 require ( github.com/VictoriaMetrics/fastcache v1.10.0 - github.com/ava-labs/avalanchego v1.10.5 + github.com/ava-labs/avalanchego v1.10.6 github.com/cespare/cp v0.1.0 github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 github.com/davecgh/go-spew v1.1.1 @@ -141,5 +141,3 @@ require ( gopkg.in/yaml.v3 v3.0.1 // indirect rsc.io/tmplfunc v0.0.3 // indirect ) - -replace github.com/ava-labs/avalanchego => github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 diff --git a/go.sum b/go.sum index a7aacd71fd..298fe30c03 100644 --- a/go.sum +++ b/go.sum @@ -51,12 +51,16 @@ github.com/VictoriaMetrics/fastcache v1.10.0 h1:5hDJnLsKLpnUEToub7ETuRu8RCkb40wo github.com/VictoriaMetrics/fastcache v1.10.0/go.mod h1:tjiYeEfYXCqacuvYw/7UoDIeJaNxq6132xHICNP77w8= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 h1:kuz7rZrL3dm2ZhK6Vg0N8No1yM3Ur0HPPHUxGiZxbjU= github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= +github.com/ava-labs/avalanchego v1.10.6 h1:O3nFob1jViBEyYIfIJ/cMHmuZ604dgD0nkgl2K6Oujk= +github.com/ava-labs/avalanchego v1.10.6/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -123,6 +127,7 @@ github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 h1:ytcWPaNPhNoG github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811/go.mod h1:Nb5lgvnQ2+oGlE/EyZy4+2/CxRh9KfvCXnag1vtpxVM= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= @@ -209,6 +214,7 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -372,13 +378,16 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8= github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE= @@ -409,6 +418,7 @@ github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4F github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= +github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.6 h1:5ibWZ6iY0NctNGWo87LalDlEZ6R41TqbbDamhfG/Qzo= github.com/magiconair/properties v1.8.6/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= @@ -450,9 +460,11 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= @@ -1024,6 +1036,7 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +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= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/gossip/gossip.go b/gossip/gossip.go index e14a715933..0f36fd7d5e 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -8,11 +8,11 @@ import ( "sync" "time" - "github.com/ava-labs/avalanchego/x/p2p" "github.com/ethereum/go-ethereum/log" "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" ) // GossipableAny exists to help create non-nil pointers to a concrete Gossipable @@ -76,39 +76,8 @@ func (g *Gossiper[T, U]) Pull( continue } - onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { - if err != nil { - log.Debug("failed gossip request", "nodeID", nodeID, "error", err) - return - } - - response := PullGossipResponse{} - if _, err := g.codec.Unmarshal(responseBytes, &response); err != nil { - log.Debug("failed to unmarshal gossip response", "error", err) - return - } - - for _, gossipBytes := range response.GossipBytes { - gossipable := U(new(T)) - if err := gossipable.Unmarshal(gossipBytes); err != nil { - log.Debug("failed to unmarshal gossip", "error", err, "nodeID", nodeID) - continue - } - - ok, err := g.set.Add(gossipable) - if err != nil { - log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetID()) - continue - } - if !ok { - log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetID()) - continue - } - } - } - for i := 0; i < g.gossipSize; i++ { - if err := g.client.AppRequestAny(context.TODO(), msgBytes, onResponse); err != nil { + if err := g.client.AppRequestAny(context.TODO(), msgBytes, g.handleResponse); err != nil { log.Warn("failed to gossip", "error", err) continue } @@ -119,3 +88,29 @@ func (g *Gossiper[T, U]) Pull( } } } + +func (g *Gossiper[T, U]) handleResponse(nodeID ids.NodeID, responseBytes []byte, err error) { + if err != nil { + log.Debug("failed gossip request", "nodeID", nodeID, "error", err) + return + } + + response := PullGossipResponse{} + if _, err := g.codec.Unmarshal(responseBytes, &response); err != nil { + log.Debug("failed to unmarshal gossip response", "error", err) + return + } + + for _, gossipBytes := range response.GossipBytes { + gossipable := U(new(T)) + if err := gossipable.Unmarshal(gossipBytes); err != nil { + log.Debug("failed to unmarshal gossip", "error", err, "nodeID", nodeID) + continue + } + + if err := g.set.Add(gossipable); err != nil { + log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetID()) + continue + } + } +} diff --git a/gossip/gossipable.go b/gossip/gossipable.go index aabaf9cebd..0f72057520 100644 --- a/gossip/gossipable.go +++ b/gossip/gossipable.go @@ -18,7 +18,7 @@ type Gossipable interface { // Set holds a set of known Gossipable items type Set[T Gossipable] interface { // Add adds a Gossipable to the set - Add(gossipable T) (bool, error) + Add(gossipable T) error // Get returns elements that match the provided filter function Get(filter func(gossipable T) bool) []T // GetBloomFilter returns a bloom filter representing the items in Set. diff --git a/gossip/handler.go b/gossip/handler.go index 93f777c17c..00233e47fd 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -7,12 +7,16 @@ import ( "context" "time" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ethereum/go-ethereum/log" bloomfilter "github.com/holiman/bloomfilter/v2" "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" ) +var _ p2p.Handler = (*Handler[Gossipable])(nil) + func NewHandler[T Gossipable](set Set[T], codec codec.Manager, codecVersion uint16) *Handler[T] { return &Handler[T]{ set: set, @@ -27,18 +31,16 @@ type Handler[T Gossipable] struct { codecVersion uint16 } -func (Handler[T]) AppGossip(context.Context, ids.NodeID, []byte) error { - return nil -} - -func (h Handler[T]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ time.Time, requestBytes []byte) ([]byte, error) { +func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time, requestBytes []byte) ([]byte, error) { request := PullGossipRequest{} if _, err := h.codec.Unmarshal(requestBytes, &request); err != nil { - return nil, err + log.Info("failed to unmarshal gossip request", "nodeID", nodeID, "err", err) + return nil, nil } peerFilter := &bloomfilter.Filter{} if err := peerFilter.UnmarshalBinary(request.BloomFilter); err != nil { - return nil, err + log.Debug("failed to unmarshal bloom filter", "nodeID", nodeID, "err", err) + return nil, nil } // filter out what the requesting peer already knows about @@ -65,6 +67,10 @@ func (h Handler[T]) AppRequest(_ context.Context, _ ids.NodeID, _ uint32, _ time return responseBytes, nil } -func (Handler[T]) CrossChainAppRequest(context.Context, ids.ID, uint32, time.Time, []byte) ([]byte, error) { +func (h Handler[T]) AppGossip(context.Context, ids.NodeID, []byte) error { + return nil +} + +func (h Handler[T]) CrossChainAppRequest(context.Context, ids.ID, time.Time, []byte) ([]byte, error) { return nil, nil } diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index f842a683e3..e32169d199 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -38,12 +38,12 @@ type testMempool struct { lock sync.Mutex } -func (t *testMempool) Add(tx *testTx) (bool, error) { +func (t *testMempool) Add(tx *testTx) error { t.lock.Lock() defer t.lock.Unlock() t.mempool = append(t.mempool, tx) - return true, nil + return nil } func (t *testMempool) Get(filter func(tx *testTx) bool) []*testTx { diff --git a/peer/network.go b/peer/network.go index b08e86a3bf..c7b3ad483d 100644 --- a/peer/network.go +++ b/peer/network.go @@ -10,9 +10,10 @@ import ( "sync" "time" - "github.com/ava-labs/avalanchego/x/p2p" "golang.org/x/sync/semaphore" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ethereum/go-ethereum/log" "github.com/ava-labs/avalanchego/codec" @@ -336,14 +337,13 @@ func (n *network) AppRequest(ctx context.Context, nodeID ids.NodeID, requestID u log.Debug("received AppRequest from node", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request)) - if err := n.router.AppRequest(ctx, nodeID, requestID, deadline, request); err == nil { - return nil - } - // if we errored, it's possible that this is a non-sdk request - var req message.Request if _, err := n.codec.Unmarshal(request, &req); err != nil { - log.Debug("failed to unmarshal app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) + // this might be an sdk request + if err := n.router.AppRequest(ctx, nodeID, requestID, deadline, request); err == nil { + log.Debug("failed to unmarshal app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) + } + return nil } @@ -384,13 +384,13 @@ func (n *network) AppResponse(ctx context.Context, nodeID ids.NodeID, requestID log.Debug("received AppResponse from peer", "nodeID", nodeID, "requestID", requestID) - if err := n.router.AppResponse(ctx, nodeID, requestID, response); err == nil { - return nil - } - // if we errored, it's possible that this is a non-sdk response - handler, exists := n.markRequestFulfilled(requestID) if !exists { + // this might be an sdk response + if err := n.router.AppResponse(ctx, nodeID, requestID, response); err == nil { + return nil + } + // Should never happen since the engine should be managing outstanding requests log.Error("received AppResponse to unknown request", "nodeID", nodeID, "requestID", requestID, "responseLen", len(response)) return nil diff --git a/peer/network_test.go b/peer/network_test.go index 87384ea9be..8fdf99c7fa 100644 --- a/peer/network_test.go +++ b/peer/network_test.go @@ -12,9 +12,10 @@ import ( "testing" "time" + "github.com/ava-labs/avalanchego/network/p2p" "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/x/p2p" ethcommon "github.com/ethereum/go-ethereum/common" @@ -55,7 +56,7 @@ var ( func TestNetworkDoesNotConnectToItself(t *testing.T) { selfNodeID := ids.GenerateTestNodeID() - n := NewNetwork(p2p.NewRouter(nil), nil, nil, nil, selfNodeID, 1, 1) + n := NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), nil, nil, nil, selfNodeID, 1, 1) assert.NoError(t, n.Connected(context.Background(), selfNodeID, defaultPeerVersion)) assert.EqualValues(t, 0, n.Size()) } @@ -91,7 +92,7 @@ func TestRequestAnyRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) net.SetRequestHandler(&HelloGreetingRequestHandler{codec: codecManager}) client := NewNetworkClient(net) nodeID := ids.GenerateTestNodeID() @@ -166,7 +167,7 @@ func TestRequestRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 16, 16) net.SetRequestHandler(&HelloGreetingRequestHandler{codec: codecManager}) client := NewNetworkClient(net) @@ -246,7 +247,7 @@ func TestAppRequestOnShutdown(t *testing.T) { codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) client := NewNetworkClient(net) nodeID := ids.GenerateTestNodeID() require.NoError(t, net.Connected(context.Background(), nodeID, defaultPeerVersion)) @@ -295,7 +296,7 @@ func TestRequestMinVersion(t *testing.T) { } // passing nil as codec works because the net.AppRequest is never called - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 16) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 16) client := NewNetworkClient(net) requestMessage := TestMessage{Message: "this is a request"} requestBytes, err := message.RequestToBytes(codecManager, requestMessage) @@ -358,7 +359,7 @@ func TestOnRequestHonoursDeadline(t *testing.T) { processingDuration: 500 * time.Millisecond, } - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetRequestHandler(requestHandler) nodeID := ids.GenerateTestNodeID() @@ -398,7 +399,7 @@ func TestGossip(t *testing.T) { } gossipHandler := &testGossipHandler{} - clientNetwork = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(gossipHandler) assert.NoError(t, clientNetwork.Connected(context.Background(), nodeID, defaultPeerVersion)) @@ -425,7 +426,7 @@ func TestHandleInvalidMessages(t *testing.T) { requestID := uint32(1) sender := testAppSender{} - clientNetwork := NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork := NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) clientNetwork.SetRequestHandler(&testRequestHandler{}) @@ -475,7 +476,7 @@ func TestNetworkPropagatesRequestHandlerError(t *testing.T) { requestID := uint32(1) sender := testAppSender{} - clientNetwork := NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + clientNetwork := NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) clientNetwork.SetRequestHandler(&testRequestHandler{err: errors.New("fail")}) // Return an error from the request handler @@ -515,7 +516,7 @@ func TestCrossChainAppRequest(t *testing.T) { }, } - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetCrossChainRequestHandler(&testCrossChainHandler{codec: crossChainCodecManager}) client := NewNetworkClient(net) @@ -570,7 +571,7 @@ func TestCrossChainRequestRequestsRoutingAndResponse(t *testing.T) { codecManager := buildCodec(t, TestMessage{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) net.SetCrossChainRequestHandler(&testCrossChainHandler{codec: crossChainCodecManager}) client := NewNetworkClient(net) @@ -630,7 +631,7 @@ func TestCrossChainRequestOnShutdown(t *testing.T) { } codecManager := buildCodec(t, TestMessage{}) crossChainCodecManager := buildCodec(t, ExampleCrossChainRequest{}, ExampleCrossChainResponse{}) - net = NewNetwork(p2p.NewRouter(nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) + net = NewNetwork(p2p.NewRouter(logging.NoLog{}, nil), sender, codecManager, crossChainCodecManager, ids.EmptyNodeID, 1, 1) client := NewNetworkClient(net) exampleCrossChainRequest := ExampleCrossChainRequest{ diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index ba6c746d15..aa9723e300 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -21,87 +21,12 @@ import ( ) var ( - _ gossip.Set[*GossipAtomicTx] = (*GossipAtomicMempool)(nil) - _ gossip.Gossipable = (*GossipAtomicTx)(nil) - _ gossip.Set[*GossipEthTx] = (*GossipEthTxPool)(nil) _ gossip.Gossipable = (*GossipEthTx)(nil) ) -func NewGossipAtomicMempool(Mempool *Mempool) (*GossipAtomicMempool, error) { - bloom, err := bloomfilter.New(gossip.DefaultBloomM, gossip.DefaultBloomK) - if err != nil { - return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) - } - - return &GossipAtomicMempool{ - mempool: Mempool, - bloom: bloom, - }, nil -} - -type GossipAtomicMempool struct { - mempool *Mempool - bloom *bloomfilter.Filter - lock sync.RWMutex -} - -func (g *GossipAtomicMempool) Add(tx *GossipAtomicTx) (bool, error) { - ok, err := g.mempool.AddTx(tx.Tx) - if err != nil { - if !tx.Local { - // unlike local txs, invalid remote txs are recorded as discarded - // so that they won't be requested again - txID := tx.Tx.ID() - g.mempool.discardedTxs.Put(txID, tx.Tx) - log.Debug("failed to issue remote tx to mempool", - "txID", txID, - "err", err, - ) - } - return false, err - } - - if !ok { - return false, nil - } - - g.lock.Lock() - defer g.lock.Unlock() - - g.bloom.Add(gossip.NewHasher(tx.GetID())) - g.bloom, _ = gossip.ResetBloomFilterIfNeeded(g.bloom, gossip.DefaultBloomMaxFilledRatio) - - return true, nil -} - -func (g *GossipAtomicMempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { - f := func(tx *Tx) bool { - return filter(&GossipAtomicTx{ - Tx: tx, - }) - } - txs := g.mempool.GetTxs(f) - gossipTxs := make([]*GossipAtomicTx, 0, len(txs)) - for _, tx := range txs { - gossipTxs = append(gossipTxs, &GossipAtomicTx{ - Tx: tx, - }) - } - - return gossipTxs -} - -func (g *GossipAtomicMempool) GetBloomFilter() ([]byte, error) { - g.lock.RLock() - defer g.lock.RUnlock() - - return g.bloom.MarshalBinary() -} - type GossipAtomicTx struct { - Tx *Tx - Local bool + Tx *Tx } func (tx *GossipAtomicTx) GetID() ids.ID { @@ -163,13 +88,12 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync // Add enqueues the transaction to the mempool. Subscribe should be called // to receive an event if tx is actually added to the mempool or not. -func (g *GossipEthTxPool) Add(tx *GossipEthTx) (bool, error) { - err := g.mempool.AddRemotes([]*types.Transaction{tx.Tx})[0] - if err != nil { - return false, err +func (g *GossipEthTxPool) Add(tx *GossipEthTx) error { + if err := g.mempool.AddRemotes([]*types.Transaction{tx.Tx})[0]; err != nil { + return err } - return true, nil + return nil } func (g *GossipEthTxPool) Get(filter func(tx *GossipEthTx) bool) []*GossipEthTx { diff --git a/plugin/evm/gossip_mempool_test.go b/plugin/evm/gossip_mempool_test.go index ec71eaa7fe..f5dbb3a697 100644 --- a/plugin/evm/gossip_mempool_test.go +++ b/plugin/evm/gossip_mempool_test.go @@ -68,17 +68,14 @@ func TestAtomicMempoolAddTx(t *testing.T) { t.Run(tt.name, func(t *testing.T) { require := require.New(t) - m := NewMempool(ids.Empty, 10) - mempool, err := NewGossipAtomicMempool(m) + m, err := NewMempool(ids.Empty, 10) require.NoError(err) for _, add := range tt.add { - ok, err := mempool.Add(add) - require.True(ok) - require.NoError(err) + require.NoError(m.Add(add)) } - txs := mempool.Get(tt.filter) + txs := m.Get(tt.filter) require.Len(txs, len(tt.expected)) for _, expected := range tt.expected { diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index 6cdc95585c..75aabbd9be 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -9,9 +9,11 @@ import ( "sync" "github.com/ethereum/go-ethereum/log" + bloomfilter "github.com/holiman/bloomfilter/v2" "github.com/ava-labs/avalanchego/cache" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/coreth/gossip" "github.com/ava-labs/coreth/metrics" ) @@ -70,12 +72,19 @@ type Mempool struct { txHeap *txHeap // utxoSpenders maps utxoIDs to the transaction consuming them in the mempool utxoSpenders map[ids.ID]*Tx + // bloom is a bloom filter containing the txs in the mempool + bloom *bloomfilter.Filter metrics *mempoolMetrics } // NewMempool returns a Mempool with [maxSize] -func NewMempool(AVAXAssetID ids.ID, maxSize int) *Mempool { +func NewMempool(AVAXAssetID ids.ID, maxSize int) (*Mempool, error) { + bloom, err := bloomfilter.New(gossip.DefaultBloomM, gossip.DefaultBloomK) + if err != nil { + return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) + } + return &Mempool{ AVAXAssetID: AVAXAssetID, issuedTxs: make(map[ids.ID]*Tx), @@ -85,8 +94,9 @@ func NewMempool(AVAXAssetID ids.ID, maxSize int) *Mempool { txHeap: newTxHeap(maxSize), maxSize: maxSize, utxoSpenders: make(map[ids.ID]*Tx), + bloom: bloom, metrics: newMempoolMetrics(), - } + }, nil } // Len returns the number of transactions in the mempool @@ -126,9 +136,13 @@ func (m *Mempool) atomicTxGasPrice(tx *Tx) (uint64, error) { return burned / gasUsed, nil } +func (m *Mempool) Add(tx *GossipAtomicTx) error { + return m.AddTx(tx.Tx) +} + // Add attempts to add [tx] to the mempool and returns an error if // it could not be addeed to the mempool. -func (m *Mempool) AddTx(tx *Tx) (bool, error) { +func (m *Mempool) AddTx(tx *Tx) error { m.lock.Lock() defer m.lock.Unlock() @@ -136,7 +150,7 @@ func (m *Mempool) AddTx(tx *Tx) (bool, error) { } // forceAddTx forcibly adds a *Tx to the mempool and bypasses all verification. -func (m *Mempool) ForceAddTx(tx *Tx) (bool, error) { +func (m *Mempool) ForceAddTx(tx *Tx) error { m.lock.Lock() defer m.lock.Unlock() @@ -177,31 +191,31 @@ func (m *Mempool) checkConflictTx(tx *Tx) (uint64, ids.ID, []*Tx, error) { // addTx adds [tx] to the mempool. Assumes [m.lock] is held. // If [force], skips conflict checks within the mempool. -func (m *Mempool) addTx(tx *Tx, force bool) (bool, error) { +func (m *Mempool) addTx(tx *Tx, force bool) error { txID := tx.ID() // If [txID] has already been issued or is in the currentTxs map // there's no need to add it. if _, exists := m.issuedTxs[txID]; exists { - return false, nil + return nil } if _, exists := m.currentTxs[txID]; exists { - return false, nil + return nil } if _, exists := m.txHeap.Get(txID); exists { - return false, nil + return nil } utxoSet := tx.InputUTXOs() gasPrice, _ := m.atomicTxGasPrice(tx) highestGasPrice, highestGasPriceConflictTxID, conflictingTxs, err := m.checkConflictTx(tx) if err != nil { - return false, err + return err } if len(conflictingTxs) != 0 && !force { // If [tx] does not have a higher fee than all of its conflicts, // we refuse to issue it to the mempool. if highestGasPrice >= gasPrice { - return false, fmt.Errorf( + return fmt.Errorf( "%w: issued tx (%s) gas price %d <= conflict tx (%s) gas price %d (%d total conflicts in mempool)", errConflictingAtomicTx, txID, @@ -226,7 +240,7 @@ func (m *Mempool) addTx(tx *Tx, force bool) (bool, error) { // submitted item, discard the submitted item (we prefer items // already in the mempool). if minGasPrice >= gasPrice { - return false, fmt.Errorf( + return fmt.Errorf( "%w currentMin=%d provided=%d", errInsufficientAtomicTxFee, minGasPrice, @@ -238,7 +252,7 @@ func (m *Mempool) addTx(tx *Tx, force bool) (bool, error) { } else { // This could occur if we have used our entire size allowance on // transactions that are currently processing. - return false, errTooManyAtomicTx + return errTooManyAtomicTx } } @@ -267,7 +281,35 @@ func (m *Mempool) addTx(tx *Tx, force bool) (bool, error) { // and CancelCurrentTx. m.newTxs = append(m.newTxs, tx) m.addPending() - return true, nil + + m.bloom.Add(gossip.NewHasher(tx.ID())) + m.bloom, _ = gossip.ResetBloomFilterIfNeeded(m.bloom, gossip.DefaultBloomMaxFilledRatio) + + return nil +} + +func (m *Mempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { + f := func(tx *Tx) bool { + return filter(&GossipAtomicTx{ + Tx: tx, + }) + } + txs := m.GetTxs(f) + gossipTxs := make([]*GossipAtomicTx, 0, len(txs)) + for _, tx := range txs { + gossipTxs = append(gossipTxs, &GossipAtomicTx{ + Tx: tx, + }) + } + + return gossipTxs +} + +func (m *Mempool) GetBloomFilter() ([]byte, error) { + m.lock.RLock() + defer m.lock.RUnlock() + + return m.bloom.MarshalBinary() } // NextTx returns a transaction to be issued from the mempool. diff --git a/plugin/evm/mempool_atomic_gossiping_test.go b/plugin/evm/mempool_atomic_gossiping_test.go index c9094ef374..684153183c 100644 --- a/plugin/evm/mempool_atomic_gossiping_test.go +++ b/plugin/evm/mempool_atomic_gossiping_test.go @@ -108,16 +108,14 @@ func TestMempoolMaxMempoolSizeHandling(t *testing.T) { // shortcut to simulated almost filled mempool mempool.maxSize = 0 - ok, err := mempool.AddTx(tx) - assert.False(ok) + err := mempool.AddTx(tx) assert.ErrorIs(err, errTooManyAtomicTx) assert.False(mempool.has(tx.ID())) // shortcut to simulated empty mempool mempool.maxSize = defaultMempoolSize - ok, err = mempool.AddTx(tx) - assert.True(ok) + err = mempool.AddTx(tx) assert.NoError(err) assert.True(mempool.has(tx.ID())) } @@ -197,19 +195,16 @@ func TestMempoolPriorityDrop(t *testing.T) { mempool.maxSize = 1 tx1 := createImportTx(t, vm, ids.ID{1}, params.AvalancheAtomicTxFee) - ok, err := mempool.AddTx(tx1) - assert.True(ok) + err := mempool.AddTx(tx1) assert.NoError(err) assert.True(mempool.has(tx1.ID())) tx2 := createImportTx(t, vm, ids.ID{2}, params.AvalancheAtomicTxFee) - ok, err = mempool.AddTx(tx2) - assert.False(ok) + err = mempool.AddTx(tx2) assert.ErrorIs(err, errInsufficientAtomicTxFee) assert.True(mempool.has(tx1.ID())) assert.False(mempool.has(tx2.ID())) tx3 := createImportTx(t, vm, ids.ID{3}, 2*params.AvalancheAtomicTxFee) - ok, err = mempool.AddTx(tx3) - assert.True(ok) + err = mempool.AddTx(tx3) assert.NoError(err) assert.False(mempool.has(tx1.ID())) assert.False(mempool.has(tx2.ID())) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 62cd5ce92a..cd76a23798 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -17,7 +17,7 @@ import ( "time" avalanchegoMetrics "github.com/ava-labs/avalanchego/api/metrics" - "github.com/ava-labs/avalanchego/x/p2p" + "github.com/ava-labs/avalanchego/network/p2p" "github.com/ava-labs/coreth/consensus/dummy" corethConstants "github.com/ava-labs/coreth/constants" @@ -265,7 +265,6 @@ type VM struct { gossiper Gossiper ethTxGossiper *gossip.Gossiper[GossipEthTx, *GossipEthTx] atomicTxGossiper *gossip.Gossiper[GossipAtomicTx, *GossipAtomicTx] - atomicMempool gossip.Set[*GossipAtomicTx] baseCodec codec.Registry codec codec.Manager @@ -504,14 +503,17 @@ func (vm *VM) Initialize( vm.codec = Codec // TODO: read size from settings - vm.mempool = NewMempool(chainCtx.AVAXAssetID, defaultMempoolSize) + vm.mempool, err = NewMempool(chainCtx.AVAXAssetID, defaultMempoolSize) + if err != nil { + return fmt.Errorf("failed to initialize mempool: %w", err) + } if err := vm.initializeMetrics(); err != nil { return err } // initialize peer network - vm.router = p2p.NewRouter(appSender) + vm.router = p2p.NewRouter(vm.ctx.Log, appSender) vm.networkCodec = message.Codec vm.Network = peer.NewNetwork(vm.router, appSender, vm.networkCodec, message.CrossChainCodec, chainCtx.NodeID, vm.config.MaxOutboundActiveRequests, vm.config.MaxOutboundActiveCrossChainRequests) vm.client = peer.NewNetworkClient(vm.Network) @@ -964,12 +966,6 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) - atomicMempool, err := NewGossipAtomicMempool(vm.mempool) - if err != nil { - return err - } - vm.atomicMempool = atomicMempool - ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, vm.codec, message.Version) ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler) if err != nil { @@ -977,7 +973,7 @@ func (vm *VM) initBlockBuilding() error { } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](atomicMempool, vm.codec, message.Version) + atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](vm.mempool, vm.codec, message.Version) atomicTxGossipClient, err := vm.router.RegisterAppProtocol(0x1, atomicTxGossipHandler) if err != nil { return err @@ -996,7 +992,7 @@ func (vm *VM) initBlockBuilding() error { go vm.ethTxGossiper.Pull(vm.shutdownChan, &vm.shutdownWg) vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( - atomicMempool, + vm.mempool, vm.atomicTxGossipClient, vm.networkCodec, message.Version, @@ -1377,8 +1373,19 @@ func (vm *VM) issueTx(tx *Tx, local bool) error { return err } - gossipTx := &GossipAtomicTx{Tx: tx, Local: local} - if _, err := vm.atomicMempool.Add(gossipTx); err != nil { + // add to mempool and possibly re-gossip + if err := vm.mempool.AddTx(tx); err != nil { + if !local { + // unlike local txs, invalid remote txs are recorded as discarded + // so that they won't be requested again + txID := tx.ID() + vm.mempool.discardedTxs.Put(tx.ID(), tx) + log.Debug("failed to issue remote tx to mempool", + "txID", txID, + "err", err, + ) + return nil + } return err } diff --git a/plugin/evm/vm_test.go b/plugin/evm/vm_test.go index 272197b08e..bd3792df26 100644 --- a/plugin/evm/vm_test.go +++ b/plugin/evm/vm_test.go @@ -1043,7 +1043,7 @@ func testConflictingImportTxs(t *testing.T, genesis string) { t.Fatal("Expected issueTx to fail due to conflicting transaction") } // Force issue transaction directly to the mempool - if _, err := vm.mempool.ForceAddTx(tx); err != nil { + if err := vm.mempool.ForceAddTx(tx); err != nil { t.Fatal(err) } <-issuer @@ -1659,7 +1659,7 @@ func TestConflictingTransitiveAncestryWithGap(t *testing.T) { t.Fatalf("Should not have been able to issue import tx with conflict") } // Force issue transaction directly into the mempool - if _, err := vm.mempool.ForceAddTx(importTx0B); err != nil { + if err := vm.mempool.ForceAddTx(importTx0B); err != nil { t.Fatal(err) } <-issuer @@ -3278,12 +3278,12 @@ func TestAtomicTxFailsEVMStateTransferBuildBlock(t *testing.T) { t.Fatal("Should have failed to issue due to an invalid export tx") } - if _, err := vm.mempool.AddTx(exportTx2); err == nil { + if err := vm.mempool.AddTx(exportTx2); err == nil { t.Fatal("Should have failed to add because conflicting") } // Manually add transaction to mempool to bypass validation - if _, err := vm.mempool.ForceAddTx(exportTx2); err != nil { + if err := vm.mempool.ForceAddTx(exportTx2); err != nil { t.Fatal(err) } <-issuer @@ -3341,7 +3341,7 @@ func TestBuildInvalidBlockHead(t *testing.T) { t.Fatal("Should have failed to issue invalid transaction") } // Force issue the transaction directly to the mempool - if _, err := vm.mempool.AddTx(tx); err != nil { + if err := vm.mempool.AddTx(tx); err != nil { t.Fatal(err) } From 020c43f3cef783ebd55bf193c06048af02045bbe Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 3 Aug 2023 15:39:31 -0400 Subject: [PATCH 19/78] nit --- go.sum | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/go.sum b/go.sum index 298fe30c03..c34286fcc9 100644 --- a/go.sum +++ b/go.sum @@ -51,8 +51,6 @@ github.com/VictoriaMetrics/fastcache v1.10.0 h1:5hDJnLsKLpnUEToub7ETuRu8RCkb40wo github.com/VictoriaMetrics/fastcache v1.10.0/go.mod h1:tjiYeEfYXCqacuvYw/7UoDIeJaNxq6132xHICNP77w8= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= @@ -127,7 +125,6 @@ github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 h1:ytcWPaNPhNoG github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811/go.mod h1:Nb5lgvnQ2+oGlE/EyZy4+2/CxRh9KfvCXnag1vtpxVM= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= @@ -214,7 +211,6 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -378,16 +374,13 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= -github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8= github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE= @@ -418,7 +411,6 @@ github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4F github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= -github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.6 h1:5ibWZ6iY0NctNGWo87LalDlEZ6R41TqbbDamhfG/Qzo= github.com/magiconair/properties v1.8.6/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= @@ -460,11 +452,9 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= @@ -1036,7 +1026,6 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -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= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= From 4b3ee0a847919ff865b60a04be3a7006cac3afd0 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 4 Aug 2023 04:52:41 -0400 Subject: [PATCH 20/78] nit --- gossip/bloom.go | 55 +++++++++++++++----- gossip/bloom_test.go | 18 ++++--- gossip/filter.go | 9 ++++ gossip/gossip.go | 9 ++-- gossip/gossipable.go | 14 ++--- gossip/handler.go | 15 +++--- gossip/message.go | 2 +- gossip/test_gossip.go | 29 +++-------- gossip/types.go | 14 +++++ plugin/evm/gossip_mempool.go | 55 ++++++-------------- plugin/evm/gossiper_atomic_gossiping_test.go | 8 +-- plugin/evm/mempool.go | 17 +++--- 12 files changed, 126 insertions(+), 119 deletions(-) create mode 100644 gossip/filter.go create mode 100644 gossip/types.go diff --git a/gossip/bloom.go b/gossip/bloom.go index 26222c0d90..b6a1d542ab 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -8,8 +8,6 @@ import ( "hash" bloomfilter "github.com/holiman/bloomfilter/v2" - - "github.com/ava-labs/avalanchego/ids" ) const ( @@ -20,35 +18,66 @@ const ( DefaultBloomMaxFilledRatio = 0.75 ) -var _ hash.Hash64 = (*hasher)(nil) +var ( + _ Filter = (*BloomFilter)(nil) + _ hash.Hash64 = (*hasher)(nil) +) + +func NewDefaultBloomFilter() (*BloomFilter, error) { + return NewBloomFilter(DefaultBloomM, DefaultBloomK) +} + +func NewBloomFilter(m, k uint64) (*BloomFilter, error) { + bloom, err := bloomfilter.New(m, k) + if err != nil { + return nil, err + } + + return &BloomFilter{ + Bloom: bloom, + }, nil +} + +type BloomFilter struct { + Bloom *bloomfilter.Filter `serialize:"true"` +} + +func (b *BloomFilter) Add(gossipable Gossipable) { + b.Bloom.Add(NewHasher(gossipable.GetHash())) +} + +func (b *BloomFilter) Has(gossipable Gossipable) bool { + return b.Bloom.Contains(NewHasher(gossipable.GetHash())) +} // ResetBloomFilterIfNeeded resets a bloom filter if it breaches a ratio of // filled elements. Returns true if the bloom filter was reset. func ResetBloomFilterIfNeeded( - bloomFilter *bloomfilter.Filter, + bloomFilter *BloomFilter, maxFilledRatio float64, -) (*bloomfilter.Filter, bool) { - if bloomFilter.PreciseFilledRatio() < maxFilledRatio { - return bloomFilter, false +) bool { + if bloomFilter.Bloom.PreciseFilledRatio() < maxFilledRatio { + return false } // it's not possible for this to error assuming that the original // bloom filter's parameters were valid - fresh, _ := bloomfilter.New((*bloomFilter).M(), (*bloomFilter).K()) - return fresh, true + fresh, _ := bloomfilter.New(bloomFilter.Bloom.M(), bloomFilter.Bloom.K()) + bloomFilter.Bloom = fresh + return true } -func NewHasher(id ids.ID) hash.Hash64 { - return hasher{id: id} +func NewHasher(hash Hash) hash.Hash64 { + return hasher{hash: hash} } type hasher struct { hash.Hash64 - id ids.ID + hash Hash } func (h hasher) Sum64() uint64 { - return binary.BigEndian.Uint64(h.id[:]) + return binary.BigEndian.Uint64(h.hash[:]) } func (h hasher) Size() int { diff --git a/gossip/bloom_test.go b/gossip/bloom_test.go index 0c27f557a2..90daf8af87 100644 --- a/gossip/bloom_test.go +++ b/gossip/bloom_test.go @@ -6,7 +6,6 @@ package gossip import ( "testing" - "github.com/ava-labs/avalanchego/ids" bloomfilter "github.com/holiman/bloomfilter/v2" "github.com/stretchr/testify/require" ) @@ -15,27 +14,27 @@ func TestBloomFilterRefresh(t *testing.T) { tests := []struct { name string refreshRatio float64 - add []ids.ID - expected []ids.ID + add []Hash + expected []Hash }{ { name: "no refresh", refreshRatio: 1, - add: []ids.ID{ + add: []Hash{ {0}, }, - expected: []ids.ID{ + expected: []Hash{ {0}, }, }, { name: "refresh", refreshRatio: 0.1, - add: []ids.ID{ + add: []Hash{ {0}, {1}, }, - expected: []ids.ID{ + expected: []Hash{ {1}, }, }, @@ -45,10 +44,13 @@ func TestBloomFilterRefresh(t *testing.T) { t.Run(tt.name, func(t *testing.T) { require := require.New(t) b, err := bloomfilter.New(10, 1) + bloom := BloomFilter{ + Bloom: b, + } require.NoError(err) for _, item := range tt.add { - b, _ = ResetBloomFilterIfNeeded(b, tt.refreshRatio) + _ = ResetBloomFilterIfNeeded(&bloom, tt.refreshRatio) b.Add(NewHasher(item)) } diff --git a/gossip/filter.go b/gossip/filter.go new file mode 100644 index 0000000000..424a636f28 --- /dev/null +++ b/gossip/filter.go @@ -0,0 +1,9 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +type Filter interface { + Add(gossipable Gossipable) + Has(gossipable Gossipable) bool +} diff --git a/gossip/gossip.go b/gossip/gossip.go index 0f36fd7d5e..1f059ecb3f 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -61,14 +61,15 @@ func (g *Gossiper[T, U]) Pull( for { select { case <-gossipTicker.C: - bloomBytes, err := g.set.GetBloomFilter() + filter := g.set.GetFilter() + filterBytes, err := g.codec.Marshal(g.codecVersion, filter) if err != nil { log.Warn("failed to marshal bloom filter", "error", err) continue } request := PullGossipRequest{ - BloomFilter: bloomBytes, + Filter: filterBytes, } msgBytes, err := g.codec.Marshal(g.codecVersion, request) if err != nil { @@ -103,13 +104,13 @@ func (g *Gossiper[T, U]) handleResponse(nodeID ids.NodeID, responseBytes []byte, for _, gossipBytes := range response.GossipBytes { gossipable := U(new(T)) - if err := gossipable.Unmarshal(gossipBytes); err != nil { + if _, err := g.codec.Unmarshal(gossipBytes, gossipable); err != nil { log.Debug("failed to unmarshal gossip", "error", err, "nodeID", nodeID) continue } if err := g.set.Add(gossipable); err != nil { - log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetID()) + log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetHash()) continue } } diff --git a/gossip/gossipable.go b/gossip/gossipable.go index 0f72057520..80c7df9722 100644 --- a/gossip/gossipable.go +++ b/gossip/gossipable.go @@ -3,16 +3,10 @@ package gossip -import "github.com/ava-labs/avalanchego/ids" - // Gossipable is an item that can be gossiped across the network type Gossipable interface { - // GetID represents the unique id of this item - GetID() ids.ID - // Marshal returns the byte representation of this item - Marshal() ([]byte, error) - // Unmarshal deserializes the provided bytes in-place - Unmarshal(b []byte) error + // GetHash represents the unique hash of this item + GetHash() Hash } // Set holds a set of known Gossipable items @@ -21,6 +15,6 @@ type Set[T Gossipable] interface { Add(gossipable T) error // Get returns elements that match the provided filter function Get(filter func(gossipable T) bool) []T - // GetBloomFilter returns a bloom filter representing the items in Set. - GetBloomFilter() ([]byte, error) + // GetFilter returns a filter containing the elements in Set + GetFilter() Filter } diff --git a/gossip/handler.go b/gossip/handler.go index 00233e47fd..57064883a2 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -7,12 +7,10 @@ import ( "context" "time" - "github.com/ava-labs/avalanchego/network/p2p" - "github.com/ethereum/go-ethereum/log" - bloomfilter "github.com/holiman/bloomfilter/v2" - "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ethereum/go-ethereum/log" ) var _ p2p.Handler = (*Handler[Gossipable])(nil) @@ -37,19 +35,20 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time log.Info("failed to unmarshal gossip request", "nodeID", nodeID, "err", err) return nil, nil } - peerFilter := &bloomfilter.Filter{} - if err := peerFilter.UnmarshalBinary(request.BloomFilter); err != nil { + var peerFilter Filter + if _, err := h.codec.Unmarshal(request.Filter, &peerFilter); err != nil { log.Debug("failed to unmarshal bloom filter", "nodeID", nodeID, "err", err) return nil, nil } // filter out what the requesting peer already knows about unknown := h.set.Get(func(gossipable T) bool { - return !peerFilter.Contains(NewHasher(gossipable.GetID())) + return !peerFilter.Has(gossipable) }) + gossipBytes := make([][]byte, 0, len(unknown)) for _, gossipable := range unknown { - bytes, err := gossipable.Marshal() + bytes, err := h.codec.Marshal(h.codecVersion, gossipable) if err != nil { return nil, err } diff --git a/gossip/message.go b/gossip/message.go index 606903028a..082670ca3a 100644 --- a/gossip/message.go +++ b/gossip/message.go @@ -4,7 +4,7 @@ package gossip type PullGossipRequest struct { - BloomFilter []byte `serialize:"true"` + Filter []byte `serialize:"true"` } type PullGossipResponse struct { diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index e32169d199..ffb93c02f5 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -7,8 +7,6 @@ import ( "sync" bloomfilter "github.com/holiman/bloomfilter/v2" - - "github.com/ava-labs/avalanchego/ids" ) var ( @@ -17,20 +15,11 @@ var ( ) type testTx struct { - id ids.ID -} - -func (t *testTx) GetID() ids.ID { - return t.id -} - -func (t *testTx) Marshal() ([]byte, error) { - return t.id[:], nil + hash Hash } -func (t *testTx) Unmarshal(b []byte) error { - copy(t.id[:], b) - return nil +func (t *testTx) GetHash() Hash { + return t.hash } type testMempool struct { @@ -61,18 +50,14 @@ func (t *testMempool) Get(filter func(tx *testTx) bool) []*testTx { return result } -func (t *testMempool) GetBloomFilter() ([]byte, error) { +func (t *testMempool) GetFilter() Filter { t.lock.Lock() defer t.lock.Unlock() - bloom, err := bloomfilter.New(DefaultBloomM, DefaultBloomK) - if err != nil { - return nil, err - } - + bloom, _ := bloomfilter.New(DefaultBloomM, DefaultBloomK) for _, tx := range t.mempool { - bloom.Add(NewHasher(tx.GetID())) + bloom.Add(NewHasher(tx.GetHash())) } - return bloom.MarshalBinary() + return &BloomFilter{Bloom: bloom} } diff --git a/gossip/types.go b/gossip/types.go new file mode 100644 index 0000000000..4839f36c4d --- /dev/null +++ b/gossip/types.go @@ -0,0 +1,14 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +const HashLength = 8 + +type Hash [HashLength]byte + +func HashFromBytes(b []byte) Hash { + h := Hash{} + copy(h[:], b) + return h +} diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index aa9723e300..9957071b8f 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -7,17 +7,12 @@ import ( "fmt" "sync" - "github.com/ava-labs/avalanchego/ids" "github.com/ethereum/go-ethereum/log" - "github.com/ava-labs/coreth/gossip" - "github.com/ava-labs/coreth/plugin/evm/message" - - bloomfilter "github.com/holiman/bloomfilter/v2" - "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/txpool" "github.com/ava-labs/coreth/core/types" + "github.com/ava-labs/coreth/gossip" ) var ( @@ -26,26 +21,16 @@ var ( ) type GossipAtomicTx struct { - Tx *Tx -} - -func (tx *GossipAtomicTx) GetID() ids.ID { - return tx.Tx.ID() -} - -func (tx *GossipAtomicTx) Marshal() ([]byte, error) { - return Codec.Marshal(message.Version, tx.Tx) + Tx *Tx `serialize:"true"` } -func (tx *GossipAtomicTx) Unmarshal(bytes []byte) error { - tx.Tx = &Tx{} - _, err := Codec.Unmarshal(bytes, tx.Tx) - - return err +func (tx *GossipAtomicTx) GetHash() gossip.Hash { + id := tx.Tx.ID() + return gossip.HashFromBytes(id[:]) } func NewGossipEthTxPool(mempool *txpool.TxPool) (*GossipEthTxPool, error) { - bloom, err := bloomfilter.New(gossip.DefaultBloomM, gossip.DefaultBloomK) + bloom, err := gossip.NewDefaultBloomFilter() if err != nil { return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) } @@ -62,7 +47,7 @@ type GossipEthTxPool struct { mempool *txpool.TxPool pendingTxs chan core.NewTxsEvent - bloom *bloomfilter.Filter + bloom *gossip.BloomFilter lock sync.RWMutex } @@ -78,8 +63,8 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync case tx := <-g.pendingTxs: g.lock.Lock() for _, tx := range tx.Txs { - g.bloom.Add(gossip.NewHasher(ids.ID(tx.Hash()))) - g.bloom, _ = gossip.ResetBloomFilterIfNeeded(g.bloom, gossip.DefaultBloomMaxFilledRatio) + g.bloom.Add(&GossipEthTx{Tx: tx}) + _ = gossip.ResetBloomFilterIfNeeded(g.bloom, gossip.DefaultBloomMaxFilledRatio) } g.lock.Unlock() } @@ -114,28 +99,18 @@ func (g *GossipEthTxPool) Get(filter func(tx *GossipEthTx) bool) []*GossipEthTx return result } -func (g *GossipEthTxPool) GetBloomFilter() ([]byte, error) { +func (g *GossipEthTxPool) GetFilter() gossip.Filter { g.lock.RLock() defer g.lock.RUnlock() - return g.bloom.MarshalBinary() + return g.bloom } type GossipEthTx struct { - Tx *types.Transaction -} - -func (tx *GossipEthTx) GetID() ids.ID { - return ids.ID(tx.Tx.Hash()) + Tx *types.Transaction `serialize:"true"` } -func (tx *GossipEthTx) Marshal() ([]byte, error) { - return tx.Tx.MarshalBinary() -} - -func (tx *GossipEthTx) Unmarshal(bytes []byte) error { - tx.Tx = &types.Transaction{} - err := tx.Tx.UnmarshalBinary(bytes) - - return err +func (tx *GossipEthTx) GetHash() gossip.Hash { + hash := tx.Tx.Hash() + return gossip.HashFromBytes(hash[:]) } diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index 3cf5824db4..2074d4cf2b 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -10,7 +10,6 @@ import ( "testing" "time" - bloomfilter "github.com/holiman/bloomfilter/v2" "github.com/stretchr/testify/require" "github.com/ava-labs/avalanchego/ids" @@ -72,9 +71,10 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { _, err := vm.networkCodec.Unmarshal(bytes, &msg) require.NoError(t, err) - bloom := &bloomfilter.Filter{} - require.NoError(t, bloom.UnmarshalBinary(msg.BloomFilter)) - if !bloom.Contains(gossip.NewHasher(tx.ID())) { + bloom := &gossip.BloomFilter{} + _, err = vm.networkCodec.Unmarshal(msg.Filter, bloom) + require.NoError(t, err) + if !bloom.Has(tx) { return nil } addedToBloomFilter = true diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index 75aabbd9be..d483361710 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -8,11 +8,10 @@ import ( "fmt" "sync" - "github.com/ethereum/go-ethereum/log" - bloomfilter "github.com/holiman/bloomfilter/v2" - "github.com/ava-labs/avalanchego/cache" "github.com/ava-labs/avalanchego/ids" + "github.com/ethereum/go-ethereum/log" + "github.com/ava-labs/coreth/gossip" "github.com/ava-labs/coreth/metrics" ) @@ -73,14 +72,14 @@ type Mempool struct { // utxoSpenders maps utxoIDs to the transaction consuming them in the mempool utxoSpenders map[ids.ID]*Tx // bloom is a bloom filter containing the txs in the mempool - bloom *bloomfilter.Filter + bloom *gossip.BloomFilter metrics *mempoolMetrics } // NewMempool returns a Mempool with [maxSize] func NewMempool(AVAXAssetID ids.ID, maxSize int) (*Mempool, error) { - bloom, err := bloomfilter.New(gossip.DefaultBloomM, gossip.DefaultBloomK) + bloom, err := gossip.NewDefaultBloomFilter() if err != nil { return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) } @@ -282,8 +281,8 @@ func (m *Mempool) addTx(tx *Tx, force bool) error { m.newTxs = append(m.newTxs, tx) m.addPending() - m.bloom.Add(gossip.NewHasher(tx.ID())) - m.bloom, _ = gossip.ResetBloomFilterIfNeeded(m.bloom, gossip.DefaultBloomMaxFilledRatio) + m.bloom.Add(&GossipAtomicTx{Tx: tx}) + _ = gossip.ResetBloomFilterIfNeeded(m.bloom, gossip.DefaultBloomMaxFilledRatio) return nil } @@ -305,11 +304,11 @@ func (m *Mempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { return gossipTxs } -func (m *Mempool) GetBloomFilter() ([]byte, error) { +func (m *Mempool) GetFilter() gossip.Filter { m.lock.RLock() defer m.lock.RUnlock() - return m.bloom.MarshalBinary() + return m.bloom } // NextTx returns a transaction to be issued from the mempool. From f5ee8468144844721e1db5e16e18a951d20ef876 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 4 Aug 2023 04:53:04 -0400 Subject: [PATCH 21/78] nit --- gossip/handler.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/gossip/handler.go b/gossip/handler.go index 57064883a2..d28e8c8e7c 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -35,6 +35,7 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time log.Info("failed to unmarshal gossip request", "nodeID", nodeID, "err", err) return nil, nil } + var peerFilter Filter if _, err := h.codec.Unmarshal(request.Filter, &peerFilter); err != nil { log.Debug("failed to unmarshal bloom filter", "nodeID", nodeID, "err", err) @@ -58,6 +59,7 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time response := PullGossipResponse{ GossipBytes: gossipBytes, } + responseBytes, err := h.codec.Marshal(h.codecVersion, response) if err != nil { return nil, err From f925c4337ab8357af9e73dd9a0e858e5bb6f52c3 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 4 Aug 2023 04:58:13 -0400 Subject: [PATCH 22/78] nit --- gossip/gossip.go | 5 +---- gossip/gossip_test.go | 2 +- plugin/evm/vm.go | 4 ++-- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/gossip/gossip.go b/gossip/gossip.go index 1f059ecb3f..326adf9637 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -48,10 +48,7 @@ type Gossiper[T any, U GossipableAny[T]] struct { frequency time.Duration } -func (g *Gossiper[T, U]) Pull( - shutdownChan chan struct{}, - shutdownWg *sync.WaitGroup, -) { +func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.WaitGroup) { gossipTicker := time.NewTicker(g.frequency) defer func() { gossipTicker.Stop() diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 51667872ff..3cc49d8b65 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -22,7 +22,7 @@ func TestPullGossiperShutdown(t *testing.T) { wg := &sync.WaitGroup{} wg.Add(1) - go puller.Pull(done, wg) + go puller.Gossip(done, wg) close(done) wg.Wait() diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index cd76a23798..b54ff043a0 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -989,7 +989,7 @@ func (vm *VM) initBlockBuilding() error { pullTxsFrequency, ) vm.shutdownWg.Add(1) - go vm.ethTxGossiper.Pull(vm.shutdownChan, &vm.shutdownWg) + go vm.ethTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( vm.mempool, @@ -1000,7 +1000,7 @@ func (vm *VM) initBlockBuilding() error { pullTxsFrequency, ) vm.shutdownWg.Add(1) - go vm.atomicTxGossiper.Pull(vm.shutdownChan, &vm.shutdownWg) + go vm.atomicTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) return nil } From dd8bb71f22e8895dbf416d39cbede6fb7dfd468a Mon Sep 17 00:00:00 2001 From: Darioush Jalali Date: Mon, 7 Aug 2023 15:20:04 -0700 Subject: [PATCH 23/78] changes --- RELEASES.md | 3 ++ go.mod | 2 +- go.sum | 6 ++-- gossip/bloom_test.go | 8 ++--- plugin/evm/gossiper_atomic_gossiping_test.go | 6 ++-- plugin/evm/mempool.go | 36 ++++++-------------- plugin/evm/message/codec.go | 7 ---- plugin/evm/vm.go | 6 ++-- scripts/versions.sh | 2 +- 9 files changed, 27 insertions(+), 49 deletions(-) diff --git a/RELEASES.md b/RELEASES.md index d09c8953ff..1304029c18 100644 --- a/RELEASES.md +++ b/RELEASES.md @@ -2,6 +2,9 @@ ## [v0.12.5](https://github.com/ava-labs/coreth/releases/tag/v0.12.5) +- Migrate geth changes from v1.11.4 through v1.12.0 +- Bump AvalancheGo dependency to v1.10.6-rc.4 to use sized LRU for Chain State Cache + ## [v0.12.4](https://github.com/ava-labs/coreth/releases/tag/v0.12.4) - Fix API handler crash for `lookupState` in `prestate` tracer diff --git a/go.mod b/go.mod index 742b59b9c3..b801d62ce7 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,7 @@ go 1.19 require ( github.com/VictoriaMetrics/fastcache v1.10.0 - github.com/ava-labs/avalanchego v1.10.6 + github.com/ava-labs/avalanchego v1.10.7 github.com/cespare/cp v0.1.0 github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 github.com/davecgh/go-spew v1.1.1 diff --git a/go.sum b/go.sum index c34286fcc9..4b2660c5c7 100644 --- a/go.sum +++ b/go.sum @@ -55,10 +55,8 @@ github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9 h1:kuz7rZrL3dm2ZhK6Vg0N8No1yM3Ur0HPPHUxGiZxbjU= -github.com/ava-labs/avalanchego v1.10.6-0.20230801003740-69f3c28399f9/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= -github.com/ava-labs/avalanchego v1.10.6 h1:O3nFob1jViBEyYIfIJ/cMHmuZ604dgD0nkgl2K6Oujk= -github.com/ava-labs/avalanchego v1.10.6/go.mod h1:rXAX4UaE9ORIEJcMyzN6ibv4rnLwv0zUIPLmzA0MCno= +github.com/ava-labs/avalanchego v1.10.7 h1:E1cjNKzWqHrxcvNPAaTwa5cnUH2h6i+6DEDFAGayHZ0= +github.com/ava-labs/avalanchego v1.10.7/go.mod h1:9ekFPMV5ZMIJfsK3BXj0UGW0TzQznke8bwciP3Aav4U= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= diff --git a/gossip/bloom_test.go b/gossip/bloom_test.go index 90daf8af87..4ef018dfb3 100644 --- a/gossip/bloom_test.go +++ b/gossip/bloom_test.go @@ -44,20 +44,20 @@ func TestBloomFilterRefresh(t *testing.T) { t.Run(tt.name, func(t *testing.T) { require := require.New(t) b, err := bloomfilter.New(10, 1) + require.NoError(err) bloom := BloomFilter{ Bloom: b, } - require.NoError(err) for _, item := range tt.add { _ = ResetBloomFilterIfNeeded(&bloom, tt.refreshRatio) - b.Add(NewHasher(item)) + bloom.Bloom.Add(NewHasher(item)) } - require.Equal(uint64(len(tt.expected)), b.N()) + require.Equal(uint64(len(tt.expected)), bloom.Bloom.N()) for _, expected := range tt.expected { - require.True(b.Contains(NewHasher(expected))) + require.True(bloom.Bloom.Contains(NewHasher(expected))) } }) } diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index 2074d4cf2b..349e893e8b 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -10,15 +10,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/coreth/gossip" - - "github.com/stretchr/testify/assert" - "github.com/ava-labs/coreth/plugin/evm/message" ) @@ -74,7 +72,7 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { bloom := &gossip.BloomFilter{} _, err = vm.networkCodec.Unmarshal(msg.Filter, bloom) require.NoError(t, err) - if !bloom.Has(tx) { + if !bloom.Has(&GossipAtomicTx{tx}) { return nil } addedToBloomFilter = true diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index d483361710..0412d15dd6 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -288,17 +288,16 @@ func (m *Mempool) addTx(tx *Tx, force bool) error { } func (m *Mempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { - f := func(tx *Tx) bool { - return filter(&GossipAtomicTx{ - Tx: tx, - }) - } - txs := m.GetTxs(f) - gossipTxs := make([]*GossipAtomicTx, 0, len(txs)) - for _, tx := range txs { - gossipTxs = append(gossipTxs, &GossipAtomicTx{ - Tx: tx, - }) + m.lock.RLock() + defer m.lock.RUnlock() + + gossipTxs := make([]*GossipAtomicTx, 0, len(m.txHeap.maxHeap.items)) + for _, item := range m.txHeap.maxHeap.items { + gossipTx := &GossipAtomicTx{Tx: item.tx} + if !filter(gossipTx) { + continue + } + gossipTxs = append(gossipTxs, gossipTx) } return gossipTxs @@ -339,21 +338,6 @@ func (m *Mempool) GetPendingTx(txID ids.ID) (*Tx, bool) { return m.txHeap.Get(txID) } -func (m *Mempool) GetTxs(filter func(tx *Tx) bool) []*Tx { - m.lock.RLock() - defer m.lock.RUnlock() - - result := make([]*Tx, 0, len(m.txHeap.maxHeap.items)) - for _, item := range m.txHeap.maxHeap.items { - if !filter(item.tx) { - continue - } - result = append(result, item.tx) - } - - return result -} - // GetTx returns the transaction [txID] if it was issued // by this node and returns whether it was dropped and whether // it exists. diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index 2ebcc1afce..2209893e2a 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -8,8 +8,6 @@ import ( "github.com/ava-labs/avalanchego/codec/linearcodec" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/utils/wrappers" - - "github.com/ava-labs/coreth/gossip" ) const ( @@ -42,11 +40,6 @@ func init() { c.RegisterType(LeafsResponse{}), c.RegisterType(CodeRequest{}), c.RegisterType(CodeResponse{}), - - // p2p sdk gossip types - c.RegisterType(gossip.PullGossipRequest{}), - c.RegisterType(gossip.PullGossipResponse{}), - Codec.RegisterCodec(Version, c), ) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index b54ff043a0..2c0263496a 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -124,9 +124,10 @@ const ( codecVersion = uint16(0) secpFactoryCacheSize = 1024 - decidedCacheSize = 100 + decidedCacheSize = 10 * units.MiB missingCacheSize = 50 - unverifiedCacheSize = 50 + unverifiedCacheSize = 5 * units.MiB + bytesToIDCacheSize = 5 * units.MiB targetAtomicTxsSize = 40 * units.KiB @@ -691,6 +692,7 @@ func (vm *VM) initChainState(lastAcceptedBlock *types.Block) error { DecidedCacheSize: decidedCacheSize, MissingCacheSize: missingCacheSize, UnverifiedCacheSize: unverifiedCacheSize, + BytesToIDCacheSize: bytesToIDCacheSize, GetBlockIDAtHeight: vm.GetBlockIDAtHeight, GetBlock: vm.getBlock, UnmarshalBlock: vm.parseBlock, diff --git a/scripts/versions.sh b/scripts/versions.sh index ea31ca7aee..378be3c8ac 100644 --- a/scripts/versions.sh +++ b/scripts/versions.sh @@ -1,4 +1,4 @@ #!/usr/bin/env bash # Don't export them as they're used in the context of other calls -avalanche_version=${AVALANCHE_VERSION:-'v1.10.5-rc.1'} +avalanche_version=${AVALANCHE_VERSION:-'v1.10.6-rc.4'} From c401a7e5765eafe5b34a666057bcfffb96097683 Mon Sep 17 00:00:00 2001 From: Darioush Jalali Date: Mon, 7 Aug 2023 15:22:28 -0700 Subject: [PATCH 24/78] revert --- plugin/evm/message/codec.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index 2209893e2a..2ebcc1afce 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -8,6 +8,8 @@ import ( "github.com/ava-labs/avalanchego/codec/linearcodec" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/utils/wrappers" + + "github.com/ava-labs/coreth/gossip" ) const ( @@ -40,6 +42,11 @@ func init() { c.RegisterType(LeafsResponse{}), c.RegisterType(CodeRequest{}), c.RegisterType(CodeResponse{}), + + // p2p sdk gossip types + c.RegisterType(gossip.PullGossipRequest{}), + c.RegisterType(gossip.PullGossipResponse{}), + Codec.RegisterCodec(Version, c), ) From 5aaff3799e8184a37a81e98eeaf7f2ebb640245d Mon Sep 17 00:00:00 2001 From: Darioush Jalali Date: Mon, 7 Aug 2023 15:40:57 -0700 Subject: [PATCH 25/78] use consistent version --- scripts/versions.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/versions.sh b/scripts/versions.sh index 378be3c8ac..ade1d20c48 100644 --- a/scripts/versions.sh +++ b/scripts/versions.sh @@ -1,4 +1,4 @@ #!/usr/bin/env bash # Don't export them as they're used in the context of other calls -avalanche_version=${AVALANCHE_VERSION:-'v1.10.6-rc.4'} +avalanche_version=${AVALANCHE_VERSION:-'v1.10.7'} From fb5b1ac6d5bddbef309253b76e9272cafb83629c Mon Sep 17 00:00:00 2001 From: Darioush Jalali Date: Mon, 7 Aug 2023 15:45:45 -0700 Subject: [PATCH 26/78] bump go version --- .github/workflows/ci.yml | 8 ++++---- Dockerfile | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index dbe7224ada..674924e847 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -31,7 +31,7 @@ jobs: token: ${{ secrets.AVALANCHE_PAT }} - uses: actions/setup-go@v3 with: - go-version: "1.19" + go-version: "~1.19.12" - name: change avalanchego dep if: ${{ github.event_name == 'workflow_dispatch' }} run: | @@ -51,7 +51,7 @@ jobs: runs-on: ${{ matrix.os }} strategy: matrix: - go: ['1.19'] + go: ['~1.19.12'] os: [macos-11.0, ubuntu-20.04, windows-latest] steps: - uses: actions/checkout@v3 @@ -84,7 +84,7 @@ jobs: runs-on: ${{ matrix.os }} strategy: matrix: - go: ['1.19'] + go: ['~1.19.12'] os: [ubuntu-20.04] steps: - uses: actions/checkout@v3 @@ -115,7 +115,7 @@ jobs: runs-on: ${{ matrix.os }} strategy: matrix: - go: [ '1.19' ] + go: [ '~1.19.12' ] os: [ ubuntu-20.04 ] steps: - uses: actions/checkout@v3 diff --git a/Dockerfile b/Dockerfile index fe7f7a7f52..c3cf033c16 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,5 +1,5 @@ # ============= Compilation Stage ================ -FROM golang:1.19.10-buster AS builder +FROM golang:1.19.12-buster AS builder RUN apt-get update && apt-get install -y --no-install-recommends bash=5.0-4 make=4.2.1-1.2 gcc=4:8.3.0-1 musl-dev=1.1.21-2 ca-certificates=20200601~deb10u2 linux-headers-amd64 From 236e426f284cfb0fc3a803c5f238746bdd7dd38d Mon Sep 17 00:00:00 2001 From: Darioush Jalali Date: Mon, 7 Aug 2023 15:47:17 -0700 Subject: [PATCH 27/78] fix --- Dockerfile | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/Dockerfile b/Dockerfile index c3cf033c16..5f088fb36c 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,7 +1,5 @@ # ============= Compilation Stage ================ -FROM golang:1.19.12-buster AS builder - -RUN apt-get update && apt-get install -y --no-install-recommends bash=5.0-4 make=4.2.1-1.2 gcc=4:8.3.0-1 musl-dev=1.1.21-2 ca-certificates=20200601~deb10u2 linux-headers-amd64 +FROM golang:1.19.12-bullseye AS builder ARG AVALANCHE_VERSION From 66bbcac96509a164a01ab8381bd88cd7e346d85a Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 03:12:52 -0400 Subject: [PATCH 28/78] nit --- gossip/bloom.go | 50 ++++--- gossip/bloom_test.go | 30 ++-- gossip/filter.go | 9 -- gossip/gossip.go | 26 ++-- gossip/gossip_test.go | 12 +- gossip/gossipable.go | 4 +- gossip/handler.go | 11 +- gossip/message.go | 3 +- gossip/test_gossip.go | 20 ++- plugin/evm/gossip_mempool.go | 37 ++++- plugin/evm/gossiper_atomic_gossiping_test.go | 7 +- plugin/evm/mempool.go | 8 +- plugin/evm/tx_gossip_test.go | 138 +++++++++++++++++++ plugin/evm/vm.go | 20 ++- 14 files changed, 267 insertions(+), 108 deletions(-) delete mode 100644 gossip/filter.go create mode 100644 plugin/evm/tx_gossip_test.go diff --git a/gossip/bloom.go b/gossip/bloom.go index b6a1d542ab..b4f33aea02 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -10,44 +10,40 @@ import ( bloomfilter "github.com/holiman/bloomfilter/v2" ) -const ( - DefaultBloomM = 8 * 1024 // 8 KiB - DefaultBloomK = 4 - // DefaultBloomMaxFilledRatio is the max ratio of filled slots in the bloom - // filter before we reset it to avoid too many collisions. - DefaultBloomMaxFilledRatio = 0.75 -) - -var ( - _ Filter = (*BloomFilter)(nil) - _ hash.Hash64 = (*hasher)(nil) -) - -func NewDefaultBloomFilter() (*BloomFilter, error) { - return NewBloomFilter(DefaultBloomM, DefaultBloomK) -} +var _ hash.Hash64 = (*hasher)(nil) -func NewBloomFilter(m, k uint64) (*BloomFilter, error) { - bloom, err := bloomfilter.New(m, k) +func NewBloomFilter(m uint64, p float64) (*BloomFilter, error) { + bloom, err := bloomfilter.NewOptimal(m, p) if err != nil { return nil, err } return &BloomFilter{ - Bloom: bloom, + bloom: bloom, }, nil } type BloomFilter struct { - Bloom *bloomfilter.Filter `serialize:"true"` + bloom *bloomfilter.Filter } func (b *BloomFilter) Add(gossipable Gossipable) { - b.Bloom.Add(NewHasher(gossipable.GetHash())) + b.bloom.Add(NewHasher(gossipable)) } func (b *BloomFilter) Has(gossipable Gossipable) bool { - return b.Bloom.Contains(NewHasher(gossipable.GetHash())) + return b.bloom.Contains(NewHasher(gossipable)) +} + +func (b *BloomFilter) Marshal() ([]byte, error) { + return b.bloom.MarshalBinary() +} + +func (b *BloomFilter) Unmarshal(data []byte) error { + bloom := &bloomfilter.Filter{} + err := bloom.UnmarshalBinary(data) + b.bloom = bloom + return err } // ResetBloomFilterIfNeeded resets a bloom filter if it breaches a ratio of @@ -56,19 +52,19 @@ func ResetBloomFilterIfNeeded( bloomFilter *BloomFilter, maxFilledRatio float64, ) bool { - if bloomFilter.Bloom.PreciseFilledRatio() < maxFilledRatio { + if bloomFilter.bloom.PreciseFilledRatio() < maxFilledRatio { return false } // it's not possible for this to error assuming that the original // bloom filter's parameters were valid - fresh, _ := bloomfilter.New(bloomFilter.Bloom.M(), bloomFilter.Bloom.K()) - bloomFilter.Bloom = fresh + fresh, _ := bloomfilter.New(bloomFilter.bloom.M(), bloomFilter.bloom.K()) + bloomFilter.bloom = fresh return true } -func NewHasher(hash Hash) hash.Hash64 { - return hasher{hash: hash} +func NewHasher(gossipable Gossipable) hash.Hash64 { + return hasher{hash: gossipable.GetHash()} } type hasher struct { diff --git a/gossip/bloom_test.go b/gossip/bloom_test.go index 4ef018dfb3..a31e0683eb 100644 --- a/gossip/bloom_test.go +++ b/gossip/bloom_test.go @@ -14,28 +14,28 @@ func TestBloomFilterRefresh(t *testing.T) { tests := []struct { name string refreshRatio float64 - add []Hash - expected []Hash + add []*testTx + expected []*testTx }{ { name: "no refresh", refreshRatio: 1, - add: []Hash{ - {0}, + add: []*testTx{ + {hash: Hash{0}}, }, - expected: []Hash{ - {0}, + expected: []*testTx{ + {hash: Hash{0}}, }, }, { name: "refresh", refreshRatio: 0.1, - add: []Hash{ - {0}, - {1}, + add: []*testTx{ + {hash: Hash{0}}, + {hash: Hash{1}}, }, - expected: []Hash{ - {1}, + expected: []*testTx{ + {hash: Hash{1}}, }, }, } @@ -46,18 +46,18 @@ func TestBloomFilterRefresh(t *testing.T) { b, err := bloomfilter.New(10, 1) require.NoError(err) bloom := BloomFilter{ - Bloom: b, + bloom: b, } for _, item := range tt.add { _ = ResetBloomFilterIfNeeded(&bloom, tt.refreshRatio) - bloom.Bloom.Add(NewHasher(item)) + bloom.Add(item) } - require.Equal(uint64(len(tt.expected)), bloom.Bloom.N()) + require.Equal(uint64(len(tt.expected)), bloom.bloom.N()) for _, expected := range tt.expected { - require.True(bloom.Bloom.Contains(NewHasher(expected))) + require.True(bloom.Has(expected)) } }) } diff --git a/gossip/filter.go b/gossip/filter.go deleted file mode 100644 index 424a636f28..0000000000 --- a/gossip/filter.go +++ /dev/null @@ -1,9 +0,0 @@ -// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package gossip - -type Filter interface { - Add(gossipable Gossipable) - Has(gossipable Gossipable) bool -} diff --git a/gossip/gossip.go b/gossip/gossip.go index 326adf9637..8cf52b29b3 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -21,35 +21,37 @@ type GossipableAny[T any] interface { Gossipable } +type Config struct { + Frequency time.Duration + PollSize int +} + func NewGossiper[T any, U GossipableAny[T]]( + config Config, set Set[U], client *p2p.Client, codec codec.Manager, codecVersion uint16, - gossipSize int, - frequency time.Duration, ) *Gossiper[T, U] { return &Gossiper[T, U]{ + config: config, set: set, client: client, codec: codec, codecVersion: codecVersion, - gossipSize: gossipSize, - frequency: frequency, } } type Gossiper[T any, U GossipableAny[T]] struct { + config Config set Set[U] client *p2p.Client codec codec.Manager codecVersion uint16 - gossipSize int - frequency time.Duration } func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.WaitGroup) { - gossipTicker := time.NewTicker(g.frequency) + gossipTicker := time.NewTicker(g.config.Frequency) defer func() { gossipTicker.Stop() shutdownWg.Done() @@ -58,15 +60,13 @@ func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.Wai for { select { case <-gossipTicker.C: - filter := g.set.GetFilter() - filterBytes, err := g.codec.Marshal(g.codecVersion, filter) + filterBytes, err := g.set.GetBloomFilter().Marshal() if err != nil { log.Warn("failed to marshal bloom filter", "error", err) - continue } request := PullGossipRequest{ - Filter: filterBytes, + FilterBytes: filterBytes, } msgBytes, err := g.codec.Marshal(g.codecVersion, request) if err != nil { @@ -74,7 +74,7 @@ func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.Wai continue } - for i := 0; i < g.gossipSize; i++ { + for i := 0; i < g.config.PollSize; i++ { if err := g.client.AppRequestAny(context.TODO(), msgBytes, g.handleResponse); err != nil { log.Warn("failed to gossip", "error", err) continue @@ -101,7 +101,7 @@ func (g *Gossiper[T, U]) handleResponse(nodeID ids.NodeID, responseBytes []byte, for _, gossipBytes := range response.GossipBytes { gossipable := U(new(T)) - if _, err := g.codec.Unmarshal(gossipBytes, gossipable); err != nil { + if err := gossipable.Unmarshal(gossipBytes); err != nil { log.Debug("failed to unmarshal gossip", "error", err, "nodeID", nodeID) continue } diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 3cc49d8b65..0963cf62b7 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -9,15 +9,9 @@ import ( "time" ) -func TestPullGossiperShutdown(t *testing.T) { - puller := NewGossiper[testTx, *testTx]( - nil, - nil, - nil, - 0, - 0, - time.Hour, - ) +func TestGossiperShutdown(t *testing.T) { + config := Config{Frequency: time.Second} + puller := NewGossiper[testTx, *testTx](config, nil, nil, nil, 0) done := make(chan struct{}) wg := &sync.WaitGroup{} diff --git a/gossip/gossipable.go b/gossip/gossipable.go index 80c7df9722..bb35dd4b35 100644 --- a/gossip/gossipable.go +++ b/gossip/gossipable.go @@ -7,6 +7,8 @@ package gossip type Gossipable interface { // GetHash represents the unique hash of this item GetHash() Hash + Marshal() ([]byte, error) + Unmarshal(bytes []byte) error } // Set holds a set of known Gossipable items @@ -16,5 +18,5 @@ type Set[T Gossipable] interface { // Get returns elements that match the provided filter function Get(filter func(gossipable T) bool) []T // GetFilter returns a filter containing the elements in Set - GetFilter() Filter + GetBloomFilter() *BloomFilter } diff --git a/gossip/handler.go b/gossip/handler.go index d28e8c8e7c..79177c0775 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -32,24 +32,24 @@ type Handler[T Gossipable] struct { func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time, requestBytes []byte) ([]byte, error) { request := PullGossipRequest{} if _, err := h.codec.Unmarshal(requestBytes, &request); err != nil { - log.Info("failed to unmarshal gossip request", "nodeID", nodeID, "err", err) + log.Debug("failed to unmarshal gossip request", "nodeID", nodeID, "err", err) return nil, nil } - var peerFilter Filter - if _, err := h.codec.Unmarshal(request.Filter, &peerFilter); err != nil { + filter := &BloomFilter{} + if err := filter.Unmarshal(request.FilterBytes); err != nil { log.Debug("failed to unmarshal bloom filter", "nodeID", nodeID, "err", err) return nil, nil } // filter out what the requesting peer already knows about unknown := h.set.Get(func(gossipable T) bool { - return !peerFilter.Has(gossipable) + return !filter.Has(gossipable) }) gossipBytes := make([][]byte, 0, len(unknown)) for _, gossipable := range unknown { - bytes, err := h.codec.Marshal(h.codecVersion, gossipable) + bytes, err := gossipable.Marshal() if err != nil { return nil, err } @@ -59,7 +59,6 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time response := PullGossipResponse{ GossipBytes: gossipBytes, } - responseBytes, err := h.codec.Marshal(h.codecVersion, response) if err != nil { return nil, err diff --git a/gossip/message.go b/gossip/message.go index 082670ca3a..49ae103d04 100644 --- a/gossip/message.go +++ b/gossip/message.go @@ -4,7 +4,8 @@ package gossip type PullGossipRequest struct { - Filter []byte `serialize:"true"` + FilterBytes []byte `serialize:"true"` + SaltBytes []byte `serialize:"true"` } type PullGossipResponse struct { diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index ffb93c02f5..3a74ad51b1 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -5,8 +5,6 @@ package gossip import ( "sync" - - bloomfilter "github.com/holiman/bloomfilter/v2" ) var ( @@ -22,6 +20,16 @@ func (t *testTx) GetHash() Hash { return t.hash } +func (t *testTx) Marshal() ([]byte, error) { + // TODO implement me + panic("implement me") +} + +func (t *testTx) Unmarshal(bytes []byte) error { + // TODO implement me + panic("implement me") +} + type testMempool struct { mempool []*testTx lock sync.Mutex @@ -50,14 +58,14 @@ func (t *testMempool) Get(filter func(tx *testTx) bool) []*testTx { return result } -func (t *testMempool) GetFilter() Filter { +func (t *testMempool) GetBloomFilter() *BloomFilter { t.lock.Lock() defer t.lock.Unlock() - bloom, _ := bloomfilter.New(DefaultBloomM, DefaultBloomK) + bloom, _ := NewBloomFilter(1000, 0.01) for _, tx := range t.mempool { - bloom.Add(NewHasher(tx.GetHash())) + bloom.Add(tx) } - return &BloomFilter{Bloom: bloom} + return bloom } diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 9957071b8f..e08307bc01 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -13,11 +13,13 @@ import ( "github.com/ava-labs/coreth/core/txpool" "github.com/ava-labs/coreth/core/types" "github.com/ava-labs/coreth/gossip" + "github.com/ava-labs/coreth/plugin/evm/message" ) var ( - _ gossip.Set[*GossipEthTx] = (*GossipEthTxPool)(nil) _ gossip.Gossipable = (*GossipEthTx)(nil) + _ gossip.Gossipable = (*GossipAtomicTx)(nil) + _ gossip.Set[*GossipEthTx] = (*GossipEthTxPool)(nil) ) type GossipAtomicTx struct { @@ -29,8 +31,17 @@ func (tx *GossipAtomicTx) GetHash() gossip.Hash { return gossip.HashFromBytes(id[:]) } +func (tx *GossipAtomicTx) Marshal() ([]byte, error) { + return Codec.Marshal(message.Version, tx) +} + +func (tx *GossipAtomicTx) Unmarshal(bytes []byte) error { + _, err := Codec.Unmarshal(bytes, tx) + return err +} + func NewGossipEthTxPool(mempool *txpool.TxPool) (*GossipEthTxPool, error) { - bloom, err := gossip.NewDefaultBloomFilter() + bloom, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomMaxFilledRatio) if err != nil { return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) } @@ -60,11 +71,14 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync case <-shutdownChan: log.Debug("shutting down subscription") return - case tx := <-g.pendingTxs: + case pendingTxs := <-g.pendingTxs: g.lock.Lock() - for _, tx := range tx.Txs { - g.bloom.Add(&GossipEthTx{Tx: tx}) - _ = gossip.ResetBloomFilterIfNeeded(g.bloom, gossip.DefaultBloomMaxFilledRatio) + for _, pendingTx := range pendingTxs.Txs { + tx := &GossipEthTx{Tx: pendingTx} + g.bloom.Add(tx) + if gossip.ResetBloomFilterIfNeeded(g.bloom, txGossipBloomMaxFilledRatio) { + log.Debug("resetting bloom filter", "reason", "reached max filled ratio") + } } g.lock.Unlock() } @@ -99,7 +113,7 @@ func (g *GossipEthTxPool) Get(filter func(tx *GossipEthTx) bool) []*GossipEthTx return result } -func (g *GossipEthTxPool) GetFilter() gossip.Filter { +func (g *GossipEthTxPool) GetBloomFilter() *gossip.BloomFilter { g.lock.RLock() defer g.lock.RUnlock() @@ -114,3 +128,12 @@ func (tx *GossipEthTx) GetHash() gossip.Hash { hash := tx.Tx.Hash() return gossip.HashFromBytes(hash[:]) } + +func (tx *GossipEthTx) Marshal() ([]byte, error) { + return tx.Tx.MarshalBinary() +} + +func (tx *GossipEthTx) Unmarshal(bytes []byte) error { + tx.Tx = &types.Transaction{} + return tx.Tx.UnmarshalBinary(bytes) +} diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index 349e893e8b..dcff4aa160 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -69,10 +69,9 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { _, err := vm.networkCodec.Unmarshal(bytes, &msg) require.NoError(t, err) - bloom := &gossip.BloomFilter{} - _, err = vm.networkCodec.Unmarshal(msg.Filter, bloom) - require.NoError(t, err) - if !bloom.Has(&GossipAtomicTx{tx}) { + filter := &gossip.BloomFilter{} + require.NoError(t, filter.Unmarshal(msg.FilterBytes)) + if !filter.Has(&GossipAtomicTx{Tx: tx}) { return nil } addedToBloomFilter = true diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index 0412d15dd6..c2c564c206 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -79,7 +79,7 @@ type Mempool struct { // NewMempool returns a Mempool with [maxSize] func NewMempool(AVAXAssetID ids.ID, maxSize int) (*Mempool, error) { - bloom, err := gossip.NewDefaultBloomFilter() + bloom, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomMaxFilledRatio) if err != nil { return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) } @@ -282,7 +282,9 @@ func (m *Mempool) addTx(tx *Tx, force bool) error { m.addPending() m.bloom.Add(&GossipAtomicTx{Tx: tx}) - _ = gossip.ResetBloomFilterIfNeeded(m.bloom, gossip.DefaultBloomMaxFilledRatio) + if gossip.ResetBloomFilterIfNeeded(m.bloom, txGossipBloomMaxFilledRatio) { + log.Debug("resetting bloom filter", "reason", "reached max filled ratio") + } return nil } @@ -303,7 +305,7 @@ func (m *Mempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { return gossipTxs } -func (m *Mempool) GetFilter() gossip.Filter { +func (m *Mempool) GetBloomFilter() *gossip.BloomFilter { m.lock.RLock() defer m.lock.RUnlock() diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go new file mode 100644 index 0000000000..b4e76fa63e --- /dev/null +++ b/plugin/evm/tx_gossip_test.go @@ -0,0 +1,138 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "context" + "math/big" + "sync" + "testing" + "time" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" + "github.com/ava-labs/avalanchego/utils/logging" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + + "github.com/ava-labs/coreth/core/types" + "github.com/ava-labs/coreth/gossip" + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/plugin/evm/message" +) + +func TestTxGossip(t *testing.T) { + require := require.New(t) + + // set up prefunded address + importAmount := uint64(1000000000) + issuer, vm, _, _, sender := GenesisVMWithUTXOs(t, true, genesisJSONApricotPhase0, "", "", map[ids.ShortID]uint64{ + testShortIDAddrs[0]: importAmount, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + importTx, err := vm.newImportTx(vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + if err != nil { + t.Fatal(err) + } + + if err := vm.issueTx(importTx, true /*=local*/); err != nil { + t.Fatal(err) + } + + <-issuer + + blk, err := vm.BuildBlock(context.Background()) + require.NoError(err) + + require.NoError(blk.Verify(context.Background())) + require.NoError(vm.SetPreference(context.Background(), blk.ID())) + require.NoError(blk.Accept(context.Background())) + + // sender for the peer requesting gossip from [vm] + ctrl := gomock.NewController(t) + peerSender := common.NewMockSender(ctrl) + router := p2p.NewRouter(logging.NoLog{}, peerSender) + + // we're only making client requests, so we don't need a server handler + client, err := router.RegisterAppProtocol(0x0, nil) + require.NoError(err) + + emptyBloomFilter, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) + require.NoError(err) + emptyBloomFilterBytes, err := emptyBloomFilter.Marshal() + require.NoError(err) + request := gossip.PullGossipRequest{ + FilterBytes: emptyBloomFilterBytes, + } + requestBytes, err := vm.networkCodec.Marshal(message.Version, request) + require.NoError(err) + + wg := &sync.WaitGroup{} + + peerSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(ctx context.Context, nodeIDs set.Set[ids.NodeID], requestID uint32, appRequestBytes []byte) { + go func() { + require.NoError(vm.AppRequest(ctx, ids.EmptyNodeID, requestID, time.Time{}, appRequestBytes)) + }() + }).AnyTimes() + + sender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, appResponseBytes []byte) error { + go func() { + require.NoError(router.AppResponse(ctx, nodeID, requestID, appResponseBytes)) + }() + return nil + } + + // Ask the VM for any new transactions. We should get nothing at first. + wg.Add(1) + onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { + require.NoError(err) + + response := gossip.PullGossipResponse{} + _, err = vm.networkCodec.Unmarshal(responseBytes, &response) + require.NoError(err) + require.Empty(response.GossipBytes) + wg.Done() + } + require.NoError(client.AppRequest(context.Background(), set.Set[ids.NodeID]{vm.ctx.NodeID: struct{}{}}, requestBytes, onResponse)) + wg.Wait() + + // Issue a tx to the VM + address := testEthAddrs[0] + key := testKeys[0].ToECDSA() + tx := types.NewTransaction(0, address, big.NewInt(10), 100_000, big.NewInt(params.LaunchMinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainID), key) + require.NoError(err) + + errs := vm.txPool.AddRemotes([]*types.Transaction{signedTx}) + require.Len(errs, 1) + require.Nil(errs[0]) + + // Ask the VM for new transactions. We should get the newly issued tx. + wg.Add(1) + onResponse = func(nodeID ids.NodeID, responseBytes []byte, err error) { + require.NoError(err) + + response := gossip.PullGossipResponse{} + _, err = vm.networkCodec.Unmarshal(responseBytes, &response) + require.NoError(err) + require.Len(response.GossipBytes, 1) + + gotTx := &GossipEthTx{} + require.NoError(gotTx.Unmarshal(response.GossipBytes[0])) + require.Equal(signedTx.Hash(), gotTx.Tx.Hash()) + + wg.Done() + } + require.NoError(client.AppRequest(context.Background(), set.Set[ids.NodeID]{vm.ctx.NodeID: struct{}{}}, requestBytes, onResponse)) + wg.Wait() +} diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 2c0263496a..ce3dc65946 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -131,11 +131,19 @@ const ( targetAtomicTxsSize = 40 * units.KiB - // How frequently we should attempt to poll other nodes for new transactions - pullTxsFrequency = 500 * time.Millisecond - pullTxsGossipSize = 10 + // threshold on how full a tx gossip bloom filter can get before it's reset + txGossipBloomMaxFilledRatio = 0.75 + // maximum anticipated amount of entries in the tx gossip bloom filter + txGossipBloomMaxItems = 1_000 + // maximum false positive rate for lookups + txGossipBloomFalsePositiveRate = 0.001 ) +var txGossipConfig = gossip.Config{ + Frequency: 500 * time.Millisecond, + PollSize: 10, +} + // Define the API endpoints for the VM const ( avaxEndpoint = "/avax" @@ -983,23 +991,21 @@ func (vm *VM) initBlockBuilding() error { vm.atomicTxGossipClient = atomicTxGossipClient vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx, *GossipEthTx]( + txGossipConfig, ethTxPool, vm.ethTxGossipClient, vm.networkCodec, message.Version, - pullTxsGossipSize, - pullTxsFrequency, ) vm.shutdownWg.Add(1) go vm.ethTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( + txGossipConfig, vm.mempool, vm.atomicTxGossipClient, vm.networkCodec, message.Version, - pullTxsGossipSize, - pullTxsFrequency, ) vm.shutdownWg.Add(1) go vm.atomicTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) From 8a500232b84506cb8e55de3b4b4f2ac6b8ce76e3 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 11:41:10 -0400 Subject: [PATCH 29/78] more unit tests --- gossip/test_gossip.go | 49 +--------------- plugin/evm/tx_gossip_test.go | 105 +++++++++++++++++++++++++++++++---- plugin/evm/vm.go | 7 ++- 3 files changed, 99 insertions(+), 62 deletions(-) diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index 3a74ad51b1..0fa160ec79 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -3,14 +3,7 @@ package gossip -import ( - "sync" -) - -var ( - _ Gossipable = (*testTx)(nil) - _ Set[*testTx] = (*testMempool)(nil) -) +var _ Gossipable = (*testTx)(nil) type testTx struct { hash Hash @@ -29,43 +22,3 @@ func (t *testTx) Unmarshal(bytes []byte) error { // TODO implement me panic("implement me") } - -type testMempool struct { - mempool []*testTx - lock sync.Mutex -} - -func (t *testMempool) Add(tx *testTx) error { - t.lock.Lock() - defer t.lock.Unlock() - - t.mempool = append(t.mempool, tx) - return nil -} - -func (t *testMempool) Get(filter func(tx *testTx) bool) []*testTx { - t.lock.Lock() - defer t.lock.Unlock() - - result := make([]*testTx, 0) - for _, tx := range t.mempool { - if !filter(tx) { - continue - } - result = append(result, tx) - } - - return result -} - -func (t *testMempool) GetBloomFilter() *BloomFilter { - t.lock.Lock() - defer t.lock.Unlock() - - bloom, _ := NewBloomFilter(1000, 0.01) - for _, tx := range t.mempool { - bloom.Add(tx) - } - - return bloom -} diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index b4e76fa63e..a106f37ec1 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -25,29 +25,23 @@ import ( "github.com/ava-labs/coreth/plugin/evm/message" ) -func TestTxGossip(t *testing.T) { +func TestEthTxGossip(t *testing.T) { require := require.New(t) // set up prefunded address - importAmount := uint64(1000000000) + importAmount := uint64(1_000_000_000) issuer, vm, _, _, sender := GenesisVMWithUTXOs(t, true, genesisJSONApricotPhase0, "", "", map[ids.ShortID]uint64{ testShortIDAddrs[0]: importAmount, }) defer func() { - if err := vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } + require.NoError(vm.Shutdown(context.Background())) }() importTx, err := vm.newImportTx(vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } + require.NoError(err) - if err := vm.issueTx(importTx, true /*=local*/); err != nil { - t.Fatal(err) - } + require.NoError(vm.issueTx(importTx, true)) <-issuer @@ -64,7 +58,7 @@ func TestTxGossip(t *testing.T) { router := p2p.NewRouter(logging.NoLog{}, peerSender) // we're only making client requests, so we don't need a server handler - client, err := router.RegisterAppProtocol(0x0, nil) + client, err := router.RegisterAppProtocol(ethTxGossipProtocol, nil) require.NoError(err) emptyBloomFilter, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) @@ -136,3 +130,90 @@ func TestTxGossip(t *testing.T) { require.NoError(client.AppRequest(context.Background(), set.Set[ids.NodeID]{vm.ctx.NodeID: struct{}{}}, requestBytes, onResponse)) wg.Wait() } + +func TestAtomicTxGossip(t *testing.T) { + require := require.New(t) + + // set up prefunded address + importAmount := uint64(1_000_000_000) + issuer, vm, _, _, sender := GenesisVMWithUTXOs(t, true, genesisJSONApricotPhase0, "", "", map[ids.ShortID]uint64{ + testShortIDAddrs[0]: importAmount, + }) + + defer func() { + require.NoError(vm.Shutdown(context.Background())) + }() + + // sender for the peer requesting gossip from [vm] + ctrl := gomock.NewController(t) + peerSender := common.NewMockSender(ctrl) + router := p2p.NewRouter(logging.NoLog{}, peerSender) + + // we're only making client requests, so we don't need a server handler + client, err := router.RegisterAppProtocol(atomicTxGossipProtocol, nil) + require.NoError(err) + + emptyBloomFilter, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) + require.NoError(err) + emptyBloomFilterBytes, err := emptyBloomFilter.Marshal() + require.NoError(err) + request := gossip.PullGossipRequest{ + FilterBytes: emptyBloomFilterBytes, + } + requestBytes, err := vm.networkCodec.Marshal(message.Version, request) + require.NoError(err) + + wg := &sync.WaitGroup{} + peerSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(ctx context.Context, nodeIDs set.Set[ids.NodeID], requestID uint32, appRequestBytes []byte) { + go func() { + require.NoError(vm.AppRequest(ctx, ids.EmptyNodeID, requestID, time.Time{}, appRequestBytes)) + }() + }).AnyTimes() + + sender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, appResponseBytes []byte) error { + go func() { + require.NoError(router.AppResponse(ctx, nodeID, requestID, appResponseBytes)) + }() + return nil + } + + // Ask the VM for any new transactions. We should get nothing at first. + wg.Add(1) + onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { + require.NoError(err) + + response := gossip.PullGossipResponse{} + _, err = vm.networkCodec.Unmarshal(responseBytes, &response) + require.NoError(err) + require.Empty(response.GossipBytes) + wg.Done() + } + require.NoError(client.AppRequest(context.Background(), set.Set[ids.NodeID]{vm.ctx.NodeID: struct{}{}}, requestBytes, onResponse)) + wg.Wait() + + // issue a new tx to the vm + importTx, err := vm.newImportTx(vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + require.NoError(err) + + require.NoError(vm.issueTx(importTx, true /*=local*/)) + <-issuer + + // Ask the VM for new transactions. We should get the newly issued tx. + wg.Add(1) + onResponse = func(nodeID ids.NodeID, responseBytes []byte, err error) { + require.NoError(err) + + response := gossip.PullGossipResponse{} + _, err = vm.networkCodec.Unmarshal(responseBytes, &response) + require.NoError(err) + require.Len(response.GossipBytes, 1) + + gotTx := &GossipAtomicTx{} + require.NoError(gotTx.Unmarshal(response.GossipBytes[0])) + require.Equal(importTx.InputUTXOs(), gotTx.Tx.InputUTXOs()) + + wg.Done() + } + require.NoError(client.AppRequest(context.Background(), set.Set[ids.NodeID]{vm.ctx.NodeID: struct{}{}}, requestBytes, onResponse)) + wg.Wait() +} diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index ce3dc65946..e5daf409c8 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -131,6 +131,9 @@ const ( targetAtomicTxsSize = 40 * units.KiB + ethTxGossipProtocol = 0x0 + atomicTxGossipProtocol = 0x1 + // threshold on how full a tx gossip bloom filter can get before it's reset txGossipBloomMaxFilledRatio = 0.75 // maximum anticipated amount of entries in the tx gossip bloom filter @@ -977,14 +980,14 @@ func (vm *VM) initBlockBuilding() error { go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, vm.codec, message.Version) - ethTxGossipClient, err := vm.router.RegisterAppProtocol(0x0, ethTxGossipHandler) + ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler) if err != nil { return err } vm.ethTxGossipClient = ethTxGossipClient atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](vm.mempool, vm.codec, message.Version) - atomicTxGossipClient, err := vm.router.RegisterAppProtocol(0x1, atomicTxGossipHandler) + atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler) if err != nil { return err } From a360c286d7c3182cff4ad7a44b154bbea402dfe2 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 14:31:52 -0400 Subject: [PATCH 30/78] nit --- gossip/bloom.go | 57 ++++++++++++-------- gossip/bloom_test.go | 4 +- gossip/gossip.go | 6 ++- gossip/gossipable.go | 4 +- gossip/handler.go | 8 ++- plugin/evm/gossip_mempool.go | 2 +- plugin/evm/gossiper_atomic_gossiping_test.go | 8 ++- plugin/evm/mempool.go | 2 +- plugin/evm/tx_gossip_test.go | 9 ++-- 9 files changed, 61 insertions(+), 39 deletions(-) diff --git a/gossip/bloom.go b/gossip/bloom.go index b4f33aea02..95c4512ee7 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -6,8 +6,10 @@ package gossip import ( "encoding/binary" "hash" + "time" bloomfilter "github.com/holiman/bloomfilter/v2" + "golang.org/x/exp/rand" ) var _ hash.Hash64 = (*hasher)(nil) @@ -18,32 +20,32 @@ func NewBloomFilter(m uint64, p float64) (*BloomFilter, error) { return nil, err } - return &BloomFilter{ - bloom: bloom, - }, nil + bloomFilter := &BloomFilter{ + Bloom: bloom, + Salt: randomSalt(), + } + return bloomFilter, nil } type BloomFilter struct { - bloom *bloomfilter.Filter + Bloom *bloomfilter.Filter + Salt []byte } func (b *BloomFilter) Add(gossipable Gossipable) { - b.bloom.Add(NewHasher(gossipable)) + salted := hasher{ + hash: gossipable.GetHash(), + salt: b.Salt, + } + b.Bloom.Add(salted) } func (b *BloomFilter) Has(gossipable Gossipable) bool { - return b.bloom.Contains(NewHasher(gossipable)) -} - -func (b *BloomFilter) Marshal() ([]byte, error) { - return b.bloom.MarshalBinary() -} - -func (b *BloomFilter) Unmarshal(data []byte) error { - bloom := &bloomfilter.Filter{} - err := bloom.UnmarshalBinary(data) - b.bloom = bloom - return err + salted := hasher{ + hash: gossipable.GetHash(), + salt: b.Salt, + } + return b.Bloom.Contains(salted) } // ResetBloomFilterIfNeeded resets a bloom filter if it breaches a ratio of @@ -52,30 +54,39 @@ func ResetBloomFilterIfNeeded( bloomFilter *BloomFilter, maxFilledRatio float64, ) bool { - if bloomFilter.bloom.PreciseFilledRatio() < maxFilledRatio { + if bloomFilter.Bloom.PreciseFilledRatio() < maxFilledRatio { return false } // it's not possible for this to error assuming that the original // bloom filter's parameters were valid - fresh, _ := bloomfilter.New(bloomFilter.bloom.M(), bloomFilter.bloom.K()) - bloomFilter.bloom = fresh + fresh, _ := bloomfilter.New(bloomFilter.Bloom.M(), bloomFilter.Bloom.K()) + bloomFilter.Bloom = fresh + bloomFilter.Salt = randomSalt() return true } -func NewHasher(gossipable Gossipable) hash.Hash64 { - return hasher{hash: gossipable.GetHash()} +func randomSalt() []byte { + salt := make([]byte, 0, HashLength) + r := rand.New(rand.NewSource(uint64(time.Now().Nanosecond()))) + _, _ = r.Read(salt) + return salt } type hasher struct { hash.Hash64 hash Hash + salt []byte } func (h hasher) Sum64() uint64 { + for i, salt := range h.salt { + h.hash[i] ^= salt + } + return binary.BigEndian.Uint64(h.hash[:]) } func (h hasher) Size() int { - return 8 + return HashLength } diff --git a/gossip/bloom_test.go b/gossip/bloom_test.go index a31e0683eb..7a17579932 100644 --- a/gossip/bloom_test.go +++ b/gossip/bloom_test.go @@ -46,7 +46,7 @@ func TestBloomFilterRefresh(t *testing.T) { b, err := bloomfilter.New(10, 1) require.NoError(err) bloom := BloomFilter{ - bloom: b, + Bloom: b, } for _, item := range tt.add { @@ -54,7 +54,7 @@ func TestBloomFilterRefresh(t *testing.T) { bloom.Add(item) } - require.Equal(uint64(len(tt.expected)), bloom.bloom.N()) + require.Equal(uint64(len(tt.expected)), bloom.Bloom.N()) for _, expected := range tt.expected { require.True(bloom.Has(expected)) diff --git a/gossip/gossip.go b/gossip/gossip.go index 8cf52b29b3..226a8e09a0 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -60,13 +60,15 @@ func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.Wai for { select { case <-gossipTicker.C: - filterBytes, err := g.set.GetBloomFilter().Marshal() + filter := g.set.GetFilter() + bloomBytes, err := filter.Bloom.MarshalBinary() if err != nil { log.Warn("failed to marshal bloom filter", "error", err) } request := PullGossipRequest{ - FilterBytes: filterBytes, + FilterBytes: bloomBytes, + SaltBytes: filter.Salt, } msgBytes, err := g.codec.Marshal(g.codecVersion, request) if err != nil { diff --git a/gossip/gossipable.go b/gossip/gossipable.go index bb35dd4b35..b9548d984a 100644 --- a/gossip/gossipable.go +++ b/gossip/gossipable.go @@ -17,6 +17,6 @@ type Set[T Gossipable] interface { Add(gossipable T) error // Get returns elements that match the provided filter function Get(filter func(gossipable T) bool) []T - // GetFilter returns a filter containing the elements in Set - GetBloomFilter() *BloomFilter + // GetFilter returns a bloom filter containing the elements in Set + GetFilter() *BloomFilter } diff --git a/gossip/handler.go b/gossip/handler.go index 79177c0775..47f948548b 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -11,6 +11,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" "github.com/ethereum/go-ethereum/log" + bloomfilter "github.com/holiman/bloomfilter/v2" ) var _ p2p.Handler = (*Handler[Gossipable])(nil) @@ -36,8 +37,11 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time return nil, nil } - filter := &BloomFilter{} - if err := filter.Unmarshal(request.FilterBytes); err != nil { + filter := &BloomFilter{ + Bloom: &bloomfilter.Filter{}, + Salt: request.SaltBytes, + } + if err := filter.Bloom.UnmarshalBinary(request.FilterBytes); err != nil { log.Debug("failed to unmarshal bloom filter", "nodeID", nodeID, "err", err) return nil, nil } diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index e08307bc01..6c8f07535a 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -113,7 +113,7 @@ func (g *GossipEthTxPool) Get(filter func(tx *GossipEthTx) bool) []*GossipEthTx return result } -func (g *GossipEthTxPool) GetBloomFilter() *gossip.BloomFilter { +func (g *GossipEthTxPool) GetFilter() *gossip.BloomFilter { g.lock.RLock() defer g.lock.RUnlock() diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index dcff4aa160..4f685eff26 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -10,6 +10,7 @@ import ( "testing" "time" + bloomfilter "github.com/holiman/bloomfilter/v2" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -69,8 +70,11 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { _, err := vm.networkCodec.Unmarshal(bytes, &msg) require.NoError(t, err) - filter := &gossip.BloomFilter{} - require.NoError(t, filter.Unmarshal(msg.FilterBytes)) + filter := &gossip.BloomFilter{ + Bloom: &bloomfilter.Filter{}, + Salt: msg.SaltBytes, + } + require.NoError(t, filter.Bloom.UnmarshalBinary(msg.FilterBytes)) if !filter.Has(&GossipAtomicTx{Tx: tx}) { return nil } diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index c2c564c206..4be0e58055 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -305,7 +305,7 @@ func (m *Mempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { return gossipTxs } -func (m *Mempool) GetBloomFilter() *gossip.BloomFilter { +func (m *Mempool) GetFilter() *gossip.BloomFilter { m.lock.RLock() defer m.lock.RUnlock() diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index a106f37ec1..f485f25a4b 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -63,7 +63,7 @@ func TestEthTxGossip(t *testing.T) { emptyBloomFilter, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) require.NoError(err) - emptyBloomFilterBytes, err := emptyBloomFilter.Marshal() + emptyBloomFilterBytes, err := emptyBloomFilter.Bloom.MarshalBinary() require.NoError(err) request := gossip.PullGossipRequest{ FilterBytes: emptyBloomFilterBytes, @@ -107,7 +107,7 @@ func TestEthTxGossip(t *testing.T) { signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainID), key) require.NoError(err) - errs := vm.txPool.AddRemotes([]*types.Transaction{signedTx}) + errs := vm.txPool.AddLocals([]*types.Transaction{signedTx}) require.Len(errs, 1) require.Nil(errs[0]) @@ -155,10 +155,11 @@ func TestAtomicTxGossip(t *testing.T) { emptyBloomFilter, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) require.NoError(err) - emptyBloomFilterBytes, err := emptyBloomFilter.Marshal() + bloomBytes, err := emptyBloomFilter.Bloom.MarshalBinary() require.NoError(err) request := gossip.PullGossipRequest{ - FilterBytes: emptyBloomFilterBytes, + FilterBytes: bloomBytes, + SaltBytes: emptyBloomFilter.Salt, } requestBytes, err := vm.networkCodec.Marshal(message.Version, request) require.NoError(err) From 088f74793f6f773be08e7e49af1560785fc4affc Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 14:38:10 -0400 Subject: [PATCH 31/78] nit --- gossip/handler.go | 6 +----- plugin/evm/mempool_atomic_gossiping_test.go | 3 +-- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/gossip/handler.go b/gossip/handler.go index 47f948548b..68bad000c2 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -63,12 +63,8 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time response := PullGossipResponse{ GossipBytes: gossipBytes, } - responseBytes, err := h.codec.Marshal(h.codecVersion, response) - if err != nil { - return nil, err - } - return responseBytes, nil + return h.codec.Marshal(h.codecVersion, response) } func (h Handler[T]) AppGossip(context.Context, ids.NodeID, []byte) error { diff --git a/plugin/evm/mempool_atomic_gossiping_test.go b/plugin/evm/mempool_atomic_gossiping_test.go index 684153183c..a2e05e6286 100644 --- a/plugin/evm/mempool_atomic_gossiping_test.go +++ b/plugin/evm/mempool_atomic_gossiping_test.go @@ -204,8 +204,7 @@ func TestMempoolPriorityDrop(t *testing.T) { assert.True(mempool.has(tx1.ID())) assert.False(mempool.has(tx2.ID())) tx3 := createImportTx(t, vm, ids.ID{3}, 2*params.AvalancheAtomicTxFee) - err = mempool.AddTx(tx3) - assert.NoError(err) + assert.NoError(mempool.AddTx(tx3)) assert.False(mempool.has(tx1.ID())) assert.False(mempool.has(tx2.ID())) assert.True(mempool.has(tx3.ID())) From 90f0301ecb433b84227e615c98783146e3503139 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 14:40:18 -0400 Subject: [PATCH 32/78] nit --- plugin/evm/vm.go | 1 - 1 file changed, 1 deletion(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index e5daf409c8..ddb192c500 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -1383,7 +1383,6 @@ func (vm *VM) issueTx(tx *Tx, local bool) error { } return err } - // add to mempool and possibly re-gossip if err := vm.mempool.AddTx(tx); err != nil { if !local { From 1fe4b830069240904a7922d7638838ac5fe7600c Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 16:26:59 -0400 Subject: [PATCH 33/78] more unit tests --- gossip/gossip_test.go | 87 ++++++++++++++++++++++++++++++++++++- gossip/test_gossip.go | 43 +++++++++++++++--- plugin/evm/message/codec.go | 3 ++ plugin/evm/vm.go | 4 +- 4 files changed, 128 insertions(+), 9 deletions(-) diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 0963cf62b7..5ceb24bc2e 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -4,19 +4,102 @@ package gossip import ( + "context" "sync" "testing" "time" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/codec/linearcodec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/utils/logging" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/avalanchego/utils/units" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" ) func TestGossiperShutdown(t *testing.T) { config := Config{Frequency: time.Second} - puller := NewGossiper[testTx, *testTx](config, nil, nil, nil, 0) + gossiper := NewGossiper[testTx](config, nil, nil, nil, 0) done := make(chan struct{}) wg := &sync.WaitGroup{} wg.Add(1) - go puller.Gossip(done, wg) + go gossiper.Gossip(done, wg) + + close(done) + wg.Wait() +} + +func TestGossiperGossip(t *testing.T) { + require := require.New(t) + ctrl := gomock.NewController(t) + + cc := codec.NewManager(2 * units.MiB) + lc := linearcodec.NewDefault() + require.NoError(lc.RegisterType(PullGossipRequest{})) + require.NoError(lc.RegisterType(PullGossipResponse{})) + require.NoError(cc.RegisterCodec(0, lc)) + + responseSender := common.NewMockSender(ctrl) + responseRouter := p2p.NewRouter(logging.NoLog{}, responseSender) + responseSet := testSet{ + set: set.Set[*testTx]{}, + } + tx := &testTx{hash: Hash{1, 2, 3, 4, 5}} + require.NoError(responseSet.Add(tx)) + handler := NewHandler[*testTx](responseSet, cc, 0) + _, err := responseRouter.RegisterAppProtocol(0x0, handler) + require.NoError(err) + + requestSender := common.NewMockSender(ctrl) + requestRouter := p2p.NewRouter(logging.NoLog{}, requestSender) + require.NoError(requestRouter.Connected(context.Background(), ids.EmptyNodeID, nil)) + + gossiped := make(chan struct{}) + requestSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + Do(func(ctx context.Context, nodeIDs set.Set[ids.NodeID], requestID uint32, request []byte) { + go func() { + require.NoError(responseRouter.AppRequest(ctx, ids.EmptyNodeID, requestID, time.Time{}, request)) + }() + }).AnyTimes() + + responseSender.EXPECT(). + SendAppResponse(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + Do(func(ctx context.Context, nodeID ids.NodeID, requestID uint32, appResponseBytes []byte) { + go func() { + require.NoError(requestRouter.AppResponse(ctx, nodeID, requestID, appResponseBytes)) + close(gossiped) + }() + }).AnyTimes() + + bloom, err := NewBloomFilter(1000, 0.01) + require.NoError(err) + requestSet := testSet{ + set: set.Set[*testTx]{}, + bloom: bloom, + } + requestClient, err := requestRouter.RegisterAppProtocol(0x0, nil) + require.NoError(err) + + config := Config{ + Frequency: time.Second, + PollSize: 1, + } + gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient, cc, 0) + done := make(chan struct{}) + wg := &sync.WaitGroup{} + + require.Len(requestSet.set, 0) + go gossiper.Gossip(done, wg) + <-gossiped + + require.Len(requestSet.set, 1) + gotTx, _ := requestSet.set.Pop() + require.Equal(tx, gotTx) close(done) wg.Wait() diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index 0fa160ec79..38bbd1b439 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -3,7 +3,14 @@ package gossip -var _ Gossipable = (*testTx)(nil) +import ( + "github.com/ava-labs/avalanchego/utils/set" +) + +var ( + _ Gossipable = (*testTx)(nil) + _ Set[*testTx] = (*testSet)(nil) +) type testTx struct { hash Hash @@ -14,11 +21,37 @@ func (t *testTx) GetHash() Hash { } func (t *testTx) Marshal() ([]byte, error) { - // TODO implement me - panic("implement me") + return t.hash[:], nil } func (t *testTx) Unmarshal(bytes []byte) error { - // TODO implement me - panic("implement me") + t.hash = Hash{} + copy(t.hash[:], bytes[:]) + return nil +} + +type testSet struct { + set set.Set[*testTx] + bloom *BloomFilter +} + +func (t testSet) Add(gossipable *testTx) error { + t.set.Add(gossipable) + return nil +} + +func (t testSet) Get(filter func(gossipable *testTx) bool) []*testTx { + result := make([]*testTx, 0) + for tx := range t.set { + if !filter(tx) { + continue + } + result = append(result, tx) + } + + return result +} + +func (t testSet) GetFilter() *BloomFilter { + return t.bloom } diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index 2ebcc1afce..c802bfe315 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -20,6 +20,7 @@ const ( var ( Codec codec.Manager CrossChainCodec codec.Manager + SdkCodec codec.Manager ) func init() { @@ -69,4 +70,6 @@ func init() { if errs.Errored() { panic(errs.Err) } + + SdkCodec = codec.NewManager(maxMessageSize) } diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index ddb192c500..d9abcd5a8f 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -993,7 +993,7 @@ func (vm *VM) initBlockBuilding() error { } vm.atomicTxGossipClient = atomicTxGossipClient - vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx, *GossipEthTx]( + vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx]( txGossipConfig, ethTxPool, vm.ethTxGossipClient, @@ -1003,7 +1003,7 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go vm.ethTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) - vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( + vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx]( txGossipConfig, vm.mempool, vm.atomicTxGossipClient, From ef521d77202f938c10acfcf27633a8b4a2d11fe4 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 16:39:11 -0400 Subject: [PATCH 34/78] nits --- gossip/gossip_test.go | 9 +++++---- plugin/evm/message/codec.go | 17 +++++++++++++---- plugin/evm/vm.go | 12 ++++++------ 3 files changed, 24 insertions(+), 14 deletions(-) diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 5ceb24bc2e..d2c6651fcf 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -85,6 +85,9 @@ func TestGossiperGossip(t *testing.T) { requestClient, err := requestRouter.RegisterAppProtocol(0x0, nil) require.NoError(err) + // We shouldn't have gotten any gossip before the test started + require.Len(requestSet.set, 0) + config := Config{ Frequency: time.Second, PollSize: 1, @@ -92,14 +95,12 @@ func TestGossiperGossip(t *testing.T) { gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient, cc, 0) done := make(chan struct{}) wg := &sync.WaitGroup{} - - require.Len(requestSet.set, 0) + wg.Add(1) go gossiper.Gossip(done, wg) <-gossiped require.Len(requestSet.set, 1) - gotTx, _ := requestSet.set.Pop() - require.Equal(tx, gotTx) + require.Contains(requestSet.set, tx) close(done) wg.Wait() diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index c802bfe315..55d8da8a51 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -44,10 +44,6 @@ func init() { c.RegisterType(CodeRequest{}), c.RegisterType(CodeResponse{}), - // p2p sdk gossip types - c.RegisterType(gossip.PullGossipRequest{}), - c.RegisterType(gossip.PullGossipResponse{}), - Codec.RegisterCodec(Version, c), ) @@ -72,4 +68,17 @@ func init() { } SdkCodec = codec.NewManager(maxMessageSize) + sdkc := linearcodec.NewDefault() + + errs = wrappers.Errs{} + errs.Add( + // p2p sdk gossip types + c.RegisterType(gossip.PullGossipRequest{}), + c.RegisterType(gossip.PullGossipResponse{}), + SdkCodec.RegisterCodec(Version, sdkc), + ) + + if errs.Errored() { + panic(errs.Err) + } } diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index d9abcd5a8f..d0561c03ed 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -979,35 +979,35 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) - ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, vm.codec, message.Version) + ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, message.SdkCodec, message.Version) ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler) if err != nil { return err } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](vm.mempool, vm.codec, message.Version) + atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SdkCodec, message.Version) atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler) if err != nil { return err } vm.atomicTxGossipClient = atomicTxGossipClient - vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx]( + vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx, *GossipEthTx]( txGossipConfig, ethTxPool, vm.ethTxGossipClient, - vm.networkCodec, + message.SdkCodec, message.Version, ) vm.shutdownWg.Add(1) go vm.ethTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) - vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx]( + vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( txGossipConfig, vm.mempool, vm.atomicTxGossipClient, - vm.networkCodec, + message.SdkCodec, message.Version, ) vm.shutdownWg.Add(1) From dc471cc09646d84a7db3814034a416659069ac4d Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 16:59:14 -0400 Subject: [PATCH 35/78] more unit tests --- gossip/gossip_test.go | 176 ++++++++++++++++++++++++++---------------- gossip/test_gossip.go | 1 + 2 files changed, 111 insertions(+), 66 deletions(-) diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index d2c6651fcf..a7f07b3e20 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -35,73 +35,117 @@ func TestGossiperShutdown(t *testing.T) { } func TestGossiperGossip(t *testing.T) { - require := require.New(t) - ctrl := gomock.NewController(t) - - cc := codec.NewManager(2 * units.MiB) - lc := linearcodec.NewDefault() - require.NoError(lc.RegisterType(PullGossipRequest{})) - require.NoError(lc.RegisterType(PullGossipResponse{})) - require.NoError(cc.RegisterCodec(0, lc)) - - responseSender := common.NewMockSender(ctrl) - responseRouter := p2p.NewRouter(logging.NoLog{}, responseSender) - responseSet := testSet{ - set: set.Set[*testTx]{}, + tests := []struct { + name string + requester []*testTx // what we have + responder []*testTx // what the peer we're requesting gossip from has + expected []*testTx // what we should have after a gossip cycle + }{ + { + name: "no gossip - no one knows anything", + }, + { + name: "no gossip - requester knows more than responder", + requester: []*testTx{{hash: Hash{0}}}, + expected: []*testTx{{hash: Hash{0}}}, + }, + { + name: "no gossip - requester knows everything responder knows", + requester: []*testTx{{hash: Hash{0}}}, + responder: []*testTx{{hash: Hash{0}}}, + expected: []*testTx{{hash: Hash{0}}}, + }, + { + name: "gossip - requester knows nothing", + responder: []*testTx{{hash: Hash{0}}}, + expected: []*testTx{{hash: Hash{0}}}, + }, + { + name: "gossip - requester knows less than responder", + requester: []*testTx{{hash: Hash{0}}}, + responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + expected: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + }, } - tx := &testTx{hash: Hash{1, 2, 3, 4, 5}} - require.NoError(responseSet.Add(tx)) - handler := NewHandler[*testTx](responseSet, cc, 0) - _, err := responseRouter.RegisterAppProtocol(0x0, handler) - require.NoError(err) - - requestSender := common.NewMockSender(ctrl) - requestRouter := p2p.NewRouter(logging.NoLog{}, requestSender) - require.NoError(requestRouter.Connected(context.Background(), ids.EmptyNodeID, nil)) - - gossiped := make(chan struct{}) - requestSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). - Do(func(ctx context.Context, nodeIDs set.Set[ids.NodeID], requestID uint32, request []byte) { - go func() { - require.NoError(responseRouter.AppRequest(ctx, ids.EmptyNodeID, requestID, time.Time{}, request)) - }() - }).AnyTimes() - - responseSender.EXPECT(). - SendAppResponse(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). - Do(func(ctx context.Context, nodeID ids.NodeID, requestID uint32, appResponseBytes []byte) { - go func() { - require.NoError(requestRouter.AppResponse(ctx, nodeID, requestID, appResponseBytes)) - close(gossiped) - }() - }).AnyTimes() - - bloom, err := NewBloomFilter(1000, 0.01) - require.NoError(err) - requestSet := testSet{ - set: set.Set[*testTx]{}, - bloom: bloom, - } - requestClient, err := requestRouter.RegisterAppProtocol(0x0, nil) - require.NoError(err) - - // We shouldn't have gotten any gossip before the test started - require.Len(requestSet.set, 0) - config := Config{ - Frequency: time.Second, - PollSize: 1, + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + require := require.New(t) + ctrl := gomock.NewController(t) + + cc := codec.NewManager(units.MiB) + lc := linearcodec.NewDefault() + require.NoError(lc.RegisterType(PullGossipRequest{})) + require.NoError(lc.RegisterType(PullGossipResponse{})) + require.NoError(cc.RegisterCodec(0, lc)) + + responseSender := common.NewMockSender(ctrl) + responseRouter := p2p.NewRouter(logging.NoLog{}, responseSender) + responseBloom, err := NewBloomFilter(1000, 0.01) + require.NoError(err) + responseSet := testSet{ + set: set.Set[*testTx]{}, + bloom: responseBloom, + } + for _, item := range tt.responder { + require.NoError(responseSet.Add(item)) + } + handler := NewHandler[*testTx](responseSet, cc, 0) + _, err = responseRouter.RegisterAppProtocol(0x0, handler) + require.NoError(err) + + requestSender := common.NewMockSender(ctrl) + requestRouter := p2p.NewRouter(logging.NoLog{}, requestSender) + require.NoError(requestRouter.Connected(context.Background(), ids.EmptyNodeID, nil)) + + gossiped := make(chan struct{}) + requestSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + Do(func(ctx context.Context, nodeIDs set.Set[ids.NodeID], requestID uint32, request []byte) { + go func() { + require.NoError(responseRouter.AppRequest(ctx, ids.EmptyNodeID, requestID, time.Time{}, request)) + }() + }).AnyTimes() + + responseSender.EXPECT(). + SendAppResponse(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + Do(func(ctx context.Context, nodeID ids.NodeID, requestID uint32, appResponseBytes []byte) { + go func() { + require.NoError(requestRouter.AppResponse(ctx, nodeID, requestID, appResponseBytes)) + close(gossiped) + }() + }).AnyTimes() + + bloom, err := NewBloomFilter(1000, 0.01) + require.NoError(err) + requestSet := testSet{ + set: set.Set[*testTx]{}, + bloom: bloom, + } + for _, item := range tt.requester { + require.NoError(requestSet.Add(item)) + } + + requestClient, err := requestRouter.RegisterAppProtocol(0x0, nil) + require.NoError(err) + + config := Config{ + Frequency: 500 * time.Millisecond, + PollSize: 1, + } + gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient, cc, 0) + done := make(chan struct{}) + wg := &sync.WaitGroup{} + wg.Add(1) + go gossiper.Gossip(done, wg) + <-gossiped + + require.Len(requestSet.set, len(tt.expected)) + for _, expected := range tt.expected { + require.Contains(requestSet.set, expected) + } + + close(done) + wg.Wait() + }) } - gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient, cc, 0) - done := make(chan struct{}) - wg := &sync.WaitGroup{} - wg.Add(1) - go gossiper.Gossip(done, wg) - <-gossiped - - require.Len(requestSet.set, 1) - require.Contains(requestSet.set, tx) - - close(done) - wg.Wait() } diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index 38bbd1b439..873c04cc2d 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -37,6 +37,7 @@ type testSet struct { func (t testSet) Add(gossipable *testTx) error { t.set.Add(gossipable) + t.bloom.Add(gossipable) return nil } From 45871c6e98f6fbdb6e93a24f8f55fb6c42d38311 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 22:44:08 -0400 Subject: [PATCH 36/78] nits --- peer/network.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/peer/network.go b/peer/network.go index c7b3ad483d..cec0a17af4 100644 --- a/peer/network.go +++ b/peer/network.go @@ -339,7 +339,7 @@ func (n *network) AppRequest(ctx context.Context, nodeID ids.NodeID, requestID u var req message.Request if _, err := n.codec.Unmarshal(request, &req); err != nil { - // this might be an sdk request + // this might be a sdk request if err := n.router.AppRequest(ctx, nodeID, requestID, deadline, request); err == nil { log.Debug("failed to unmarshal app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) } @@ -386,7 +386,7 @@ func (n *network) AppResponse(ctx context.Context, nodeID ids.NodeID, requestID handler, exists := n.markRequestFulfilled(requestID) if !exists { - // this might be an sdk response + // this might be a sdk response if err := n.router.AppResponse(ctx, nodeID, requestID, response); err == nil { return nil } From a22203959004f963c08c5c4f0fe0a391f8efbf8e Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 8 Aug 2023 22:44:15 -0400 Subject: [PATCH 37/78] fix ut --- plugin/evm/gossiper_atomic_gossiping_test.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index 4f685eff26..ab457c9e1a 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -10,6 +10,7 @@ import ( "testing" "time" + "github.com/ava-labs/avalanchego/snow" bloomfilter "github.com/holiman/bloomfilter/v2" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -25,7 +26,7 @@ import ( func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { assert := assert.New(t) - _, vm, _, sharedMemory, sender := GenesisVM(t, true, "", "", "") + _, vm, _, sharedMemory, sender := GenesisVM(t, false, "", "", "") defer func() { assert.NoError(vm.Shutdown(context.Background())) }() @@ -65,7 +66,7 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { gossipedLock.Lock() defer gossipedLock.Unlock() - bytes = bytes[1:] // first byte is an identifier + bytes = bytes[1:] // first byte is an sdk identifier msg := gossip.PullGossipRequest{} _, err := vm.networkCodec.Unmarshal(bytes, &msg) require.NoError(t, err) @@ -81,6 +82,7 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { addedToBloomFilter = true return nil } + assert.NoError(vm.SetState(context.Background(), snow.NormalOp)) // Optimistically gossip raw tx assert.NoError(vm.issueTx(tx, true /*=local*/)) From 3d1a15ea87e9c2bdaac1f3ac91678ebe8462323c Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 9 Aug 2023 11:57:17 -0400 Subject: [PATCH 38/78] nit --- gossip/gossip.go | 1 + plugin/evm/gossip_mempool.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/gossip/gossip.go b/gossip/gossip.go index 226a8e09a0..5a4fe7f616 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -64,6 +64,7 @@ func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.Wai bloomBytes, err := filter.Bloom.MarshalBinary() if err != nil { log.Warn("failed to marshal bloom filter", "error", err) + continue } request := PullGossipRequest{ diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 6c8f07535a..2c3c0428e3 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -121,7 +121,7 @@ func (g *GossipEthTxPool) GetFilter() *gossip.BloomFilter { } type GossipEthTx struct { - Tx *types.Transaction `serialize:"true"` + Tx *types.Transaction } func (tx *GossipEthTx) GetHash() gossip.Hash { From 74496f16dbc038021e43448fea3faa33e0308d6a Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 11 Aug 2023 17:25:23 -0400 Subject: [PATCH 39/78] fix fickle test --- plugin/evm/tx_gossip_test.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index f485f25a4b..f5c789c361 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -19,6 +19,7 @@ import ( "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" + "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/types" "github.com/ava-labs/coreth/gossip" "github.com/ava-labs/coreth/params" @@ -30,14 +31,16 @@ func TestEthTxGossip(t *testing.T) { // set up prefunded address importAmount := uint64(1_000_000_000) - issuer, vm, _, _, sender := GenesisVMWithUTXOs(t, true, genesisJSONApricotPhase0, "", "", map[ids.ShortID]uint64{ + issuer, vm, _, _, sender := GenesisVMWithUTXOs(t, true, genesisJSONLatest, "", "", map[ids.ShortID]uint64{ testShortIDAddrs[0]: importAmount, }) - defer func() { require.NoError(vm.Shutdown(context.Background())) }() + importAccepted := make(chan core.NewTxPoolHeadEvent) + vm.txPool.SubscribeNewHeadEvent(importAccepted) + importTx, err := vm.newImportTx(vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) require.NoError(err) @@ -51,6 +54,7 @@ func TestEthTxGossip(t *testing.T) { require.NoError(blk.Verify(context.Background())) require.NoError(vm.SetPreference(context.Background(), blk.ID())) require.NoError(blk.Accept(context.Background())) + <-importAccepted // sender for the peer requesting gossip from [vm] ctrl := gomock.NewController(t) From 48964aae40534ed2d66f0dbb757a71d579e81ffe Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 14 Aug 2023 03:17:21 -0400 Subject: [PATCH 40/78] remove useless goroutine --- gossip/gossip_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index a7f07b3e20..346e31f696 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -109,10 +109,8 @@ func TestGossiperGossip(t *testing.T) { responseSender.EXPECT(). SendAppResponse(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Do(func(ctx context.Context, nodeID ids.NodeID, requestID uint32, appResponseBytes []byte) { - go func() { - require.NoError(requestRouter.AppResponse(ctx, nodeID, requestID, appResponseBytes)) - close(gossiped) - }() + require.NoError(requestRouter.AppResponse(ctx, nodeID, requestID, appResponseBytes)) + close(gossiped) }).AnyTimes() bloom, err := NewBloomFilter(1000, 0.01) From ff9a8424ebf203c9cc208b8196fb3a41ff7df5ef Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 14 Aug 2023 22:35:09 -0400 Subject: [PATCH 41/78] fix bloom params --- plugin/evm/gossip_mempool.go | 2 +- plugin/evm/mempool.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 2c3c0428e3..cfc90f9837 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -41,7 +41,7 @@ func (tx *GossipAtomicTx) Unmarshal(bytes []byte) error { } func NewGossipEthTxPool(mempool *txpool.TxPool) (*GossipEthTxPool, error) { - bloom, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomMaxFilledRatio) + bloom, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) if err != nil { return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) } diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index 4be0e58055..94e3e6d7d2 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -79,7 +79,7 @@ type Mempool struct { // NewMempool returns a Mempool with [maxSize] func NewMempool(AVAXAssetID ids.ID, maxSize int) (*Mempool, error) { - bloom, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomMaxFilledRatio) + bloom, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) if err != nil { return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) } From 4fc08550ad0586cfd099e7c30887a512b48cbc22 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 14 Aug 2023 23:40:08 -0400 Subject: [PATCH 42/78] fix salt --- gossip/bloom.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gossip/bloom.go b/gossip/bloom.go index 95c4512ee7..09b74e0ca4 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -67,7 +67,7 @@ func ResetBloomFilterIfNeeded( } func randomSalt() []byte { - salt := make([]byte, 0, HashLength) + salt := make([]byte, HashLength) r := rand.New(rand.NewSource(uint64(time.Now().Nanosecond()))) _, _ = r.Read(salt) return salt From 1f8a298323cf2ad8ef7a26a556e6dcd309fe435c Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 15:12:56 -0400 Subject: [PATCH 43/78] nits --- gossip/bloom.go | 8 +++--- gossip/gossip.go | 48 +++++++++++++++++++----------------- gossip/gossip_test.go | 17 ++++++++----- gossip/test_gossip.go | 4 +++ peer/network.go | 7 ++++-- plugin/evm/gossip_mempool.go | 7 ++++++ plugin/evm/mempool.go | 4 +++ plugin/evm/mempool_test.go | 35 ++++++++++++++++++++++++++ plugin/evm/message/codec.go | 10 ++++---- plugin/evm/tx_gossip_test.go | 9 ++++--- plugin/evm/vm.go | 10 ++++---- 11 files changed, 113 insertions(+), 46 deletions(-) create mode 100644 plugin/evm/mempool_test.go diff --git a/gossip/bloom.go b/gossip/bloom.go index 09b74e0ca4..453c0bbf89 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -8,6 +8,7 @@ import ( "hash" "time" + safemath "github.com/ava-labs/avalanchego/utils/math" bloomfilter "github.com/holiman/bloomfilter/v2" "golang.org/x/exp/rand" ) @@ -29,7 +30,8 @@ func NewBloomFilter(m uint64, p float64) (*BloomFilter, error) { type BloomFilter struct { Bloom *bloomfilter.Filter - Salt []byte + // Salt is provided to eventually unblock collisions in Bloom + Salt []byte } func (b *BloomFilter) Add(gossipable Gossipable) { @@ -80,8 +82,8 @@ type hasher struct { } func (h hasher) Sum64() uint64 { - for i, salt := range h.salt { - h.hash[i] ^= salt + for i := 0; i < safemath.Min(len(h.hash), len(h.salt)); i++ { + h.hash[i] ^= h.salt[i] } return binary.BigEndian.Uint64(h.hash[:]) diff --git a/gossip/gossip.go b/gossip/gossip.go index 5a4fe7f616..475531e1a8 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -60,28 +60,8 @@ func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.Wai for { select { case <-gossipTicker.C: - filter := g.set.GetFilter() - bloomBytes, err := filter.Bloom.MarshalBinary() - if err != nil { - log.Warn("failed to marshal bloom filter", "error", err) - continue - } - - request := PullGossipRequest{ - FilterBytes: bloomBytes, - SaltBytes: filter.Salt, - } - msgBytes, err := g.codec.Marshal(g.codecVersion, request) - if err != nil { - log.Warn("failed to marshal gossip request", "error", err) - continue - } - - for i := 0; i < g.config.PollSize; i++ { - if err := g.client.AppRequestAny(context.TODO(), msgBytes, g.handleResponse); err != nil { - log.Warn("failed to gossip", "error", err) - continue - } + if err := g.gossip(); err != nil { + log.Warn("failed to gossip", "error", err) } case <-shutdownChan: log.Debug("shutting down gossip") @@ -89,6 +69,30 @@ func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.Wai } } } +func (g *Gossiper[T, U]) gossip() error { + filter := g.set.GetFilter() + bloomBytes, err := filter.Bloom.MarshalBinary() + if err != nil { + return err + } + + request := PullGossipRequest{ + FilterBytes: bloomBytes, + SaltBytes: filter.Salt, + } + msgBytes, err := g.codec.Marshal(g.codecVersion, request) + if err != nil { + return err + } + + for i := 0; i < g.config.PollSize; i++ { + if err := g.client.AppRequestAny(context.TODO(), msgBytes, g.handleResponse); err != nil { + return err + } + } + + return nil +} func (g *Gossiper[T, U]) handleResponse(nodeID ids.NodeID, responseBytes []byte, err error) { if err != nil { diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 346e31f696..44bfcc8010 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -131,10 +131,12 @@ func TestGossiperGossip(t *testing.T) { PollSize: 1, } gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient, cc, 0) - done := make(chan struct{}) - wg := &sync.WaitGroup{} - wg.Add(1) - go gossiper.Gossip(done, wg) + received := set.Set[*testTx]{} + requestSet.onAdd = func(tx *testTx) { + received.Add(tx) + } + + require.NoError(gossiper.gossip()) <-gossiped require.Len(requestSet.set, len(tt.expected)) @@ -142,8 +144,11 @@ func TestGossiperGossip(t *testing.T) { require.Contains(requestSet.set, expected) } - close(done) - wg.Wait() + // we should not receive anything that we already had before we + // requested the gossip + for _, tx := range tt.requester { + require.NotContains(received, tx) + } }) } } diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index 873c04cc2d..e246c978a2 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -33,11 +33,15 @@ func (t *testTx) Unmarshal(bytes []byte) error { type testSet struct { set set.Set[*testTx] bloom *BloomFilter + onAdd func(tx *testTx) } func (t testSet) Add(gossipable *testTx) error { t.set.Add(gossipable) t.bloom.Add(gossipable) + if t.onAdd != nil { + t.onAdd(gossipable) + } return nil } diff --git a/peer/network.go b/peer/network.go index cec0a17af4..f9fda1f294 100644 --- a/peer/network.go +++ b/peer/network.go @@ -339,9 +339,12 @@ func (n *network) AppRequest(ctx context.Context, nodeID ids.NodeID, requestID u var req message.Request if _, err := n.codec.Unmarshal(request, &req); err != nil { + log.Debug("failed to unmarshal app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) + // this might be a sdk request - if err := n.router.AppRequest(ctx, nodeID, requestID, deadline, request); err == nil { - log.Debug("failed to unmarshal app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) + if err := n.router.AppRequest(ctx, nodeID, requestID, deadline, request); err != nil { + log.Debug("failed to handle app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) + return nil } return nil diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index cfc90f9837..16266e2566 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -78,6 +78,13 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync g.bloom.Add(tx) if gossip.ResetBloomFilterIfNeeded(g.bloom, txGossipBloomMaxFilledRatio) { log.Debug("resetting bloom filter", "reason", "reached max filled ratio") + + pending, _ := g.mempool.Content() + for _, pendingTxs := range pending { + for _, pendingTx := range pendingTxs { + g.bloom.Add(&GossipEthTx{Tx: pendingTx}) + } + } } } g.lock.Unlock() diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index 94e3e6d7d2..1b555c023d 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -284,6 +284,10 @@ func (m *Mempool) addTx(tx *Tx, force bool) error { m.bloom.Add(&GossipAtomicTx{Tx: tx}) if gossip.ResetBloomFilterIfNeeded(m.bloom, txGossipBloomMaxFilledRatio) { log.Debug("resetting bloom filter", "reason", "reached max filled ratio") + + for _, pendingTx := range m.txHeap.minHeap.items { + m.bloom.Add(&GossipAtomicTx{Tx: pendingTx.tx}) + } } return nil diff --git a/plugin/evm/mempool_test.go b/plugin/evm/mempool_test.go new file mode 100644 index 0000000000..6d719a0ee2 --- /dev/null +++ b/plugin/evm/mempool_test.go @@ -0,0 +1,35 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "testing" + + "github.com/ava-labs/avalanchego/ids" + "github.com/stretchr/testify/require" +) + +func TestMempoolAddTx(t *testing.T) { + require := require.New(t) + m, err := NewMempool(ids.Empty, 5_000) + require.NoError(err) + + txs := make([]*GossipAtomicTx, 0) + for i := 0; i < 3_000; i++ { + tx := &GossipAtomicTx{ + Tx: &Tx{ + UnsignedAtomicTx: &TestUnsignedTx{ + IDV: ids.GenerateTestID(), + }, + }, + } + + txs = append(txs, tx) + require.NoError(m.Add(tx)) + } + + for _, tx := range txs { + require.True(m.bloom.Has(tx)) + } +} diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index 55d8da8a51..804fb3190e 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -20,7 +20,7 @@ const ( var ( Codec codec.Manager CrossChainCodec codec.Manager - SdkCodec codec.Manager + SDKCodec codec.Manager ) func init() { @@ -67,15 +67,15 @@ func init() { panic(errs.Err) } - SdkCodec = codec.NewManager(maxMessageSize) + SDKCodec = codec.NewManager(maxMessageSize) sdkc := linearcodec.NewDefault() errs = wrappers.Errs{} errs.Add( // p2p sdk gossip types - c.RegisterType(gossip.PullGossipRequest{}), - c.RegisterType(gossip.PullGossipResponse{}), - SdkCodec.RegisterCodec(Version, sdkc), + sdkc.RegisterType(gossip.PullGossipRequest{}), + sdkc.RegisterType(gossip.PullGossipResponse{}), + SDKCodec.RegisterCodec(Version, sdkc), ) if errs.Errored() { diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index f5c789c361..f1e73314cc 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -12,7 +12,8 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" - "github.com/ava-labs/avalanchego/snow/engine/common" + commonEng "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" @@ -58,7 +59,7 @@ func TestEthTxGossip(t *testing.T) { // sender for the peer requesting gossip from [vm] ctrl := gomock.NewController(t) - peerSender := common.NewMockSender(ctrl) + peerSender := commonEng.NewMockSender(ctrl) router := p2p.NewRouter(logging.NoLog{}, peerSender) // we're only making client requests, so we don't need a server handler @@ -71,7 +72,9 @@ func TestEthTxGossip(t *testing.T) { require.NoError(err) request := gossip.PullGossipRequest{ FilterBytes: emptyBloomFilterBytes, + SaltBytes: utils.RandomBytes(10), } + requestBytes, err := vm.networkCodec.Marshal(message.Version, request) require.NoError(err) @@ -150,7 +153,7 @@ func TestAtomicTxGossip(t *testing.T) { // sender for the peer requesting gossip from [vm] ctrl := gomock.NewController(t) - peerSender := common.NewMockSender(ctrl) + peerSender := commonEng.NewMockSender(ctrl) router := p2p.NewRouter(logging.NoLog{}, peerSender) // we're only making client requests, so we don't need a server handler diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index d0561c03ed..a0c4a1ec30 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -137,7 +137,7 @@ const ( // threshold on how full a tx gossip bloom filter can get before it's reset txGossipBloomMaxFilledRatio = 0.75 // maximum anticipated amount of entries in the tx gossip bloom filter - txGossipBloomMaxItems = 1_000 + txGossipBloomMaxItems = 4096 // maximum false positive rate for lookups txGossipBloomFalsePositiveRate = 0.001 ) @@ -979,14 +979,14 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) - ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, message.SdkCodec, message.Version) + ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, message.SDKCodec, message.Version) ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler) if err != nil { return err } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SdkCodec, message.Version) + atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SDKCodec, message.Version) atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler) if err != nil { return err @@ -997,7 +997,7 @@ func (vm *VM) initBlockBuilding() error { txGossipConfig, ethTxPool, vm.ethTxGossipClient, - message.SdkCodec, + message.SDKCodec, message.Version, ) vm.shutdownWg.Add(1) @@ -1007,7 +1007,7 @@ func (vm *VM) initBlockBuilding() error { txGossipConfig, vm.mempool, vm.atomicTxGossipClient, - message.SdkCodec, + message.SDKCodec, message.Version, ) vm.shutdownWg.Add(1) From 9e987210200953c100405b92586c5fc144e8c75a Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 15:13:56 -0400 Subject: [PATCH 44/78] increase gossip frequency --- plugin/evm/vm.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index a0c4a1ec30..5dfd8aa83a 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -143,7 +143,7 @@ const ( ) var txGossipConfig = gossip.Config{ - Frequency: 500 * time.Millisecond, + Frequency: 10 * time.Second, PollSize: 10, } From 369bd96f4b3e2a68fbd1c09349184aca7f7c76a7 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 15:19:04 -0400 Subject: [PATCH 45/78] nit --- gossip/gossip.go | 1 + 1 file changed, 1 insertion(+) diff --git a/gossip/gossip.go b/gossip/gossip.go index 475531e1a8..fb784ea788 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -113,6 +113,7 @@ func (g *Gossiper[T, U]) handleResponse(nodeID ids.NodeID, responseBytes []byte, continue } + log.Debug("received gossip", "nodeID", nodeID, "hash", gossipable.GetHash()) if err := g.set.Add(gossipable); err != nil { log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetHash()) continue From b74f3f09544484133c0067f1e933ea9c5261ec15 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 15:36:42 -0400 Subject: [PATCH 46/78] fix test flake --- plugin/evm/gossiper_atomic_gossiping_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index ab457c9e1a..4b331e2fb5 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -86,7 +86,7 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { // Optimistically gossip raw tx assert.NoError(vm.issueTx(tx, true /*=local*/)) - time.Sleep(1 * time.Second) + time.Sleep(2 * txGossipConfig.Frequency) gossipedLock.Lock() assert.Equal(1, gossiped) gossipedLock.Unlock() From 957ee8ec6d1e7f6b8562f538ce43e46bb6eb4818 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 15:54:27 -0400 Subject: [PATCH 47/78] nit --- plugin/evm/vm.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 5dfd8aa83a..0e36550624 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -143,7 +143,7 @@ const ( ) var txGossipConfig = gossip.Config{ - Frequency: 10 * time.Second, + Frequency: time.Second, PollSize: 10, } From 4446f4fc693041cc7b993a11bc9d27ff8d73de96 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 16:04:49 -0400 Subject: [PATCH 48/78] nit --- plugin/evm/vm.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 0e36550624..5dfd8aa83a 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -143,7 +143,7 @@ const ( ) var txGossipConfig = gossip.Config{ - Frequency: time.Second, + Frequency: 10 * time.Second, PollSize: 10, } From a4d1c9fe412741ccfeb3a98fbb13dcf79f23a269 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 16:08:18 -0400 Subject: [PATCH 49/78] nit --- gossip/gossip.go | 1 + 1 file changed, 1 insertion(+) diff --git a/gossip/gossip.go b/gossip/gossip.go index fb784ea788..3c642a7a15 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -69,6 +69,7 @@ func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.Wai } } } + func (g *Gossiper[T, U]) gossip() error { filter := g.set.GetFilter() bloomBytes, err := filter.Bloom.MarshalBinary() From b06bc7e849c2f51faf6c93aa776acd41ac1fa8ef Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 15 Aug 2023 16:08:46 -0400 Subject: [PATCH 50/78] nit --- gossip/test_gossip.go | 1 + 1 file changed, 1 insertion(+) diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index e246c978a2..e7aa2f8f37 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -42,6 +42,7 @@ func (t testSet) Add(gossipable *testTx) error { if t.onAdd != nil { t.onAdd(gossipable) } + return nil } From 2de3042fa95c0f3aa804a66a1ada09ffe62456d8 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 16 Aug 2023 13:13:33 -0400 Subject: [PATCH 51/78] comment --- gossip/bloom.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/gossip/bloom.go b/gossip/bloom.go index 453c0bbf89..914bdf7672 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -30,7 +30,9 @@ func NewBloomFilter(m uint64, p float64) (*BloomFilter, error) { type BloomFilter struct { Bloom *bloomfilter.Filter - // Salt is provided to eventually unblock collisions in Bloom + // Salt is provided to eventually unblock collisions in Bloom. It's possible + // that conflicting Gossipable items collide in the bloom filter, so a salt + // is generated to eventually resolve collisions. Salt []byte } From a45b07eae19dc205bd59355b2caf65c5a33399a6 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 22 Aug 2023 02:08:17 -0400 Subject: [PATCH 52/78] nit --- plugin/evm/gossip_mempool.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 16266e2566..6cdb3dab75 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -79,7 +79,7 @@ func (g *GossipEthTxPool) Subscribe(shutdownChan chan struct{}, shutdownWg *sync if gossip.ResetBloomFilterIfNeeded(g.bloom, txGossipBloomMaxFilledRatio) { log.Debug("resetting bloom filter", "reason", "reached max filled ratio") - pending, _ := g.mempool.Content() + pending := g.mempool.Pending(false) for _, pendingTxs := range pending { for _, pendingTx := range pendingTxs { g.bloom.Add(&GossipEthTx{Tx: pendingTx}) @@ -103,7 +103,7 @@ func (g *GossipEthTxPool) Add(tx *GossipEthTx) error { } func (g *GossipEthTxPool) Get(filter func(tx *GossipEthTx) bool) []*GossipEthTx { - pending, _ := g.mempool.Content() + pending := g.mempool.Pending(false) result := make([]*GossipEthTx, 0) for _, txs := range pending { From cc64cdd49564c98e6c0d8013757b4388ef7539e0 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 22 Aug 2023 02:32:47 -0400 Subject: [PATCH 53/78] drop messages from non-validators --- plugin/evm/handler.go | 41 +++++++++++++++++++++++++++++++++++++++++ plugin/evm/vm.go | 16 ++++++++++++---- 2 files changed, 53 insertions(+), 4 deletions(-) create mode 100644 plugin/evm/handler.go diff --git a/plugin/evm/handler.go b/plugin/evm/handler.go new file mode 100644 index 0000000000..66f061812d --- /dev/null +++ b/plugin/evm/handler.go @@ -0,0 +1,41 @@ +// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "context" + "time" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" +) + +var _ p2p.Handler = (*validatorHandler)(nil) + +// validatorHandler drops messages from non-validators +// TODO return an application-level error to non-validators when sdk supports it +type validatorHandler struct { + validators *p2p.Validators + handler p2p.Handler +} + +func (v validatorHandler) AppGossip(ctx context.Context, nodeID ids.NodeID, gossipBytes []byte) error { + if !v.validators.Has(ctx, nodeID) { + return nil + } + + return v.handler.AppGossip(ctx, nodeID, gossipBytes) +} + +func (v validatorHandler) AppRequest(ctx context.Context, nodeID ids.NodeID, deadline time.Time, requestBytes []byte) ([]byte, error) { + if !v.validators.Has(ctx, nodeID) { + return nil, nil + } + + return v.handler.AppRequest(ctx, nodeID, deadline, requestBytes) +} + +func (v validatorHandler) CrossChainAppRequest(ctx context.Context, chainID ids.ID, deadline time.Time, requestBytes []byte) ([]byte, error) { + return v.handler.CrossChainAppRequest(ctx, chainID, deadline, requestBytes) +} diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 5dfd8aa83a..9ff53702ac 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -296,6 +296,7 @@ type VM struct { client peer.NetworkClient networkCodec codec.Manager + validators *p2p.Validators router *p2p.Router ethTxGossipClient *p2p.Client atomicTxGossipClient *p2p.Client @@ -525,6 +526,7 @@ func (vm *VM) Initialize( } // initialize peer network + vm.validators = p2p.NewValidators(vm.ctx.SubnetID, vm.ctx.ValidatorState) vm.router = p2p.NewRouter(vm.ctx.Log, appSender) vm.networkCodec = message.Codec vm.Network = peer.NewNetwork(vm.router, appSender, vm.networkCodec, message.CrossChainCodec, chainCtx.NodeID, vm.config.MaxOutboundActiveRequests, vm.config.MaxOutboundActiveCrossChainRequests) @@ -979,15 +981,21 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) - ethTxGossipHandler := gossip.NewHandler[*GossipEthTx](ethTxPool, message.SDKCodec, message.Version) - ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler) + ethTxGossipHandler := validatorHandler{ + validators: vm.validators, + handler: gossip.NewHandler[*GossipEthTx](ethTxPool, message.SDKCodec, message.Version), + } + ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler, vm.validators) if err != nil { return err } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SDKCodec, message.Version) - atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler) + atomicTxGossipHandler := validatorHandler{ + validators: vm.validators, + handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SDKCodec, message.Version), + } + atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler, vm.validators) if err != nil { return err } From 1b94dc4cb5ff1fa70e2791bd32c11e3d9f0c4092 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 22 Aug 2023 12:00:22 -0400 Subject: [PATCH 54/78] nit --- peer/network.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/peer/network.go b/peer/network.go index f9fda1f294..d8bc6e93e6 100644 --- a/peer/network.go +++ b/peer/network.go @@ -513,9 +513,6 @@ func (n *network) Connected(ctx context.Context, nodeID ids.NodeID, nodeVersion return nil } - if err := n.router.Connected(ctx, nodeID, nodeVersion); err != nil { - return err - } n.peers.Connected(nodeID, nodeVersion) return nil } @@ -530,9 +527,6 @@ func (n *network) Disconnected(ctx context.Context, nodeID ids.NodeID) error { return nil } - if err := n.router.Disconnected(ctx, nodeID); err != nil { - return err - } n.peers.Disconnected(nodeID) return nil } From 1363616bad45a38d19302b8c109b2fa6f7097bf5 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 28 Aug 2023 17:49:48 -0400 Subject: [PATCH 55/78] nit --- plugin/evm/handler.go | 41 ----------------------------------------- plugin/evm/vm.go | 26 ++++++++++++++------------ 2 files changed, 14 insertions(+), 53 deletions(-) delete mode 100644 plugin/evm/handler.go diff --git a/plugin/evm/handler.go b/plugin/evm/handler.go deleted file mode 100644 index 66f061812d..0000000000 --- a/plugin/evm/handler.go +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package evm - -import ( - "context" - "time" - - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/network/p2p" -) - -var _ p2p.Handler = (*validatorHandler)(nil) - -// validatorHandler drops messages from non-validators -// TODO return an application-level error to non-validators when sdk supports it -type validatorHandler struct { - validators *p2p.Validators - handler p2p.Handler -} - -func (v validatorHandler) AppGossip(ctx context.Context, nodeID ids.NodeID, gossipBytes []byte) error { - if !v.validators.Has(ctx, nodeID) { - return nil - } - - return v.handler.AppGossip(ctx, nodeID, gossipBytes) -} - -func (v validatorHandler) AppRequest(ctx context.Context, nodeID ids.NodeID, deadline time.Time, requestBytes []byte) ([]byte, error) { - if !v.validators.Has(ctx, nodeID) { - return nil, nil - } - - return v.handler.AppRequest(ctx, nodeID, deadline, requestBytes) -} - -func (v validatorHandler) CrossChainAppRequest(ctx context.Context, chainID ids.ID, deadline time.Time, requestBytes []byte) ([]byte, error) { - return v.handler.CrossChainAppRequest(ctx, chainID, deadline, requestBytes) -} diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 9ff53702ac..607650725e 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -131,15 +131,17 @@ const ( targetAtomicTxsSize = 40 * units.KiB + // p2p app protocols ethTxGossipProtocol = 0x0 atomicTxGossipProtocol = 0x1 - // threshold on how full a tx gossip bloom filter can get before it's reset - txGossipBloomMaxFilledRatio = 0.75 - // maximum anticipated amount of entries in the tx gossip bloom filter - txGossipBloomMaxItems = 4096 - // maximum false positive rate for lookups + // gossip constants + txGossipBloomMaxFilledRatio = 0.75 + txGossipBloomMaxItems = 4096 txGossipBloomFalsePositiveRate = 0.001 + maxValidatorSetStaleness = time.Minute + throttlingPeriod = time.Second + throttlingLimit = 1 ) var txGossipConfig = gossip.Config{ @@ -526,7 +528,7 @@ func (vm *VM) Initialize( } // initialize peer network - vm.validators = p2p.NewValidators(vm.ctx.SubnetID, vm.ctx.ValidatorState) + vm.validators = p2p.NewValidators(vm.ctx.Log, vm.ctx.SubnetID, vm.ctx.ValidatorState, maxValidatorSetStaleness) vm.router = p2p.NewRouter(vm.ctx.Log, appSender) vm.networkCodec = message.Codec vm.Network = peer.NewNetwork(vm.router, appSender, vm.networkCodec, message.CrossChainCodec, chainCtx.NodeID, vm.config.MaxOutboundActiveRequests, vm.config.MaxOutboundActiveCrossChainRequests) @@ -981,9 +983,9 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) - ethTxGossipHandler := validatorHandler{ - validators: vm.validators, - handler: gossip.NewHandler[*GossipEthTx](ethTxPool, message.SDKCodec, message.Version), + ethTxGossipHandler := &p2p.ThrottledHandler{ + Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), + Handler: gossip.NewHandler[*GossipEthTx](ethTxPool, message.SDKCodec, message.Version), } ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler, vm.validators) if err != nil { @@ -991,9 +993,9 @@ func (vm *VM) initBlockBuilding() error { } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := validatorHandler{ - validators: vm.validators, - handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SDKCodec, message.Version), + atomicTxGossipHandler := &p2p.ThrottledHandler{ + Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), + Handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SDKCodec, message.Version), } atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler, vm.validators) if err != nil { From 9d6675c81ff6d3ac20c5c1a467e19f6e37b5dea5 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 28 Aug 2023 17:50:43 -0400 Subject: [PATCH 56/78] Squashed commit of the following: commit 4ef26a0bbed8dfad0c6a1dcbd7079f5ce61aab93 Author: Darioush Jalali Date: Mon Aug 28 06:38:44 2023 -0700 bump go version in dockerfile (#311) commit 505a6abb1d68c26b06ed92d9cb48e0338cf1d08d Author: aaronbuchwald Date: Fri Aug 25 15:43:09 2023 -0400 core/state/snapshot: increase batch size during diffToDisk (#309) commit cca3e00f8b53434c63761e068ffd79324030c26e Author: marun Date: Fri Aug 25 09:44:42 2023 -0700 Bump golang version to 1.19.12 for CI (#306) * Bump golang version to 1.19.12 for CI Also bump targeted version of avalanchego to the v1.10.8 (which also uses golang 1.19.12). * fixup: Update go-version specification for consistency with other repos commit 42ba830f2e05d0f3475a1e1dc656a0720e436d50 Author: Darioush Jalali Date: Fri Aug 25 09:44:12 2023 -0700 add upstream test: TestClientBatchRequest_len (#307) commit 08e2b6eda862055c5ccdb36591656763530f7aab Author: Stephen Buttolph Date: Wed Aug 23 18:09:51 2023 -0400 Update avalanchego dependency to v1.10.9-rc.4 (#308) commit 5cd24c3ad3dab4b285eb7a1ac99da7b119ecfff7 Author: Darioush Jalali Date: Wed Aug 23 08:48:01 2023 -0700 node/config.go: remove unused options (#304) * node/config.go: remove unused graphql options * remove more unused config commit 8ea8b18e88d0ae02fdbece92784f9428c5f8f4df Author: aaronbuchwald Date: Tue Aug 15 11:18:29 2023 -0400 Add security file based on AvalancheGo (#303) commit 00214b143a8a5a02c1614fd32d151d9a0300b73e Author: Stephen Buttolph Date: Mon Aug 14 14:58:07 2023 -0400 Update AVAX client implementation and interface to align types (#301) commit 56665535dea6149886b6ec9f15aa3c94880cd274 Author: Ceyhun Onur Date: Mon Aug 14 21:38:04 2023 +0300 renamed flags (#298) * renamed flags * rename admin api flags * add deprecation commit e7645fa60b1a85b490010bd2f982045ef141caa2 Author: Ikko Eltociear Ashimine Date: Thu Aug 10 01:21:44 2023 +0900 Fix typo in sync/README.md (#296) --- .github/workflows/ci.yml | 15 ++-- SECURITY.md | 20 +++++ core/state/snapshot/snapshot.go | 4 +- go.mod | 6 +- go.sum | 13 ++- node/config.go | 106 +---------------------- node/defaults.go | 49 ----------- plugin/evm/client.go | 144 +++++++++++++++++--------------- plugin/evm/config.go | 46 ++++++++-- plugin/evm/formatting.go | 11 +++ plugin/evm/gossiper.go | 12 +-- plugin/evm/service.go | 31 ++++--- plugin/evm/vm.go | 13 ++- rpc/client_test.go | 48 +++++++++++ scripts/versions.sh | 2 +- sync/README.md | 4 +- 16 files changed, 252 insertions(+), 272 deletions(-) create mode 100644 SECURITY.md delete mode 100644 node/defaults.go diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 674924e847..ec6f3f2355 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -31,7 +31,8 @@ jobs: token: ${{ secrets.AVALANCHE_PAT }} - uses: actions/setup-go@v3 with: - go-version: "~1.19.12" + go-version: '~1.19.12' + check-latest: true - name: change avalanchego dep if: ${{ github.event_name == 'workflow_dispatch' }} run: | @@ -51,7 +52,6 @@ jobs: runs-on: ${{ matrix.os }} strategy: matrix: - go: ['~1.19.12'] os: [macos-11.0, ubuntu-20.04, windows-latest] steps: - uses: actions/checkout@v3 @@ -65,7 +65,8 @@ jobs: token: ${{ secrets.AVALANCHE_PAT }} - uses: actions/setup-go@v3 with: - go-version: ${{ matrix.go }} + go-version: '~1.19.12' + check-latest: true - name: change avalanchego dep if: ${{ github.event_name == 'workflow_dispatch' }} run: | @@ -84,7 +85,6 @@ jobs: runs-on: ${{ matrix.os }} strategy: matrix: - go: ['~1.19.12'] os: [ubuntu-20.04] steps: - uses: actions/checkout@v3 @@ -98,7 +98,8 @@ jobs: token: ${{ secrets.AVALANCHE_PAT }} - uses: actions/setup-go@v3 with: - go-version: ${{ matrix.go }} + go-version: '~1.19.12' + check-latest: true - name: change avalanchego dep if: ${{ github.event_name == 'workflow_dispatch' }} run: | @@ -115,7 +116,6 @@ jobs: runs-on: ${{ matrix.os }} strategy: matrix: - go: [ '~1.19.12' ] os: [ ubuntu-20.04 ] steps: - uses: actions/checkout@v3 @@ -129,7 +129,8 @@ jobs: token: ${{ secrets.AVALANCHE_PAT }} - uses: actions/setup-go@v3 with: - go-version: ${{ matrix.go }} + go-version: '~1.19.12' + check-latest: true - name: change avalanchego dep if: ${{ github.event_name == 'workflow_dispatch' }} run: | diff --git a/SECURITY.md b/SECURITY.md new file mode 100644 index 0000000000..e1bb59fa04 --- /dev/null +++ b/SECURITY.md @@ -0,0 +1,20 @@ +# Security Policy + +Avalanche takes the security of the platform and of its users very seriously. We and our community recognize the critical role of external security researchers and developers and welcome +responsible disclosures. Valid reports will be eligible for a reward (terms and conditions apply). + +## Reporting a Vulnerability + +**Please do not file a public ticket** mentioning the vulnerability. To disclose a vulnerability submit it through our [Bug Bounty Program](https://hackenproof.com/avalanche). + +Vulnerabilities must be disclosed to us privately with reasonable time to respond, and avoid compromise of other users and accounts, or loss of funds that are not your own. We do not reward spam or +social engineering vulnerabilities. + +Do not test for or validate any security issues in the live Avalanche networks (Mainnet and Fuji testnet), confirm all exploits in a local private testnet. + +Please refer to the [Bug Bounty Page](https://hackenproof.com/avalanche) for the most up-to-date program rules and scope. + +## Supported Versions + +Please use the [most recently released version](https://github.com/ava-labs/coreth/releases/latest) to perform testing and to validate security issues. + diff --git a/core/state/snapshot/snapshot.go b/core/state/snapshot/snapshot.go index 743db48ef3..ad7c9d6d41 100644 --- a/core/state/snapshot/snapshot.go +++ b/core/state/snapshot/snapshot.go @@ -634,7 +634,7 @@ func diffToDisk(bottom *diffLayer) (*diskLayer, bool, error) { // Ensure we don't delete too much data blindly (contract can be // huge). It's ok to flush, the root will go missing in case of a // crash and we'll detect and regenerate the snapshot. - if batch.ValueSize() > ethdb.IdealBatchSize { + if batch.ValueSize() > 64*1024*1024 { if err := batch.Write(); err != nil { log.Crit("Failed to write storage deletions", "err", err) } @@ -660,7 +660,7 @@ func diffToDisk(bottom *diffLayer) (*diskLayer, bool, error) { // Ensure we don't write too much data blindly. It's ok to flush, the // root will go missing in case of a crash and we'll detect and regen // the snapshot. - if batch.ValueSize() > ethdb.IdealBatchSize { + if batch.ValueSize() > 64*1024*1024 { if err := batch.Write(); err != nil { log.Crit("Failed to write storage deletions", "err", err) } diff --git a/go.mod b/go.mod index b801d62ce7..ed7007a6f0 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,7 @@ go 1.19 require ( github.com/VictoriaMetrics/fastcache v1.10.0 - github.com/ava-labs/avalanchego v1.10.7 + github.com/ava-labs/avalanchego v1.10.9-rc.4 github.com/cespare/cp v0.1.0 github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 github.com/davecgh/go-spew v1.1.1 @@ -75,7 +75,6 @@ require ( github.com/gofrs/flock v0.8.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang-jwt/jwt/v4 v4.3.0 // indirect - github.com/golang/mock v1.6.0 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb // indirect github.com/google/pprof v0.0.0-20230207041349-798e818bf904 // indirect @@ -113,7 +112,7 @@ require ( github.com/spf13/afero v1.8.2 // indirect github.com/spf13/jwalterweatherman v1.1.0 // indirect github.com/subosito/gotenv v1.3.0 // indirect - github.com/supranational/blst v0.3.11-0.20230406105308-e9dfc5ee724b // indirect + github.com/supranational/blst v0.3.11 // indirect github.com/tklauser/go-sysconf v0.3.5 // indirect github.com/tklauser/numcpus v0.2.2 // indirect github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect @@ -127,6 +126,7 @@ require ( go.opentelemetry.io/otel/trace v1.11.0 // indirect go.opentelemetry.io/proto/otlp v0.19.0 // indirect go.uber.org/atomic v1.10.0 // indirect + go.uber.org/mock v0.2.0 // indirect go.uber.org/multierr v1.8.0 // indirect go.uber.org/zap v1.24.0 // indirect golang.org/x/net v0.8.0 // indirect diff --git a/go.sum b/go.sum index 4b2660c5c7..651ec8cf22 100644 --- a/go.sum +++ b/go.sum @@ -55,8 +55,8 @@ github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/ava-labs/avalanchego v1.10.7 h1:E1cjNKzWqHrxcvNPAaTwa5cnUH2h6i+6DEDFAGayHZ0= -github.com/ava-labs/avalanchego v1.10.7/go.mod h1:9ekFPMV5ZMIJfsK3BXj0UGW0TzQznke8bwciP3Aav4U= +github.com/ava-labs/avalanchego v1.10.9-rc.4 h1:vtavPfRiF6r1Zc6RV8/arEfVpe9GQsLWHbMfIWkHbMI= +github.com/ava-labs/avalanchego v1.10.9-rc.4/go.mod h1:vTBLl1zK36olfLRA7IUfdbvphWqlkuarIoXxvZTHZVw= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -251,8 +251,6 @@ 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.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.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -561,8 +559,8 @@ github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKs github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/subosito/gotenv v1.3.0 h1:mjC+YW8QpAdXibNi+vNWgzmgBH4+5l5dCXv8cNysBLI= github.com/subosito/gotenv v1.3.0/go.mod h1:YzJjq/33h7nrwdY+iHMhEOEEbW0ovIz0tB6t6PwAXzs= -github.com/supranational/blst v0.3.11-0.20230406105308-e9dfc5ee724b h1:u49mjRnygnB34h8OKbnNJFVUtWSKIKb1KukdV8bILUM= -github.com/supranational/blst v0.3.11-0.20230406105308-e9dfc5ee724b/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= +github.com/supranational/blst v0.3.11 h1:LyU6FolezeWAhvQk0k6O/d49jqgO52MSDDfYgbeoEm4= +github.com/supranational/blst v0.3.11/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7/go.mod h1:q4W45IWZaF22tdD+VEXcAWRA037jwmWEB5VWYORlTpc= github.com/syndtr/goleveldb v1.0.1-0.20220614013038-64ee5596c38a h1:1ur3QoCqvE5fl+nylMaIr9PVV1w343YRDtsy+Rwu7XI= github.com/syndtr/goleveldb v1.0.1-0.20220614013038-64ee5596c38a/go.mod h1:RRCYJbIwD5jmqPI9XoAFR0OcDxqUctll6zUj/+B4S48= @@ -631,6 +629,8 @@ go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/goleak v1.2.1 h1:NBol2c7O1ZokfZ0LEU9K6Whx/KnwvepVetCUhtKja4A= go.uber.org/goleak v1.2.1/go.mod h1:qlT2yGI9QafXHhZZLxlSuNsMw3FFLxBr+tBRlmO1xH4= +go.uber.org/mock v0.2.0 h1:TaP3xedm7JaAgScZO7tlvlKrqT0p7I6OsdGB5YNSMDU= +go.uber.org/mock v0.2.0/go.mod h1:J0y0rp9L3xiff1+ZBfKxlC1fz2+aO16tw0tsDOixfuM= go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= @@ -907,7 +907,6 @@ golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= -golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= diff --git a/node/config.go b/node/config.go index a4af5ed92b..83addddbad 100644 --- a/node/config.go +++ b/node/config.go @@ -34,7 +34,6 @@ import ( "github.com/ava-labs/coreth/accounts" "github.com/ava-labs/coreth/accounts/external" "github.com/ava-labs/coreth/accounts/keystore" - "github.com/ava-labs/coreth/rpc" "github.com/ethereum/go-ethereum/log" ) @@ -61,115 +60,14 @@ type Config struct { // InsecureUnlockAllowed allows user to unlock accounts in unsafe http environment. InsecureUnlockAllowed bool `toml:",omitempty"` - // HTTPHost is the host interface on which to start the HTTP RPC server. If this - // field is empty, no HTTP API endpoint will be started. - HTTPHost string - - // HTTPPort is the TCP port number on which to start the HTTP RPC server. The - // default zero value is/ valid and will pick a port number randomly (useful - // for ephemeral nodes). - HTTPPort int `toml:",omitempty"` - - // HTTPCors is the Cross-Origin Resource Sharing header to send to requesting - // clients. Please be aware that CORS is a browser enforced security, it's fully - // useless for custom HTTP clients. - HTTPCors []string `toml:",omitempty"` - - // HTTPVirtualHosts is the list of virtual hostnames which are allowed on incoming requests. - // This is by default {'localhost'}. Using this prevents attacks like - // DNS rebinding, which bypasses SOP by simply masquerading as being within the same - // origin. These attacks do not utilize CORS, since they are not cross-domain. - // By explicitly checking the Host-header, the server will not allow requests - // made against the server with a malicious host domain. - // Requests using ip address directly are not affected - HTTPVirtualHosts []string `toml:",omitempty"` - - // HTTPModules is a list of API modules to expose via the HTTP RPC interface. - // If the module list is empty, all RPC API endpoints designated public will be - // exposed. - HTTPModules []string - - // HTTPTimeouts allows for customization of the timeout values used by the HTTP RPC - // interface. - HTTPTimeouts rpc.HTTPTimeouts - - // WSHost is the host interface on which to start the websocket RPC server. If - // this field is empty, no websocket API endpoint will be started. - WSHost string - - // WSPort is the TCP port number on which to start the websocket RPC server. The - // default zero value is/ valid and will pick a port number randomly (useful for - // ephemeral nodes). - WSPort int `toml:",omitempty"` - - // WSOrigins is the list of domain to accept websocket requests from. Please be - // aware that the server can only act upon the HTTP request the client sends and - // cannot verify the validity of the request header. - WSOrigins []string `toml:",omitempty"` - - // WSModules is a list of API modules to expose via the websocket RPC interface. - // If the module list is empty, all RPC API endpoints designated public will be - // exposed. - WSModules []string - - // WSExposeAll exposes all API modules via the WebSocket RPC interface rather - // than just the public ones. - // - // *WARNING* Only set this if the node is running in a trusted network, exposing - // private APIs to untrusted users is a major security risk. - WSExposeAll bool `toml:",omitempty"` - - // GraphQLCors is the Cross-Origin Resource Sharing header to send to requesting - // clients. Please be aware that CORS is a browser enforced security, it's fully - // useless for custom HTTP clients. - GraphQLCors []string `toml:",omitempty"` - - // GraphQLVirtualHosts is the list of virtual hostnames which are allowed on incoming requests. - // This is by default {'localhost'}. Using this prevents attacks like - // DNS rebinding, which bypasses SOP by simply masquerading as being within the same - // origin. These attacks do not utilize CORS, since they are not cross-domain. - // By explicitly checking the Host-header, the server will not allow requests - // made against the server with a malicious host domain. - // Requests using ip address directly are not affected - GraphQLVirtualHosts []string `toml:",omitempty"` - CorethVersion string } -// HTTPEndpoint resolves an HTTP endpoint based on the configured host interface -// and port parameters. -func (c *Config) HTTPEndpoint() string { - if c.HTTPHost == "" { - return "" - } - return fmt.Sprintf("%s:%d", c.HTTPHost, c.HTTPPort) -} - -// DefaultHTTPEndpoint returns the HTTP endpoint used by default. -func DefaultHTTPEndpoint() string { - config := &Config{HTTPHost: DefaultHTTPHost, HTTPPort: DefaultHTTPPort} - return config.HTTPEndpoint() -} - -// WSEndpoint resolves a websocket endpoint based on the configured host interface -// and port parameters. -func (c *Config) WSEndpoint() string { - if c.WSHost == "" { - return "" - } - return fmt.Sprintf("%s:%d", c.WSHost, c.WSPort) -} - -// DefaultWSEndpoint returns the websocket endpoint used by default. -func DefaultWSEndpoint() string { - config := &Config{WSHost: DefaultWSHost, WSPort: DefaultWSPort} - return config.WSEndpoint() -} - // ExtRPCEnabled returns the indicator whether node enables the external // RPC(http, ws or graphql). func (c *Config) ExtRPCEnabled() bool { - return c.HTTPHost != "" || c.WSHost != "" + // In avalanche, we always disable the external RPC. + return false } // KeyDirConfig determines the settings for keydirectory diff --git a/node/defaults.go b/node/defaults.go deleted file mode 100644 index 2e636e2abd..0000000000 --- a/node/defaults.go +++ /dev/null @@ -1,49 +0,0 @@ -// (c) 2019-2020, Ava Labs, Inc. -// -// This file is a derived work, based on the go-ethereum library whose original -// notices appear below. -// -// It is distributed under a license compatible with the licensing terms of the -// original code from which it is derived. -// -// Much love to the original authors for their work. -// ********** -// Copyright 2016 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package node - -import ( - "github.com/ava-labs/coreth/rpc" -) - -const ( - DefaultHTTPHost = "localhost" // Default host interface for the HTTP RPC server - DefaultHTTPPort = 8545 // Default TCP port for the HTTP RPC server - DefaultWSHost = "localhost" // Default host interface for the websocket RPC server - DefaultWSPort = 8546 // Default TCP port for the websocket RPC server -) - -// DefaultConfig contains reasonable default settings. -var DefaultConfig = Config{ - HTTPPort: DefaultHTTPPort, - HTTPModules: []string{"net", "web3"}, - HTTPVirtualHosts: []string{"localhost"}, - HTTPTimeouts: rpc.DefaultHTTPTimeouts, - WSPort: DefaultWSPort, - WSModules: []string{"net", "web3"}, - GraphQLVirtualHosts: []string{"localhost"}, -} diff --git a/plugin/evm/client.go b/plugin/evm/client.go index 4e0163a208..0fa4650130 100644 --- a/plugin/evm/client.go +++ b/plugin/evm/client.go @@ -7,15 +7,16 @@ import ( "context" "fmt" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/log" "github.com/ava-labs/avalanchego/api" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" "github.com/ava-labs/avalanchego/utils/formatting" + "github.com/ava-labs/avalanchego/utils/formatting/address" + "github.com/ava-labs/avalanchego/utils/json" "github.com/ava-labs/avalanchego/utils/rpc" - - cjson "github.com/ava-labs/avalanchego/utils/json" ) // Interface compliance @@ -23,22 +24,21 @@ var _ Client = (*client)(nil) // Client interface for interacting with EVM [chain] type Client interface { - IssueTx(ctx context.Context, txBytes []byte) (ids.ID, error) - GetAtomicTxStatus(ctx context.Context, txID ids.ID) (Status, error) - GetAtomicTx(ctx context.Context, txID ids.ID) ([]byte, error) - GetAtomicUTXOs(ctx context.Context, addrs []string, sourceChain string, limit uint32, startAddress, startUTXOID string) ([][]byte, api.Index, error) - ListAddresses(ctx context.Context, userPass api.UserPass) ([]string, error) - ExportKey(ctx context.Context, userPass api.UserPass, addr string) (*secp256k1.PrivateKey, string, error) - ImportKey(ctx context.Context, userPass api.UserPass, privateKey *secp256k1.PrivateKey) (string, error) - Import(ctx context.Context, userPass api.UserPass, to string, sourceChain string) (ids.ID, error) - ExportAVAX(ctx context.Context, userPass api.UserPass, amount uint64, to string) (ids.ID, error) - Export(ctx context.Context, userPass api.UserPass, amount uint64, to string, assetID string) (ids.ID, error) - StartCPUProfiler(ctx context.Context) error - StopCPUProfiler(ctx context.Context) error - MemoryProfile(ctx context.Context) error - LockProfile(ctx context.Context) error - SetLogLevel(ctx context.Context, level log.Lvl) error - GetVMConfig(ctx context.Context) (*Config, error) + IssueTx(ctx context.Context, txBytes []byte, options ...rpc.Option) (ids.ID, error) + GetAtomicTxStatus(ctx context.Context, txID ids.ID, options ...rpc.Option) (Status, error) + GetAtomicTx(ctx context.Context, txID ids.ID, options ...rpc.Option) ([]byte, error) + GetAtomicUTXOs(ctx context.Context, addrs []ids.ShortID, sourceChain string, limit uint32, startAddress ids.ShortID, startUTXOID ids.ID, options ...rpc.Option) ([][]byte, ids.ShortID, ids.ID, error) + ExportKey(ctx context.Context, userPass api.UserPass, addr common.Address, options ...rpc.Option) (*secp256k1.PrivateKey, string, error) + ImportKey(ctx context.Context, userPass api.UserPass, privateKey *secp256k1.PrivateKey, options ...rpc.Option) (common.Address, error) + Import(ctx context.Context, userPass api.UserPass, to common.Address, sourceChain string, options ...rpc.Option) (ids.ID, error) + ExportAVAX(ctx context.Context, userPass api.UserPass, amount uint64, to ids.ShortID, targetChain string, options ...rpc.Option) (ids.ID, error) + Export(ctx context.Context, userPass api.UserPass, amount uint64, to ids.ShortID, targetChain string, assetID string, options ...rpc.Option) (ids.ID, error) + StartCPUProfiler(ctx context.Context, options ...rpc.Option) error + StopCPUProfiler(ctx context.Context, options ...rpc.Option) error + MemoryProfile(ctx context.Context, options ...rpc.Option) error + LockProfile(ctx context.Context, options ...rpc.Option) error + SetLogLevel(ctx context.Context, level log.Lvl, options ...rpc.Option) error + GetVMConfig(ctx context.Context, options ...rpc.Option) (*Config, error) } // Client implementation for interacting with EVM [chain] @@ -61,7 +61,7 @@ func NewCChainClient(uri string) Client { } // IssueTx issues a transaction to a node and returns the TxID -func (c *client) IssueTx(ctx context.Context, txBytes []byte) (ids.ID, error) { +func (c *client) IssueTx(ctx context.Context, txBytes []byte, options ...rpc.Option) (ids.ID, error) { res := &api.JSONTxID{} txStr, err := formatting.Encode(formatting.Hex, txBytes) if err != nil { @@ -70,26 +70,26 @@ func (c *client) IssueTx(ctx context.Context, txBytes []byte) (ids.ID, error) { err = c.requester.SendRequest(ctx, "avax.issueTx", &api.FormattedTx{ Tx: txStr, Encoding: formatting.Hex, - }, res) + }, res, options...) return res.TxID, err } // GetAtomicTxStatus returns the status of [txID] -func (c *client) GetAtomicTxStatus(ctx context.Context, txID ids.ID) (Status, error) { +func (c *client) GetAtomicTxStatus(ctx context.Context, txID ids.ID, options ...rpc.Option) (Status, error) { res := &GetAtomicTxStatusReply{} err := c.requester.SendRequest(ctx, "avax.getAtomicTxStatus", &api.JSONTxID{ TxID: txID, - }, res) + }, res, options...) return res.Status, err } // GetAtomicTx returns the byte representation of [txID] -func (c *client) GetAtomicTx(ctx context.Context, txID ids.ID) ([]byte, error) { +func (c *client) GetAtomicTx(ctx context.Context, txID ids.ID, options ...rpc.Option) ([]byte, error) { res := &api.FormattedTx{} err := c.requester.SendRequest(ctx, "avax.getAtomicTx", &api.GetTxArgs{ TxID: txID, Encoding: formatting.Hex, - }, res) + }, res, options...) if err != nil { return nil, err } @@ -99,70 +99,71 @@ func (c *client) GetAtomicTx(ctx context.Context, txID ids.ID) ([]byte, error) { // GetAtomicUTXOs returns the byte representation of the atomic UTXOs controlled by [addresses] // from [sourceChain] -func (c *client) GetAtomicUTXOs(ctx context.Context, addrs []string, sourceChain string, limit uint32, startAddress, startUTXOID string) ([][]byte, api.Index, error) { +func (c *client) GetAtomicUTXOs(ctx context.Context, addrs []ids.ShortID, sourceChain string, limit uint32, startAddress ids.ShortID, startUTXOID ids.ID, options ...rpc.Option) ([][]byte, ids.ShortID, ids.ID, error) { res := &api.GetUTXOsReply{} err := c.requester.SendRequest(ctx, "avax.getUTXOs", &api.GetUTXOsArgs{ - Addresses: addrs, + Addresses: ids.ShortIDsToStrings(addrs), SourceChain: sourceChain, - Limit: cjson.Uint32(limit), + Limit: json.Uint32(limit), StartIndex: api.Index{ - Address: startAddress, - UTXO: startUTXOID, + Address: startAddress.String(), + UTXO: startUTXOID.String(), }, Encoding: formatting.Hex, - }, res) + }, res, options...) if err != nil { - return nil, api.Index{}, err + return nil, ids.ShortID{}, ids.Empty, err } utxos := make([][]byte, len(res.UTXOs)) for i, utxo := range res.UTXOs { - b, err := formatting.Decode(formatting.Hex, utxo) + utxoBytes, err := formatting.Decode(res.Encoding, utxo) if err != nil { - return nil, api.Index{}, err + return nil, ids.ShortID{}, ids.Empty, err } - utxos[i] = b + utxos[i] = utxoBytes } - return utxos, res.EndIndex, nil -} - -// ListAddresses returns all addresses on this chain controlled by [user] -func (c *client) ListAddresses(ctx context.Context, user api.UserPass) ([]string, error) { - res := &api.JSONAddresses{} - err := c.requester.SendRequest(ctx, "avax.listAddresses", &user, res) - return res.Addresses, err + endAddr, err := address.ParseToID(res.EndIndex.Address) + if err != nil { + return nil, ids.ShortID{}, ids.Empty, err + } + endUTXOID, err := ids.FromString(res.EndIndex.UTXO) + return utxos, endAddr, endUTXOID, err } // ExportKey returns the private key corresponding to [addr] controlled by [user] // in both Avalanche standard format and hex format -func (c *client) ExportKey(ctx context.Context, user api.UserPass, addr string) (*secp256k1.PrivateKey, string, error) { +func (c *client) ExportKey(ctx context.Context, user api.UserPass, addr common.Address, options ...rpc.Option) (*secp256k1.PrivateKey, string, error) { res := &ExportKeyReply{} err := c.requester.SendRequest(ctx, "avax.exportKey", &ExportKeyArgs{ UserPass: user, - Address: addr, - }, res) + Address: addr.Hex(), + }, res, options...) return res.PrivateKey, res.PrivateKeyHex, err } // ImportKey imports [privateKey] to [user] -func (c *client) ImportKey(ctx context.Context, user api.UserPass, privateKey *secp256k1.PrivateKey) (string, error) { +func (c *client) ImportKey(ctx context.Context, user api.UserPass, privateKey *secp256k1.PrivateKey, options ...rpc.Option) (common.Address, error) { res := &api.JSONAddress{} err := c.requester.SendRequest(ctx, "avax.importKey", &ImportKeyArgs{ UserPass: user, PrivateKey: privateKey, - }, res) - return res.Address, err + }, res, options...) + if err != nil { + return common.Address{}, err + } + return ParseEthAddress(res.Address) } // Import sends an import transaction to import funds from [sourceChain] and // returns the ID of the newly created transaction -func (c *client) Import(ctx context.Context, user api.UserPass, to, sourceChain string) (ids.ID, error) { +func (c *client) Import(ctx context.Context, user api.UserPass, to common.Address, sourceChain string, options ...rpc.Option) (ids.ID, error) { res := &api.JSONTxID{} err := c.requester.SendRequest(ctx, "avax.import", &ImportArgs{ UserPass: user, To: to, SourceChain: sourceChain, - }, res) + }, res, options...) return res.TxID, err } @@ -172,9 +173,11 @@ func (c *client) ExportAVAX( ctx context.Context, user api.UserPass, amount uint64, - to string, + to ids.ShortID, + targetChain string, + options ...rpc.Option, ) (ids.ID, error) { - return c.Export(ctx, user, amount, to, "AVAX") + return c.Export(ctx, user, amount, to, targetChain, "AVAX", options...) } // Export sends an asset from this chain to the P/C-Chain. @@ -184,47 +187,50 @@ func (c *client) Export( ctx context.Context, user api.UserPass, amount uint64, - to string, + to ids.ShortID, + targetChain string, assetID string, + options ...rpc.Option, ) (ids.ID, error) { res := &api.JSONTxID{} err := c.requester.SendRequest(ctx, "avax.export", &ExportArgs{ ExportAVAXArgs: ExportAVAXArgs{ - UserPass: user, - Amount: cjson.Uint64(amount), - To: to, + UserPass: user, + Amount: json.Uint64(amount), + TargetChain: targetChain, + To: to.String(), }, AssetID: assetID, - }, res) + }, res, options...) return res.TxID, err } -func (c *client) StartCPUProfiler(ctx context.Context) error { - return c.adminRequester.SendRequest(ctx, "admin.startCPUProfiler", struct{}{}, &api.EmptyReply{}) +func (c *client) StartCPUProfiler(ctx context.Context, options ...rpc.Option) error { + return c.adminRequester.SendRequest(ctx, "admin.startCPUProfiler", struct{}{}, &api.EmptyReply{}, options...) } -func (c *client) StopCPUProfiler(ctx context.Context) error { - return c.adminRequester.SendRequest(ctx, "admin.stopCPUProfiler", struct{}{}, &api.EmptyReply{}) +func (c *client) StopCPUProfiler(ctx context.Context, options ...rpc.Option) error { + return c.adminRequester.SendRequest(ctx, "admin.stopCPUProfiler", struct{}{}, &api.EmptyReply{}, options...) } -func (c *client) MemoryProfile(ctx context.Context) error { - return c.adminRequester.SendRequest(ctx, "admin.memoryProfile", struct{}{}, &api.EmptyReply{}) +func (c *client) MemoryProfile(ctx context.Context, options ...rpc.Option) error { + return c.adminRequester.SendRequest(ctx, "admin.memoryProfile", struct{}{}, &api.EmptyReply{}, options...) } -func (c *client) LockProfile(ctx context.Context) error { - return c.adminRequester.SendRequest(ctx, "admin.lockProfile", struct{}{}, &api.EmptyReply{}) +func (c *client) LockProfile(ctx context.Context, options ...rpc.Option) error { + return c.adminRequester.SendRequest(ctx, "admin.lockProfile", struct{}{}, &api.EmptyReply{}, options...) } // SetLogLevel dynamically sets the log level for the C Chain -func (c *client) SetLogLevel(ctx context.Context, level log.Lvl) error { +func (c *client) SetLogLevel(ctx context.Context, level log.Lvl, options ...rpc.Option) error { return c.adminRequester.SendRequest(ctx, "admin.setLogLevel", &SetLogLevelArgs{ Level: level.String(), - }, &api.EmptyReply{}) + }, &api.EmptyReply{}, options...) } // GetVMConfig returns the current config of the VM -func (c *client) GetVMConfig(ctx context.Context) (*Config, error) { +func (c *client) GetVMConfig(ctx context.Context, options ...rpc.Option) (*Config, error) { res := &ConfigReply{} - err := c.adminRequester.SendRequest(ctx, "admin.getVMConfig", struct{}{}, res) + err := c.adminRequester.SendRequest(ctx, "admin.getVMConfig", struct{}{}, res, options...) return res.Config, err } diff --git a/plugin/evm/config.go b/plugin/evm/config.go index 90e91bb521..7fa3eb33a0 100644 --- a/plugin/evm/config.go +++ b/plugin/evm/config.go @@ -78,8 +78,10 @@ type Duration struct { type Config struct { // Coreth APIs SnowmanAPIEnabled bool `json:"snowman-api-enabled"` - CorethAdminAPIEnabled bool `json:"coreth-admin-api-enabled"` - CorethAdminAPIDir string `json:"coreth-admin-api-dir"` + AdminAPIEnabled bool `json:"admin-api-enabled"` + AdminAPIDir string `json:"admin-api-dir"` + CorethAdminAPIEnabled bool `json:"coreth-admin-api-enabled"` // Deprecated: use AdminAPIEnabled instead + CorethAdminAPIDir string `json:"coreth-admin-api-dir"` // Deprecated: use AdminAPIDir instead // EnabledEthAPIs is a list of Ethereum services that should be enabled // If none is specified, then we use the default list [defaultEnabledAPIs] @@ -144,9 +146,12 @@ type Config struct { KeystoreInsecureUnlockAllowed bool `json:"keystore-insecure-unlock-allowed"` // Gossip Settings - RemoteTxGossipOnlyEnabled bool `json:"remote-tx-gossip-only-enabled"` - TxRegossipFrequency Duration `json:"tx-regossip-frequency"` - TxRegossipMaxSize int `json:"tx-regossip-max-size"` + RemoteGossipOnlyEnabled bool `json:"remote-gossip-only-enabled"` + RegossipFrequency Duration `json:"regossip-frequency"` + RegossipMaxTxs int `json:"regossip-max-txs"` + RemoteTxGossipOnlyEnabled bool `json:"remote-tx-gossip-only-enabled"` // Deprecated: use RemoteGossipOnlyEnabled instead + TxRegossipFrequency Duration `json:"tx-regossip-frequency"` // Deprecated: use RegossipFrequency instead + TxRegossipMaxSize int `json:"tx-regossip-max-size"` // Deprecated: use RegossipMaxTxs instead // Log LogLevel string `json:"log-level"` @@ -230,8 +235,8 @@ func (c *Config) SetDefaults() { c.SnapshotCache = defaultSnapshotCache c.AcceptorQueueLimit = defaultAcceptorQueueLimit c.SnapshotWait = defaultSnapshotWait - c.TxRegossipFrequency.Duration = defaultTxRegossipFrequency - c.TxRegossipMaxSize = defaultTxRegossipMaxSize + c.RegossipFrequency.Duration = defaultTxRegossipFrequency + c.RegossipMaxTxs = defaultTxRegossipMaxSize c.OfflinePruningBloomFilterSize = defaultOfflinePruningBloomFilterSize c.LogLevel = defaultLogLevel c.PopulateMissingTriesParallelism = defaultPopulateMissingTriesParallelism @@ -285,3 +290,30 @@ func (c *Config) Validate() error { return nil } + +func (c *Config) Deprecate() string { + msg := "" + // Deprecate the old config options and set the new ones. + if c.CorethAdminAPIEnabled { + msg += "coreth-admin-api-enabled is deprecated, use admin-api-enabled instead. " + c.AdminAPIEnabled = c.CorethAdminAPIEnabled + } + if c.CorethAdminAPIDir != "" { + msg += "coreth-admin-api-dir is deprecated, use admin-api-dir instead. " + c.AdminAPIDir = c.CorethAdminAPIDir + } + if c.RemoteTxGossipOnlyEnabled { + msg += "remote-tx-gossip-only-enabled is deprecated, use tx-gossip-enabled instead. " + c.RemoteGossipOnlyEnabled = c.RemoteTxGossipOnlyEnabled + } + if c.TxRegossipFrequency != (Duration{}) { + msg += "tx-regossip-frequency is deprecated, use regossip-frequency instead. " + c.RegossipFrequency = c.TxRegossipFrequency + } + if c.TxRegossipMaxSize != 0 { + msg += "tx-regossip-max-size is deprecated, use regossip-max-txs instead. " + c.RegossipMaxTxs = c.TxRegossipMaxSize + } + + return msg +} diff --git a/plugin/evm/formatting.go b/plugin/evm/formatting.go index a586a4558e..ba9cea589f 100644 --- a/plugin/evm/formatting.go +++ b/plugin/evm/formatting.go @@ -14,6 +14,17 @@ import ( "github.com/ethereum/go-ethereum/crypto" ) +// ParseServiceAddress get address ID from address string, being it either localized (using address manager, +// doing also components validations), or not localized. +// If both attempts fail, reports error from localized address parsing +func (vm *VM) ParseServiceAddress(addrStr string) (ids.ShortID, error) { + addr, err := ids.ShortFromString(addrStr) + if err == nil { + return addr, nil + } + return vm.ParseLocalAddress(addrStr) +} + // ParseLocalAddress takes in an address for this chain and produces the ID func (vm *VM) ParseLocalAddress(addrStr string) (ids.ShortID, error) { chainID, addr, err := vm.ParseAddress(addrStr) diff --git a/plugin/evm/gossiper.go b/plugin/evm/gossiper.go index 2ced1de529..0f7d1a287d 100644 --- a/plugin/evm/gossiper.go +++ b/plugin/evm/gossiper.go @@ -140,7 +140,7 @@ func (n *pushGossiper) queueExecutableTxs(state *state.StateDB, baseFee *big.Int } // Don't try to regossip a transaction too frequently - if time.Since(tx.FirstSeen()) < n.config.TxRegossipFrequency.Duration { + if time.Since(tx.FirstSeen()) < n.config.RegossipFrequency.Duration { continue } @@ -197,14 +197,14 @@ func (n *pushGossiper) queueRegossipTxs() types.Transactions { ) return nil } - localQueued := n.queueExecutableTxs(state, tip.BaseFee, localTxs, n.config.TxRegossipMaxSize) + localQueued := n.queueExecutableTxs(state, tip.BaseFee, localTxs, n.config.RegossipMaxTxs) localCount := len(localQueued) n.stats.IncEthTxsRegossipQueuedLocal(localCount) - if localCount >= n.config.TxRegossipMaxSize { + if localCount >= n.config.RegossipMaxTxs { n.stats.IncEthTxsRegossipQueued() return localQueued } - remoteQueued := n.queueExecutableTxs(state, tip.BaseFee, remoteTxs, n.config.TxRegossipMaxSize-localCount) + remoteQueued := n.queueExecutableTxs(state, tip.BaseFee, remoteTxs, n.config.RegossipMaxTxs-localCount) n.stats.IncEthTxsRegossipQueuedRemote(len(remoteQueued)) if localCount+len(remoteQueued) > 0 { // only increment the regossip stat when there are any txs queued @@ -220,7 +220,7 @@ func (n *pushGossiper) awaitEthTxGossip() { go n.ctx.Log.RecoverAndPanic(func() { var ( gossipTicker = time.NewTicker(ethTxsGossipInterval) - regossipTicker = time.NewTicker(n.config.TxRegossipFrequency.Duration) + regossipTicker = time.NewTicker(n.config.RegossipFrequency.Duration) ) defer func() { gossipTicker.Stop() @@ -349,7 +349,7 @@ func (n *pushGossiper) gossipEthTxs(force bool) (int, error) { continue } - if n.config.RemoteTxGossipOnlyEnabled && n.txPool.HasLocal(txHash) { + if n.config.RemoteGossipOnlyEnabled && n.txPool.HasLocal(txHash) { continue } diff --git a/plugin/evm/service.go b/plugin/evm/service.go index 5d5af42301..f2dd1a552f 100644 --- a/plugin/evm/service.go +++ b/plugin/evm/service.go @@ -174,7 +174,7 @@ type ImportArgs struct { SourceChain string `json:"sourceChain"` // The address that will receive the imported funds - To string `json:"to"` + To common.Address `json:"to"` } // ImportAVAX is a deprecated name for Import. @@ -192,11 +192,6 @@ func (service *AvaxAPI) Import(_ *http.Request, args *ImportArgs, response *api. return fmt.Errorf("problem parsing chainID %q: %w", args.SourceChain, err) } - to, err := ParseEthAddress(args.To) - if err != nil { // Parse address - return fmt.Errorf("couldn't parse argument 'to' to an address: %w", err) - } - // Get the user's info db, err := service.vm.ctx.Keystore.GetDatabase(args.Username, args.Password) if err != nil { @@ -224,7 +219,7 @@ func (service *AvaxAPI) Import(_ *http.Request, args *ImportArgs, response *api. baseFee = args.BaseFee.ToInt() } - tx, err := service.vm.newImportTx(chainID, to, baseFee, privKeys) + tx, err := service.vm.newImportTx(chainID, args.To, baseFee, privKeys) if err != nil { return err } @@ -243,8 +238,12 @@ type ExportAVAXArgs struct { // Amount of asset to send Amount json.Uint64 `json:"amount"` - // ID of the address that will receive the AVAX. This address includes the - // chainID, which is used to determine what the destination chain is. + // Chain the funds are going to. Optional. Used if To address does not + // include the chainID. + TargetChain string `json:"targetChain"` + + // ID of the address that will receive the AVAX. This address may include + // the chainID, which is used to determine what the destination chain is. To string `json:"to"` } @@ -278,9 +277,17 @@ func (service *AvaxAPI) Export(_ *http.Request, args *ExportArgs, response *api. return errors.New("argument 'amount' must be > 0") } + // Get the chainID and parse the to address chainID, to, err := service.vm.ParseAddress(args.To) if err != nil { - return err + chainID, err = service.vm.ctx.BCLookup.Lookup(args.TargetChain) + if err != nil { + return err + } + to, err = ids.ShortFromString(args.To) + if err != nil { + return err + } } // Get this user's data @@ -350,7 +357,7 @@ func (service *AvaxAPI) GetUTXOs(r *http.Request, args *api.GetUTXOsArgs, reply addrSet := set.Set[ids.ShortID]{} for _, addrStr := range args.Addresses { - addr, err := service.vm.ParseLocalAddress(addrStr) + addr, err := service.vm.ParseServiceAddress(addrStr) if err != nil { return fmt.Errorf("couldn't parse address %q: %w", addrStr, err) } @@ -360,7 +367,7 @@ func (service *AvaxAPI) GetUTXOs(r *http.Request, args *api.GetUTXOsArgs, reply startAddr := ids.ShortEmpty startUTXO := ids.Empty if args.StartIndex.Address != "" || args.StartIndex.UTXO != "" { - startAddr, err = service.vm.ParseLocalAddress(args.StartIndex.Address) + startAddr, err = service.vm.ParseServiceAddress(args.StartIndex.Address) if err != nil { return fmt.Errorf("couldn't parse start index address %q: %w", args.StartIndex.Address, err) } diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 607650725e..fad4e3084f 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -111,7 +111,6 @@ var ( _ block.ChainVM = &VM{} _ block.StateSyncableVM = &VM{} - _ block.HeightIndexedChainVM = &VM{} _ statesyncclient.EthBlockParser = &VM{} ) @@ -359,6 +358,10 @@ func (vm *VM) Initialize( if err := vm.config.Validate(); err != nil { return err } + // We should deprecate config flags as the first thing, before we do anything else + // because this can set old flags to new flags. log the message after we have + // initialized the logger. + deprecateMsg := vm.config.Deprecate() vm.ctx = chainCtx @@ -381,6 +384,10 @@ func (vm *VM) Initialize( log.Info("Initializing Coreth VM", "Version", Version, "Config", vm.config) + if deprecateMsg != "" { + log.Warn("Deprecation Warning", "msg", deprecateMsg) + } + if len(fxs) > 0 { return errUnsupportedFXs } @@ -1238,8 +1245,8 @@ func (vm *VM) CreateHandlers(context.Context) (map[string]*commonEng.HTTPHandler enabledAPIs = append(enabledAPIs, "avax") apis[avaxEndpoint] = avaxAPI - if vm.config.CorethAdminAPIEnabled { - adminAPI, err := newHandler("admin", NewAdminService(vm, os.ExpandEnv(fmt.Sprintf("%s_coreth_performance_%s", vm.config.CorethAdminAPIDir, primaryAlias)))) + if vm.config.AdminAPIEnabled { + adminAPI, err := newHandler("admin", NewAdminService(vm, os.ExpandEnv(fmt.Sprintf("%s_coreth_performance_%s", vm.config.AdminAPIDir, primaryAlias)))) if err != nil { return nil, fmt.Errorf("failed to register service for admin API due to %w", err) } diff --git a/rpc/client_test.go b/rpc/client_test.go index 0d00ee3405..4ec0ef4122 100644 --- a/rpc/client_test.go +++ b/rpc/client_test.go @@ -29,6 +29,7 @@ package rpc import ( "context" "encoding/json" + "errors" "fmt" "math/rand" "net" @@ -176,6 +177,53 @@ func TestClientBatchRequest(t *testing.T) { } } +func TestClientBatchRequest_len(t *testing.T) { + b, err := json.Marshal([]jsonrpcMessage{ + {Version: "2.0", ID: json.RawMessage("1"), Method: "foo", Result: json.RawMessage(`"0x1"`)}, + {Version: "2.0", ID: json.RawMessage("2"), Method: "bar", Result: json.RawMessage(`"0x2"`)}, + }) + if err != nil { + t.Fatal("failed to encode jsonrpc message:", err) + } + s := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, r *http.Request) { + _, err := rw.Write(b) + if err != nil { + t.Error("failed to write response:", err) + } + })) + t.Cleanup(s.Close) + + client, err := Dial(s.URL) + if err != nil { + t.Fatal("failed to dial test server:", err) + } + defer client.Close() + + t.Run("too-few", func(t *testing.T) { + batch := []BatchElem{ + {Method: "foo"}, + {Method: "bar"}, + {Method: "baz"}, + } + ctx, cancelFn := context.WithTimeout(context.Background(), time.Second) + defer cancelFn() + if err := client.BatchCallContext(ctx, batch); !errors.Is(err, ErrBadResult) { + t.Errorf("expected %q but got: %v", ErrBadResult, err) + } + }) + + t.Run("too-many", func(t *testing.T) { + batch := []BatchElem{ + {Method: "foo"}, + } + ctx, cancelFn := context.WithTimeout(context.Background(), time.Second) + defer cancelFn() + if err := client.BatchCallContext(ctx, batch); !errors.Is(err, ErrBadResult) { + t.Errorf("expected %q but got: %v", ErrBadResult, err) + } + }) +} + func TestClientNotify(t *testing.T) { server := newTestServer() defer server.Stop() diff --git a/scripts/versions.sh b/scripts/versions.sh index ade1d20c48..999778cf5f 100644 --- a/scripts/versions.sh +++ b/scripts/versions.sh @@ -1,4 +1,4 @@ #!/usr/bin/env bash # Don't export them as they're used in the context of other calls -avalanche_version=${AVALANCHE_VERSION:-'v1.10.7'} +avalanche_version=${AVALANCHE_VERSION:-'v1.10.8'} diff --git a/sync/README.md b/sync/README.md index f473a9ac06..b5dc1ae3b7 100644 --- a/sync/README.md +++ b/sync/README.md @@ -28,7 +28,7 @@ State sync code is structured as follows: - `CodeRequestHandler`: handles requests for contract code - `BlockRequestHandler`: handles requests for blocks - _Note: There are response size and time limits in place so peers joining the network do not overload peers providing data. Additionally, the engine tracks the CPU usage of each peer for such messsages and throttles inbound requests accordingly._ -- `sync/client`: Validates reponses from peers and provides support for syncing tries. +- `sync/client`: Validates responses from peers and provides support for syncing tries. - `sync/statesync`: Uses `sync/client` to sync EVM related state: Accounts, storage tries, and contract code. - `plugin/evm/atomicSyncer`: Uses `sync/client` to sync the atomic trie. - `plugin/evm/`: The engine expects the VM to implement `StateSyncableVM` interface, @@ -127,4 +127,4 @@ While state sync is faster than normal bootstrapping, the process may take sever | `state-sync-skip-resume` | `bool` | set to true to avoid resuming an ongoing sync | `false` | | `state-sync-min-blocks` | `uint64` | Minimum number of blocks the chain must be ahead of local state to prefer state sync over bootstrapping | `300,000` | | `state-sync-server-trie-cache` | `int` | Size of trie cache to serve state sync data in MB. Should be set to multiples of `64`. | `64` | -| `state-sync-ids` | `string` | a comma seperated list of `NodeID-` prefixed node IDs to sync data from. If not provided, peers are randomly selected. | | \ No newline at end of file +| `state-sync-ids` | `string` | a comma seperated list of `NodeID-` prefixed node IDs to sync data from. If not provided, peers are randomly selected. | | From 732f12467a1e1e1f60ab6c7642757378ee4532aa Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Tue, 29 Aug 2023 16:56:58 -0400 Subject: [PATCH 57/78] nit --- gossip/handler.go | 5 +++-- plugin/evm/vm.go | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/gossip/handler.go b/gossip/handler.go index 68bad000c2..c045dc1829 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -7,11 +7,12 @@ import ( "context" "time" + "github.com/ethereum/go-ethereum/log" + bloomfilter "github.com/holiman/bloomfilter/v2" + "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" - "github.com/ethereum/go-ethereum/log" - bloomfilter "github.com/holiman/bloomfilter/v2" ) var _ p2p.Handler = (*Handler[Gossipable])(nil) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index fad4e3084f..0d022ee211 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -139,8 +139,8 @@ const ( txGossipBloomMaxItems = 4096 txGossipBloomFalsePositiveRate = 0.001 maxValidatorSetStaleness = time.Minute - throttlingPeriod = time.Second - throttlingLimit = 1 + throttlingPeriod = 5 * time.Second + throttlingLimit = 5 ) var txGossipConfig = gossip.Config{ From 580b9f0a0d364e7296093778d4db97371007e680 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 10:05:32 -0400 Subject: [PATCH 58/78] nit --- plugin/evm/vm.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 0d022ee211..35ea3dfbb2 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -990,7 +990,7 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) - ethTxGossipHandler := &p2p.ThrottledHandler{ + ethTxGossipHandler := &p2p.ThrottlerHandler{ Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), Handler: gossip.NewHandler[*GossipEthTx](ethTxPool, message.SDKCodec, message.Version), } @@ -1000,7 +1000,7 @@ func (vm *VM) initBlockBuilding() error { } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := &p2p.ThrottledHandler{ + atomicTxGossipHandler := &p2p.ThrottlerHandler{ Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), Handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SDKCodec, message.Version), } From 9c069ede121858db538c7e6999ed92e723655a47 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 10:12:13 -0400 Subject: [PATCH 59/78] nit --- go.mod | 2 +- go.sum | 13 +++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index ed7007a6f0..0abc8e0794 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,7 @@ go 1.19 require ( github.com/VictoriaMetrics/fastcache v1.10.0 - github.com/ava-labs/avalanchego v1.10.9-rc.4 + github.com/ava-labs/avalanchego v1.10.10-rc.0 github.com/cespare/cp v0.1.0 github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 github.com/davecgh/go-spew v1.1.1 diff --git a/go.sum b/go.sum index 651ec8cf22..e83a87d369 100644 --- a/go.sum +++ b/go.sum @@ -51,12 +51,16 @@ github.com/VictoriaMetrics/fastcache v1.10.0 h1:5hDJnLsKLpnUEToub7ETuRu8RCkb40wo github.com/VictoriaMetrics/fastcache v1.10.0/go.mod h1:tjiYeEfYXCqacuvYw/7UoDIeJaNxq6132xHICNP77w8= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/ava-labs/avalanchego v1.10.9-rc.4 h1:vtavPfRiF6r1Zc6RV8/arEfVpe9GQsLWHbMfIWkHbMI= github.com/ava-labs/avalanchego v1.10.9-rc.4/go.mod h1:vTBLl1zK36olfLRA7IUfdbvphWqlkuarIoXxvZTHZVw= +github.com/ava-labs/avalanchego v1.10.10-rc.0 h1:6VjkpwhAJ0tDNJK+UIUD8WIb5VelgH3w61mgk7JAkDQ= +github.com/ava-labs/avalanchego v1.10.10-rc.0/go.mod h1:C8R5uiltpc8MQ62ixxgODR+15mesWF0aAw3H+Qrl9Iw= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -123,6 +127,7 @@ github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 h1:ytcWPaNPhNoG github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811/go.mod h1:Nb5lgvnQ2+oGlE/EyZy4+2/CxRh9KfvCXnag1vtpxVM= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= @@ -209,6 +214,7 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -370,13 +376,16 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8= github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE= @@ -407,6 +416,7 @@ github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4F github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= +github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.6 h1:5ibWZ6iY0NctNGWo87LalDlEZ6R41TqbbDamhfG/Qzo= github.com/magiconair/properties v1.8.6/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= @@ -448,9 +458,11 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= @@ -1023,6 +1035,7 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +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= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= From d8aab627221be7a62bbff4af3d6190f47e8145c3 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 10:28:24 -0400 Subject: [PATCH 60/78] Squashed commit of the following: commit 3f5dc8a60f6a73a718596fcfe1d0f924c32298b4 Author: marun Date: Tue Aug 29 10:17:05 2023 -0700 e2e: Add avalanchego e2e job (#305) * e2e: Add avalanchego e2e job * fixup: Use currently supported golang version * fixup: Enable configurable avalanchego clone path commit 4ef26a0bbed8dfad0c6a1dcbd7079f5ce61aab93 Author: Darioush Jalali Date: Mon Aug 28 06:38:44 2023 -0700 bump go version in dockerfile (#311) commit 505a6abb1d68c26b06ed92d9cb48e0338cf1d08d Author: aaronbuchwald Date: Fri Aug 25 15:43:09 2023 -0400 core/state/snapshot: increase batch size during diffToDisk (#309) commit cca3e00f8b53434c63761e068ffd79324030c26e Author: marun Date: Fri Aug 25 09:44:42 2023 -0700 Bump golang version to 1.19.12 for CI (#306) * Bump golang version to 1.19.12 for CI Also bump targeted version of avalanchego to the v1.10.8 (which also uses golang 1.19.12). * fixup: Update go-version specification for consistency with other repos commit 42ba830f2e05d0f3475a1e1dc656a0720e436d50 Author: Darioush Jalali Date: Fri Aug 25 09:44:12 2023 -0700 add upstream test: TestClientBatchRequest_len (#307) commit 08e2b6eda862055c5ccdb36591656763530f7aab Author: Stephen Buttolph Date: Wed Aug 23 18:09:51 2023 -0400 Update avalanchego dependency to v1.10.9-rc.4 (#308) commit 5cd24c3ad3dab4b285eb7a1ac99da7b119ecfff7 Author: Darioush Jalali Date: Wed Aug 23 08:48:01 2023 -0700 node/config.go: remove unused options (#304) * node/config.go: remove unused graphql options * remove more unused config commit 8ea8b18e88d0ae02fdbece92784f9428c5f8f4df Author: aaronbuchwald Date: Tue Aug 15 11:18:29 2023 -0400 Add security file based on AvalancheGo (#303) commit 00214b143a8a5a02c1614fd32d151d9a0300b73e Author: Stephen Buttolph Date: Mon Aug 14 14:58:07 2023 -0400 Update AVAX client implementation and interface to align types (#301) commit 56665535dea6149886b6ec9f15aa3c94880cd274 Author: Ceyhun Onur Date: Mon Aug 14 21:38:04 2023 +0300 renamed flags (#298) * renamed flags * rename admin api flags * add deprecation commit e7645fa60b1a85b490010bd2f982045ef141caa2 Author: Ikko Eltociear Ashimine Date: Thu Aug 10 01:21:44 2023 +0900 Fix typo in sync/README.md (#296) --- .github/workflows/ci.yml | 23 ++++++++++++++++++ .gitignore | 3 +++ scripts/tests.e2e.sh | 52 ++++++++++++++++++++++++++++++++++++++++ 3 files changed, 78 insertions(+) create mode 100755 scripts/tests.e2e.sh diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index ec6f3f2355..ceb4d32e6a 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -148,3 +148,26 @@ jobs: DOCKER_PASS: ${{ secrets.DOCKER_PASS }} KURTOSIS_CLIENT_ID: ${{ secrets.KURTOSIS_CLIENT_ID }} KURTOSIS_CLIENT_SECRET: ${{ secrets.KURTOSIS_CLIENT_SECRET }} + avalanchego_e2e: + name: AvalancheGo E2E Tests v${{ matrix.go }} (${{ matrix.os }}) + runs-on: ${{ matrix.os }} + strategy: + matrix: + os: [ ubuntu-20.04 ] + steps: + - uses: actions/checkout@v3 + - name: check out ${{ github.event.inputs.avalanchegoRepo }} ${{ github.event.inputs.avalanchegoBranch }} + if: ${{ github.event_name == 'workflow_dispatch' }} + uses: actions/checkout@v3 + with: + repository: ${{ github.event.inputs.avalanchegoRepo }} + ref: ${{ github.event.inputs.avalanchegoBranch }} + path: avalanchego + token: ${{ secrets.AVALANCHE_PAT }} + - uses: actions/setup-go@v3 + with: + go-version: '~1.19.12' + check-latest: true + - name: Run e2e tests + run: E2E_SERIAL=1 ./scripts/tests.e2e.sh + shell: bash diff --git a/.gitignore b/.gitignore index 87ff040a44..a93619ce95 100644 --- a/.gitignore +++ b/.gitignore @@ -44,3 +44,6 @@ awscpu bin/ build/ + +# Used for e2e testing +avalanchego diff --git a/scripts/tests.e2e.sh b/scripts/tests.e2e.sh new file mode 100755 index 0000000000..31cf438e56 --- /dev/null +++ b/scripts/tests.e2e.sh @@ -0,0 +1,52 @@ +#!/usr/bin/env bash + +set -euo pipefail + +# Run AvalancheGo e2e tests from the target version against the current state of coreth. + +# e.g., +# ./scripts/tests.e2e.sh +# AVALANCHE_VERSION=v1.10.x ./scripts/tests.e2e.sh +if ! [[ "$0" =~ scripts/tests.e2e.sh ]]; then + echo "must be run from repository root" + exit 255 +fi + +# Coreth root directory +CORETH_PATH=$( cd "$( dirname "${BASH_SOURCE[0]}" )"; cd .. && pwd ) + +# Allow configuring the clone path to point to an existing clone +AVALANCHEGO_CLONE_PATH="${AVALANCHEGO_CLONE_PATH:-avalanchego}" + +# Load the version +source "$CORETH_PATH"/scripts/versions.sh + +# Always return to the coreth path on exit +function cleanup { + cd "${CORETH_PATH}" +} +trap cleanup EXIT + +echo "checking out target AvalancheGo version ${avalanche_version}" +if [[ -d "${AVALANCHEGO_CLONE_PATH}" ]]; then + echo "updating existing clone" + cd "${AVALANCHEGO_CLONE_PATH}" + git fetch + git checkout -B "${avalanche_version}" +else + echo "creating new clone" + git clone -b "${avalanche_version}"\ + --single-branch https://github.com/ava-labs/avalanchego.git\ + "${AVALANCHEGO_CLONE_PATH}" + cd "${AVALANCHEGO_CLONE_PATH}" +fi + +echo "updating coreth dependency to point to ${CORETH_PATH}" +go mod edit -replace "github.com/ava-labs/coreth=${CORETH_PATH}" +go mod tidy + +echo "building avalanchego" +./scripts/build.sh -r + +echo "running AvalancheGo e2e tests" +./scripts/tests.e2e.sh ./build/avalanchego From 601cc4d927259885b64d7fa4f189dd0d04eb3902 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 10:46:08 -0400 Subject: [PATCH 61/78] fix unit tests --- gossip/gossip_test.go | 10 +++--- plugin/evm/gossiper_atomic_gossiping_test.go | 33 +++----------------- plugin/evm/tx_gossip_test.go | 6 ++-- 3 files changed, 13 insertions(+), 36 deletions(-) diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 44bfcc8010..e56b35dfb9 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -17,8 +17,8 @@ import ( "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/avalanchego/utils/units" - "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" ) func TestGossiperShutdown(t *testing.T) { @@ -90,13 +90,15 @@ func TestGossiperGossip(t *testing.T) { for _, item := range tt.responder { require.NoError(responseSet.Add(item)) } + peers := &p2p.Peers{} + require.NoError(peers.Connected(context.Background(), ids.EmptyNodeID, nil)) + handler := NewHandler[*testTx](responseSet, cc, 0) - _, err = responseRouter.RegisterAppProtocol(0x0, handler) + _, err = responseRouter.RegisterAppProtocol(0x0, handler, peers) require.NoError(err) requestSender := common.NewMockSender(ctrl) requestRouter := p2p.NewRouter(logging.NoLog{}, requestSender) - require.NoError(requestRouter.Connected(context.Background(), ids.EmptyNodeID, nil)) gossiped := make(chan struct{}) requestSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). @@ -123,7 +125,7 @@ func TestGossiperGossip(t *testing.T) { require.NoError(requestSet.Add(item)) } - requestClient, err := requestRouter.RegisterAppProtocol(0x0, nil) + requestClient, err := requestRouter.RegisterAppProtocol(0x0, nil, peers) require.NoError(err) config := Config{ diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/gossiper_atomic_gossiping_test.go index 4b331e2fb5..6ded11967b 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/gossiper_atomic_gossiping_test.go @@ -10,15 +10,11 @@ import ( "testing" "time" - "github.com/ava-labs/avalanchego/snow" - bloomfilter "github.com/holiman/bloomfilter/v2" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow" "github.com/ava-labs/avalanchego/utils/set" + "github.com/stretchr/testify/assert" - "github.com/ava-labs/coreth/gossip" "github.com/ava-labs/coreth/plugin/evm/message" ) @@ -61,36 +57,15 @@ func TestMempoolAtmTxsIssueTxAndGossiping(t *testing.T) { return nil } - addedToBloomFilter := false - sender.SendAppRequestF = func(ctx context.Context, _ set.Set[ids.NodeID], _ uint32, bytes []byte) error { - gossipedLock.Lock() - defer gossipedLock.Unlock() - - bytes = bytes[1:] // first byte is an sdk identifier - msg := gossip.PullGossipRequest{} - _, err := vm.networkCodec.Unmarshal(bytes, &msg) - require.NoError(t, err) - - filter := &gossip.BloomFilter{ - Bloom: &bloomfilter.Filter{}, - Salt: msg.SaltBytes, - } - require.NoError(t, filter.Bloom.UnmarshalBinary(msg.FilterBytes)) - if !filter.Has(&GossipAtomicTx{Tx: tx}) { - return nil - } - addedToBloomFilter = true - return nil - } assert.NoError(vm.SetState(context.Background(), snow.NormalOp)) // Optimistically gossip raw tx assert.NoError(vm.issueTx(tx, true /*=local*/)) - time.Sleep(2 * txGossipConfig.Frequency) + time.Sleep(500 * time.Millisecond) gossipedLock.Lock() assert.Equal(1, gossiped) gossipedLock.Unlock() - assert.True(addedToBloomFilter) + assert.True(vm.mempool.bloom.Has(&GossipAtomicTx{Tx: tx})) // Test hash on retry assert.NoError(vm.gossiper.GossipAtomicTxs([]*Tx{tx})) diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index f1e73314cc..2ea8a24895 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -17,8 +17,8 @@ import ( "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" - "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/types" @@ -63,7 +63,7 @@ func TestEthTxGossip(t *testing.T) { router := p2p.NewRouter(logging.NoLog{}, peerSender) // we're only making client requests, so we don't need a server handler - client, err := router.RegisterAppProtocol(ethTxGossipProtocol, nil) + client, err := router.RegisterAppProtocol(ethTxGossipProtocol, nil, nil) require.NoError(err) emptyBloomFilter, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) @@ -157,7 +157,7 @@ func TestAtomicTxGossip(t *testing.T) { router := p2p.NewRouter(logging.NoLog{}, peerSender) // we're only making client requests, so we don't need a server handler - client, err := router.RegisterAppProtocol(atomicTxGossipProtocol, nil) + client, err := router.RegisterAppProtocol(atomicTxGossipProtocol, nil, nil) require.NoError(err) emptyBloomFilter, err := gossip.NewBloomFilter(txGossipBloomMaxItems, txGossipBloomFalsePositiveRate) From 4edcf60dfe13ca9caab90b73df7f8211c27fdeb6 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 10:49:11 -0400 Subject: [PATCH 62/78] clean diff --- go.sum | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/go.sum b/go.sum index 480048ea5a..d0a850c213 100644 --- a/go.sum +++ b/go.sum @@ -51,8 +51,6 @@ github.com/VictoriaMetrics/fastcache v1.10.0 h1:5hDJnLsKLpnUEToub7ETuRu8RCkb40wo github.com/VictoriaMetrics/fastcache v1.10.0/go.mod h1:tjiYeEfYXCqacuvYw/7UoDIeJaNxq6132xHICNP77w8= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= @@ -125,7 +123,6 @@ github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 h1:ytcWPaNPhNoG github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811/go.mod h1:Nb5lgvnQ2+oGlE/EyZy4+2/CxRh9KfvCXnag1vtpxVM= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= @@ -212,7 +209,6 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -374,16 +370,13 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= -github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8= github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE= @@ -414,7 +407,6 @@ github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4F github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= -github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.6 h1:5ibWZ6iY0NctNGWo87LalDlEZ6R41TqbbDamhfG/Qzo= github.com/magiconair/properties v1.8.6/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= @@ -456,11 +448,9 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= @@ -1033,7 +1023,6 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -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= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= From dc3a6d0cc7a0920bf697471b0ef60107cbf4a103 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 10:53:37 -0400 Subject: [PATCH 63/78] clean diff --- gossip/handler.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/gossip/handler.go b/gossip/handler.go index c045dc1829..584f4ad96b 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -19,6 +19,7 @@ var _ p2p.Handler = (*Handler[Gossipable])(nil) func NewHandler[T Gossipable](set Set[T], codec codec.Manager, codecVersion uint16) *Handler[T] { return &Handler[T]{ + Handler: p2p.NoOpHandler{}, set: set, codec: codec, codecVersion: codecVersion, @@ -26,6 +27,7 @@ func NewHandler[T Gossipable](set Set[T], codec codec.Manager, codecVersion uint } type Handler[T Gossipable] struct { + p2p.Handler set Set[T] codec codec.Manager codecVersion uint16 @@ -67,11 +69,3 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time return h.codec.Marshal(h.codecVersion, response) } - -func (h Handler[T]) AppGossip(context.Context, ids.NodeID, []byte) error { - return nil -} - -func (h Handler[T]) CrossChainAppRequest(context.Context, ids.ID, time.Time, []byte) ([]byte, error) { - return nil, nil -} From a738369bb798feca988635e310cdd4c0d919d73f Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 10:59:21 -0400 Subject: [PATCH 64/78] clean diff --- peer/network.go | 4 ++-- peer/network_test.go | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/peer/network.go b/peer/network.go index d8bc6e93e6..fa685e215c 100644 --- a/peer/network.go +++ b/peer/network.go @@ -498,7 +498,7 @@ func (n *network) AppGossip(_ context.Context, nodeID ids.NodeID, gossipBytes [] } // Connected adds the given nodeID to the peer list so that it can receive messages -func (n *network) Connected(ctx context.Context, nodeID ids.NodeID, nodeVersion *version.Application) error { +func (n *network) Connected(_ context.Context, nodeID ids.NodeID, nodeVersion *version.Application) error { log.Debug("adding new peer", "nodeID", nodeID) n.lock.Lock() @@ -518,7 +518,7 @@ func (n *network) Connected(ctx context.Context, nodeID ids.NodeID, nodeVersion } // Disconnected removes given [nodeID] from the peer list -func (n *network) Disconnected(ctx context.Context, nodeID ids.NodeID) error { +func (n *network) Disconnected(_ context.Context, nodeID ids.NodeID) error { log.Debug("disconnecting peer", "nodeID", nodeID) n.lock.Lock() defer n.lock.Unlock() diff --git a/peer/network_test.go b/peer/network_test.go index 8fdf99c7fa..d29067681a 100644 --- a/peer/network_test.go +++ b/peer/network_test.go @@ -16,7 +16,6 @@ import ( "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" - ethcommon "github.com/ethereum/go-ethereum/common" "github.com/ava-labs/coreth/plugin/evm/message" From d1485a8b1851fc77f04e7515c8004d8427a48409 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 11:23:25 -0400 Subject: [PATCH 65/78] update bloom filter constants --- plugin/evm/vm.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 35ea3dfbb2..5569752001 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -136,8 +136,8 @@ const ( // gossip constants txGossipBloomMaxFilledRatio = 0.75 - txGossipBloomMaxItems = 4096 - txGossipBloomFalsePositiveRate = 0.001 + txGossipBloomMaxItems = 8 * 1024 + txGossipBloomFalsePositiveRate = 0.01 maxValidatorSetStaleness = time.Minute throttlingPeriod = 5 * time.Second throttlingLimit = 5 From b1492e5306a95d1fb89b1f3561963eefc7a2c46c Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 15:24:40 -0400 Subject: [PATCH 66/78] use proto --- gossip/gossip.go | 50 ++++++++++++++++------------------- gossip/gossip_test.go | 29 +++++++++----------- gossip/handler.go | 30 ++++++++++----------- gossip/message.go | 13 --------- gossip/proto/message.proto | 14 ++++++++++ plugin/evm/message/codec.go | 18 ------------- plugin/evm/tx_gossip_test.go | 51 +++++++++++++++++------------------- plugin/evm/vm.go | 24 ++++++++--------- 8 files changed, 98 insertions(+), 131 deletions(-) delete mode 100644 gossip/message.go create mode 100644 gossip/proto/message.proto diff --git a/gossip/gossip.go b/gossip/gossip.go index 3c642a7a15..9b2749f68e 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -5,14 +5,15 @@ package gossip import ( "context" - "sync" "time" "github.com/ethereum/go-ethereum/log" + "github.com/golang/protobuf/proto" - "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" + + "github.com/ava-labs/coreth/gossip/proto/pb" ) // GossipableAny exists to help create non-nil pointers to a concrete Gossipable @@ -30,64 +31,57 @@ func NewGossiper[T any, U GossipableAny[T]]( config Config, set Set[U], client *p2p.Client, - codec codec.Manager, - codecVersion uint16, ) *Gossiper[T, U] { return &Gossiper[T, U]{ - config: config, - set: set, - client: client, - codec: codec, - codecVersion: codecVersion, + config: config, + set: set, + client: client, } } type Gossiper[T any, U GossipableAny[T]] struct { - config Config - set Set[U] - client *p2p.Client - codec codec.Manager - codecVersion uint16 + config Config + set Set[U] + client *p2p.Client } -func (g *Gossiper[T, U]) Gossip(shutdownChan chan struct{}, shutdownWg *sync.WaitGroup) { +func (g *Gossiper[_, _]) Gossip(ctx context.Context) { gossipTicker := time.NewTicker(g.config.Frequency) defer func() { gossipTicker.Stop() - shutdownWg.Done() }() for { select { case <-gossipTicker.C: - if err := g.gossip(); err != nil { + if err := g.gossip(ctx); err != nil { log.Warn("failed to gossip", "error", err) } - case <-shutdownChan: + case <-ctx.Done(): log.Debug("shutting down gossip") return } } } -func (g *Gossiper[T, U]) gossip() error { +func (g *Gossiper[_, _]) gossip(ctx context.Context) error { filter := g.set.GetFilter() bloomBytes, err := filter.Bloom.MarshalBinary() if err != nil { return err } - request := PullGossipRequest{ - FilterBytes: bloomBytes, - SaltBytes: filter.Salt, + request := &pb.PullGossipRequest{ + Filter: bloomBytes, + Salt: filter.Salt, } - msgBytes, err := g.codec.Marshal(g.codecVersion, request) + msgBytes, err := proto.Marshal(request) if err != nil { return err } for i := 0; i < g.config.PollSize; i++ { - if err := g.client.AppRequestAny(context.TODO(), msgBytes, g.handleResponse); err != nil { + if err := g.client.AppRequestAny(ctx, msgBytes, g.handleResponse); err != nil { return err } } @@ -101,15 +95,15 @@ func (g *Gossiper[T, U]) handleResponse(nodeID ids.NodeID, responseBytes []byte, return } - response := PullGossipResponse{} - if _, err := g.codec.Unmarshal(responseBytes, &response); err != nil { + response := &pb.PullGossipResponse{} + if err := proto.Unmarshal(responseBytes, response); err != nil { log.Debug("failed to unmarshal gossip response", "error", err) return } - for _, gossipBytes := range response.GossipBytes { + for _, bytes := range response.Gossip { gossipable := U(new(T)) - if err := gossipable.Unmarshal(gossipBytes); err != nil { + if err := gossipable.Unmarshal(bytes); err != nil { log.Debug("failed to unmarshal gossip", "error", err, "nodeID", nodeID) continue } diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index e56b35dfb9..1cdc0a0139 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -9,28 +9,29 @@ import ( "testing" "time" - "github.com/ava-labs/avalanchego/codec" - "github.com/ava-labs/avalanchego/codec/linearcodec" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/utils/units" "github.com/stretchr/testify/require" "go.uber.org/mock/gomock" ) func TestGossiperShutdown(t *testing.T) { config := Config{Frequency: time.Second} - gossiper := NewGossiper[testTx](config, nil, nil, nil, 0) - done := make(chan struct{}) - wg := &sync.WaitGroup{} + gossiper := NewGossiper[testTx](config, nil, nil) + ctx, cancel := context.WithCancel(context.Background()) + wg := &sync.WaitGroup{} wg.Add(1) - go gossiper.Gossip(done, wg) - close(done) + go func() { + gossiper.Gossip(ctx) + wg.Done() + }() + + cancel() wg.Wait() } @@ -73,12 +74,6 @@ func TestGossiperGossip(t *testing.T) { require := require.New(t) ctrl := gomock.NewController(t) - cc := codec.NewManager(units.MiB) - lc := linearcodec.NewDefault() - require.NoError(lc.RegisterType(PullGossipRequest{})) - require.NoError(lc.RegisterType(PullGossipResponse{})) - require.NoError(cc.RegisterCodec(0, lc)) - responseSender := common.NewMockSender(ctrl) responseRouter := p2p.NewRouter(logging.NoLog{}, responseSender) responseBloom, err := NewBloomFilter(1000, 0.01) @@ -93,7 +88,7 @@ func TestGossiperGossip(t *testing.T) { peers := &p2p.Peers{} require.NoError(peers.Connected(context.Background(), ids.EmptyNodeID, nil)) - handler := NewHandler[*testTx](responseSet, cc, 0) + handler := NewHandler[*testTx](responseSet) _, err = responseRouter.RegisterAppProtocol(0x0, handler, peers) require.NoError(err) @@ -132,13 +127,13 @@ func TestGossiperGossip(t *testing.T) { Frequency: 500 * time.Millisecond, PollSize: 1, } - gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient, cc, 0) + gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient) received := set.Set[*testTx]{} requestSet.onAdd = func(tx *testTx) { received.Add(tx) } - require.NoError(gossiper.gossip()) + require.NoError(gossiper.gossip(context.Background())) <-gossiped require.Len(requestSet.set, len(tt.expected)) diff --git a/gossip/handler.go b/gossip/handler.go index 584f4ad96b..2b412c151c 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -8,43 +8,41 @@ import ( "time" "github.com/ethereum/go-ethereum/log" + "github.com/golang/protobuf/proto" bloomfilter "github.com/holiman/bloomfilter/v2" - "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" + + "github.com/ava-labs/coreth/gossip/proto/pb" ) var _ p2p.Handler = (*Handler[Gossipable])(nil) -func NewHandler[T Gossipable](set Set[T], codec codec.Manager, codecVersion uint16) *Handler[T] { +func NewHandler[T Gossipable](set Set[T]) *Handler[T] { return &Handler[T]{ - Handler: p2p.NoOpHandler{}, - set: set, - codec: codec, - codecVersion: codecVersion, + Handler: p2p.NoOpHandler{}, + set: set, } } type Handler[T Gossipable] struct { p2p.Handler - set Set[T] - codec codec.Manager - codecVersion uint16 + set Set[T] } func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time, requestBytes []byte) ([]byte, error) { - request := PullGossipRequest{} - if _, err := h.codec.Unmarshal(requestBytes, &request); err != nil { + request := &pb.PullGossipRequest{} + if err := proto.Unmarshal(requestBytes, request); err != nil { log.Debug("failed to unmarshal gossip request", "nodeID", nodeID, "err", err) return nil, nil } filter := &BloomFilter{ Bloom: &bloomfilter.Filter{}, - Salt: request.SaltBytes, + Salt: request.Salt, } - if err := filter.Bloom.UnmarshalBinary(request.FilterBytes); err != nil { + if err := filter.Bloom.UnmarshalBinary(request.Filter); err != nil { log.Debug("failed to unmarshal bloom filter", "nodeID", nodeID, "err", err) return nil, nil } @@ -63,9 +61,9 @@ func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time gossipBytes = append(gossipBytes, bytes) } - response := PullGossipResponse{ - GossipBytes: gossipBytes, + response := &pb.PullGossipResponse{ + Gossip: gossipBytes, } - return h.codec.Marshal(h.codecVersion, response) + return proto.Marshal(response) } diff --git a/gossip/message.go b/gossip/message.go deleted file mode 100644 index 49ae103d04..0000000000 --- a/gossip/message.go +++ /dev/null @@ -1,13 +0,0 @@ -// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package gossip - -type PullGossipRequest struct { - FilterBytes []byte `serialize:"true"` - SaltBytes []byte `serialize:"true"` -} - -type PullGossipResponse struct { - GossipBytes [][]byte `serialize:"true"` -} diff --git a/gossip/proto/message.proto b/gossip/proto/message.proto new file mode 100644 index 0000000000..92aaac4e4b --- /dev/null +++ b/gossip/proto/message.proto @@ -0,0 +1,14 @@ +syntax = "proto3"; + +package gossip; + +option go_package="github.com/ava-labs/coreth/gossip/proto/pb"; + +message PullGossipRequest { + bytes filter = 1; + bytes salt = 2; +} + +message PullGossipResponse { + repeated bytes gossip = 1; +} diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index 804fb3190e..a698e82124 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -8,8 +8,6 @@ import ( "github.com/ava-labs/avalanchego/codec/linearcodec" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/utils/wrappers" - - "github.com/ava-labs/coreth/gossip" ) const ( @@ -20,7 +18,6 @@ const ( var ( Codec codec.Manager CrossChainCodec codec.Manager - SDKCodec codec.Manager ) func init() { @@ -66,19 +63,4 @@ func init() { if errs.Errored() { panic(errs.Err) } - - SDKCodec = codec.NewManager(maxMessageSize) - sdkc := linearcodec.NewDefault() - - errs = wrappers.Errs{} - errs.Add( - // p2p sdk gossip types - sdkc.RegisterType(gossip.PullGossipRequest{}), - sdkc.RegisterType(gossip.PullGossipResponse{}), - SDKCodec.RegisterCodec(Version, sdkc), - ) - - if errs.Errored() { - panic(errs.Err) - } } diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index 2ea8a24895..0fbd3012ba 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -19,12 +19,13 @@ import ( "github.com/ava-labs/avalanchego/utils/set" "github.com/stretchr/testify/require" "go.uber.org/mock/gomock" + "google.golang.org/protobuf/proto" "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/types" "github.com/ava-labs/coreth/gossip" + "github.com/ava-labs/coreth/gossip/proto/pb" "github.com/ava-labs/coreth/params" - "github.com/ava-labs/coreth/plugin/evm/message" ) func TestEthTxGossip(t *testing.T) { @@ -70,12 +71,12 @@ func TestEthTxGossip(t *testing.T) { require.NoError(err) emptyBloomFilterBytes, err := emptyBloomFilter.Bloom.MarshalBinary() require.NoError(err) - request := gossip.PullGossipRequest{ - FilterBytes: emptyBloomFilterBytes, - SaltBytes: utils.RandomBytes(10), + request := &pb.PullGossipRequest{ + Filter: emptyBloomFilterBytes, + Salt: utils.RandomBytes(10), } - requestBytes, err := vm.networkCodec.Marshal(message.Version, request) + requestBytes, err := proto.Marshal(request) require.NoError(err) wg := &sync.WaitGroup{} @@ -98,10 +99,9 @@ func TestEthTxGossip(t *testing.T) { onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { require.NoError(err) - response := gossip.PullGossipResponse{} - _, err = vm.networkCodec.Unmarshal(responseBytes, &response) - require.NoError(err) - require.Empty(response.GossipBytes) + response := &pb.PullGossipResponse{} + require.NoError(proto.Unmarshal(responseBytes, response)) + require.Empty(response.Gossip) wg.Done() } require.NoError(client.AppRequest(context.Background(), set.Set[ids.NodeID]{vm.ctx.NodeID: struct{}{}}, requestBytes, onResponse)) @@ -123,13 +123,12 @@ func TestEthTxGossip(t *testing.T) { onResponse = func(nodeID ids.NodeID, responseBytes []byte, err error) { require.NoError(err) - response := gossip.PullGossipResponse{} - _, err = vm.networkCodec.Unmarshal(responseBytes, &response) - require.NoError(err) - require.Len(response.GossipBytes, 1) + response := &pb.PullGossipResponse{} + require.NoError(proto.Unmarshal(responseBytes, response)) + require.Len(response.Gossip, 1) gotTx := &GossipEthTx{} - require.NoError(gotTx.Unmarshal(response.GossipBytes[0])) + require.NoError(gotTx.Unmarshal(response.Gossip[0])) require.Equal(signedTx.Hash(), gotTx.Tx.Hash()) wg.Done() @@ -164,11 +163,11 @@ func TestAtomicTxGossip(t *testing.T) { require.NoError(err) bloomBytes, err := emptyBloomFilter.Bloom.MarshalBinary() require.NoError(err) - request := gossip.PullGossipRequest{ - FilterBytes: bloomBytes, - SaltBytes: emptyBloomFilter.Salt, + request := &pb.PullGossipRequest{ + Filter: bloomBytes, + Salt: emptyBloomFilter.Salt, } - requestBytes, err := vm.networkCodec.Marshal(message.Version, request) + requestBytes, err := proto.Marshal(request) require.NoError(err) wg := &sync.WaitGroup{} @@ -190,10 +189,9 @@ func TestAtomicTxGossip(t *testing.T) { onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { require.NoError(err) - response := gossip.PullGossipResponse{} - _, err = vm.networkCodec.Unmarshal(responseBytes, &response) - require.NoError(err) - require.Empty(response.GossipBytes) + response := &pb.PullGossipResponse{} + require.NoError(proto.Unmarshal(responseBytes, response)) + require.Empty(response.Gossip) wg.Done() } require.NoError(client.AppRequest(context.Background(), set.Set[ids.NodeID]{vm.ctx.NodeID: struct{}{}}, requestBytes, onResponse)) @@ -211,13 +209,12 @@ func TestAtomicTxGossip(t *testing.T) { onResponse = func(nodeID ids.NodeID, responseBytes []byte, err error) { require.NoError(err) - response := gossip.PullGossipResponse{} - _, err = vm.networkCodec.Unmarshal(responseBytes, &response) - require.NoError(err) - require.Len(response.GossipBytes, 1) + response := &pb.PullGossipResponse{} + require.NoError(proto.Unmarshal(responseBytes, response)) + require.Len(response.Gossip, 1) gotTx := &GossipAtomicTx{} - require.NoError(gotTx.Unmarshal(response.GossipBytes[0])) + require.NoError(gotTx.Unmarshal(response.Gossip[0])) require.Equal(importTx.InputUTXOs(), gotTx.Tx.InputUTXOs()) wg.Done() diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 5569752001..4c61c24253 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -228,7 +228,9 @@ func init() { // VM implements the snowman.ChainVM interface type VM struct { - ctx *snow.Context + ctx *snow.Context // TODO rename to snowCtx + backgroundCtx context.Context // TODO rename to ctx + cancel context.CancelFunc // *chain.State helps to implement the VM interface by wrapping blocks // with an efficient caching layer. *chain.State @@ -339,7 +341,7 @@ func (vm *VM) GetActivationTime() time.Time { // Initialize implements the snowman.ChainVM interface func (vm *VM) Initialize( - _ context.Context, + ctx context.Context, chainCtx *snow.Context, dbManager manager.Manager, genesisBytes []byte, @@ -364,6 +366,9 @@ func (vm *VM) Initialize( deprecateMsg := vm.config.Deprecate() vm.ctx = chainCtx + ctx, cancel := context.WithCancel(ctx) + vm.backgroundCtx = ctx + vm.cancel = cancel // Create logger alias, err := vm.ctx.BCLookup.PrimaryAlias(vm.ctx.ChainID) @@ -992,7 +997,7 @@ func (vm *VM) initBlockBuilding() error { ethTxGossipHandler := &p2p.ThrottlerHandler{ Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), - Handler: gossip.NewHandler[*GossipEthTx](ethTxPool, message.SDKCodec, message.Version), + Handler: gossip.NewHandler[*GossipEthTx](ethTxPool), } ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler, vm.validators) if err != nil { @@ -1002,7 +1007,7 @@ func (vm *VM) initBlockBuilding() error { atomicTxGossipHandler := &p2p.ThrottlerHandler{ Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), - Handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool, message.SDKCodec, message.Version), + Handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool), } atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler, vm.validators) if err != nil { @@ -1014,21 +1019,15 @@ func (vm *VM) initBlockBuilding() error { txGossipConfig, ethTxPool, vm.ethTxGossipClient, - message.SDKCodec, - message.Version, ) - vm.shutdownWg.Add(1) - go vm.ethTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) + go vm.ethTxGossiper.Gossip(vm.backgroundCtx) vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( txGossipConfig, vm.mempool, vm.atomicTxGossipClient, - message.SDKCodec, - message.Version, ) - vm.shutdownWg.Add(1) - go vm.atomicTxGossiper.Gossip(vm.shutdownChan, &vm.shutdownWg) + go vm.atomicTxGossiper.Gossip(vm.backgroundCtx) return nil } @@ -1068,6 +1067,7 @@ func (vm *VM) Shutdown(context.Context) error { if vm.ctx == nil { return nil } + vm.cancel() vm.Network.Shutdown() if err := vm.StateSyncClient.Shutdown(); err != nil { log.Error("error stopping state syncer", "err", err) From b749986d0cf69bf90ae71583fb9d98522d04da97 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 15:26:01 -0400 Subject: [PATCH 67/78] use 32 byte hash --- gossip/types.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gossip/types.go b/gossip/types.go index 4839f36c4d..cdb9e8280c 100644 --- a/gossip/types.go +++ b/gossip/types.go @@ -3,7 +3,7 @@ package gossip -const HashLength = 8 +const HashLength = 32 type Hash [HashLength]byte From 0cf39894fe310a152beb9ba60403b5d623f30b09 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 23:49:01 -0400 Subject: [PATCH 68/78] Update gossip/gossip.go Co-authored-by: Stephen Buttolph Signed-off-by: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> --- gossip/gossip.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/gossip/gossip.go b/gossip/gossip.go index 9b2749f68e..093d77b22b 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -47,9 +47,7 @@ type Gossiper[T any, U GossipableAny[T]] struct { func (g *Gossiper[_, _]) Gossip(ctx context.Context) { gossipTicker := time.NewTicker(g.config.Frequency) - defer func() { - gossipTicker.Stop() - }() + defer gossipTicker.Stop() for { select { From b405041c142c1da678ce63b344985fc6efd8e178 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Wed, 30 Aug 2023 23:49:23 -0400 Subject: [PATCH 69/78] Update plugin/evm/gossip_mempool.go Co-authored-by: Stephen Buttolph Signed-off-by: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> --- plugin/evm/gossip_mempool.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 6cdb3dab75..a05fb3c616 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -46,12 +46,11 @@ func NewGossipEthTxPool(mempool *txpool.TxPool) (*GossipEthTxPool, error) { return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) } - g := &GossipEthTxPool{ + return &GossipEthTxPool{ mempool: mempool, pendingTxs: make(chan core.NewTxsEvent), bloom: bloom, - } - return g, nil + }, nil } type GossipEthTxPool struct { From 43be5e749c58b98870d266601d92fc9bd33397d9 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 31 Aug 2023 01:04:42 -0400 Subject: [PATCH 70/78] nits --- gossip/bloom.go | 67 +++++++++++++++++++++++++++--------- gossip/gossip.go | 3 +- gossip/gossip_test.go | 49 ++++++++++++++++---------- gossip/handler.go | 40 ++++++++++++++------- gossip/types.go | 14 -------- peer/network.go | 9 +++-- peer/network_test.go | 1 - plugin/evm/gossip_mempool.go | 12 +++++-- plugin/evm/tx_gossip_test.go | 33 ++++++++++++++---- plugin/evm/vm.go | 23 ++++++++----- 10 files changed, 168 insertions(+), 83 deletions(-) delete mode 100644 gossip/types.go diff --git a/gossip/bloom.go b/gossip/bloom.go index 914bdf7672..c4f845d2f6 100644 --- a/gossip/bloom.go +++ b/gossip/bloom.go @@ -8,15 +8,27 @@ import ( "hash" "time" - safemath "github.com/ava-labs/avalanchego/utils/math" bloomfilter "github.com/holiman/bloomfilter/v2" "golang.org/x/exp/rand" ) +const hashLength = 32 + var _ hash.Hash64 = (*hasher)(nil) -func NewBloomFilter(m uint64, p float64) (*BloomFilter, error) { - bloom, err := bloomfilter.NewOptimal(m, p) +type Hash [hashLength]byte + +// NewBloomFilter returns a new instance of a bloom filter with at most +// [maxExpectedElements] elements anticipated at any moment, and a collision +// rate of [falsePositiveProbability]. +func NewBloomFilter( + maxExpectedElements uint64, + falsePositiveProbability float64, +) (*BloomFilter, error) { + bloom, err := bloomfilter.NewOptimal( + maxExpectedElements, + falsePositiveProbability, + ) if err != nil { return nil, err } @@ -33,20 +45,22 @@ type BloomFilter struct { // Salt is provided to eventually unblock collisions in Bloom. It's possible // that conflicting Gossipable items collide in the bloom filter, so a salt // is generated to eventually resolve collisions. - Salt []byte + Salt Hash } func (b *BloomFilter) Add(gossipable Gossipable) { + h := gossipable.GetHash() salted := hasher{ - hash: gossipable.GetHash(), + hash: h[:], salt: b.Salt, } b.Bloom.Add(salted) } func (b *BloomFilter) Has(gossipable Gossipable) bool { + h := gossipable.GetHash() salted := hasher{ - hash: gossipable.GetHash(), + hash: h[:], salt: b.Salt, } return b.Bloom.Contains(salted) @@ -70,27 +84,48 @@ func ResetBloomFilterIfNeeded( return true } -func randomSalt() []byte { - salt := make([]byte, HashLength) +func randomSalt() Hash { + salt := Hash{} r := rand.New(rand.NewSource(uint64(time.Now().Nanosecond()))) - _, _ = r.Read(salt) + _, _ = r.Read(salt[:]) return salt } +var _ hash.Hash64 = (*hasher)(nil) + type hasher struct { - hash.Hash64 - hash Hash - salt []byte + hash []byte + salt Hash +} + +func (h hasher) Write(p []byte) (n int, err error) { + h.hash = append(h.hash, p...) + return len(p), nil +} + +func (h hasher) Sum(b []byte) []byte { + h.hash = append(h.hash, b...) + return h.hash +} + +func (h hasher) Reset() { + reset := Hash{} + h.hash = reset[:] +} + +func (h hasher) BlockSize() int { + return hashLength } func (h hasher) Sum64() uint64 { - for i := 0; i < safemath.Min(len(h.hash), len(h.salt)); i++ { - h.hash[i] ^= h.salt[i] + salted := Hash{} + for i := 0; i < len(h.hash) && i < hashLength; i++ { + salted[i] = h.hash[i] ^ h.salt[i] } - return binary.BigEndian.Uint64(h.hash[:]) + return binary.BigEndian.Uint64(salted[:]) } func (h hasher) Size() int { - return HashLength + return len(h.hash) } diff --git a/gossip/gossip.go b/gossip/gossip.go index 093d77b22b..8eb370a241 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -17,6 +17,7 @@ import ( ) // GossipableAny exists to help create non-nil pointers to a concrete Gossipable +// ref: https://stackoverflow.com/questions/69573113/how-can-i-instantiate-a-non-nil-pointer-of-type-argument-with-generic-go type GossipableAny[T any] interface { *T Gossipable @@ -71,7 +72,7 @@ func (g *Gossiper[_, _]) gossip(ctx context.Context) error { request := &pb.PullGossipRequest{ Filter: bloomBytes, - Salt: filter.Salt, + Salt: filter.Salt[:], } msgBytes, err := proto.Marshal(request) if err != nil { diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 1cdc0a0139..515f83e092 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -37,35 +37,46 @@ func TestGossiperShutdown(t *testing.T) { func TestGossiperGossip(t *testing.T) { tests := []struct { - name string - requester []*testTx // what we have - responder []*testTx // what the peer we're requesting gossip from has - expected []*testTx // what we should have after a gossip cycle + name string + maxResponseSize int + requester []*testTx // what we have + responder []*testTx // what the peer we're requesting gossip from has + expected []*testTx // what we should have after a gossip cycle }{ { name: "no gossip - no one knows anything", }, { - name: "no gossip - requester knows more than responder", - requester: []*testTx{{hash: Hash{0}}}, - expected: []*testTx{{hash: Hash{0}}}, + name: "no gossip - requester knows more than responder", + maxResponseSize: 1024, + requester: []*testTx{{hash: Hash{0}}}, + expected: []*testTx{{hash: Hash{0}}}, }, { - name: "no gossip - requester knows everything responder knows", - requester: []*testTx{{hash: Hash{0}}}, - responder: []*testTx{{hash: Hash{0}}}, - expected: []*testTx{{hash: Hash{0}}}, + name: "no gossip - requester knows everything responder knows", + maxResponseSize: 1024, + requester: []*testTx{{hash: Hash{0}}}, + responder: []*testTx{{hash: Hash{0}}}, + expected: []*testTx{{hash: Hash{0}}}, }, { - name: "gossip - requester knows nothing", - responder: []*testTx{{hash: Hash{0}}}, - expected: []*testTx{{hash: Hash{0}}}, + name: "gossip - requester knows nothing", + maxResponseSize: 1024, + responder: []*testTx{{hash: Hash{0}}}, + expected: []*testTx{{hash: Hash{0}}}, }, { - name: "gossip - requester knows less than responder", - requester: []*testTx{{hash: Hash{0}}}, - responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, - expected: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + name: "gossip - requester knows less than responder", + maxResponseSize: 1024, + requester: []*testTx{{hash: Hash{0}}}, + responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + expected: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + }, + { + name: "gossip - max response size exceeded", + maxResponseSize: 32, + responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + expected: []*testTx{{hash: Hash{0}}}, }, } @@ -88,7 +99,7 @@ func TestGossiperGossip(t *testing.T) { peers := &p2p.Peers{} require.NoError(peers.Connected(context.Background(), ids.EmptyNodeID, nil)) - handler := NewHandler[*testTx](responseSet) + handler := NewHandler[*testTx](responseSet, tt.maxResponseSize) _, err = responseRouter.RegisterAppProtocol(0x0, handler, peers) require.NoError(err) diff --git a/gossip/handler.go b/gossip/handler.go index 2b412c151c..bb1b096ba9 100644 --- a/gossip/handler.go +++ b/gossip/handler.go @@ -5,9 +5,9 @@ package gossip import ( "context" + "errors" "time" - "github.com/ethereum/go-ethereum/log" "github.com/golang/protobuf/proto" bloomfilter "github.com/holiman/bloomfilter/v2" @@ -17,47 +17,63 @@ import ( "github.com/ava-labs/coreth/gossip/proto/pb" ) -var _ p2p.Handler = (*Handler[Gossipable])(nil) +var ( + _ p2p.Handler = (*Handler[Gossipable])(nil) -func NewHandler[T Gossipable](set Set[T]) *Handler[T] { + ErrInvalidHash = errors.New("invalid hash") +) + +func NewHandler[T Gossipable](set Set[T], maxResponseSize int) *Handler[T] { return &Handler[T]{ - Handler: p2p.NoOpHandler{}, - set: set, + Handler: p2p.NoOpHandler{}, + set: set, + maxResponseSize: maxResponseSize, } } type Handler[T Gossipable] struct { p2p.Handler - set Set[T] + set Set[T] + maxResponseSize int } -func (h Handler[T]) AppRequest(_ context.Context, nodeID ids.NodeID, _ time.Time, requestBytes []byte) ([]byte, error) { +func (h Handler[T]) AppRequest(_ context.Context, _ ids.NodeID, _ time.Time, requestBytes []byte) ([]byte, error) { request := &pb.PullGossipRequest{} if err := proto.Unmarshal(requestBytes, request); err != nil { - log.Debug("failed to unmarshal gossip request", "nodeID", nodeID, "err", err) - return nil, nil + return nil, err } + if len(request.Salt) != hashLength { + return nil, ErrInvalidHash + } filter := &BloomFilter{ Bloom: &bloomfilter.Filter{}, - Salt: request.Salt, + Salt: Hash{}, } if err := filter.Bloom.UnmarshalBinary(request.Filter); err != nil { - log.Debug("failed to unmarshal bloom filter", "nodeID", nodeID, "err", err) - return nil, nil + return nil, err } + copy(filter.Salt[:], request.Salt) + // filter out what the requesting peer already knows about unknown := h.set.Get(func(gossipable T) bool { return !filter.Has(gossipable) }) + responseSize := 0 gossipBytes := make([][]byte, 0, len(unknown)) for _, gossipable := range unknown { bytes, err := gossipable.Marshal() if err != nil { return nil, err } + + responseSize += len(bytes) + if responseSize > h.maxResponseSize { + break + } + gossipBytes = append(gossipBytes, bytes) } diff --git a/gossip/types.go b/gossip/types.go deleted file mode 100644 index cdb9e8280c..0000000000 --- a/gossip/types.go +++ /dev/null @@ -1,14 +0,0 @@ -// Copyright (C) 2019-2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package gossip - -const HashLength = 32 - -type Hash [HashLength]byte - -func HashFromBytes(b []byte) Hash { - h := Hash{} - copy(h[:], b) - return h -} diff --git a/peer/network.go b/peer/network.go index fa685e215c..561501605c 100644 --- a/peer/network.go +++ b/peer/network.go @@ -411,7 +411,7 @@ func (n *network) AppResponse(ctx context.Context, nodeID ids.NodeID, requestID // - request times out before a response is provided // error returned by this function is expected to be treated as fatal by the engine // returns error only when the response handler returns an error -func (n *network) AppRequestFailed(_ context.Context, nodeID ids.NodeID, requestID uint32) error { +func (n *network) AppRequestFailed(ctx context.Context, nodeID ids.NodeID, requestID uint32) error { n.lock.Lock() defer n.lock.Unlock() @@ -423,8 +423,11 @@ func (n *network) AppRequestFailed(_ context.Context, nodeID ids.NodeID, request handler, exists := n.markRequestFulfilled(requestID) if !exists { - // Should never happen since the engine should be managing outstanding requests - log.Error("received AppRequestFailed to unknown request", "nodeID", nodeID, "requestID", requestID) + // this must have been a sdk request + log.Debug("received AppRequestFailed to unknown request", "nodeID", nodeID, "requestID", requestID) + if err := n.router.AppRequestFailed(ctx, nodeID, requestID); err != nil { + return err + } return nil } diff --git a/peer/network_test.go b/peer/network_test.go index d29067681a..d65a17bbc2 100644 --- a/peer/network_test.go +++ b/peer/network_test.go @@ -464,7 +464,6 @@ func TestHandleInvalidMessages(t *testing.T) { assert.NoError(t, clientNetwork.AppResponse(context.Background(), nodeID, requestID, garbageResponse)) assert.NoError(t, clientNetwork.AppResponse(context.Background(), nodeID, requestID, emptyResponse)) assert.NoError(t, clientNetwork.AppResponse(context.Background(), nodeID, requestID, nilResponse)) - assert.NoError(t, clientNetwork.AppRequestFailed(context.Background(), nodeID, requestID)) } func TestNetworkPropagatesRequestHandlerError(t *testing.T) { diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index a05fb3c616..1f131c73ec 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -28,7 +28,10 @@ type GossipAtomicTx struct { func (tx *GossipAtomicTx) GetHash() gossip.Hash { id := tx.Tx.ID() - return gossip.HashFromBytes(id[:]) + hash := gossip.Hash{} + copy(hash[:], id[:]) + + return hash } func (tx *GossipAtomicTx) Marshal() ([]byte, error) { @@ -131,8 +134,11 @@ type GossipEthTx struct { } func (tx *GossipEthTx) GetHash() gossip.Hash { - hash := tx.Tx.Hash() - return gossip.HashFromBytes(hash[:]) + txHash := tx.Tx.Hash() + hash := gossip.Hash{} + copy(hash[:], txHash[:]) + + return hash } func (tx *GossipEthTx) Marshal() ([]byte, error) { diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index 0fbd3012ba..3a5656b6b3 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -13,6 +13,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" commonEng "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/snow/validators" "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" "github.com/ava-labs/avalanchego/utils/logging" @@ -45,9 +46,7 @@ func TestEthTxGossip(t *testing.T) { importTx, err := vm.newImportTx(vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) require.NoError(err) - require.NoError(vm.issueTx(importTx, true)) - <-issuer blk, err := vm.BuildBlock(context.Background()) @@ -73,7 +72,7 @@ func TestEthTxGossip(t *testing.T) { require.NoError(err) request := &pb.PullGossipRequest{ Filter: emptyBloomFilterBytes, - Salt: utils.RandomBytes(10), + Salt: utils.RandomBytes(32), } requestBytes, err := proto.Marshal(request) @@ -81,9 +80,10 @@ func TestEthTxGossip(t *testing.T) { wg := &sync.WaitGroup{} + requestingNodeID := ids.GenerateTestNodeID() peerSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(ctx context.Context, nodeIDs set.Set[ids.NodeID], requestID uint32, appRequestBytes []byte) { go func() { - require.NoError(vm.AppRequest(ctx, ids.EmptyNodeID, requestID, time.Time{}, appRequestBytes)) + require.NoError(vm.AppRequest(ctx, requestingNodeID, requestID, time.Time{}, appRequestBytes)) }() }).AnyTimes() @@ -94,6 +94,16 @@ func TestEthTxGossip(t *testing.T) { return nil } + // we only accept gossip requests from validators + mockValidatorSet, ok := vm.ctx.ValidatorState.(*validators.TestState) + require.True(ok) + mockValidatorSet.GetCurrentHeightF = func(context.Context) (uint64, error) { + return 0, nil + } + mockValidatorSet.GetValidatorSetF = func(context.Context, uint64, ids.ID) (map[ids.NodeID]*validators.GetValidatorOutput, error) { + return map[ids.NodeID]*validators.GetValidatorOutput{requestingNodeID: nil}, nil + } + // Ask the VM for any new transactions. We should get nothing at first. wg.Add(1) onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { @@ -165,15 +175,16 @@ func TestAtomicTxGossip(t *testing.T) { require.NoError(err) request := &pb.PullGossipRequest{ Filter: bloomBytes, - Salt: emptyBloomFilter.Salt, + Salt: emptyBloomFilter.Salt[:], } requestBytes, err := proto.Marshal(request) require.NoError(err) + requestingNodeID := ids.GenerateTestNodeID() wg := &sync.WaitGroup{} peerSender.EXPECT().SendAppRequest(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(ctx context.Context, nodeIDs set.Set[ids.NodeID], requestID uint32, appRequestBytes []byte) { go func() { - require.NoError(vm.AppRequest(ctx, ids.EmptyNodeID, requestID, time.Time{}, appRequestBytes)) + require.NoError(vm.AppRequest(ctx, requestingNodeID, requestID, time.Time{}, appRequestBytes)) }() }).AnyTimes() @@ -184,6 +195,16 @@ func TestAtomicTxGossip(t *testing.T) { return nil } + // we only accept gossip requests from validators + mockValidatorSet, ok := vm.ctx.ValidatorState.(*validators.TestState) + require.True(ok) + mockValidatorSet.GetCurrentHeightF = func(context.Context) (uint64, error) { + return 0, nil + } + mockValidatorSet.GetValidatorSetF = func(context.Context, uint64, ids.ID) (map[ids.NodeID]*validators.GetValidatorOutput, error) { + return map[ids.NodeID]*validators.GetValidatorOutput{requestingNodeID: nil}, nil + } + // Ask the VM for any new transactions. We should get nothing at first. wg.Add(1) onResponse := func(nodeID ids.NodeID, responseBytes []byte, err error) { diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 4c61c24253..2070e9aa44 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -135,12 +135,13 @@ const ( atomicTxGossipProtocol = 0x1 // gossip constants + txGossipMaxResponseSize = 20 * units.KiB txGossipBloomMaxFilledRatio = 0.75 txGossipBloomMaxItems = 8 * 1024 txGossipBloomFalsePositiveRate = 0.01 maxValidatorSetStaleness = time.Minute - throttlingPeriod = 5 * time.Second - throttlingLimit = 5 + throttlingPeriod = 10 * time.Second + throttlingLimit = 1 ) var txGossipConfig = gossip.Config{ @@ -995,9 +996,12 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go ethTxPool.Subscribe(vm.shutdownChan, &vm.shutdownWg) - ethTxGossipHandler := &p2p.ThrottlerHandler{ - Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), - Handler: gossip.NewHandler[*GossipEthTx](ethTxPool), + ethTxGossipHandler := &p2p.ValidatorHandler{ + ValidatorSet: vm.validators, + Handler: &p2p.ThrottlerHandler{ + Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), + Handler: gossip.NewHandler[*GossipEthTx](ethTxPool, txGossipMaxResponseSize), + }, } ethTxGossipClient, err := vm.router.RegisterAppProtocol(ethTxGossipProtocol, ethTxGossipHandler, vm.validators) if err != nil { @@ -1005,9 +1009,12 @@ func (vm *VM) initBlockBuilding() error { } vm.ethTxGossipClient = ethTxGossipClient - atomicTxGossipHandler := &p2p.ThrottlerHandler{ - Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), - Handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool), + atomicTxGossipHandler := &p2p.ValidatorHandler{ + ValidatorSet: vm.validators, + Handler: &p2p.ThrottlerHandler{ + Throttler: p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), + Handler: gossip.NewHandler[*GossipAtomicTx](vm.mempool, txGossipMaxResponseSize), + }, } atomicTxGossipClient, err := vm.router.RegisterAppProtocol(atomicTxGossipProtocol, atomicTxGossipHandler, vm.validators) if err != nil { From f79dbc0b256e9e0c9ee2d0d8ba0bcb097af5c78e Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 31 Aug 2023 01:26:09 -0400 Subject: [PATCH 71/78] update go.mod --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 0abc8e0794..10ced6c0fd 100644 --- a/go.mod +++ b/go.mod @@ -44,6 +44,7 @@ require ( golang.org/x/sys v0.8.0 golang.org/x/text v0.8.0 golang.org/x/time v0.0.0-20220922220347-f3bd1da661af + google.golang.org/protobuf v1.30.0 gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) @@ -134,7 +135,6 @@ require ( gonum.org/v1/gonum v0.11.0 // indirect google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4 // indirect google.golang.org/grpc v1.55.0 // indirect - google.golang.org/protobuf v1.30.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce // indirect gopkg.in/yaml.v2 v2.4.0 // indirect From 74a65f03558d0a6fbaaf7117e783624c32b65e0b Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 31 Aug 2023 12:05:45 -0400 Subject: [PATCH 72/78] Update peer/network.go Co-authored-by: Stephen Buttolph Signed-off-by: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> --- peer/network.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/peer/network.go b/peer/network.go index 561501605c..22f9ace624 100644 --- a/peer/network.go +++ b/peer/network.go @@ -425,10 +425,7 @@ func (n *network) AppRequestFailed(ctx context.Context, nodeID ids.NodeID, reque if !exists { // this must have been a sdk request log.Debug("received AppRequestFailed to unknown request", "nodeID", nodeID, "requestID", requestID) - if err := n.router.AppRequestFailed(ctx, nodeID, requestID); err != nil { - return err - } - return nil + return n.router.AppRequestFailed(ctx, nodeID, requestID) } // We must release the slot From 4031484f1006090df6bc38f2d5882fc46fe34573 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 31 Aug 2023 16:04:36 -0400 Subject: [PATCH 73/78] use avalanchego logger --- gossip/gossip.go | 44 ++++++++++++++++++++++++++++++++++--------- gossip/gossip_test.go | 4 ++-- plugin/evm/vm.go | 2 ++ 3 files changed, 39 insertions(+), 11 deletions(-) diff --git a/gossip/gossip.go b/gossip/gossip.go index 8eb370a241..163aa87d91 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -7,8 +7,9 @@ import ( "context" "time" - "github.com/ethereum/go-ethereum/log" + "github.com/ava-labs/avalanchego/utils/logging" "github.com/golang/protobuf/proto" + "go.uber.org/zap" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" @@ -30,11 +31,13 @@ type Config struct { func NewGossiper[T any, U GossipableAny[T]]( config Config, + log logging.Logger, set Set[U], client *p2p.Client, ) *Gossiper[T, U] { return &Gossiper[T, U]{ config: config, + log: log, set: set, client: client, } @@ -42,6 +45,7 @@ func NewGossiper[T any, U GossipableAny[T]]( type Gossiper[T any, U GossipableAny[T]] struct { config Config + log logging.Logger set Set[U] client *p2p.Client } @@ -54,10 +58,10 @@ func (g *Gossiper[_, _]) Gossip(ctx context.Context) { select { case <-gossipTicker.C: if err := g.gossip(ctx); err != nil { - log.Warn("failed to gossip", "error", err) + g.log.Warn("failed to gossip", zap.Error(err)) } case <-ctx.Done(): - log.Debug("shutting down gossip") + g.log.Debug("shutting down gossip") return } } @@ -88,28 +92,50 @@ func (g *Gossiper[_, _]) gossip(ctx context.Context) error { return nil } -func (g *Gossiper[T, U]) handleResponse(nodeID ids.NodeID, responseBytes []byte, err error) { +func (g *Gossiper[T, U]) handleResponse( + nodeID ids.NodeID, + responseBytes []byte, + err error, +) { if err != nil { - log.Debug("failed gossip request", "nodeID", nodeID, "error", err) + g.log.Debug( + "failed gossip request", + zap.Stringer("nodeID", nodeID), + zap.Error(err), + ) return } response := &pb.PullGossipResponse{} if err := proto.Unmarshal(responseBytes, response); err != nil { - log.Debug("failed to unmarshal gossip response", "error", err) + g.log.Debug("failed to unmarshal gossip response", zap.Error(err)) return } for _, bytes := range response.Gossip { gossipable := U(new(T)) if err := gossipable.Unmarshal(bytes); err != nil { - log.Debug("failed to unmarshal gossip", "error", err, "nodeID", nodeID) + g.log.Debug( + "failed to unmarshal gossip", + zap.Stringer("nodeID", nodeID), + zap.Error(err), + ) continue } - log.Debug("received gossip", "nodeID", nodeID, "hash", gossipable.GetHash()) + hash := gossipable.GetHash() + g.log.Debug( + "received gossip", + zap.Stringer("nodeID", nodeID), + zap.Binary("hash", hash[:]), + ) if err := g.set.Add(gossipable); err != nil { - log.Debug("failed to add gossip to the known set", "error", err, "nodeID", nodeID, "id", gossipable.GetHash()) + g.log.Debug( + "failed to add gossip to the known set", + zap.Stringer("nodeID", nodeID), + zap.Binary("id", hash[:]), + zap.Error(err), + ) continue } } diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index 515f83e092..b693dd033e 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -20,7 +20,7 @@ import ( func TestGossiperShutdown(t *testing.T) { config := Config{Frequency: time.Second} - gossiper := NewGossiper[testTx](config, nil, nil) + gossiper := NewGossiper[testTx](config, logging.NoLog{}, nil, nil) ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} @@ -138,7 +138,7 @@ func TestGossiperGossip(t *testing.T) { Frequency: 500 * time.Millisecond, PollSize: 1, } - gossiper := NewGossiper[testTx, *testTx](config, requestSet, requestClient) + gossiper := NewGossiper[testTx, *testTx](config, logging.NoLog{}, requestSet, requestClient) received := set.Set[*testTx]{} requestSet.onAdd = func(tx *testTx) { received.Add(tx) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 2070e9aa44..b31ab0bed5 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -1024,6 +1024,7 @@ func (vm *VM) initBlockBuilding() error { vm.ethTxGossiper = gossip.NewGossiper[GossipEthTx, *GossipEthTx]( txGossipConfig, + vm.ctx.Log, ethTxPool, vm.ethTxGossipClient, ) @@ -1031,6 +1032,7 @@ func (vm *VM) initBlockBuilding() error { vm.atomicTxGossiper = gossip.NewGossiper[GossipAtomicTx, *GossipAtomicTx]( txGossipConfig, + vm.ctx.Log, vm.mempool, vm.atomicTxGossipClient, ) From f06e0371271e2214adbbd6916e231cd915743613 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 31 Aug 2023 16:08:47 -0400 Subject: [PATCH 74/78] fix flakey test --- gossip/gossip_test.go | 66 +++++++++++++++++++++++-------------------- 1 file changed, 35 insertions(+), 31 deletions(-) diff --git a/gossip/gossip_test.go b/gossip/gossip_test.go index b693dd033e..dc9cef3944 100644 --- a/gossip/gossip_test.go +++ b/gossip/gossip_test.go @@ -37,46 +37,52 @@ func TestGossiperShutdown(t *testing.T) { func TestGossiperGossip(t *testing.T) { tests := []struct { - name string - maxResponseSize int - requester []*testTx // what we have - responder []*testTx // what the peer we're requesting gossip from has - expected []*testTx // what we should have after a gossip cycle + name string + maxResponseSize int + requester []*testTx // what we have + responder []*testTx // what the peer we're requesting gossip from has + expectedPossibleValues []*testTx // possible values we can have + expectedLen int }{ { name: "no gossip - no one knows anything", }, { - name: "no gossip - requester knows more than responder", - maxResponseSize: 1024, - requester: []*testTx{{hash: Hash{0}}}, - expected: []*testTx{{hash: Hash{0}}}, + name: "no gossip - requester knows more than responder", + maxResponseSize: 1024, + requester: []*testTx{{hash: Hash{0}}}, + expectedPossibleValues: []*testTx{{hash: Hash{0}}}, + expectedLen: 1, }, { - name: "no gossip - requester knows everything responder knows", - maxResponseSize: 1024, - requester: []*testTx{{hash: Hash{0}}}, - responder: []*testTx{{hash: Hash{0}}}, - expected: []*testTx{{hash: Hash{0}}}, + name: "no gossip - requester knows everything responder knows", + maxResponseSize: 1024, + requester: []*testTx{{hash: Hash{0}}}, + responder: []*testTx{{hash: Hash{0}}}, + expectedPossibleValues: []*testTx{{hash: Hash{0}}}, + expectedLen: 1, }, { - name: "gossip - requester knows nothing", - maxResponseSize: 1024, - responder: []*testTx{{hash: Hash{0}}}, - expected: []*testTx{{hash: Hash{0}}}, + name: "gossip - requester knows nothing", + maxResponseSize: 1024, + responder: []*testTx{{hash: Hash{0}}}, + expectedPossibleValues: []*testTx{{hash: Hash{0}}}, + expectedLen: 1, }, { - name: "gossip - requester knows less than responder", - maxResponseSize: 1024, - requester: []*testTx{{hash: Hash{0}}}, - responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, - expected: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + name: "gossip - requester knows less than responder", + maxResponseSize: 1024, + requester: []*testTx{{hash: Hash{0}}}, + responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + expectedPossibleValues: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + expectedLen: 2, }, { - name: "gossip - max response size exceeded", - maxResponseSize: 32, - responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, - expected: []*testTx{{hash: Hash{0}}}, + name: "gossip - max response size exceeded", + maxResponseSize: 32, + responder: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + expectedPossibleValues: []*testTx{{hash: Hash{0}}, {hash: Hash{1}}}, + expectedLen: 1, }, } @@ -147,10 +153,8 @@ func TestGossiperGossip(t *testing.T) { require.NoError(gossiper.gossip(context.Background())) <-gossiped - require.Len(requestSet.set, len(tt.expected)) - for _, expected := range tt.expected { - require.Contains(requestSet.set, expected) - } + require.Len(requestSet.set, tt.expectedLen) + require.Subset(tt.expectedPossibleValues, requestSet.set.List()) // we should not receive anything that we already had before we // requested the gossip From c0318058f4444c2741ca83c9ca1fe5d9416664f7 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 1 Sep 2023 17:04:49 -0400 Subject: [PATCH 75/78] fix throttling in unit tests --- plugin/evm/tx_gossip_test.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index 3a5656b6b3..be078dc0b2 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -128,6 +128,9 @@ func TestEthTxGossip(t *testing.T) { require.Len(errs, 1) require.Nil(errs[0]) + // wait so we aren't throttled by the vm + time.Sleep(throttlingPeriod / throttlingLimit) + // Ask the VM for new transactions. We should get the newly issued tx. wg.Add(1) onResponse = func(nodeID ids.NodeID, responseBytes []byte, err error) { @@ -225,6 +228,9 @@ func TestAtomicTxGossip(t *testing.T) { require.NoError(vm.issueTx(importTx, true /*=local*/)) <-issuer + // wait so we aren't throttled by the vm + time.Sleep(throttlingPeriod / throttlingLimit) + // Ask the VM for new transactions. We should get the newly issued tx. wg.Add(1) onResponse = func(nodeID ids.NodeID, responseBytes []byte, err error) { From f1128391c3a5ad59c81157720c645895e0abf9a4 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 1 Sep 2023 17:04:49 -0400 Subject: [PATCH 76/78] fix throttling in unit tests --- plugin/evm/vm.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index b31ab0bed5..c9ff043640 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -141,7 +141,7 @@ const ( txGossipBloomFalsePositiveRate = 0.01 maxValidatorSetStaleness = time.Minute throttlingPeriod = 10 * time.Second - throttlingLimit = 1 + throttlingLimit = 2 ) var txGossipConfig = gossip.Config{ From f00b4d596fc9332e04d99b72c9eed4459ff3aaa7 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 1 Sep 2023 19:10:20 -0400 Subject: [PATCH 77/78] address race comments --- core/txpool/txpool.go | 13 +++++++++++++ gossip/gossip.go | 7 +++---- gossip/gossipable.go | 5 +++-- gossip/test_gossip.go | 5 +++-- plugin/evm/gossip_mempool.go | 26 +++++++++++++++----------- plugin/evm/mempool.go | 5 +++-- 6 files changed, 40 insertions(+), 21 deletions(-) diff --git a/core/txpool/txpool.go b/core/txpool/txpool.go index 29be40f192..5095396794 100644 --- a/core/txpool/txpool.go +++ b/core/txpool/txpool.go @@ -630,6 +630,19 @@ func (pool *TxPool) PendingSize() int { return count } +// IteratePending iterates over the [pool.pending] until [f] returns false. +func (pool *TxPool) IteratePending(f func(tx *types.Transaction) bool) { + pending := pool.Pending(true) + + for _, list := range pending { + for _, tx := range list { + if !f(tx) { + return + } + } + } +} + // Locals retrieves the accounts currently considered local by the pool. func (pool *TxPool) Locals() []common.Address { pool.mu.Lock() diff --git a/gossip/gossip.go b/gossip/gossip.go index 163aa87d91..ccd39a3fc2 100644 --- a/gossip/gossip.go +++ b/gossip/gossip.go @@ -68,15 +68,14 @@ func (g *Gossiper[_, _]) Gossip(ctx context.Context) { } func (g *Gossiper[_, _]) gossip(ctx context.Context) error { - filter := g.set.GetFilter() - bloomBytes, err := filter.Bloom.MarshalBinary() + bloom, salt, err := g.set.GetFilter() if err != nil { return err } request := &pb.PullGossipRequest{ - Filter: bloomBytes, - Salt: filter.Salt[:], + Filter: bloom, + Salt: salt, } msgBytes, err := proto.Marshal(request) if err != nil { diff --git a/gossip/gossipable.go b/gossip/gossipable.go index b9548d984a..24ee8d8543 100644 --- a/gossip/gossipable.go +++ b/gossip/gossipable.go @@ -17,6 +17,7 @@ type Set[T Gossipable] interface { Add(gossipable T) error // Get returns elements that match the provided filter function Get(filter func(gossipable T) bool) []T - // GetFilter returns a bloom filter containing the elements in Set - GetFilter() *BloomFilter + // GetFilter returns the byte representation of bloom filter and its + // corresponding salt. + GetFilter() (bloom []byte, salt []byte, err error) } diff --git a/gossip/test_gossip.go b/gossip/test_gossip.go index e7aa2f8f37..f153abf984 100644 --- a/gossip/test_gossip.go +++ b/gossip/test_gossip.go @@ -58,6 +58,7 @@ func (t testSet) Get(filter func(gossipable *testTx) bool) []*testTx { return result } -func (t testSet) GetFilter() *BloomFilter { - return t.bloom +func (t testSet) GetFilter() ([]byte, []byte, error) { + bloom, err := t.bloom.Bloom.MarshalBinary() + return bloom, t.bloom.Salt[:], err } diff --git a/plugin/evm/gossip_mempool.go b/plugin/evm/gossip_mempool.go index 1f131c73ec..0ba204a49e 100644 --- a/plugin/evm/gossip_mempool.go +++ b/plugin/evm/gossip_mempool.go @@ -105,28 +105,32 @@ func (g *GossipEthTxPool) Add(tx *GossipEthTx) error { } func (g *GossipEthTxPool) Get(filter func(tx *GossipEthTx) bool) []*GossipEthTx { - pending := g.mempool.Pending(false) + limit := 1000 + resultSize := 0 result := make([]*GossipEthTx, 0) - for _, txs := range pending { - for _, tx := range txs { - gossipTx := &GossipEthTx{Tx: tx} - if !filter(gossipTx) { - continue - } + g.mempool.IteratePending(func(tx *types.Transaction) bool { + resultSize += int(tx.Size()) + if resultSize > limit { + return false + } - result = append(result, gossipTx) + gossipTx := &GossipEthTx{ + Tx: tx, } - } + result = append(result, gossipTx) + return true + }) return result } -func (g *GossipEthTxPool) GetFilter() *gossip.BloomFilter { +func (g *GossipEthTxPool) GetFilter() ([]byte, []byte, error) { g.lock.RLock() defer g.lock.RUnlock() - return g.bloom + bloom, err := g.bloom.Bloom.MarshalBinary() + return bloom, g.bloom.Salt[:], err } type GossipEthTx struct { diff --git a/plugin/evm/mempool.go b/plugin/evm/mempool.go index 1b555c023d..aa1a036fd9 100644 --- a/plugin/evm/mempool.go +++ b/plugin/evm/mempool.go @@ -309,11 +309,12 @@ func (m *Mempool) Get(filter func(tx *GossipAtomicTx) bool) []*GossipAtomicTx { return gossipTxs } -func (m *Mempool) GetFilter() *gossip.BloomFilter { +func (m *Mempool) GetFilter() ([]byte, []byte, error) { m.lock.RLock() defer m.lock.RUnlock() - return m.bloom + bloom, err := m.bloom.Bloom.MarshalBinary() + return bloom, m.bloom.Salt[:], err } // NextTx returns a transaction to be issued from the mempool. From cfedffe19f1d3cb52baa961d5f9bb3c246b52e69 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 1 Sep 2023 19:13:47 -0400 Subject: [PATCH 78/78] go get pb --- go.sum | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/go.sum b/go.sum index d0a850c213..480048ea5a 100644 --- a/go.sum +++ b/go.sum @@ -51,6 +51,8 @@ github.com/VictoriaMetrics/fastcache v1.10.0 h1:5hDJnLsKLpnUEToub7ETuRu8RCkb40wo github.com/VictoriaMetrics/fastcache v1.10.0/go.mod h1:tjiYeEfYXCqacuvYw/7UoDIeJaNxq6132xHICNP77w8= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= @@ -123,6 +125,7 @@ github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 h1:ytcWPaNPhNoG github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811/go.mod h1:Nb5lgvnQ2+oGlE/EyZy4+2/CxRh9KfvCXnag1vtpxVM= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= @@ -209,6 +212,7 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -370,13 +374,16 @@ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7Bd github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8= github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE= @@ -407,6 +414,7 @@ github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4F github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= +github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.6 h1:5ibWZ6iY0NctNGWo87LalDlEZ6R41TqbbDamhfG/Qzo= github.com/magiconair/properties v1.8.6/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= @@ -448,9 +456,11 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= @@ -1023,6 +1033,7 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +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= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=