From 1b8954cdd4b462e2787913cd82cc8a1b30c46393 Mon Sep 17 00:00:00 2001 From: gammazero <11790789+gammazero@users.noreply.github.com> Date: Wed, 30 Oct 2024 10:40:30 -1000 Subject: [PATCH 01/36] Remove dependency on goprocess The dependency on goprocess is not needed by boxo, and removing it removes the need to support it in code dependent on boxo. Closes #709 --- CHANGELOG.md | 2 + bitswap/bitswap.go | 7 +- bitswap/client/client.go | 38 ++-- bitswap/server/internal/decision/engine.go | 48 +++-- .../server/internal/decision/engine_test.go | 202 ++++++++---------- bitswap/server/server.go | 108 +++++----- bootstrap/bootstrap.go | 108 ++++++---- go.mod | 2 +- go.sum | 1 - namesys/republisher/repub.go | 23 +- namesys/republisher/repub_test.go | 9 +- 11 files changed, 272 insertions(+), 276 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 176a0fb7e..6c5c621d2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,6 +18,8 @@ The following emojis are used to highlight certain changes: ### Changed +- No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. + ### Removed ### Fixed diff --git a/bitswap/bitswap.go b/bitswap/bitswap.go index 393ab96ad..ddc50f6dd 100644 --- a/bitswap/bitswap.go +++ b/bitswap/bitswap.go @@ -144,10 +144,9 @@ func (bs *Bitswap) Stat() (*Stat, error) { func (bs *Bitswap) Close() error { bs.net.Stop() - return multierr.Combine( - bs.Client.Close(), - bs.Server.Close(), - ) + bs.Client.Close() + bs.Server.Close() + return nil } func (bs *Bitswap) WantlistForPeer(p peer.ID) []cid.Cid { diff --git a/bitswap/client/client.go b/bitswap/client/client.go index fc735f448..b4bc91b58 100644 --- a/bitswap/client/client.go +++ b/bitswap/client/client.go @@ -31,8 +31,6 @@ import ( delay "github.com/ipfs/go-ipfs-delay" logging "github.com/ipfs/go-log/v2" "github.com/ipfs/go-metrics-interface" - process "github.com/jbenet/goprocess" - procctx "github.com/jbenet/goprocess/context" "github.com/libp2p/go-libp2p/core/peer" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" @@ -117,10 +115,6 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore // exclusively. We should probably find another way to share logging data ctx, cancelFunc := context.WithCancel(parent) - px := process.WithTeardown(func() error { - return nil - }) - // onDontHaveTimeout is called when a want-block is sent to a peer that // has an old version of Bitswap that doesn't support DONT_HAVE messages, // or when no response is received within a timeout. @@ -165,7 +159,8 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore bs = &Client{ blockstore: bstore, network: network, - process: px, + cancel: cancelFunc, + closing: make(chan struct{}), pm: pm, sm: sm, sim: sim, @@ -185,16 +180,6 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore pqm.Startup() - // bind the context and process. - // do it over here to avoid closing before all setup is done. - go func() { - <-px.Closing() // process closes first - sm.Shutdown() - cancelFunc() - notif.Shutdown() - }() - procctx.CloseAfterContext(px, ctx) // parent cancelled first - return bs } @@ -212,7 +197,9 @@ type Client struct { // manages channels of outgoing blocks for sessions notif notifications.PubSub - process process.Process + cancel context.CancelFunc + closing chan struct{} + closeOnce sync.Once // Counters for various statistics counterLk sync.Mutex @@ -287,7 +274,7 @@ func (bs *Client) NotifyNewBlocks(ctx context.Context, blks ...blocks.Block) err defer span.End() select { - case <-bs.process.Closing(): + case <-bs.closing: return errors.New("bitswap is closed") default: } @@ -310,10 +297,10 @@ func (bs *Client) NotifyNewBlocks(ctx context.Context, blks ...blocks.Block) err return nil } -// receiveBlocksFrom process blocks received from the network +// receiveBlocksFrom processes blocks received from the network func (bs *Client) receiveBlocksFrom(ctx context.Context, from peer.ID, blks []blocks.Block, haves []cid.Cid, dontHaves []cid.Cid) error { select { - case <-bs.process.Closing(): + case <-bs.closing: return errors.New("bitswap is closed") default: } @@ -465,8 +452,13 @@ func (bs *Client) ReceiveError(err error) { } // Close is called to shutdown the Client -func (bs *Client) Close() error { - return bs.process.Close() +func (bs *Client) Close() { + bs.closeOnce.Do(func() { + close(bs.closing) + bs.sm.Shutdown() + bs.cancel() + bs.notif.Shutdown() + }) } // GetWantlist returns the current local wantlist (both want-blocks and diff --git a/bitswap/server/internal/decision/engine.go b/bitswap/server/internal/decision/engine.go index 5e4463e33..a46d67dd7 100644 --- a/bitswap/server/internal/decision/engine.go +++ b/bitswap/server/internal/decision/engine.go @@ -25,7 +25,6 @@ import ( "github.com/ipfs/go-peertaskqueue" "github.com/ipfs/go-peertaskqueue/peertask" "github.com/ipfs/go-peertaskqueue/peertracker" - process "github.com/jbenet/goprocess" "github.com/libp2p/go-libp2p/core/peer" mh "github.com/multiformats/go-multihash" ) @@ -195,6 +194,9 @@ type Engine struct { taskWorkerLock sync.Mutex taskWorkerCount int + waitWorkers sync.WaitGroup + cancel context.CancelFunc + closeOnce sync.Once targetMessageSize int @@ -376,12 +378,13 @@ func wrapTaskComparator(tc TaskComparator) peertask.QueueTaskComparator { // maxOutstandingBytesPerPeer hints to the peer task queue not to give a peer // more tasks if it has some maximum work already outstanding. func NewEngine( - ctx context.Context, bs bstore.Blockstore, peerTagger PeerTagger, self peer.ID, opts ...Option, ) *Engine { + ctx, cancel := context.WithCancel(context.Background()) + e := &Engine{ scoreLedger: NewDefaultScoreLedger(), bstoreWorkerCount: defaults.BitswapEngineBlockstoreWorkerCount, @@ -401,6 +404,7 @@ func NewEngine( tagUseful: fmt.Sprintf(tagFormat, "useful", uuid.New().String()), maxQueuedWantlistEntriesPerPeer: defaults.MaxQueuedWantlistEntiresPerPeer, maxCidSize: defaults.MaximumAllowedCid, + cancel: cancel, } for _, opt := range opts { @@ -437,6 +441,8 @@ func NewEngine( log.Infow("Replace WantHave with WantBlock is enabled", "maxSize", e.wantHaveReplaceSize) } + e.startWorkers(ctx) + return e } @@ -462,7 +468,7 @@ func (e *Engine) SetSendDontHaves(send bool) { // Starts the score ledger. Before start the function checks and, // if it is unset, initializes the scoreLedger with the default // implementation. -func (e *Engine) startScoreLedger(px process.Process) { +func (e *Engine) startScoreLedger() { e.scoreLedger.Start(func(p peer.ID, score int) { if score == 0 { e.peerTagger.UntagPeer(p, e.tagUseful) @@ -470,35 +476,34 @@ func (e *Engine) startScoreLedger(px process.Process) { e.peerTagger.TagPeer(p, e.tagUseful, score) } }) - px.Go(func(ppx process.Process) { - <-ppx.Closing() - e.scoreLedger.Stop() - }) } -func (e *Engine) startBlockstoreManager(px process.Process) { +// startWorkers starts workers to handle requests from other nodes for the data +// on this node. +func (e *Engine) startWorkers(ctx context.Context) { e.bsm.start() - px.Go(func(ppx process.Process) { - <-ppx.Closing() - e.bsm.stop() - }) -} - -// Start up workers to handle requests from other nodes for the data on this node -func (e *Engine) StartWorkers(ctx context.Context, px process.Process) { - e.startBlockstoreManager(px) - e.startScoreLedger(px) + e.startScoreLedger() e.taskWorkerLock.Lock() defer e.taskWorkerLock.Unlock() + e.waitWorkers.Add(e.taskWorkerCount) for i := 0; i < e.taskWorkerCount; i++ { - px.Go(func(_ process.Process) { - e.taskWorker(ctx) - }) + go e.taskWorker(ctx) } } +// Close shuts down the decision engine and returns after all workers have +// finished. Safe to call multiple times/concurrently. +func (e *Engine) Close() { + e.closeOnce.Do(func() { + e.cancel() + e.bsm.stop() + e.scoreLedger.Stop() + }) + e.waitWorkers.Wait() +} + func (e *Engine) onPeerAdded(p peer.ID) { e.peerTagger.TagPeer(p, e.tagQueued, queuedTagWeight) } @@ -524,6 +529,7 @@ func (e *Engine) LedgerForPeer(p peer.ID) *Receipt { // and adds them to an envelope that is passed off to the bitswap workers, // which send the message to the network. func (e *Engine) taskWorker(ctx context.Context) { + defer e.waitWorkers.Done() defer e.taskWorkerExit() for { oneTimeUse := make(chan *Envelope, 1) // buffer to prevent blocking diff --git a/bitswap/server/internal/decision/engine_test.go b/bitswap/server/internal/decision/engine_test.go index 5cc1375c7..d549e1218 100644 --- a/bitswap/server/internal/decision/engine_test.go +++ b/bitswap/server/internal/decision/engine_test.go @@ -23,7 +23,6 @@ import ( ds "github.com/ipfs/go-datastore" dssync "github.com/ipfs/go-datastore/sync" "github.com/ipfs/go-test/random" - process "github.com/jbenet/goprocess" peer "github.com/libp2p/go-libp2p/core/peer" libp2ptest "github.com/libp2p/go-libp2p/core/test" mh "github.com/multiformats/go-multihash" @@ -95,15 +94,14 @@ type engineSet struct { Blockstore blockstore.Blockstore } -func newTestEngine(ctx context.Context, idStr string, opts ...Option) engineSet { - return newTestEngineWithSampling(ctx, idStr, shortTerm, nil, clock.New(), opts...) +func newTestEngine(idStr string, opts ...Option) engineSet { + return newTestEngineWithSampling(idStr, shortTerm, nil, clock.New(), opts...) } -func newTestEngineWithSampling(ctx context.Context, idStr string, peerSampleInterval time.Duration, sampleCh chan struct{}, clock clock.Clock, opts ...Option) engineSet { +func newTestEngineWithSampling(idStr string, peerSampleInterval time.Duration, sampleCh chan struct{}, clock clock.Clock, opts ...Option) engineSet { fpt := &fakePeerTagger{} bs := blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())) - e := newEngineForTesting(ctx, bs, fpt, "localhost", 0, append(opts[:len(opts):len(opts)], WithScoreLedger(NewTestScoreLedger(peerSampleInterval, sampleCh, clock)), WithBlockstoreWorkerCount(4))...) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, fpt, "localhost", 0, append(opts[:len(opts):len(opts)], WithScoreLedger(NewTestScoreLedger(peerSampleInterval, sampleCh, clock)), WithBlockstoreWorkerCount(4))...) return engineSet{ Peer: peer.ID(idStr), PeerTagger: fpt, @@ -113,20 +111,19 @@ func newTestEngineWithSampling(ctx context.Context, idStr string, peerSampleInte } func TestConsistentAccounting(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - sender := newTestEngine(ctx, "Ernie") - receiver := newTestEngine(ctx, "Bert") + sender := newTestEngine("Ernie") + defer sender.Engine.Close() + receiver := newTestEngine("Bert") + defer receiver.Engine.Close() // Send messages from Ernie to Bert for i := 0; i < 1000; i++ { - m := message.New(false) content := []string{"this", "is", "message", "i"} m.AddBlock(blocks.NewBlock([]byte(strings.Join(content, " ")))) sender.Engine.MessageSent(receiver.Peer, m) - receiver.Engine.MessageReceived(ctx, sender.Peer, m) + receiver.Engine.MessageReceived(context.Background(), sender.Peer, m) receiver.Engine.ReceivedBlocks(sender.Peer, m.Blocks()) } @@ -148,17 +145,17 @@ func TestConsistentAccounting(t *testing.T) { } func TestPeerIsAddedToPeersWhenMessageSent(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - sanfrancisco := newTestEngine(ctx, "sf") - seattle := newTestEngine(ctx, "sea") + sanfrancisco := newTestEngine("sf") + defer sanfrancisco.Engine.Close() + seattle := newTestEngine("sea") + defer seattle.Engine.Close() m := message.New(true) // We need to request something for it to add us as partner. m.AddEntry(blocks.NewBlock([]byte("Hæ")).Cid(), 0, pb.Message_Wantlist_Block, true) - seattle.Engine.MessageReceived(ctx, sanfrancisco.Peer, m) + seattle.Engine.MessageReceived(context.Background(), sanfrancisco.Peer, m) if seattle.Peer == sanfrancisco.Peer { t.Fatal("Sanity Check: Peers have same Key!") @@ -184,7 +181,6 @@ func peerIsPartner(p peer.ID, e *Engine) bool { } func newEngineForTesting( - ctx context.Context, bs blockstore.Blockstore, peerTagger PeerTagger, self peer.ID, @@ -192,14 +188,13 @@ func newEngineForTesting( opts ...Option, ) *Engine { opts = append(opts, WithWantHaveReplaceSize(wantHaveReplaceSize)) - return NewEngine(ctx, bs, peerTagger, self, opts...) + return NewEngine(bs, peerTagger, self, opts...) } func TestOutboxClosedWhenEngineClosed(t *testing.T) { t.SkipNow() // TODO implement *Engine.Close - ctx := context.Background() - e := newEngineForTesting(ctx, blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())), &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())), &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) + defer e.Close() var wg sync.WaitGroup wg.Add(1) go func() { @@ -526,9 +521,8 @@ func TestPartnerWantHaveWantBlockNonActive(t *testing.T) { testCases = onlyTestCases } - ctx := context.Background() - e := newEngineForTesting(ctx, bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) + defer e.Close() for i, testCase := range testCases { t.Logf("Test case %d:", i) for _, wl := range testCase.wls { @@ -683,9 +677,8 @@ func TestPartnerWantHaveWantBlockActive(t *testing.T) { testCases = onlyTestCases } - ctx := context.Background() - e := newEngineForTesting(ctx, bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) + defer e.Close() var next envChan for i, testCase := range testCases { @@ -866,11 +859,10 @@ func TestPartnerWantsThenCancels(t *testing.T) { } } - ctx := context.Background() for i := 0; i < numRounds; i++ { expected := make([][]string, 0, len(testcases)) - e := newEngineForTesting(ctx, bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) + defer e.Close() for _, testcase := range testcases { set := testcase[0] cancels := testcase[1] @@ -894,9 +886,8 @@ func TestSendReceivedBlocksToPeersThatWantThem(t *testing.T) { partner := libp2ptest.RandPeerIDFatal(t) otherPeer := libp2ptest.RandPeerIDFatal(t) - ctx := context.Background() - e := newEngineForTesting(ctx, bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) + defer e.Close() blks := random.BlocksOfSize(4, 8*1024) msg := message.New(false) @@ -940,9 +931,8 @@ func TestSendDontHave(t *testing.T) { partner := libp2ptest.RandPeerIDFatal(t) otherPeer := libp2ptest.RandPeerIDFatal(t) - ctx := context.Background() - e := newEngineForTesting(ctx, bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) + defer e.Close() blks := random.BlocksOfSize(4, 8*1024) msg := message.New(false) @@ -1006,9 +996,8 @@ func TestWantlistForPeer(t *testing.T) { partner := libp2ptest.RandPeerIDFatal(t) otherPeer := libp2ptest.RandPeerIDFatal(t) - ctx := context.Background() - e := newEngineForTesting(ctx, bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, &fakePeerTagger{}, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4)) + defer e.Close() blks := random.BlocksOfSize(4, 8*1024) msg := message.New(false) @@ -1039,9 +1028,6 @@ func TestWantlistForPeer(t *testing.T) { } func TestTaskComparator(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) - defer cancel() - keys := []string{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"} cids := make(map[cid.Cid]int) blks := make([]blocks.Block, 0, len(keys)) @@ -1054,19 +1040,22 @@ func TestTaskComparator(t *testing.T) { fpt := &fakePeerTagger{} sl := NewTestScoreLedger(shortTerm, nil, clock.New()) bs := blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())) + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + if err := bs.PutMany(ctx, blks); err != nil { t.Fatal(err) } // use a single task worker so that the order of outgoing messages is deterministic - e := newEngineForTesting(ctx, bs, fpt, "localhost", 0, WithScoreLedger(sl), WithBlockstoreWorkerCount(4), WithTaskWorkerCount(1), + e := newEngineForTesting(bs, fpt, "localhost", 0, WithScoreLedger(sl), WithBlockstoreWorkerCount(4), WithTaskWorkerCount(1), // if this Option is omitted, the test fails WithTaskComparator(func(ta, tb *TaskInfo) bool { // prioritize based on lexicographic ordering of block content return cids[ta.Cid] < cids[tb.Cid] }), ) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + defer e.Close() // rely on randomness of Go map's iteration order to add Want entries in random order peerIDs := make([]peer.ID, len(keys)) @@ -1093,9 +1082,6 @@ func TestTaskComparator(t *testing.T) { } func TestPeerBlockFilter(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) - defer cancel() - // Generate a few keys keys := []string{"a", "b", "c", "d"} blks := make([]blocks.Block, 0, len(keys)) @@ -1114,11 +1100,14 @@ func TestPeerBlockFilter(t *testing.T) { fpt := &fakePeerTagger{} sl := NewTestScoreLedger(shortTerm, nil, clock.New()) bs := blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())) + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + if err := bs.PutMany(ctx, blks); err != nil { t.Fatal(err) } - e := newEngineForTesting(ctx, bs, fpt, "localhost", 0, WithScoreLedger(sl), WithBlockstoreWorkerCount(4), + e := newEngineForTesting(bs, fpt, "localhost", 0, WithScoreLedger(sl), WithBlockstoreWorkerCount(4), WithPeerBlockRequestFilter(func(p peer.ID, c cid.Cid) bool { // peer 0 has access to everything if p == peerIDs[0] { @@ -1132,7 +1121,7 @@ func TestPeerBlockFilter(t *testing.T) { return blks[3].Cid().Equals(c) }), ) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + defer e.Close() // Setup the test type testCaseEntry struct { @@ -1252,9 +1241,6 @@ func TestPeerBlockFilter(t *testing.T) { } func TestPeerBlockFilterMutability(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) - defer cancel() - // Generate a few keys keys := []string{"a", "b", "c", "d"} blks := make([]blocks.Block, 0, len(keys)) @@ -1269,18 +1255,21 @@ func TestPeerBlockFilterMutability(t *testing.T) { fpt := &fakePeerTagger{} sl := NewTestScoreLedger(shortTerm, nil, clock.New()) bs := blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())) + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + if err := bs.PutMany(ctx, blks); err != nil { t.Fatal(err) } filterAllowList := make(map[cid.Cid]bool) - e := newEngineForTesting(ctx, bs, fpt, "localhost", 0, WithScoreLedger(sl), WithBlockstoreWorkerCount(4), + e := newEngineForTesting(bs, fpt, "localhost", 0, WithScoreLedger(sl), WithBlockstoreWorkerCount(4), WithPeerBlockRequestFilter(func(p peer.ID, c cid.Cid) bool { return filterAllowList[c] }), ) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + defer e.Close() // Setup the test type testCaseEntry struct { @@ -1421,10 +1410,10 @@ func TestPeerBlockFilterMutability(t *testing.T) { } func TestTaggingPeers(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) - defer cancel() - sanfrancisco := newTestEngine(ctx, "sf") - seattle := newTestEngine(ctx, "sea") + sanfrancisco := newTestEngine("sf") + defer sanfrancisco.Engine.Close() + seattle := newTestEngine("sea") + defer seattle.Engine.Close() keys := []string{"a", "b", "c", "d", "e"} for _, letter := range keys { @@ -1451,12 +1440,10 @@ func TestTaggingPeers(t *testing.T) { func TestTaggingUseful(t *testing.T) { const peerSampleIntervalHalf = 10 * time.Millisecond - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - defer cancel() - sampleCh := make(chan struct{}) mockClock := clock.NewMock() - me := newTestEngineWithSampling(ctx, "engine", peerSampleIntervalHalf*2, sampleCh, mockClock) + me := newTestEngineWithSampling("engine", peerSampleIntervalHalf*2, sampleCh, mockClock) + defer me.Engine.Close() mockClock.Add(1 * time.Millisecond) friend := peer.ID("friend") @@ -1544,9 +1531,6 @@ func partnerCancels(e *Engine, keys []string, partner peer.ID) { type envChan <-chan *Envelope func getNextEnvelope(e *Engine, next envChan, t time.Duration) (envChan, *Envelope) { - ctx, cancel := context.WithTimeout(context.Background(), t) - defer cancel() - if next == nil { next = <-e.Outbox() // returns immediately } @@ -1558,7 +1542,7 @@ func getNextEnvelope(e *Engine, next envChan, t time.Duration) (envChan, *Envelo return nil, nil } return nil, env - case <-ctx.Done(): + case <-time.After(t): // log.Warnf("got timeout") } return next, nil @@ -1606,12 +1590,11 @@ func stringsComplement(set, subset []string) []string { } func TestWantlistDoesNotGrowPastLimit(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - const limit = 32 - warsaw := newTestEngine(ctx, "warsaw", WithMaxQueuedWantlistEntriesPerPeer(limit)) - riga := newTestEngine(ctx, "riga") + warsaw := newTestEngine("warsaw", WithMaxQueuedWantlistEntriesPerPeer(limit)) + defer warsaw.Engine.Close() + riga := newTestEngine("riga") + defer riga.Engine.Close() // Send in two messages to test reslicing. for i := 2; i != 0; i-- { @@ -1619,7 +1602,7 @@ func TestWantlistDoesNotGrowPastLimit(t *testing.T) { for j := limit * 3 / 4; j != 0; j-- { m.AddEntry(blocks.NewBlock([]byte(fmt.Sprint(i, j))).Cid(), 0, pb.Message_Wantlist_Block, true) } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) } if warsaw.Peer == riga.Peer { @@ -1633,19 +1616,19 @@ func TestWantlistDoesNotGrowPastLimit(t *testing.T) { } func TestWantlistGrowsToLimit(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - const limit = 32 - warsaw := newTestEngine(ctx, "warsaw", WithMaxQueuedWantlistEntriesPerPeer(limit)) - riga := newTestEngine(ctx, "riga") + warsaw := newTestEngine("warsaw", WithMaxQueuedWantlistEntriesPerPeer(limit)) + defer warsaw.Engine.Close() + riga := newTestEngine("riga") + defer riga.Engine.Close() // Send in two messages to test reslicing. m := message.New(false) for j := limit; j != 0; j-- { m.AddEntry(blocks.NewBlock([]byte(strconv.Itoa(j))).Cid(), 0, pb.Message_Wantlist_Block, true) } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) if warsaw.Peer == riga.Peer { t.Fatal("Sanity Check: Peers have same Key!") @@ -1658,12 +1641,11 @@ func TestWantlistGrowsToLimit(t *testing.T) { } func TestIgnoresCidsAboveLimit(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - const cidLimit = 64 - warsaw := newTestEngine(ctx, "warsaw", WithMaxCidSize(cidLimit)) - riga := newTestEngine(ctx, "riga") + warsaw := newTestEngine("warsaw", WithMaxCidSize(cidLimit)) + defer warsaw.Engine.Close() + riga := newTestEngine("riga") + defer riga.Engine.Close() // Send in two messages to test reslicing. m := message.New(true) @@ -1678,7 +1660,7 @@ func TestIgnoresCidsAboveLimit(t *testing.T) { rand.Read(hash[startOfDigest:]) m.AddEntry(cid.NewCidV1(cid.Raw, hash), 0, pb.Message_Wantlist_Block, true) - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) if warsaw.Peer == riga.Peer { t.Fatal("Sanity Check: Peers have same Key!") @@ -1691,11 +1673,10 @@ func TestIgnoresCidsAboveLimit(t *testing.T) { } func TestKillConnectionForInlineCid(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - warsaw := newTestEngine(ctx, "warsaw") - riga := newTestEngine(ctx, "riga") + warsaw := newTestEngine("warsaw") + defer warsaw.Engine.Close() + riga := newTestEngine("riga") + defer riga.Engine.Close() if warsaw.Peer == riga.Peer { t.Fatal("Sanity Check: Peers have same Key!") @@ -1715,7 +1696,7 @@ func TestKillConnectionForInlineCid(t *testing.T) { rand.Read(hash[startOfDigest:]) m.AddEntry(cid.NewCidV1(cid.Raw, hash), 0, pb.Message_Wantlist_Block, true) - if !warsaw.Engine.MessageReceived(ctx, riga.Peer, m) { + if !warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) { t.Fatal("connection was not killed when receiving inline in cancel") } @@ -1724,15 +1705,12 @@ func TestKillConnectionForInlineCid(t *testing.T) { m.AddEntry(blocks.NewBlock([]byte("Hæ")).Cid(), 0, pb.Message_Wantlist_Block, true) m.Cancel(cid.NewCidV1(cid.Raw, hash)) - if !warsaw.Engine.MessageReceived(ctx, riga.Peer, m) { + if !warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) { t.Fatal("connection was not killed when receiving inline in cancel") } } func TestWantlistBlocked(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - const limit = 32 bs := blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())) @@ -1752,15 +1730,17 @@ func TestWantlistBlocked(t *testing.T) { } fpt := &fakePeerTagger{} - e := newEngineForTesting(ctx, bs, fpt, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4), WithMaxQueuedWantlistEntriesPerPeer(limit)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, fpt, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4), WithMaxQueuedWantlistEntriesPerPeer(limit)) + defer e.Close() + warsaw := engineSet{ Peer: peer.ID("warsaw"), PeerTagger: fpt, Blockstore: bs, Engine: e, } - riga := newTestEngine(ctx, "riga") + riga := newTestEngine("riga") + defer riga.Engine.Close() if warsaw.Peer == riga.Peer { t.Fatal("Sanity Check: Peers have same Key!") } @@ -1773,7 +1753,7 @@ func TestWantlistBlocked(t *testing.T) { m.AddEntry(c, 1, pb.Message_Wantlist_Block, true) dontHaveCids[i] = c } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) wl := warsaw.Engine.WantlistForPeer(riga.Peer) // Check that all the dontHave wants are on the wantlist. for _, c := range dontHaveCids { @@ -1787,7 +1767,7 @@ func TestWantlistBlocked(t *testing.T) { for _, c := range haveCids { m.AddEntry(c, 1, pb.Message_Wantlist_Block, true) } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) wl = warsaw.Engine.WantlistForPeer(riga.Peer) // Check that all the dontHave wants are on the wantlist. for _, c := range haveCids { @@ -1804,7 +1784,7 @@ func TestWantlistBlocked(t *testing.T) { m.AddEntry(c, 1, pb.Message_Wantlist_Block, true) dontHaveCids[i] = c } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) // Check that all the new dontHave wants are not on the wantlist. for _, c := range dontHaveCids { if findCid(c, wl) { @@ -1815,9 +1795,6 @@ func TestWantlistBlocked(t *testing.T) { } func TestWantlistOverflow(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - const limit = 32 bs := blockstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())) @@ -1838,20 +1815,21 @@ func TestWantlistOverflow(t *testing.T) { } fpt := &fakePeerTagger{} - e := newEngineForTesting(ctx, bs, fpt, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4), WithMaxQueuedWantlistEntriesPerPeer(limit)) - e.StartWorkers(ctx, process.WithTeardown(func() error { return nil })) + e := newEngineForTesting(bs, fpt, "localhost", 0, WithScoreLedger(NewTestScoreLedger(shortTerm, nil, clock.New())), WithBlockstoreWorkerCount(4), WithMaxQueuedWantlistEntriesPerPeer(limit)) + defer e.Close() warsaw := engineSet{ Peer: peer.ID("warsaw"), PeerTagger: fpt, Blockstore: bs, Engine: e, } - riga := newTestEngine(ctx, "riga") + riga := newTestEngine("riga") + defer riga.Engine.Close() if warsaw.Peer == riga.Peer { t.Fatal("Sanity Check: Peers have same Key!") } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) // Check that the wantlist is at the size limit. wl := warsaw.Engine.WantlistForPeer(riga.Peer) if len(wl) != limit { @@ -1867,7 +1845,7 @@ func TestWantlistOverflow(t *testing.T) { m.AddEntry(c, 0, pb.Message_Wantlist_Block, true) lowPrioCids[i] = c } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) wl = warsaw.Engine.WantlistForPeer(riga.Peer) if len(wl) != limit { t.Fatal("wantlist size", len(wl), "does not match limit", limit) @@ -1893,7 +1871,7 @@ func TestWantlistOverflow(t *testing.T) { m.AddEntry(c, 10, pb.Message_Wantlist_Block, true) highPrioCids[i] = c } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) wl = warsaw.Engine.WantlistForPeer(riga.Peer) if len(wl) != limit { t.Fatal("wantlist size", len(wl), "does not match limit", limit) @@ -1918,7 +1896,7 @@ func TestWantlistOverflow(t *testing.T) { m.AddEntry(c, 0, pb.Message_Wantlist_Block, true) blockCids[i] = c } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) wl = warsaw.Engine.WantlistForPeer(riga.Peer) if len(wl) != limit { t.Fatal("wantlist size", len(wl), "does not match limit", limit) @@ -1942,7 +1920,7 @@ func TestWantlistOverflow(t *testing.T) { for _, c := range origCids { m.AddEntry(c, 0, pb.Message_Wantlist_Block, true) } - warsaw.Engine.MessageReceived(ctx, riga.Peer, m) + warsaw.Engine.MessageReceived(context.Background(), riga.Peer, m) wl = warsaw.Engine.WantlistForPeer(riga.Peer) for _, c := range origCids { if !findCid(c, wl) { diff --git a/bitswap/server/server.go b/bitswap/server/server.go index 46d29a8fc..2b45b324b 100644 --- a/bitswap/server/server.go +++ b/bitswap/server/server.go @@ -20,8 +20,6 @@ import ( "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "github.com/ipfs/go-metrics-interface" - process "github.com/jbenet/goprocess" - procctx "github.com/jbenet/goprocess/context" "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/zap" ) @@ -57,7 +55,12 @@ type Server struct { // the total number of simultaneous threads sending outgoing messages taskWorkerCount int - process process.Process + // Cancel stops the server + cancel context.CancelFunc + closing chan struct{} + closeOnce sync.Once + // waitWorkers waits for all worker goroutines to exit. + waitWorkers sync.WaitGroup // newBlocks is a channel for newly added blocks to be provided to the // network. blocks pushed down this channel get buffered and fed to the @@ -78,20 +81,13 @@ type Server struct { func New(ctx context.Context, network bsnet.BitSwapNetwork, bstore blockstore.Blockstore, options ...Option) *Server { ctx, cancel := context.WithCancel(ctx) - px := process.WithTeardown(func() error { - return nil - }) - go func() { - <-px.Closing() // process closes first - cancel() - }() - s := &Server{ sentHistogram: bmetrics.SentHist(ctx), sendTimeHistogram: bmetrics.SendTimeHist(ctx), taskWorkerCount: defaults.BitswapTaskWorkerCount, network: network, - process: px, + cancel: cancel, + closing: make(chan struct{}), provideEnabled: true, hasBlockBufferSize: defaults.HasBlockBufferSize, provideKeys: make(chan cid.Cid, provideKeysBufferSize), @@ -103,7 +99,6 @@ func New(ctx context.Context, network bsnet.BitSwapNetwork, bstore blockstore.Bl } s.engine = decision.NewEngine( - ctx, bstore, network.ConnectionManager(), network.Self(), @@ -111,7 +106,7 @@ func New(ctx context.Context, network bsnet.BitSwapNetwork, bstore blockstore.Bl ) s.engineOptions = nil - s.startWorkers(ctx, px) + s.startWorkers(ctx) return s } @@ -293,33 +288,31 @@ func (bs *Server) WantlistForPeer(p peer.ID) []cid.Cid { return out } -func (bs *Server) startWorkers(ctx context.Context, px process.Process) { - bs.engine.StartWorkers(ctx, px) - +func (bs *Server) startWorkers(ctx context.Context) { // Start up workers to handle requests from other nodes for the data on this node + bs.waitWorkers.Add(bs.taskWorkerCount) for i := 0; i < bs.taskWorkerCount; i++ { i := i - px.Go(func(px process.Process) { - bs.taskWorker(ctx, i) - }) + go bs.taskWorker(ctx, i) } if bs.provideEnabled { - // Start up a worker to manage sending out provides messages - px.Go(func(px process.Process) { - bs.provideCollector(ctx) - }) + bs.waitWorkers.Add(1) + go bs.provideCollector(ctx) // Spawn up multiple workers to handle incoming blocks // consider increasing number if providing blocks bottlenecks // file transfers - px.Go(bs.provideWorker) + bs.waitWorkers.Add(1) + go bs.provideWorker(ctx) } } func (bs *Server) taskWorker(ctx context.Context, id int) { - defer log.Debug("bitswap task worker shutting down...") + defer bs.waitWorkers.Done() + log := log.With("ID", id) + defer log.Debug("bitswap task worker shutting down...") for { log.Debug("Bitswap.TaskWorker.Loop") select { @@ -341,8 +334,7 @@ func (bs *Server) taskWorker(ctx context.Context, id int) { } bs.sendBlocks(ctx, envelope) - dur := time.Since(start) - bs.sendTimeHistogram.Observe(dur.Seconds()) + bs.sendTimeHistogram.Observe(time.Since(start).Seconds()) case <-ctx.Done(): return @@ -452,7 +444,7 @@ func (bs *Server) Stat() (Stat, error) { // that those blocks are available in the blockstore before calling this function. func (bs *Server) NotifyNewBlocks(ctx context.Context, blks ...blocks.Block) error { select { - case <-bs.process.Closing(): + case <-bs.closing: return errors.New("bitswap is closed") default: } @@ -466,8 +458,8 @@ func (bs *Server) NotifyNewBlocks(ctx context.Context, blks ...blocks.Block) err select { case bs.newBlocks <- blk.Cid(): // send block off to be reprovided - case <-bs.process.Closing(): - return bs.process.Close() + case <-bs.closing: + return nil } } } @@ -476,6 +468,7 @@ func (bs *Server) NotifyNewBlocks(ctx context.Context, blks ...blocks.Block) err } func (bs *Server) provideCollector(ctx context.Context) { + defer bs.waitWorkers.Done() defer close(bs.provideKeys) var toProvide []cid.Cid var nextKey cid.Cid @@ -508,18 +501,16 @@ func (bs *Server) provideCollector(ctx context.Context) { } } -func (bs *Server) provideWorker(px process.Process) { - // FIXME: OnClosingContext returns a _custom_ context type. - // Unfortunately, deriving a new cancelable context from this custom - // type fires off a goroutine. To work around this, we create a single - // cancelable context up-front and derive all sub-contexts from that. - // - // See: https://github.com/ipfs/go-ipfs/issues/5810 - ctx := procctx.OnClosingContext(px) - ctx, cancel := context.WithCancel(ctx) - defer cancel() - +func (bs *Server) provideWorker(ctx context.Context) { limit := make(chan struct{}, provideWorkerMax) + defer func() { + // Wait until all limitGoProvide goroutines are done before declaring + // this worker as done. + for i := 0; i < provideWorkerMax; i++ { + limit <- struct{}{} + } + bs.waitWorkers.Done() + }() limitedGoProvide := func(k cid.Cid, wid int) { defer func() { @@ -540,25 +531,18 @@ func (bs *Server) provideWorker(px process.Process) { // worker spawner, reads from bs.provideKeys until it closes, spawning a // _ratelimited_ number of workers to handle each key. - for wid := 2; ; wid++ { + wid := 2 + for k := range bs.provideKeys { log.Debug("Bitswap.ProvideWorker.Loop") - select { - case <-px.Closing(): + case limit <- struct{}{}: + go limitedGoProvide(k, wid) + case <-ctx.Done(): return - case k, ok := <-bs.provideKeys: - if !ok { - log.Debug("provideKeys channel closed") - return - } - select { - case <-px.Closing(): - return - case limit <- struct{}{}: - go limitedGoProvide(k, wid) - } } + wid++ } + log.Debug("provideKeys channel closed") } func (bs *Server) ReceiveMessage(ctx context.Context, p peer.ID, incoming message.BitSwapMessage) { @@ -597,7 +581,13 @@ func (bs *Server) PeerDisconnected(p peer.ID) { bs.engine.PeerDisconnected(p) } -// Close is called to shutdown the Client -func (bs *Server) Close() error { - return bs.process.Close() +// Close is called to shutdown the Server. Returns when all workers and +// decision engine have finished. Safe to calling multiple times/concurrently. +func (bs *Server) Close() { + bs.closeOnce.Do(func() { + close(bs.closing) + bs.cancel() + }) + bs.engine.Close() + bs.waitWorkers.Wait() } diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index 28b004559..eea833be3 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -10,9 +10,6 @@ import ( "time" logging "github.com/ipfs/go-log/v2" - "github.com/jbenet/goprocess" - goprocessctx "github.com/jbenet/goprocess/context" - periodicproc "github.com/jbenet/goprocess/periodic" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" @@ -112,78 +109,105 @@ func (cfg *BootstrapConfig) SetBackupPeers(load func(context.Context) []peer.Add // connections to well-known bootstrap peers. It also kicks off subsystem // bootstrapping (i.e. routing). func Bootstrap(id peer.ID, host host.Host, rt routing.Routing, cfg BootstrapConfig) (io.Closer, error) { - // make a signal to wait for one bootstrap round to complete. - doneWithRound := make(chan struct{}) - if len(cfg.BootstrapPeers()) == 0 { // We *need* to bootstrap but we have no bootstrap peers // configured *at all*, inform the user. log.Warn("no bootstrap nodes configured: go-ipfs may have difficulty connecting to the network") } - // the periodic bootstrap function -- the connection supervisor - periodic := func(worker goprocess.Process) { - ctx := goprocessctx.OnClosingContext(worker) + ctx, cancel := context.WithCancel(context.Background()) - if err := bootstrapRound(ctx, host, cfg); err != nil { - log.Debugf("%s bootstrap error: %s", id, err) + // Signal when first bootstrap round is complete, started independent of ticker. + doneWithRound := make(chan struct{}) + + go func() { + // the periodic bootstrap function -- the connection supervisor + periodic := func() { + if err := bootstrapRound(ctx, host, cfg); err != nil { + log.Debugf("%s bootstrap error: %s", id, err) + } } - // Exit the first call (triggered independently by `proc.Go`, not `Tick`) - // only after being done with the *single* Routing.Bootstrap call. Following - // periodic calls (`Tick`) will not block on this. + ticker := time.NewTicker(cfg.Period) + defer ticker.Stop() + + // Run first round independent of ticker. + periodic() <-doneWithRound - } + if ctx.Err() != nil { + return + } - // kick off the node's periodic bootstrapping - proc := periodicproc.Tick(cfg.Period, periodic) - proc.Go(periodic) // run one right now. + for { + select { + case <-ticker.C: + periodic() + case <-ctx.Done(): + return + } + } + }() // kick off Routing.Bootstrap if rt != nil { - ctx := goprocessctx.OnClosingContext(proc) if err := rt.Bootstrap(ctx); err != nil { - proc.Close() + cancel() + close(doneWithRound) return nil, err } } - doneWithRound <- struct{}{} - close(doneWithRound) // it no longer blocks periodic - // If loadBackupBootstrapPeers is not nil then saveBackupBootstrapPeers // must also not be nil. if cfg.loadBackupBootstrapPeers != nil { - startSavePeersAsTemporaryBootstrapProc(cfg, host, proc) + doneWithRound <- struct{}{} // wait for first bootstrap + startSavePeersAsTemporaryBootstrapProc(ctx, cfg, host) } - return proc, nil + return &bootstrapCloser{ + cancel: cancel, + }, nil +} + +type bootstrapCloser struct { + cancel context.CancelFunc +} + +func (bsc *bootstrapCloser) Close() error { + bsc.cancel() + return nil } // Aside of the main bootstrap process we also run a secondary one that saves // connected peers as a backup measure if we can't connect to the official // bootstrap ones. These peers will serve as *temporary* bootstrap nodes. -func startSavePeersAsTemporaryBootstrapProc(cfg BootstrapConfig, host host.Host, bootstrapProc goprocess.Process) { - savePeersFn := func(worker goprocess.Process) { - ctx := goprocessctx.OnClosingContext(worker) +func startSavePeersAsTemporaryBootstrapProc(ctx context.Context, cfg BootstrapConfig, host host.Host) { + go func() { + periodic := func() { + if err := saveConnectedPeersAsTemporaryBootstrap(ctx, host, cfg); err != nil { + log.Debugf("saveConnectedPeersAsTemporaryBootstrap error: %s", err) + } + } + + ticker := time.NewTicker(cfg.BackupBootstrapInterval) + defer ticker.Stop() - if err := saveConnectedPeersAsTemporaryBootstrap(ctx, host, cfg); err != nil { - log.Debugf("saveConnectedPeersAsTemporaryBootstrap error: %s", err) + // Run the first round now (after the first bootstrap process has + // finished) as the SavePeersPeriod can be much longer than bootstrap. + periodic() + if ctx.Err() != nil { + return } - } - savePeersProc := periodicproc.Tick(cfg.BackupBootstrapInterval, savePeersFn) - // When the main bootstrap process ends also terminate the 'save connected - // peers' ones. Coupling the two seems the easiest way to handle this backup - // process without additional complexity. - go func() { - <-bootstrapProc.Closing() - savePeersProc.Close() + for { + select { + case <-ticker.C: + periodic() + case <-ctx.Done(): + return + } + } }() - - // Run the first round now (after the first bootstrap process has finished) - // as the SavePeersPeriod can be much longer than bootstrap. - savePeersProc.Go(savePeersFn) } func saveConnectedPeersAsTemporaryBootstrap(ctx context.Context, host host.Host, cfg BootstrapConfig) error { diff --git a/go.mod b/go.mod index 54cb060e8..3054db658 100644 --- a/go.mod +++ b/go.mod @@ -37,7 +37,6 @@ require ( github.com/ipld/go-car/v2 v2.14.2 github.com/ipld/go-codec-dagpb v1.6.0 github.com/ipld/go-ipld-prime v0.21.0 - github.com/jbenet/goprocess v0.1.4 github.com/libp2p/go-buffer-pool v0.1.0 github.com/libp2p/go-doh-resolver v0.4.0 github.com/libp2p/go-libp2p v0.37.0 @@ -118,6 +117,7 @@ require ( github.com/ipfs/go-verifcid v0.0.3 // indirect github.com/jackpal/go-nat-pmp v1.0.2 // indirect github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect + github.com/jbenet/goprocess v0.1.4 // indirect github.com/klauspost/compress v1.17.11 // indirect github.com/klauspost/cpuid/v2 v2.2.8 // indirect github.com/koron/go-ssdp v0.0.4 // indirect diff --git a/go.sum b/go.sum index 303230175..57bb266c6 100644 --- a/go.sum +++ b/go.sum @@ -233,7 +233,6 @@ github.com/ipld/go-ipld-prime/storage/bsadapter v0.0.0-20230102063945-1a409dc236 github.com/ipld/go-ipld-prime/storage/bsadapter v0.0.0-20230102063945-1a409dc236dd/go.mod h1:wZ8hH8UxeryOs4kJEJaiui/s00hDSbE37OKsL47g+Sw= github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= -github.com/jbenet/go-cienv v0.1.0 h1:Vc/s0QbQtoxX8MwwSLWWh+xNNZvM3Lw7NsTcHrvvhMc= github.com/jbenet/go-cienv v0.1.0/go.mod h1:TqNnHUmJgXau0nCzC7kXWeotg3J9W34CUv5Djy1+FlA= github.com/jbenet/go-temp-err-catcher v0.1.0 h1:zpb3ZH6wIE8Shj2sKS+khgRvf7T7RABoLk/+KKHggpk= github.com/jbenet/go-temp-err-catcher v0.1.0/go.mod h1:0kJRvmDZXNMIiJirNPEYfhpPwbGVtZVWC34vc5WLsDk= diff --git a/namesys/republisher/repub.go b/namesys/republisher/repub.go index 7ca2ae932..95e440436 100644 --- a/namesys/republisher/repub.go +++ b/namesys/republisher/repub.go @@ -16,8 +16,6 @@ import ( "github.com/ipfs/boxo/ipns" ds "github.com/ipfs/go-datastore" logging "github.com/ipfs/go-log/v2" - "github.com/jbenet/goprocess" - gpctx "github.com/jbenet/goprocess/context" ic "github.com/libp2p/go-libp2p/core/crypto" "github.com/libp2p/go-libp2p/core/peer" ) @@ -67,8 +65,17 @@ func NewRepublisher(ns namesys.Publisher, ds ds.Datastore, self ic.PrivKey, ks k } } -// Run starts the republisher facility. It can be stopped by stopping the provided proc. -func (rp *Republisher) Run(proc goprocess.Process) { +// Run starts the republisher facility. It can be stopped by calling the returned function.. +func (rp *Republisher) Run() func() { + ctx, cancel := context.WithCancel(context.Background()) + go rp.run(ctx) + return func() { + log.Debug("stopping republisher") + cancel() + } +} + +func (rp *Republisher) run(ctx context.Context) { timer := time.NewTimer(InitialRebroadcastDelay) defer timer.Stop() if rp.Interval < InitialRebroadcastDelay { @@ -79,21 +86,21 @@ func (rp *Republisher) Run(proc goprocess.Process) { select { case <-timer.C: timer.Reset(rp.Interval) - err := rp.republishEntries(proc) + err := rp.republishEntries(ctx) if err != nil { log.Info("republisher failed to republish: ", err) if FailureRetryInterval < rp.Interval { timer.Reset(FailureRetryInterval) } } - case <-proc.Closing(): + case <-ctx.Done(): return } } } -func (rp *Republisher) republishEntries(p goprocess.Process) error { - ctx, cancel := context.WithCancel(gpctx.OnClosingContext(p)) +func (rp *Republisher) republishEntries(ctx context.Context) error { + ctx, cancel := context.WithCancel(ctx) defer cancel() ctx, span := startSpan(ctx, "Republisher.RepublishEntries") defer span.End() diff --git a/namesys/republisher/repub_test.go b/namesys/republisher/repub_test.go index 88ec04dae..db42b02a9 100644 --- a/namesys/republisher/repub_test.go +++ b/namesys/republisher/repub_test.go @@ -6,7 +6,6 @@ import ( "testing" "time" - "github.com/jbenet/goprocess" "github.com/libp2p/go-libp2p" dht "github.com/libp2p/go-libp2p-kad-dht" ic "github.com/libp2p/go-libp2p/core/crypto" @@ -125,8 +124,8 @@ func TestRepublish(t *testing.T) { repub.Interval = time.Second repub.RecordLifetime = time.Second * 5 - proc := goprocess.Go(repub.Run) - defer proc.Close() + stop := repub.Run() + defer stop() // now wait a couple seconds for it to fire time.Sleep(time.Second * 2) @@ -182,8 +181,8 @@ func TestLongEOLRepublish(t *testing.T) { repub.Interval = time.Millisecond * 500 repub.RecordLifetime = time.Second - proc := goprocess.Go(repub.Run) - defer proc.Close() + stop := repub.Run() + defer stop() // now wait a couple seconds for it to fire a few times time.Sleep(time.Second * 2) From 4a077bcb32c2a7be901576cf226ab670d5a3863e Mon Sep 17 00:00:00 2001 From: gammazero <11790789+gammazero@users.noreply.github.com> Date: Wed, 30 Oct 2024 11:20:43 -1000 Subject: [PATCH 02/36] client Close needs to return error --- bitswap/client/client.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bitswap/client/client.go b/bitswap/client/client.go index b4bc91b58..bab03c3cd 100644 --- a/bitswap/client/client.go +++ b/bitswap/client/client.go @@ -452,13 +452,14 @@ func (bs *Client) ReceiveError(err error) { } // Close is called to shutdown the Client -func (bs *Client) Close() { +func (bs *Client) Close() error { bs.closeOnce.Do(func() { close(bs.closing) bs.sm.Shutdown() bs.cancel() bs.notif.Shutdown() }) + return nil } // GetWantlist returns the current local wantlist (both want-blocks and From 19d1d6b9df169d38489da37aaefb78e269607ff4 Mon Sep 17 00:00:00 2001 From: gammazero <11790789+gammazero@users.noreply.github.com> Date: Tue, 5 Nov 2024 09:51:11 -1000 Subject: [PATCH 03/36] Simplify provide woker goroutines --- bitswap/server/server.go | 72 +++++++++++++++------------------------- 1 file changed, 26 insertions(+), 46 deletions(-) diff --git a/bitswap/server/server.go b/bitswap/server/server.go index 2b45b324b..2cc3e2474 100644 --- a/bitswap/server/server.go +++ b/bitswap/server/server.go @@ -299,12 +299,7 @@ func (bs *Server) startWorkers(ctx context.Context) { if bs.provideEnabled { bs.waitWorkers.Add(1) go bs.provideCollector(ctx) - - // Spawn up multiple workers to handle incoming blocks - // consider increasing number if providing blocks bottlenecks - // file transfers - bs.waitWorkers.Add(1) - go bs.provideWorker(ctx) + bs.startProvideWorkers(ctx) } } @@ -501,48 +496,33 @@ func (bs *Server) provideCollector(ctx context.Context) { } } -func (bs *Server) provideWorker(ctx context.Context) { - limit := make(chan struct{}, provideWorkerMax) - defer func() { - // Wait until all limitGoProvide goroutines are done before declaring - // this worker as done. - for i := 0; i < provideWorkerMax; i++ { - limit <- struct{}{} - } - bs.waitWorkers.Done() - }() - - limitedGoProvide := func(k cid.Cid, wid int) { - defer func() { - // replace token when done - <-limit - }() - - log.Debugw("Bitswap.ProvideWorker.Start", "ID", wid, "cid", k) - defer log.Debugw("Bitswap.ProvideWorker.End", "ID", wid, "cid", k) - - ctx, cancel := context.WithTimeout(ctx, defaults.ProvideTimeout) // timeout ctx - defer cancel() - - if err := bs.network.Provide(ctx, k); err != nil { - log.Warn(err) - } - } +// startProvideWorkers starts provide worker goroutines that provide CID +// supplied by provideCollector. +// +// If providing blocks bottlenecks file transfers then consider increasing +// provideWorkerMax, +func (bs *Server) startProvideWorkers(ctx context.Context) { + bs.waitWorkers.Add(provideWorkerMax) + for id := 0; id < provideWorkerMax; id++ { + go func(wid int) { + defer bs.waitWorkers.Done() + + var runCount int + // Read bs.proviudeKeys until closed, when provideCollector exits. + for k := range bs.provideKeys { + runCount++ + log.Debugw("Bitswap provider worker start", "ID", wid, "run", runCount, "cid", k) + + ctx, cancel := context.WithTimeout(ctx, defaults.ProvideTimeout) + if err := bs.network.Provide(ctx, k); err != nil { + log.Warn(err) + } + cancel() - // worker spawner, reads from bs.provideKeys until it closes, spawning a - // _ratelimited_ number of workers to handle each key. - wid := 2 - for k := range bs.provideKeys { - log.Debug("Bitswap.ProvideWorker.Loop") - select { - case limit <- struct{}{}: - go limitedGoProvide(k, wid) - case <-ctx.Done(): - return - } - wid++ + log.Debugw("Bitswap provider worker done", "ID", wid, "run", runCount, "cid", k) + } + }(id) } - log.Debug("provideKeys channel closed") } func (bs *Server) ReceiveMessage(ctx context.Context, p peer.ID, incoming message.BitSwapMessage) { From 1bf14880fbc356983e99a0085968caa7592882dc Mon Sep 17 00:00:00 2001 From: Daniel Norman <1992255+2color@users.noreply.github.com> Date: Tue, 19 Nov 2024 01:07:11 +0100 Subject: [PATCH 04/36] feat(routing/http/server): expose prometheus metrics (#718) * feat: add instrumentation to delegated routing endpoint * feat: allow passing in custom registry * docs: update changelog * chore: go mod tidy * fix: create new registry unless set * chore: adjust histogram buckets * refactor: rename histogram --------- Co-authored-by: Daniel N <2color@users.noreply.github.com> Co-authored-by: Marcin Rataj --- CHANGELOG.md | 2 ++ examples/go.sum | 5 +++-- go.mod | 2 +- go.sum | 2 ++ routing/http/server/server.go | 37 ++++++++++++++++++++++++++++++----- 5 files changed, 40 insertions(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6c5c621d2..faca07433 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,8 @@ The following emojis are used to highlight certain changes: ### Added +- `routing/http/server`: added Prometheus instrumentation to http delegated routing endpoints. + ### Changed - No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. diff --git a/examples/go.sum b/examples/go.sum index 323084c69..8d7be4a96 100644 --- a/examples/go.sum +++ b/examples/go.sum @@ -97,9 +97,8 @@ github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXP github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= -github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -467,6 +466,8 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537/go.mod h1:QJTqeLYEDaXHZDBsXlPCDqdhQuJkuw4NOtaxYe3xii4= github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5kWdCj2z2KEozexVbfEZIWiTjhE0+UjmZgPqehw= github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= +github.com/slok/go-http-metrics v0.12.0 h1:mAb7hrX4gB4ItU6NkFoKYdBslafg3o60/HbGBRsKaG8= +github.com/slok/go-http-metrics v0.12.0/go.mod h1:Ee/mdT9BYvGrlGzlClkK05pP2hRHmVbRF9dtUVS8LNA= github.com/smartystreets/assertions v1.2.0 h1:42S6lae5dvLc7BrLu/0ugRtcFVjoJNMC/N3yZFZkDFs= github.com/smartystreets/assertions v1.2.0/go.mod h1:tcbTF8ujkAEcZ8TElKY+i30BzYlVhC/LOxJk7iOWnoo= github.com/smartystreets/goconvey v1.7.2 h1:9RBaZCeXEQ3UselpuwUQHltGVXvdwm6cv1hgR6gDIPg= diff --git a/go.mod b/go.mod index 3054db658..db188a203 100644 --- a/go.mod +++ b/go.mod @@ -57,6 +57,7 @@ require ( github.com/polydawn/refmt v0.89.0 github.com/prometheus/client_golang v1.20.5 github.com/samber/lo v1.47.0 + github.com/slok/go-http-metrics v0.12.0 github.com/spaolacci/murmur3 v1.1.0 github.com/stretchr/testify v1.9.0 github.com/whyrusleeping/base32 v0.0.0-20170828182744-c30ac30633cc @@ -96,7 +97,6 @@ require ( github.com/go-logr/stdr v1.2.2 // indirect github.com/go-task/slim-sprig/v3 v3.0.0 // indirect github.com/godbus/dbus/v5 v5.1.0 // indirect - github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/google/gopacket v1.1.19 // indirect github.com/google/pprof v0.0.0-20241017200806-017d972448fc // indirect github.com/gopherjs/gopherjs v0.0.0-20190430165422-3e4dfb77656c // indirect diff --git a/go.sum b/go.sum index 57bb266c6..f3c5da72b 100644 --- a/go.sum +++ b/go.sum @@ -467,6 +467,8 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537/go.mod h1:QJTqeLYEDaXHZDBsXlPCDqdhQuJkuw4NOtaxYe3xii4= github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5kWdCj2z2KEozexVbfEZIWiTjhE0+UjmZgPqehw= github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= +github.com/slok/go-http-metrics v0.12.0 h1:mAb7hrX4gB4ItU6NkFoKYdBslafg3o60/HbGBRsKaG8= +github.com/slok/go-http-metrics v0.12.0/go.mod h1:Ee/mdT9BYvGrlGzlClkK05pP2hRHmVbRF9dtUVS8LNA= github.com/smartystreets/assertions v1.2.0 h1:42S6lae5dvLc7BrLu/0ugRtcFVjoJNMC/N3yZFZkDFs= github.com/smartystreets/assertions v1.2.0/go.mod h1:tcbTF8ujkAEcZ8TElKY+i30BzYlVhC/LOxJk7iOWnoo= github.com/smartystreets/goconvey v1.7.2 h1:9RBaZCeXEQ3UselpuwUQHltGVXvdwm6cv1hgR6gDIPg= diff --git a/routing/http/server/server.go b/routing/http/server/server.go index a7f9385b6..6177da125 100644 --- a/routing/http/server/server.go +++ b/routing/http/server/server.go @@ -25,8 +25,12 @@ import ( "github.com/libp2p/go-libp2p/core/routing" "github.com/multiformats/go-multiaddr" "github.com/multiformats/go-multibase" + "github.com/prometheus/client_golang/prometheus" logging "github.com/ipfs/go-log/v2" + metrics "github.com/slok/go-http-metrics/metrics/prometheus" + "github.com/slok/go-http-metrics/middleware" + middlewarestd "github.com/slok/go-http-metrics/middleware/std" ) const ( @@ -122,6 +126,12 @@ func WithStreamingRecordsLimit(limit int) Option { } } +func WithPrometheusRegistry(reg prometheus.Registerer) Option { + return func(s *server) { + s.promRegistry = reg + } +} + func Handler(svc ContentRouter, opts ...Option) http.Handler { server := &server{ svc: svc, @@ -133,12 +143,28 @@ func Handler(svc ContentRouter, opts ...Option) http.Handler { opt(server) } + if server.promRegistry == nil { + server.promRegistry = prometheus.NewRegistry() + } + + // Create middleware with prometheus recorder + mdlw := middleware.New(middleware.Config{ + Recorder: metrics.NewRecorder(metrics.Config{ + Registry: server.promRegistry, + Prefix: "delegated_routing_server", + + DurationBuckets: []float64{0.1, 0.5, 1, 2, 5, 8, 10, 20, 30}, + }), + }) + r := mux.NewRouter() - r.HandleFunc(findProvidersPath, server.findProviders).Methods(http.MethodGet) - r.HandleFunc(providePath, server.provide).Methods(http.MethodPut) - r.HandleFunc(findPeersPath, server.findPeers).Methods(http.MethodGet) - r.HandleFunc(GetIPNSPath, server.GetIPNS).Methods(http.MethodGet) - r.HandleFunc(GetIPNSPath, server.PutIPNS).Methods(http.MethodPut) + // Wrap each handler with the metrics middleware + r.Handle(findProvidersPath, middlewarestd.Handler(findProvidersPath, mdlw, http.HandlerFunc(server.findProviders))).Methods(http.MethodGet) + r.Handle(providePath, middlewarestd.Handler(providePath, mdlw, http.HandlerFunc(server.provide))).Methods(http.MethodPut) + r.Handle(findPeersPath, middlewarestd.Handler(findPeersPath, mdlw, http.HandlerFunc(server.findPeers))).Methods(http.MethodGet) + r.Handle(GetIPNSPath, middlewarestd.Handler(GetIPNSPath, mdlw, http.HandlerFunc(server.GetIPNS))).Methods(http.MethodGet) + r.Handle(GetIPNSPath, middlewarestd.Handler(GetIPNSPath, mdlw, http.HandlerFunc(server.PutIPNS))).Methods(http.MethodPut) + return r } @@ -147,6 +173,7 @@ type server struct { disableNDJSON bool recordsLimit int streamingRecordsLimit int + promRegistry prometheus.Registerer } func (s *server) detectResponseType(r *http.Request) (string, error) { From e38f236348d62c9c75494c563975e3eccc745b10 Mon Sep 17 00:00:00 2001 From: Daniel Norman <1992255+2color@users.noreply.github.com> Date: Tue, 19 Nov 2024 01:30:55 +0100 Subject: [PATCH 05/36] feat(routing/http/server): add routing timeout (#720) * feat: add routing timeouts to delegated server * chore: update changelog * docs: CHANGELOG.md --------- Co-authored-by: Daniel N <2color@users.noreply.github.com> Co-authored-by: Marcin Rataj --- CHANGELOG.md | 1 + routing/http/server/server.go | 30 ++++++++++++++++++++++++++---- 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index faca07433..8ed1fb078 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,6 +17,7 @@ The following emojis are used to highlight certain changes: ### Added - `routing/http/server`: added Prometheus instrumentation to http delegated routing endpoints. +- `routing/http/server`: added configurable routing timeout (`DefaultRoutingTimeout` being 30s) to prevent indefinite hangs during content/peer routing. Set custom duration via `WithRoutingTimeout`. ### Changed diff --git a/routing/http/server/server.go b/routing/http/server/server.go index 6177da125..f6d0f1993 100644 --- a/routing/http/server/server.go +++ b/routing/http/server/server.go @@ -41,6 +41,7 @@ const ( DefaultRecordsLimit = 20 DefaultStreamingRecordsLimit = 0 + DefaultRoutingTimeout = 30 * time.Second ) var logger = logging.Logger("routing/http/server") @@ -132,11 +133,18 @@ func WithPrometheusRegistry(reg prometheus.Registerer) Option { } } +func WithRoutingTimeout(timeout time.Duration) Option { + return func(s *server) { + s.routingTimeout = timeout + } +} + func Handler(svc ContentRouter, opts ...Option) http.Handler { server := &server{ svc: svc, recordsLimit: DefaultRecordsLimit, streamingRecordsLimit: DefaultStreamingRecordsLimit, + routingTimeout: DefaultRoutingTimeout, } for _, opt := range opts { @@ -174,6 +182,7 @@ type server struct { recordsLimit int streamingRecordsLimit int promRegistry prometheus.Registerer + routingTimeout time.Duration } func (s *server) detectResponseType(r *http.Request) (string, error) { @@ -246,7 +255,10 @@ func (s *server) findProviders(w http.ResponseWriter, httpReq *http.Request) { recordsLimit = s.recordsLimit } - provIter, err := s.svc.FindProviders(httpReq.Context(), cid, recordsLimit) + ctx, cancel := context.WithTimeout(httpReq.Context(), s.routingTimeout) + defer cancel() + + provIter, err := s.svc.FindProviders(ctx, cid, recordsLimit) if err != nil { if errors.Is(err, routing.ErrNotFound) { // handlerFunc takes care of setting the 404 and necessary headers @@ -335,7 +347,11 @@ func (s *server) findPeers(w http.ResponseWriter, r *http.Request) { recordsLimit = s.recordsLimit } - provIter, err := s.svc.FindPeers(r.Context(), pid, recordsLimit) + // Add timeout to the routing operation + ctx, cancel := context.WithTimeout(r.Context(), s.routingTimeout) + defer cancel() + + provIter, err := s.svc.FindPeers(ctx, pid, recordsLimit) if err != nil { if errors.Is(err, routing.ErrNotFound) { // handlerFunc takes care of setting the 404 and necessary headers @@ -466,7 +482,10 @@ func (s *server) GetIPNS(w http.ResponseWriter, r *http.Request) { return } - record, err := s.svc.GetIPNS(r.Context(), name) + ctx, cancel := context.WithTimeout(r.Context(), s.routingTimeout) + defer cancel() + + record, err := s.svc.GetIPNS(ctx, name) if err != nil { if errors.Is(err, routing.ErrNotFound) { writeErr(w, "GetIPNS", http.StatusNotFound, fmt.Errorf("delegate error: %w", err)) @@ -550,7 +569,10 @@ func (s *server) PutIPNS(w http.ResponseWriter, r *http.Request) { return } - err = s.svc.PutIPNS(r.Context(), name, record) + ctx, cancel := context.WithTimeout(r.Context(), s.routingTimeout) + defer cancel() + + err = s.svc.PutIPNS(ctx, name, record) if err != nil { writeErr(w, "PutIPNS", http.StatusInternalServerError, fmt.Errorf("delegate error: %w", err)) return From 6043002188bc5a71b9a75f387d238428ece5f08b Mon Sep 17 00:00:00 2001 From: Daniel N <2color@users.noreply.github.com> Date: Tue, 19 Nov 2024 10:08:33 +0100 Subject: [PATCH 06/36] fix: use more realistic size buckets for metrics --- routing/http/server/server.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/routing/http/server/server.go b/routing/http/server/server.go index f6d0f1993..fd964cbab 100644 --- a/routing/http/server/server.go +++ b/routing/http/server/server.go @@ -158,9 +158,9 @@ func Handler(svc ContentRouter, opts ...Option) http.Handler { // Create middleware with prometheus recorder mdlw := middleware.New(middleware.Config{ Recorder: metrics.NewRecorder(metrics.Config{ - Registry: server.promRegistry, - Prefix: "delegated_routing_server", - + Registry: server.promRegistry, + Prefix: "delegated_routing_server", + SizeBuckets: prometheus.ExponentialBuckets(100, 4, 8), // [100 400 1600 6400 25600 102400 409600 1.6384e+06] DurationBuckets: []float64{0.1, 0.5, 1, 2, 5, 8, 10, 20, 30}, }), }) From 139b7a58476260a8027735c7d7b19e298563c2d7 Mon Sep 17 00:00:00 2001 From: Daniel N <2color@users.noreply.github.com> Date: Tue, 19 Nov 2024 10:13:23 +0100 Subject: [PATCH 07/36] chore: update changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8ed1fb078..7751ee6a3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,6 +22,7 @@ The following emojis are used to highlight certain changes: ### Changed - No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. +- `routing/http/server`: changed default Prometheus buckets for response size and duration to match real world data. ### Removed From cfe68d89a6af313b4e3d72ad0443e616fee85841 Mon Sep 17 00:00:00 2001 From: Daniel Norman <1992255+2color@users.noreply.github.com> Date: Tue, 19 Nov 2024 16:17:09 +0100 Subject: [PATCH 08/36] docs: fix url of tracing env vars (#719) --- docs/tracing.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tracing.md b/docs/tracing.md index 868b68d95..dae37024f 100644 --- a/docs/tracing.md +++ b/docs/tracing.md @@ -134,6 +134,6 @@ the complete trace of this request. [Open Telemetry]: https://opentelemetry.io/ [opentelemetry-go]: https://github.com/open-telemetry/opentelemetry-go [Trace Context]: https://www.w3.org/TR/trace-context -[OpenTelemetry Environment Variable Specification]: https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/sdk-environment-variables.md +[OpenTelemetry Environment Variable Specification]: https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/configuration/sdk-environment-variables.md [OpenTelemetry Protocol Exporter]: https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/protocol/exporter.md [Jaeger UI]: https://github.com/jaegertracing/jaeger-ui From 5929aca3037f59a8a59045c5c53c22ca600bada0 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Tue, 19 Nov 2024 05:32:47 -1000 Subject: [PATCH 09/36] fix(bitswap/server): pass context to server engine to register metrics (#723) pass context to server engine to register metrics --- bitswap/server/internal/decision/engine.go | 3 ++- bitswap/server/internal/decision/engine_test.go | 2 +- bitswap/server/server.go | 1 + 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/bitswap/server/internal/decision/engine.go b/bitswap/server/internal/decision/engine.go index a46d67dd7..9b929b99c 100644 --- a/bitswap/server/internal/decision/engine.go +++ b/bitswap/server/internal/decision/engine.go @@ -378,12 +378,13 @@ func wrapTaskComparator(tc TaskComparator) peertask.QueueTaskComparator { // maxOutstandingBytesPerPeer hints to the peer task queue not to give a peer // more tasks if it has some maximum work already outstanding. func NewEngine( + ctx context.Context, bs bstore.Blockstore, peerTagger PeerTagger, self peer.ID, opts ...Option, ) *Engine { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancel(ctx) e := &Engine{ scoreLedger: NewDefaultScoreLedger(), diff --git a/bitswap/server/internal/decision/engine_test.go b/bitswap/server/internal/decision/engine_test.go index d549e1218..cab5b5691 100644 --- a/bitswap/server/internal/decision/engine_test.go +++ b/bitswap/server/internal/decision/engine_test.go @@ -188,7 +188,7 @@ func newEngineForTesting( opts ...Option, ) *Engine { opts = append(opts, WithWantHaveReplaceSize(wantHaveReplaceSize)) - return NewEngine(bs, peerTagger, self, opts...) + return NewEngine(context.Background(), bs, peerTagger, self, opts...) } func TestOutboxClosedWhenEngineClosed(t *testing.T) { diff --git a/bitswap/server/server.go b/bitswap/server/server.go index 2cc3e2474..6416da034 100644 --- a/bitswap/server/server.go +++ b/bitswap/server/server.go @@ -99,6 +99,7 @@ func New(ctx context.Context, network bsnet.BitSwapNetwork, bstore blockstore.Bl } s.engine = decision.NewEngine( + ctx, bstore, network.ConnectionManager(), network.Self(), From b3631993f07e4476c423fcd06e3958fe4686634a Mon Sep 17 00:00:00 2001 From: Marcin Rataj Date: Tue, 19 Nov 2024 17:25:51 +0100 Subject: [PATCH 10/36] docs: update CHANGELOG.md --- CHANGELOG.md | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7751ee6a3..6d32d0b6d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,13 +16,12 @@ The following emojis are used to highlight certain changes: ### Added -- `routing/http/server`: added Prometheus instrumentation to http delegated routing endpoints. -- `routing/http/server`: added configurable routing timeout (`DefaultRoutingTimeout` being 30s) to prevent indefinite hangs during content/peer routing. Set custom duration via `WithRoutingTimeout`. +- `routing/http/server`: added built-in Prometheus instrumentation to http delegated `/routing/v1/` endpoints, with custom buckets for response size and duration to match real world data observed at [the `delegated-ipfs.dev` instance](https://docs.ipfs.tech/concepts/public-utilities/#delegated-routing). [#718](https://github.com/ipfs/boxo/pull/718) [#724](https://github.com/ipfs/boxo/pull/724) +- `routing/http/server`: added configurable routing timeout (`DefaultRoutingTimeout` being 30s) to prevent indefinite hangs during content/peer routing. Set custom duration via `WithRoutingTimeout`. [#720](https://github.com/ipfs/boxo/pull/720) ### Changed -- No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. -- `routing/http/server`: changed default Prometheus buckets for response size and duration to match real world data. +- No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. [#710](https://github.com/ipfs/boxo/pull/710) ### Removed From 13d0b3255920dfca2e61173abaa05e8b6a96dfaf Mon Sep 17 00:00:00 2001 From: Adin Schmahmann Date: Tue, 19 Nov 2024 12:46:35 -0500 Subject: [PATCH 11/36] fix(bitswap/client/providerquerymanager): use non-timed out context for tracing (#721) --- .../internal/providerquerymanager/providerquerymanager.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bitswap/client/internal/providerquerymanager/providerquerymanager.go b/bitswap/client/internal/providerquerymanager/providerquerymanager.go index ab4273a82..e57726ddc 100644 --- a/bitswap/client/internal/providerquerymanager/providerquerymanager.go +++ b/bitswap/client/internal/providerquerymanager/providerquerymanager.go @@ -261,7 +261,7 @@ func (pqm *ProviderQueryManager) findProviderWorker() { span.AddEvent("ConnectedToProvider", trace.WithAttributes(attribute.Stringer("peer", p))) select { case pqm.providerQueryMessages <- &receivedProviderMessage{ - ctx: findProviderCtx, + ctx: fpr.ctx, k: k, p: p, }: @@ -274,7 +274,7 @@ func (pqm *ProviderQueryManager) findProviderWorker() { cancel() select { case pqm.providerQueryMessages <- &finishedProviderQueryMessage{ - ctx: findProviderCtx, + ctx: fpr.ctx, k: k, }: case <-pqm.ctx.Done(): From c91cc1d5a0552c653936bd5051b6c0972d57743c Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Thu, 21 Nov 2024 09:51:32 -1000 Subject: [PATCH 12/36] =?UTF-8?q?fix(bitswap/client/providerquerymanager):?= =?UTF-8?q?=20don't=20end=20trace=20span=20until=20=E2=80=A6=20(#725)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit fix(bitswap/client/providerquerymanager): don't end trace span until all providers are returned --- .../internal/providerquerymanager/providerquerymanager.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/bitswap/client/internal/providerquerymanager/providerquerymanager.go b/bitswap/client/internal/providerquerymanager/providerquerymanager.go index e57726ddc..c85efe737 100644 --- a/bitswap/client/internal/providerquerymanager/providerquerymanager.go +++ b/bitswap/client/internal/providerquerymanager/providerquerymanager.go @@ -126,7 +126,6 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, var span trace.Span sessionCtx, span = internal.StartSpan(sessionCtx, "ProviderQueryManager.FindProvidersAsync", trace.WithAttributes(attribute.Stringer("cid", k))) - defer span.End() select { case pqm.providerQueryMessages <- &newProvideQueryMessage{ @@ -137,6 +136,7 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, case <-pqm.ctx.Done(): ch := make(chan peer.ID) close(ch) + span.End() return ch case <-sessionCtx.Done(): ch := make(chan peer.ID) @@ -152,14 +152,15 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, case <-pqm.ctx.Done(): ch := make(chan peer.ID) close(ch) + span.End() return ch case receivedInProgressRequest = <-inProgressRequestChan: } - return pqm.receiveProviders(sessionCtx, k, receivedInProgressRequest) + return pqm.receiveProviders(sessionCtx, k, receivedInProgressRequest, func() { span.End() }) } -func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k cid.Cid, receivedInProgressRequest inProgressRequest) <-chan peer.ID { +func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k cid.Cid, receivedInProgressRequest inProgressRequest, onCloseFn func()) <-chan peer.ID { // maintains an unbuffered queue for incoming providers for given request for a given session // essentially, as a provider comes in, for a given CID, we want to immediately broadcast to all // sessions that queried that CID, without worrying about whether the client code is actually @@ -171,6 +172,7 @@ func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k go func() { defer close(returnedProviders) + defer onCloseFn() outgoingProviders := func() chan<- peer.ID { if len(receivedProviders) == 0 { return nil From 37756ce2eeb14d446222739107c421f266e5af78 Mon Sep 17 00:00:00 2001 From: Adin Schmahmann Date: Mon, 25 Nov 2024 16:09:08 -0500 Subject: [PATCH 13/36] Bitswap: move providing -> Exchange-layer, providerQueryManager -> routing (#641) Bitswap: refactor content provider elements This PR performs a rather large and touchy refactor of things related to Content providing and Content discovery previously embedded into Bitswap. The motivations: * Make ProviderQueryManager options configurable * Align and separate coalesced layers: content routing must not be part of bitswap as in the future we will be using different exchanges (bitswap, http) for retrieval and content routing should be above exchange. * Align content routing interfaces with libp2p: to avoid crust, wrappers and user confusion, align Providers and Discovery types to libp2p.ContentRouting. * Reduce duplicated functionality: i.e. code that handles providing in multiple places and fails to take advantage of ProvideMany optimizations. As a result: * ProviderQueryManager is now part of the routing module * A new providing.Exchange has been created * Bitswap initialization params have changed and Bitswap Network doesn't provide anymore (see changelog for more details) Co-authored-by: Hector Sanjuan Co-authored-by: Andrew Gillis <11790789+gammazero@users.noreply.github.com> --- CHANGELOG.md | 47 +++++ bitswap/benchmarks_test.go | 43 ++--- bitswap/bitswap.go | 16 +- bitswap/bitswap_test.go | 102 +++++----- bitswap/client/bitswap_with_sessions_test.go | 58 +++--- bitswap/client/client.go | 99 +++++++--- .../internal/messagequeue/messagequeue.go | 2 +- .../messagequeue/messagequeue_test.go | 2 +- bitswap/client/internal/session/session.go | 10 +- .../client/internal/session/session_test.go | 4 +- bitswap/internal/defaults/defaults.go | 5 - bitswap/network/interface.go | 6 +- bitswap/network/ipfs_impl.go | 44 +---- bitswap/network/ipfs_impl_test.go | 23 +-- bitswap/options.go | 12 +- bitswap/server/server.go | 138 +------------- bitswap/testinstance/testinstance.go | 35 ++-- bitswap/testnet/network_test.go | 3 +- bitswap/testnet/peernet.go | 12 +- bitswap/testnet/virtual.go | 46 +---- blockservice/test/mock.go | 9 +- examples/bitswap-transfer/main.go | 7 +- examples/go.mod | 2 +- exchange/providing/providing.go | 46 +++++ exchange/providing/providing_test.go | 74 ++++++++ fetcher/helpers/block_visitor_test.go | 18 +- fetcher/impl/blockservice/fetcher_test.go | 46 ++--- provider/noop.go | 2 +- provider/provider.go | 2 +- provider/reprovider.go | 2 +- provider/reprovider_test.go | 2 +- routing/mock/centralized_client.go | 3 +- routing/mock/centralized_server.go | 7 +- .../providerquerymanager.go | 178 +++++++++++++----- .../providerquerymanager_test.go | 170 ++++++++++------- 35 files changed, 725 insertions(+), 550 deletions(-) create mode 100644 exchange/providing/providing.go create mode 100644 exchange/providing/providing_test.go rename {bitswap/client/internal => routing}/providerquerymanager/providerquerymanager.go (69%) rename {bitswap/client/internal => routing}/providerquerymanager/providerquerymanager_test.go (69%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6d32d0b6d..84e8558b5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -14,6 +14,52 @@ The following emojis are used to highlight certain changes: ## [Unreleased] +- `bitswap`, `routing`, `exchange` ([#641](https://github.com/ipfs/boxo/pull/641)): + - ✨ Bitswap is no longer in charge of providing blocks to the newtork: providing functionality is now handled by a `exchange/providing.Exchange`, meant to be used with `provider.System` so that all provides follow the same rules (multiple parts of the code where handling provides) before. + - 🛠 `bitswap/client/internal/providerquerymanager` has been moved to `routing/providerquerymanager` where it belongs. In order to keep compatibility, Bitswap now receives a `routing.ContentDiscovery` parameter which implements `FindProvidersAsync(...)` and uses it to create a `providerquerymanager` with the default settings as before. Custom settings can be used by using a custom `providerquerymanager` to manually wrap a `ContentDiscovery` object and pass that in as `ContentDiscovery` on initialization while setting `bitswap.WithDefaultProviderQueryManager(false)` (to avoid re-wrapping it again). + - The renovated `providedQueryManager` will trigger lookups until it manages to connect to `MaxProviders`. Before it would lookup at most `MaxInProcessRequests*MaxProviders` and connection failures may have limited the actual number of providers found. + - 🛠 We have aligned our routing-related interfaces with the libp2p [`routing`](https://pkg.go.dev/github.com/libp2p/go-libp2p/core/routing#ContentRouting) ones, including in the `reprovider.System`. + - In order to obtain exactly the same behaviour as before (i.e. particularly ensuring that new blocks are still provided), what was done like: + +```go + bswapnet := network.NewFromIpfsHost(host, contentRouter) + bswap := bitswap.New(p.ctx, bswapnet, blockstore) + bserv = blockservice.New(blockstore, bswap) +``` + - becomes: + +```go + // Create network: no contentRouter anymore + bswapnet := network.NewFromIpfsHost(host) + // Create Bitswap: a new "discovery" parameter, usually the "contentRouter" + // which does both discovery and providing. + bswap := bitswap.New(p.ctx, bswapnet, discovery, blockstore) + // A provider system that handles concurrent provides etc. "contentProvider" + // is usually the "contentRouter" which does both discovery and providing. + // "contentProvider" could be used directly without wrapping, but it is recommended + // to do so to provide more efficiently. + provider := provider.New(datastore, provider.Online(contentProvider) + // A wrapped providing exchange using the previous exchange and the provider. + exch := providing.New(bswap, provider) + + // Finally the blockservice + bserv := blockservice.New(blockstore, exch) + ... +``` + + - The above is only necessary if content routing is needed. Otherwise: + +``` + // Create network: no contentRouter anymore + bswapnet := network.NewFromIpfsHost(host) + // Create Bitswap: a new "discovery" parameter set to nil (disable content discovery) + bswap := bitswap.New(p.ctx, bswapnet, nil, blockstore) + // Finally the blockservice + bserv := blockservice.New(blockstore, exch) +``` + + + ### Added - `routing/http/server`: added built-in Prometheus instrumentation to http delegated `/routing/v1/` endpoints, with custom buckets for response size and duration to match real world data observed at [the `delegated-ipfs.dev` instance](https://docs.ipfs.tech/concepts/public-utilities/#delegated-routing). [#718](https://github.com/ipfs/boxo/pull/718) [#724](https://github.com/ipfs/boxo/pull/724) @@ -117,6 +163,7 @@ The following emojis are used to highlight certain changes: - `bitswap/client` fix memory leak in BlockPresenceManager due to unlimited map growth. [#636](https://github.com/ipfs/boxo/pull/636) - `bitswap/network` fixed race condition when a timeout occurred before hole punching completed while establishing a first-time stream to a peer behind a NAT [#651](https://github.com/ipfs/boxo/pull/651) - `bitswap`: wantlist overflow handling now cancels existing entries to make room for newer entries. This fix prevents the wantlist from filling up with CIDs that the server does not have. [#629](https://github.com/ipfs/boxo/pull/629) +- 🛠 `bitswap` & `bitswap/server` no longer provide to content routers, instead you can use the `provider` package because it uses a datastore queue and batches calls to ProvideMany. ## [v0.21.0] diff --git a/bitswap/benchmarks_test.go b/bitswap/benchmarks_test.go index bd8f342ea..d1930c900 100644 --- a/bitswap/benchmarks_test.go +++ b/bitswap/benchmarks_test.go @@ -12,17 +12,16 @@ import ( "testing" "time" - blocks "github.com/ipfs/go-block-format" - "github.com/ipfs/go-test/random" - protocol "github.com/libp2p/go-libp2p/core/protocol" - "github.com/ipfs/boxo/bitswap" bsnet "github.com/ipfs/boxo/bitswap/network" testinstance "github.com/ipfs/boxo/bitswap/testinstance" tn "github.com/ipfs/boxo/bitswap/testnet" mockrouting "github.com/ipfs/boxo/routing/mock" + blocks "github.com/ipfs/go-block-format" cid "github.com/ipfs/go-cid" delay "github.com/ipfs/go-ipfs-delay" + "github.com/ipfs/go-test/random" + protocol "github.com/libp2p/go-libp2p/core/protocol" ) type fetchFunc func(b *testing.B, bs *bitswap.Bitswap, ks []cid.Cid) @@ -135,6 +134,7 @@ func BenchmarkFetchFromOldBitswap(b *testing.B) { benchmarkLog = nil fixedDelay := delay.Fixed(10 * time.Millisecond) bstoreLatency := time.Duration(0) + router := mockrouting.NewServer() for _, bch := range mixedBenches { b.Run(bch.name, func(b *testing.B) { @@ -142,17 +142,17 @@ func BenchmarkFetchFromOldBitswap(b *testing.B) { oldSeedCount := bch.oldSeedCount newSeedCount := bch.nodeCount - (fetcherCount + oldSeedCount) - net := tn.VirtualNetwork(mockrouting.NewServer(), fixedDelay) + net := tn.VirtualNetwork(fixedDelay) // Simulate an older Bitswap node (old protocol ID) that doesn't // send DONT_HAVE responses oldProtocol := []protocol.ID{bsnet.ProtocolBitswapOneOne} oldNetOpts := []bsnet.NetOpt{bsnet.SupportedProtocols(oldProtocol)} oldBsOpts := []bitswap.Option{bitswap.SetSendDontHaves(false)} - oldNodeGenerator := testinstance.NewTestInstanceGenerator(net, oldNetOpts, oldBsOpts) + oldNodeGenerator := testinstance.NewTestInstanceGenerator(net, router, oldNetOpts, oldBsOpts) // Regular new Bitswap node - newNodeGenerator := testinstance.NewTestInstanceGenerator(net, nil, nil) + newNodeGenerator := testinstance.NewTestInstanceGenerator(net, router, nil, nil) var instances []testinstance.Instance // Create new nodes (fetchers + seeds) @@ -294,9 +294,10 @@ func BenchmarkDatacenterMultiLeechMultiSeed(b *testing.B) { numblks := 1000 for i := 0; i < b.N; i++ { - net := tn.RateLimitedVirtualNetwork(mockrouting.NewServer(), d, rateLimitGenerator) + net := tn.RateLimitedVirtualNetwork(d, rateLimitGenerator) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() instances := ig.Instances(numnodes) @@ -312,9 +313,9 @@ func BenchmarkDatacenterMultiLeechMultiSeed(b *testing.B) { func subtestDistributeAndFetch(b *testing.B, numnodes, numblks int, d delay.D, bstoreLatency time.Duration, df distFunc, ff fetchFunc) { for i := 0; i < b.N; i++ { - net := tn.VirtualNetwork(mockrouting.NewServer(), d) - - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(d) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) instances := ig.Instances(numnodes) rootBlock := random.BlocksOfSize(1, rootBlockSize) @@ -327,9 +328,9 @@ func subtestDistributeAndFetch(b *testing.B, numnodes, numblks int, d delay.D, b func subtestDistributeAndFetchRateLimited(b *testing.B, numnodes, numblks int, d delay.D, rateLimitGenerator tn.RateLimitGenerator, blockSize int64, bstoreLatency time.Duration, df distFunc, ff fetchFunc) { for i := 0; i < b.N; i++ { - net := tn.RateLimitedVirtualNetwork(mockrouting.NewServer(), d, rateLimitGenerator) - - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.RateLimitedVirtualNetwork(d, rateLimitGenerator) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() instances := ig.Instances(numnodes) @@ -437,7 +438,7 @@ func runDistribution(b *testing.B, instances []testinstance.Instance, blocks []b func allToAll(b *testing.B, provs []testinstance.Instance, blocks []blocks.Block) { for _, p := range provs { - if err := p.Blockstore().PutMany(context.Background(), blocks); err != nil { + if err := p.Blockstore.PutMany(context.Background(), blocks); err != nil { b.Fatal(err) } } @@ -452,10 +453,10 @@ func overlap1(b *testing.B, provs []testinstance.Instance, blks []blocks.Block) bill := provs[0] jeff := provs[1] - if err := bill.Blockstore().PutMany(context.Background(), blks[:75]); err != nil { + if err := bill.Blockstore.PutMany(context.Background(), blks[:75]); err != nil { b.Fatal(err) } - if err := jeff.Blockstore().PutMany(context.Background(), blks[25:]); err != nil { + if err := jeff.Blockstore.PutMany(context.Background(), blks[25:]); err != nil { b.Fatal(err) } } @@ -473,12 +474,12 @@ func overlap2(b *testing.B, provs []testinstance.Instance, blks []blocks.Block) even := i%2 == 0 third := i%3 == 0 if third || even { - if err := bill.Blockstore().Put(context.Background(), blk); err != nil { + if err := bill.Blockstore.Put(context.Background(), blk); err != nil { b.Fatal(err) } } if third || !even { - if err := jeff.Blockstore().Put(context.Background(), blk); err != nil { + if err := jeff.Blockstore.Put(context.Background(), blk); err != nil { b.Fatal(err) } } @@ -490,7 +491,7 @@ func overlap2(b *testing.B, provs []testinstance.Instance, blks []blocks.Block) // but we're mostly just testing performance of the sync algorithm func onePeerPerBlock(b *testing.B, provs []testinstance.Instance, blks []blocks.Block) { for _, blk := range blks { - err := provs[rand.Intn(len(provs))].Blockstore().Put(context.Background(), blk) + err := provs[rand.Intn(len(provs))].Blockstore.Put(context.Background(), blk) if err != nil { b.Fatal(err) } diff --git a/bitswap/bitswap.go b/bitswap/bitswap.go index ddc50f6dd..558eb12e0 100644 --- a/bitswap/bitswap.go +++ b/bitswap/bitswap.go @@ -5,7 +5,6 @@ import ( "fmt" "github.com/ipfs/boxo/bitswap/client" - "github.com/ipfs/boxo/bitswap/internal/defaults" "github.com/ipfs/boxo/bitswap/message" "github.com/ipfs/boxo/bitswap/network" "github.com/ipfs/boxo/bitswap/server" @@ -45,9 +44,8 @@ type bitswap interface { } var ( - _ exchange.SessionExchange = (*Bitswap)(nil) - _ bitswap = (*Bitswap)(nil) - HasBlockBufferSize = defaults.HasBlockBufferSize + _ exchange.SessionExchange = (*Bitswap)(nil) + _ bitswap = (*Bitswap)(nil) ) type Bitswap struct { @@ -58,7 +56,7 @@ type Bitswap struct { net network.BitSwapNetwork } -func New(ctx context.Context, net network.BitSwapNetwork, bstore blockstore.Blockstore, options ...Option) *Bitswap { +func New(ctx context.Context, net network.BitSwapNetwork, providerFinder client.ProviderFinder, bstore blockstore.Blockstore, options ...Option) *Bitswap { bs := &Bitswap{ net: net, } @@ -85,14 +83,10 @@ func New(ctx context.Context, net network.BitSwapNetwork, bstore blockstore.Bloc serverOptions = append(serverOptions, server.WithTracer(tracer)) } - if HasBlockBufferSize != defaults.HasBlockBufferSize { - serverOptions = append(serverOptions, server.HasBlockBufferSize(HasBlockBufferSize)) - } - ctx = metrics.CtxSubScope(ctx, "bitswap") bs.Server = server.New(ctx, net, bstore, serverOptions...) - bs.Client = client.New(ctx, net, bstore, append(clientOptions, client.WithBlockReceivedNotifier(bs.Server))...) + bs.Client = client.New(ctx, net, providerFinder, bstore, append(clientOptions, client.WithBlockReceivedNotifier(bs.Server))...) net.Start(bs) // use the polyfill receiver to log received errors and trace messages only once return bs @@ -115,7 +109,6 @@ type Stat struct { MessagesReceived uint64 BlocksSent uint64 DataSent uint64 - ProvideBufLen int } func (bs *Bitswap) Stat() (*Stat, error) { @@ -138,7 +131,6 @@ func (bs *Bitswap) Stat() (*Stat, error) { Peers: ss.Peers, BlocksSent: ss.BlocksSent, DataSent: ss.DataSent, - ProvideBufLen: ss.ProvideBufLen, }, nil } diff --git a/bitswap/bitswap_test.go b/bitswap/bitswap_test.go index 85055879c..2fb32aa61 100644 --- a/bitswap/bitswap_test.go +++ b/bitswap/bitswap_test.go @@ -36,7 +36,7 @@ func isCI() bool { func addBlock(t *testing.T, ctx context.Context, inst testinstance.Instance, blk blocks.Block) { t.Helper() - err := inst.Blockstore().Put(ctx, blk) + err := inst.Blockstore.Put(ctx, blk) if err != nil { t.Fatal(err) } @@ -51,8 +51,9 @@ func addBlock(t *testing.T, ctx context.Context, inst testinstance.Instance, blk const kNetworkDelay = 0 * time.Millisecond func TestClose(t *testing.T) { - vnet := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(vnet, nil, nil) + vnet := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) defer ig.Close() block := random.BlocksOfSize(1, blockSize)[0] bitswap := ig.Next() @@ -65,14 +66,14 @@ func TestClose(t *testing.T) { } func TestProviderForKeyButNetworkCannotFind(t *testing.T) { // TODO revisit this - rs := mockrouting.NewServer() - net := tn.VirtualNetwork(rs, delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() block := blocks.NewBlock([]byte("block")) pinfo := p2ptestutil.RandTestBogusIdentityOrFatal(t) - err := rs.Client(pinfo).Provide(context.Background(), block.Cid(), true) // but not on network + err := router.Client(pinfo).Provide(context.Background(), block.Cid(), true) // but not on network if err != nil { t.Fatal(err) } @@ -90,9 +91,10 @@ func TestProviderForKeyButNetworkCannotFind(t *testing.T) { // TODO revisit this } func TestGetBlockFromPeerAfterPeerAnnounces(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) block := blocks.NewBlock([]byte("block")) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() peers := ig.Instances(2) @@ -118,10 +120,11 @@ func TestGetBlockFromPeerAfterPeerAnnounces(t *testing.T) { } func TestDoesNotProvideWhenConfiguredNotTo(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) block := blocks.NewBlock([]byte("block")) - bsOpts := []bitswap.Option{bitswap.ProvideEnabled(false), bitswap.ProviderSearchDelay(50 * time.Millisecond)} - ig := testinstance.NewTestInstanceGenerator(net, nil, bsOpts) + bsOpts := []bitswap.Option{bitswap.ProviderSearchDelay(50 * time.Millisecond)} + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, bsOpts) defer ig.Close() hasBlock := ig.Next() @@ -150,12 +153,13 @@ func TestDoesNotProvideWhenConfiguredNotTo(t *testing.T) { // Tests that a received block is not stored in the blockstore if the block was // not requested by the client func TestUnwantedBlockNotAdded(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) block := blocks.NewBlock([]byte("block")) bsMessage := bsmsg.New(true) bsMessage.AddBlock(block) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() peers := ig.Instances(2) @@ -170,9 +174,9 @@ func TestUnwantedBlockNotAdded(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() - doesNotWantBlock.Exchange.ReceiveMessage(ctx, hasBlock.Peer, bsMessage) + doesNotWantBlock.Exchange.ReceiveMessage(ctx, hasBlock.Identity.ID(), bsMessage) - blockInStore, err := doesNotWantBlock.Blockstore().Has(ctx, block.Cid()) + blockInStore, err := doesNotWantBlock.Blockstore.Has(ctx, block.Cid()) if err != nil || blockInStore { t.Fatal("Unwanted block added to block store") } @@ -186,10 +190,11 @@ func TestUnwantedBlockNotAdded(t *testing.T) { // (because the live request queue is full) func TestPendingBlockAdded(t *testing.T) { ctx := context.Background() - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) sessionBroadcastWantCapacity := 4 - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() instance := ig.Instances(1)[0] @@ -277,8 +282,9 @@ func PerformDistributionTest(t *testing.T, numInstances, numBlocks int) { if testing.Short() { t.SkipNow() } - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, []bitswap.Option{ + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, []bitswap.Option{ bitswap.TaskWorkerCount(5), bitswap.EngineTaskWorkerCount(5), bitswap.MaxOutstandingBytesPerPeer(1 << 20), @@ -333,16 +339,17 @@ func TestSendToWantingPeer(t *testing.T) { t.SkipNow() } - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() peers := ig.Instances(2) peerA := peers[0] peerB := peers[1] - t.Logf("Session %v\n", peerA.Peer) - t.Logf("Session %v\n", peerB.Peer) + t.Logf("Session %v\n", peerA.Identity.ID()) + t.Logf("Session %v\n", peerB.Identity.ID()) waitTime := time.Second * 5 @@ -370,8 +377,9 @@ func TestSendToWantingPeer(t *testing.T) { } func TestEmptyKey(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() bs := ig.Instances(1)[0].Exchange @@ -403,8 +411,9 @@ func assertStat(t *testing.T, st *bitswap.Stat, sblks, rblks, sdata, rdata uint6 } func TestBasicBitswap(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() t.Log("Test a one node trying to get one block from another") @@ -428,7 +437,7 @@ func TestBasicBitswap(t *testing.T) { // When second peer receives block, it should send out a cancel, so third // peer should no longer keep second peer's want if err = tu.WaitFor(ctx, func() error { - if len(instances[2].Exchange.WantlistForPeer(instances[1].Peer)) != 0 { + if len(instances[2].Exchange.WantlistForPeer(instances[1].Identity.ID())) != 0 { return errors.New("should have no items in other peers wantlist") } if len(instances[1].Exchange.GetWantlist()) != 0 { @@ -474,8 +483,9 @@ func TestBasicBitswap(t *testing.T) { } func TestDoubleGet(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() t.Log("Test a one node trying to get one block from another") @@ -518,7 +528,7 @@ func TestDoubleGet(t *testing.T) { } t.Log(blk) case <-time.After(time.Second * 5): - p1wl := instances[0].Exchange.WantlistForPeer(instances[1].Peer) + p1wl := instances[0].Exchange.WantlistForPeer(instances[1].Identity.ID()) if len(p1wl) != 1 { t.Logf("wantlist view didnt have 1 item (had %d)", len(p1wl)) } else if !p1wl[0].Equals(blocks[0].Cid()) { @@ -538,8 +548,9 @@ func TestDoubleGet(t *testing.T) { } func TestWantlistCleanup(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() instances := ig.Instances(2) @@ -659,8 +670,9 @@ func newReceipt(sent, recv, exchanged uint64) *server.Receipt { } func TestBitswapLedgerOneWay(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() t.Log("Test ledgers match when one peer sends block to another") @@ -676,8 +688,8 @@ func TestBitswapLedgerOneWay(t *testing.T) { t.Fatal(err) } - ra := instances[0].Exchange.LedgerForPeer(instances[1].Peer) - rb := instances[1].Exchange.LedgerForPeer(instances[0].Peer) + ra := instances[0].Exchange.LedgerForPeer(instances[1].Identity.ID()) + rb := instances[1].Exchange.LedgerForPeer(instances[0].Identity.ID()) // compare peer ledger receipts err = assertLedgerMatch(ra, rb) @@ -707,8 +719,9 @@ func TestBitswapLedgerOneWay(t *testing.T) { } func TestBitswapLedgerTwoWay(t *testing.T) { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(net, router, nil, nil) defer ig.Close() t.Log("Test ledgers match when two peers send one block to each other") @@ -732,8 +745,8 @@ func TestBitswapLedgerTwoWay(t *testing.T) { t.Fatal(err) } - ra := instances[0].Exchange.LedgerForPeer(instances[1].Peer) - rb := instances[1].Exchange.LedgerForPeer(instances[0].Peer) + ra := instances[0].Exchange.LedgerForPeer(instances[1].Identity.ID()) + rb := instances[1].Exchange.LedgerForPeer(instances[0].Identity.ID()) // compare peer ledger receipts err = assertLedgerMatch(ra, rb) @@ -795,9 +808,10 @@ func (tsl *testingScoreLedger) Stop() { // Tests start and stop of a custom decision logic func TestWithScoreLedger(t *testing.T) { tsl := newTestingScoreLedger() - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(kNetworkDelay)) + net := tn.VirtualNetwork(delay.Fixed(kNetworkDelay)) + router := mockrouting.NewServer() bsOpts := []bitswap.Option{bitswap.WithScoreLedger(tsl)} - ig := testinstance.NewTestInstanceGenerator(net, nil, bsOpts) + ig := testinstance.NewTestInstanceGenerator(net, router, nil, bsOpts) defer ig.Close() i := ig.Next() defer i.Exchange.Close() diff --git a/bitswap/client/bitswap_with_sessions_test.go b/bitswap/client/bitswap_with_sessions_test.go index 6241865ef..62695de8b 100644 --- a/bitswap/client/bitswap_with_sessions_test.go +++ b/bitswap/client/bitswap_with_sessions_test.go @@ -26,12 +26,12 @@ const blockSize = 4 func getVirtualNetwork() tn.Network { // FIXME: the tests are really sensitive to the network delay. fix them to work // well under varying conditions - return tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0)) + return tn.VirtualNetwork(delay.Fixed(0)) } func addBlock(t *testing.T, ctx context.Context, inst testinstance.Instance, blk blocks.Block) { t.Helper() - err := inst.Blockstore().Put(ctx, blk) + err := inst.Blockstore.Put(ctx, blk) if err != nil { t.Fatal(err) } @@ -39,6 +39,10 @@ func addBlock(t *testing.T, ctx context.Context, inst testinstance.Instance, blk if err != nil { t.Fatal(err) } + err = inst.Routing.Provide(ctx, blk.Cid(), true) + if err != nil { + t.Fatal(err) + } } func TestBasicSessions(t *testing.T) { @@ -46,7 +50,8 @@ func TestBasicSessions(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) defer ig.Close() block := random.BlocksOfSize(1, blockSize)[0] @@ -56,7 +61,7 @@ func TestBasicSessions(t *testing.T) { b := inst[1] // Add a block to Peer B - if err := b.Blockstore().Put(ctx, block); err != nil { + if err := b.Blockstore.Put(ctx, block); err != nil { t.Fatal(err) } @@ -78,7 +83,7 @@ func TestBasicSessions(t *testing.T) { t.Fatal("did not get tracable block") } - if traceBlock.From != b.Peer { + if traceBlock.From != b.Identity.ID() { t.Fatal("should have received block from peer B, did not") } } @@ -111,15 +116,16 @@ func TestSessionBetweenPeers(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - vnet := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(vnet, nil, []bitswap.Option{bitswap.SetSimulateDontHavesOnTimeout(false)}) + vnet := tn.VirtualNetwork(delay.Fixed(time.Millisecond)) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, []bitswap.Option{bitswap.SetSimulateDontHavesOnTimeout(false)}) defer ig.Close() inst := ig.Instances(10) // Add 101 blocks to Peer A blks := random.BlocksOfSize(101, blockSize) - if err := inst[0].Blockstore().PutMany(ctx, blks); err != nil { + if err := inst[0].Blockstore.PutMany(ctx, blks); err != nil { t.Fatal(err) } @@ -147,7 +153,7 @@ func TestSessionBetweenPeers(t *testing.T) { for b := range ch { got = append(got, b) } - if err := assertBlockListsFrom(inst[0].Peer, got, blks[i*10:(i+1)*10]); err != nil { + if err := assertBlockListsFrom(inst[0].Identity.ID(), got, blks[i*10:(i+1)*10]); err != nil { t.Fatal(err) } } @@ -171,7 +177,8 @@ func TestSessionSplitFetch(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) defer ig.Close() inst := ig.Instances(11) @@ -179,7 +186,7 @@ func TestSessionSplitFetch(t *testing.T) { // Add 10 distinct blocks to each of 10 peers blks := random.BlocksOfSize(100, blockSize) for i := 0; i < 10; i++ { - if err := inst[i].Blockstore().PutMany(ctx, blks[i*10:(i+1)*10]); err != nil { + if err := inst[i].Blockstore.PutMany(ctx, blks[i*10:(i+1)*10]); err != nil { t.Fatal(err) } } @@ -203,7 +210,7 @@ func TestSessionSplitFetch(t *testing.T) { for b := range ch { got = append(got, b) } - if err := assertBlockListsFrom(inst[i].Peer, got, blks[i*10:(i+1)*10]); err != nil { + if err := assertBlockListsFrom(inst[i].Identity.ID(), got, blks[i*10:(i+1)*10]); err != nil { t.Fatal(err) } } @@ -214,7 +221,8 @@ func TestFetchNotConnected(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, []bitswap.Option{bitswap.ProviderSearchDelay(10 * time.Millisecond)}) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, []bitswap.Option{bitswap.ProviderSearchDelay(10 * time.Millisecond)}) defer ig.Close() other := ig.Next() @@ -236,7 +244,6 @@ func TestFetchNotConnected(t *testing.T) { thisNode := ig.Next() ses := thisNode.Exchange.NewSession(ctx).(*session.Session) ses.SetBaseTickDelay(time.Millisecond * 10) - ch, err := ses.GetBlocks(ctx, cids) if err != nil { t.Fatal(err) @@ -246,7 +253,7 @@ func TestFetchNotConnected(t *testing.T) { for b := range ch { got = append(got, b) } - if err := assertBlockListsFrom(other.Peer, got, blks); err != nil { + if err := assertBlockListsFrom(other.Identity.ID(), got, blks); err != nil { t.Fatal(err) } } @@ -256,7 +263,8 @@ func TestFetchAfterDisconnect(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, []bitswap.Option{ + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, []bitswap.Option{ bitswap.ProviderSearchDelay(10 * time.Millisecond), bitswap.RebroadcastDelay(delay.Fixed(15 * time.Millisecond)), }) @@ -294,12 +302,12 @@ func TestFetchAfterDisconnect(t *testing.T) { got = append(got, b) } - if err := assertBlockListsFrom(peerA.Peer, got, blks[:5]); err != nil { + if err := assertBlockListsFrom(peerA.Identity.ID(), got, blks[:5]); err != nil { t.Fatal(err) } // Break connection - err = peerA.Adapter.DisconnectFrom(ctx, peerB.Peer) + err = peerA.Adapter.DisconnectFrom(ctx, peerB.Identity.ID()) if err != nil { t.Fatal(err) } @@ -323,7 +331,7 @@ func TestFetchAfterDisconnect(t *testing.T) { } } - if err := assertBlockListsFrom(peerA.Peer, got, blks); err != nil { + if err := assertBlockListsFrom(peerA.Identity.ID(), got, blks); err != nil { t.Fatal(err) } } @@ -333,7 +341,8 @@ func TestInterestCacheOverflow(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) defer ig.Close() blks := random.BlocksOfSize(2049, blockSize) @@ -382,7 +391,8 @@ func TestPutAfterSessionCacheEvict(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) defer ig.Close() blks := random.BlocksOfSize(2500, blockSize) @@ -419,7 +429,8 @@ func TestMultipleSessions(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) defer ig.Close() blk := random.BlocksOfSize(1, blockSize)[0] @@ -459,7 +470,8 @@ func TestWantlistClearsOnCancel(t *testing.T) { defer cancel() vnet := getVirtualNetwork() - ig := testinstance.NewTestInstanceGenerator(vnet, nil, nil) + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) defer ig.Close() blks := random.BlocksOfSize(10, blockSize) diff --git a/bitswap/client/client.go b/bitswap/client/client.go index bab03c3cd..13d8a006a 100644 --- a/bitswap/client/client.go +++ b/bitswap/client/client.go @@ -13,7 +13,6 @@ import ( bsmq "github.com/ipfs/boxo/bitswap/client/internal/messagequeue" "github.com/ipfs/boxo/bitswap/client/internal/notifications" bspm "github.com/ipfs/boxo/bitswap/client/internal/peermanager" - bspqm "github.com/ipfs/boxo/bitswap/client/internal/providerquerymanager" bssession "github.com/ipfs/boxo/bitswap/client/internal/session" bssim "github.com/ipfs/boxo/bitswap/client/internal/sessioninterestmanager" bssm "github.com/ipfs/boxo/bitswap/client/internal/sessionmanager" @@ -26,6 +25,7 @@ import ( "github.com/ipfs/boxo/bitswap/tracer" blockstore "github.com/ipfs/boxo/blockstore" exchange "github.com/ipfs/boxo/exchange" + rpqm "github.com/ipfs/boxo/routing/providerquerymanager" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" delay "github.com/ipfs/go-ipfs-delay" @@ -97,6 +97,19 @@ func WithoutDuplicatedBlockStats() Option { } } +// WithDefaultProviderQueryManager indicates wether we should use a the +// default ProviderQueryManager, a wrapper of the content Router which +// provides bounded paralelism and limits for these lookups. The +// ProviderQueryManager setup by default uses maxInProcessRequests = 6 and +// maxProviders = 10. To use a custom ProviderQueryManager, set to false and +// wrap directly the content router provided with the WithContentRouting() +// option. Only takes effect if WithContentRouting is set. +func WithDefaultProviderQueryManager(defaultProviderQueryManager bool) Option { + return func(bs *Client) { + bs.defaultProviderQueryManager = defaultProviderQueryManager + } +} + type BlockReceivedNotifier interface { // ReceivedBlocks notifies the decision engine that a peer is well-behaving // and gave us useful data, potentially increasing its score and making us @@ -104,8 +117,16 @@ type BlockReceivedNotifier interface { ReceivedBlocks(peer.ID, []blocks.Block) } +// ProviderFinder is a subset of +// https://pkg.go.dev/github.com/libp2p/go-libp2p@v0.37.0/core/routing#ContentRouting +type ProviderFinder interface { + FindProvidersAsync(context.Context, cid.Cid, int) <-chan peer.AddrInfo +} + // New initializes a Bitswap client that runs until client.Close is called. -func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore.Blockstore, options ...Option) *Client { +// The Content providerFinder paramteter can be nil to disable content-routing +// lookups for content (rely only on bitswap for discovery). +func New(parent context.Context, network bsnet.BitSwapNetwork, providerFinder ProviderFinder, bstore blockstore.Blockstore, options ...Option) *Client { // important to use provided parent context (since it may include important // loggable data). It's probably not a good idea to allow bitswap to be // coupled to the concerns of the ipfs daemon in this way. @@ -115,11 +136,30 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore // exclusively. We should probably find another way to share logging data ctx, cancelFunc := context.WithCancel(parent) + bs := &Client{ + network: network, + providerFinder: providerFinder, + blockstore: bstore, + cancel: cancelFunc, + closing: make(chan struct{}), + counters: new(counters), + dupMetric: bmetrics.DupHist(ctx), + allMetric: bmetrics.AllHist(ctx), + provSearchDelay: defaults.ProvSearchDelay, + rebroadcastDelay: delay.Fixed(defaults.RebroadcastDelay), + simulateDontHavesOnTimeout: true, + defaultProviderQueryManager: true, + } + + // apply functional options before starting and running bitswap + for _, option := range options { + option(bs) + } + // onDontHaveTimeout is called when a want-block is sent to a peer that // has an old version of Bitswap that doesn't support DONT_HAVE messages, // or when no response is received within a timeout. var sm *bssm.SessionManager - var bs *Client onDontHaveTimeout := func(p peer.ID, dontHaves []cid.Cid) { // Simulate a message arriving with DONT_HAVEs if bs.simulateDontHavesOnTimeout { @@ -133,7 +173,17 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore sim := bssim.New() bpm := bsbpm.New() pm := bspm.New(ctx, peerQueueFactory, network.Self()) - pqm := bspqm.New(ctx, network) + + if bs.providerFinder != nil && bs.defaultProviderQueryManager { + // network can do dialing. + pqm, err := rpqm.New(ctx, network, bs.providerFinder, rpqm.WithMaxProviders(10)) + if err != nil { + // Should not be possible to hit this + panic(err) + } + pqm.Startup() + bs.pqm = pqm + } sessionFactory := func( sessctx context.Context, @@ -148,6 +198,14 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore rebroadcastDelay delay.D, self peer.ID, ) bssm.Session { + // careful when bs.pqm is nil. Since we are type-casting it + // into session.ProviderFinder when passing it, it will become + // not nil. Related: + // https://groups.google.com/g/golang-nuts/c/wnH302gBa4I?pli=1 + var pqm bssession.ProviderFinder + if bs.pqm != nil { + pqm = bs.pqm + } return bssession.New(sessctx, sessmgr, id, spm, pqm, sim, pm, bpm, notif, provSearchDelay, rebroadcastDelay, self) } sessionPeerManagerFactory := func(ctx context.Context, id uint64) bssession.SessionPeerManager { @@ -156,29 +214,10 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore notif := notifications.New() sm = bssm.New(ctx, sessionFactory, sim, sessionPeerManagerFactory, bpm, pm, notif, network.Self()) - bs = &Client{ - blockstore: bstore, - network: network, - cancel: cancelFunc, - closing: make(chan struct{}), - pm: pm, - sm: sm, - sim: sim, - notif: notif, - counters: new(counters), - dupMetric: bmetrics.DupHist(ctx), - allMetric: bmetrics.AllHist(ctx), - provSearchDelay: defaults.ProvSearchDelay, - rebroadcastDelay: delay.Fixed(defaults.RebroadcastDelay), - simulateDontHavesOnTimeout: true, - } - - // apply functional options before starting and running bitswap - for _, option := range options { - option(bs) - } - - pqm.Startup() + bs.sm = sm + bs.notif = notif + bs.pm = pm + bs.sim = sim return bs } @@ -187,6 +226,12 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, bstore blockstore type Client struct { pm *bspm.PeerManager + providerFinder ProviderFinder + + // the provider query manager manages requests to find providers + pqm *rpqm.ProviderQueryManager + defaultProviderQueryManager bool + // network delivers messages on behalf of the session network bsnet.BitSwapNetwork diff --git a/bitswap/client/internal/messagequeue/messagequeue.go b/bitswap/client/internal/messagequeue/messagequeue.go index edea20b9c..11c25089e 100644 --- a/bitswap/client/internal/messagequeue/messagequeue.go +++ b/bitswap/client/internal/messagequeue/messagequeue.go @@ -51,7 +51,7 @@ const ( // MessageNetwork is any network that can connect peers and generate a message // sender. type MessageNetwork interface { - ConnectTo(context.Context, peer.ID) error + Connect(context.Context, peer.AddrInfo) error NewMessageSender(context.Context, peer.ID, *bsnet.MessageSenderOpts) (bsnet.MessageSender, error) Latency(peer.ID) time.Duration Ping(context.Context, peer.ID) ping.Result diff --git a/bitswap/client/internal/messagequeue/messagequeue_test.go b/bitswap/client/internal/messagequeue/messagequeue_test.go index 3a9c21309..1073a9f74 100644 --- a/bitswap/client/internal/messagequeue/messagequeue_test.go +++ b/bitswap/client/internal/messagequeue/messagequeue_test.go @@ -27,7 +27,7 @@ type fakeMessageNetwork struct { messageSender bsnet.MessageSender } -func (fmn *fakeMessageNetwork) ConnectTo(context.Context, peer.ID) error { +func (fmn *fakeMessageNetwork) Connect(context.Context, peer.AddrInfo) error { return fmn.connectError } diff --git a/bitswap/client/internal/session/session.go b/bitswap/client/internal/session/session.go index 6f99dec0e..3e2a9b53d 100644 --- a/bitswap/client/internal/session/session.go +++ b/bitswap/client/internal/session/session.go @@ -75,7 +75,7 @@ type SessionPeerManager interface { // ProviderFinder is used to find providers for a given key type ProviderFinder interface { // FindProvidersAsync searches for peers that provide the given CID - FindProvidersAsync(ctx context.Context, k cid.Cid) <-chan peer.ID + FindProvidersAsync(ctx context.Context, k cid.Cid, max int) <-chan peer.AddrInfo } // opType is the kind of operation that is being processed by the event loop @@ -403,14 +403,18 @@ func (s *Session) handlePeriodicSearch(ctx context.Context) { // findMorePeers attempts to find more peers for a session by searching for // providers for the given Cid func (s *Session) findMorePeers(ctx context.Context, c cid.Cid) { + // noop when provider finder is disabled + if s.providerFinder == nil { + return + } go func(k cid.Cid) { ctx, span := internal.StartSpan(ctx, "Session.FindMorePeers") defer span.End() - for p := range s.providerFinder.FindProvidersAsync(ctx, k) { + for p := range s.providerFinder.FindProvidersAsync(ctx, k, 0) { // When a provider indicates that it has a cid, it's equivalent to // the providing peer sending a HAVE span.AddEvent("FoundPeer") - s.sws.Update(p, nil, []cid.Cid{c}, nil) + s.sws.Update(p.ID, nil, []cid.Cid{c}, nil) } }(c) } diff --git a/bitswap/client/internal/session/session_test.go b/bitswap/client/internal/session/session_test.go index a14fdffd0..061e298e5 100644 --- a/bitswap/client/internal/session/session_test.go +++ b/bitswap/client/internal/session/session_test.go @@ -116,7 +116,7 @@ func newFakeProviderFinder() *fakeProviderFinder { } } -func (fpf *fakeProviderFinder) FindProvidersAsync(ctx context.Context, k cid.Cid) <-chan peer.ID { +func (fpf *fakeProviderFinder) FindProvidersAsync(ctx context.Context, k cid.Cid, max int) <-chan peer.AddrInfo { go func() { select { case fpf.findMorePeersRequested <- k: @@ -124,7 +124,7 @@ func (fpf *fakeProviderFinder) FindProvidersAsync(ctx context.Context, k cid.Cid } }() - return make(chan peer.ID) + return make(chan peer.AddrInfo) } type wantReq struct { diff --git a/bitswap/internal/defaults/defaults.go b/bitswap/internal/defaults/defaults.go index b30bcc87f..dbcd62a31 100644 --- a/bitswap/internal/defaults/defaults.go +++ b/bitswap/internal/defaults/defaults.go @@ -20,11 +20,6 @@ const ( BitswapMaxOutstandingBytesPerPeer = 1 << 20 // the number of bytes we attempt to make each outgoing bitswap message BitswapEngineTargetMessageSize = 16 * 1024 - // HasBlockBufferSize is the buffer size of the channel for new blocks - // that need to be provided. They should get pulled over by the - // provideCollector even before they are actually provided. - // TODO: Does this need to be this large givent that? - HasBlockBufferSize = 256 // Maximum size of the wantlist we are willing to keep in memory. MaxQueuedWantlistEntiresPerPeer = 1024 diff --git a/bitswap/network/interface.go b/bitswap/network/interface.go index 6ea0fc525..6c56bab14 100644 --- a/bitswap/network/interface.go +++ b/bitswap/network/interface.go @@ -40,7 +40,7 @@ type BitSwapNetwork interface { // Stop stops the network service. Stop() - ConnectTo(context.Context, peer.ID) error + Connect(context.Context, peer.AddrInfo) error DisconnectFrom(context.Context, peer.ID) error NewMessageSender(context.Context, peer.ID, *MessageSenderOpts) (MessageSender, error) @@ -49,8 +49,6 @@ type BitSwapNetwork interface { Stats() Stats - Routing - Pinger } @@ -88,7 +86,7 @@ type Receiver interface { // network. type Routing interface { // FindProvidersAsync returns a channel of providers for the given key. - FindProvidersAsync(context.Context, cid.Cid, int) <-chan peer.ID + FindProvidersAsync(context.Context, cid.Cid, int) <-chan peer.AddrInfo // Provide provides the key to the network. Provide(context.Context, cid.Cid) error diff --git a/bitswap/network/ipfs_impl.go b/bitswap/network/ipfs_impl.go index f01adb996..993b64429 100644 --- a/bitswap/network/ipfs_impl.go +++ b/bitswap/network/ipfs_impl.go @@ -11,15 +11,12 @@ import ( bsmsg "github.com/ipfs/boxo/bitswap/message" "github.com/ipfs/boxo/bitswap/network/internal" - "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p/core/connmgr" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" - "github.com/libp2p/go-libp2p/core/peerstore" "github.com/libp2p/go-libp2p/core/protocol" - "github.com/libp2p/go-libp2p/core/routing" "github.com/libp2p/go-libp2p/p2p/protocol/ping" "github.com/libp2p/go-msgio" ma "github.com/multiformats/go-multiaddr" @@ -36,12 +33,11 @@ var ( ) // NewFromIpfsHost returns a BitSwapNetwork supported by underlying IPFS host. -func NewFromIpfsHost(host host.Host, r routing.ContentRouting, opts ...NetOpt) BitSwapNetwork { +func NewFromIpfsHost(host host.Host, opts ...NetOpt) BitSwapNetwork { s := processSettings(opts...) bitswapNetwork := impl{ - host: host, - routing: r, + host: host, protocolBitswapNoVers: s.ProtocolPrefix + ProtocolBitswapNoVers, protocolBitswapOneZero: s.ProtocolPrefix + ProtocolBitswapOneZero, @@ -73,7 +69,6 @@ type impl struct { stats Stats host host.Host - routing routing.ContentRouting connectEvtMgr *connectEventManager protocolBitswapNoVers protocol.ID @@ -104,7 +99,7 @@ func (s *streamMessageSender) Connect(ctx context.Context) (network.Stream, erro tctx, cancel := context.WithTimeout(ctx, s.opts.SendTimeout) defer cancel() - if err := s.bsnet.ConnectTo(tctx, s.to); err != nil { + if err := s.bsnet.Connect(ctx, peer.AddrInfo{ID: s.to}); err != nil { return nil, err } @@ -363,40 +358,17 @@ func (bsnet *impl) Stop() { bsnet.host.Network().StopNotify((*netNotifiee)(bsnet)) } -func (bsnet *impl) ConnectTo(ctx context.Context, p peer.ID) error { - return bsnet.host.Connect(ctx, peer.AddrInfo{ID: p}) +func (bsnet *impl) Connect(ctx context.Context, p peer.AddrInfo) error { + if p.ID == bsnet.host.ID() { + return nil + } + return bsnet.host.Connect(ctx, p) } func (bsnet *impl) DisconnectFrom(ctx context.Context, p peer.ID) error { return bsnet.host.Network().ClosePeer(p) } -// FindProvidersAsync returns a channel of providers for the given key. -func (bsnet *impl) FindProvidersAsync(ctx context.Context, k cid.Cid, max int) <-chan peer.ID { - out := make(chan peer.ID, max) - go func() { - defer close(out) - providers := bsnet.routing.FindProvidersAsync(ctx, k, max) - for info := range providers { - if info.ID == bsnet.host.ID() { - continue // ignore self as provider - } - bsnet.host.Peerstore().AddAddrs(info.ID, info.Addrs, peerstore.TempAddrTTL) - select { - case <-ctx.Done(): - return - case out <- info.ID: - } - } - }() - return out -} - -// Provide provides the key to the network -func (bsnet *impl) Provide(ctx context.Context, k cid.Cid) error { - return bsnet.routing.Provide(ctx, k, true) -} - // handleNewStream receives a new stream from the network. func (bsnet *impl) handleNewStream(s network.Stream) { defer s.Close() diff --git a/bitswap/network/ipfs_impl_test.go b/bitswap/network/ipfs_impl_test.go index 91e998846..bfba5709d 100644 --- a/bitswap/network/ipfs_impl_test.go +++ b/bitswap/network/ipfs_impl_test.go @@ -13,8 +13,6 @@ import ( bsnet "github.com/ipfs/boxo/bitswap/network" "github.com/ipfs/boxo/bitswap/network/internal" tn "github.com/ipfs/boxo/bitswap/testnet" - mockrouting "github.com/ipfs/boxo/routing/mock" - ds "github.com/ipfs/go-datastore" "github.com/ipfs/go-test/random" tnet "github.com/libp2p/go-libp2p-testing/net" "github.com/libp2p/go-libp2p/core/host" @@ -170,8 +168,7 @@ func TestMessageSendAndReceive(t *testing.T) { defer cancel() mn := mocknet.New() defer mn.Close() - mr := mockrouting.NewServer() - streamNet, err := tn.StreamNet(ctx, mn, mr) + streamNet, err := tn.StreamNet(ctx, mn) if err != nil { t.Fatal("Unable to setup network") } @@ -191,7 +188,7 @@ func TestMessageSendAndReceive(t *testing.T) { if err != nil { t.Fatal(err) } - err = bsnet1.ConnectTo(ctx, p2.ID()) + err = bsnet1.Connect(ctx, peer.AddrInfo{ID: p2.ID()}) if err != nil { t.Fatal(err) } @@ -200,7 +197,7 @@ func TestMessageSendAndReceive(t *testing.T) { t.Fatal("did not connect peer") case <-r1.connectionEvent: } - err = bsnet2.ConnectTo(ctx, p1.ID()) + err = bsnet2.Connect(ctx, peer.AddrInfo{ID: p1.ID()}) if err != nil { t.Fatal(err) } @@ -275,7 +272,6 @@ func prepareNetwork(t *testing.T, ctx context.Context, p1 tnet.Identity, r1 *rec // create network mn := mocknet.New() defer mn.Close() - mr := mockrouting.NewServer() // Host 1 h1, err := mn.AddPeer(p1.PrivateKey(), p1.Address()) @@ -283,8 +279,7 @@ func prepareNetwork(t *testing.T, ctx context.Context, p1 tnet.Identity, r1 *rec t.Fatal(err) } eh1 := &ErrHost{Host: h1} - routing1 := mr.ClientWithDatastore(context.TODO(), p1, ds.NewMapDatastore()) - bsnet1 := bsnet.NewFromIpfsHost(eh1, routing1) + bsnet1 := bsnet.NewFromIpfsHost(eh1) bsnet1.Start(r1) t.Cleanup(bsnet1.Stop) if r1.listener != nil { @@ -297,8 +292,7 @@ func prepareNetwork(t *testing.T, ctx context.Context, p1 tnet.Identity, r1 *rec t.Fatal(err) } eh2 := &ErrHost{Host: h2} - routing2 := mr.ClientWithDatastore(context.TODO(), p2, ds.NewMapDatastore()) - bsnet2 := bsnet.NewFromIpfsHost(eh2, routing2) + bsnet2 := bsnet.NewFromIpfsHost(eh2) bsnet2.Start(r2) t.Cleanup(bsnet2.Stop) if r2.listener != nil { @@ -310,7 +304,7 @@ func prepareNetwork(t *testing.T, ctx context.Context, p1 tnet.Identity, r1 *rec if err != nil { t.Fatal(err) } - err = bsnet1.ConnectTo(ctx, p2.ID()) + err = bsnet1.Connect(ctx, peer.AddrInfo{ID: p2.ID()}) if err != nil { t.Fatal(err) } @@ -319,7 +313,7 @@ func prepareNetwork(t *testing.T, ctx context.Context, p1 tnet.Identity, r1 *rec t.Fatal("Expected connect event") } - err = bsnet2.ConnectTo(ctx, p1.ID()) + err = bsnet2.Connect(ctx, peer.AddrInfo{ID: p1.ID()}) if err != nil { t.Fatal(err) } @@ -454,8 +448,7 @@ func TestSupportsHave(t *testing.T) { ctx := context.Background() mn := mocknet.New() defer mn.Close() - mr := mockrouting.NewServer() - streamNet, err := tn.StreamNet(ctx, mn, mr) + streamNet, err := tn.StreamNet(ctx, mn) if err != nil { t.Fatalf("Unable to setup network: %s", err) } diff --git a/bitswap/options.go b/bitswap/options.go index 6a98b27db..736b58914 100644 --- a/bitswap/options.go +++ b/bitswap/options.go @@ -43,10 +43,6 @@ func TaskWorkerCount(count int) Option { return Option{server.TaskWorkerCount(count)} } -func ProvideEnabled(enabled bool) Option { - return Option{server.ProvideEnabled(enabled)} -} - func SetSendDontHaves(send bool) Option { return Option{server.SetSendDontHaves(send)} } @@ -106,3 +102,11 @@ func WithTracer(tap tracer.Tracer) Option { }), } } + +func WithClientOption(opt client.Option) Option { + return Option{opt} +} + +func WithServerOption(opt server.Option) Option { + return Option{opt} +} diff --git a/bitswap/server/server.go b/bitswap/server/server.go index 6416da034..5bb277dfb 100644 --- a/bitswap/server/server.go +++ b/bitswap/server/server.go @@ -24,15 +24,11 @@ import ( "go.uber.org/zap" ) -var provideKeysBufferSize = 2048 - var ( log = logging.Logger("bitswap/server") sflog = log.Desugar() ) -const provideWorkerMax = 6 - type Option func(*Server) type Server struct { @@ -62,37 +58,21 @@ type Server struct { // waitWorkers waits for all worker goroutines to exit. waitWorkers sync.WaitGroup - // newBlocks is a channel for newly added blocks to be provided to the - // network. blocks pushed down this channel get buffered and fed to the - // provideKeys channel later on to avoid too much network activity - newBlocks chan cid.Cid - // provideKeys directly feeds provide workers - provideKeys chan cid.Cid - // Extra options to pass to the decision manager engineOptions []decision.Option - - // the size of channel buffer to use - hasBlockBufferSize int - // whether or not to make provide announcements - provideEnabled bool } func New(ctx context.Context, network bsnet.BitSwapNetwork, bstore blockstore.Blockstore, options ...Option) *Server { ctx, cancel := context.WithCancel(ctx) s := &Server{ - sentHistogram: bmetrics.SentHist(ctx), - sendTimeHistogram: bmetrics.SendTimeHist(ctx), - taskWorkerCount: defaults.BitswapTaskWorkerCount, - network: network, - cancel: cancel, - closing: make(chan struct{}), - provideEnabled: true, - hasBlockBufferSize: defaults.HasBlockBufferSize, - provideKeys: make(chan cid.Cid, provideKeysBufferSize), + sentHistogram: bmetrics.SentHist(ctx), + sendTimeHistogram: bmetrics.SendTimeHist(ctx), + taskWorkerCount: defaults.BitswapTaskWorkerCount, + network: network, + cancel: cancel, + closing: make(chan struct{}), } - s.newBlocks = make(chan cid.Cid, s.hasBlockBufferSize) for _, o := range options { o(s) @@ -127,13 +107,6 @@ func WithTracer(tap tracer.Tracer) Option { } } -// ProvideEnabled is an option for enabling/disabling provide announcements -func ProvideEnabled(enabled bool) Option { - return func(bs *Server) { - bs.provideEnabled = enabled - } -} - func WithPeerBlockRequestFilter(pbrf decision.PeerBlockRequestFilter) Option { o := decision.WithPeerBlockRequestFilter(pbrf) return func(bs *Server) { @@ -237,16 +210,6 @@ func MaxCidSize(n uint) Option { } } -// HasBlockBufferSize configure how big the new blocks buffer should be. -func HasBlockBufferSize(count int) Option { - if count < 0 { - panic("cannot have negative buffer size") - } - return func(bs *Server) { - bs.hasBlockBufferSize = count - } -} - // WithWantHaveReplaceSize sets the maximum size of a block in bytes up to // which the bitswap server will replace a WantHave with a WantBlock response. // @@ -296,12 +259,6 @@ func (bs *Server) startWorkers(ctx context.Context) { i := i go bs.taskWorker(ctx, i) } - - if bs.provideEnabled { - bs.waitWorkers.Add(1) - go bs.provideCollector(ctx) - bs.startProvideWorkers(ctx) - } } func (bs *Server) taskWorker(ctx context.Context, id int) { @@ -410,10 +367,9 @@ func (bs *Server) sendBlocks(ctx context.Context, env *decision.Envelope) { } type Stat struct { - Peers []string - ProvideBufLen int - BlocksSent uint64 - DataSent uint64 + Peers []string + BlocksSent uint64 + DataSent uint64 } // Stat returns aggregated statistics about bitswap operations @@ -421,7 +377,6 @@ func (bs *Server) Stat() (Stat, error) { bs.counterLk.Lock() s := bs.counters bs.counterLk.Unlock() - s.ProvideBufLen = len(bs.newBlocks) peers := bs.engine.Peers() peersStr := make([]string, len(peers)) @@ -448,84 +403,9 @@ func (bs *Server) NotifyNewBlocks(ctx context.Context, blks ...blocks.Block) err // Send wanted blocks to decision engine bs.engine.NotifyNewBlocks(blks) - // If the reprovider is enabled, send block to reprovider - if bs.provideEnabled { - for _, blk := range blks { - select { - case bs.newBlocks <- blk.Cid(): - // send block off to be reprovided - case <-bs.closing: - return nil - } - } - } - return nil } -func (bs *Server) provideCollector(ctx context.Context) { - defer bs.waitWorkers.Done() - defer close(bs.provideKeys) - var toProvide []cid.Cid - var nextKey cid.Cid - var keysOut chan cid.Cid - - for { - select { - case blkey, ok := <-bs.newBlocks: - if !ok { - log.Debug("newBlocks channel closed") - return - } - - if keysOut == nil { - nextKey = blkey - keysOut = bs.provideKeys - } else { - toProvide = append(toProvide, blkey) - } - case keysOut <- nextKey: - if len(toProvide) > 0 { - nextKey = toProvide[0] - toProvide = toProvide[1:] - } else { - keysOut = nil - } - case <-ctx.Done(): - return - } - } -} - -// startProvideWorkers starts provide worker goroutines that provide CID -// supplied by provideCollector. -// -// If providing blocks bottlenecks file transfers then consider increasing -// provideWorkerMax, -func (bs *Server) startProvideWorkers(ctx context.Context) { - bs.waitWorkers.Add(provideWorkerMax) - for id := 0; id < provideWorkerMax; id++ { - go func(wid int) { - defer bs.waitWorkers.Done() - - var runCount int - // Read bs.proviudeKeys until closed, when provideCollector exits. - for k := range bs.provideKeys { - runCount++ - log.Debugw("Bitswap provider worker start", "ID", wid, "run", runCount, "cid", k) - - ctx, cancel := context.WithTimeout(ctx, defaults.ProvideTimeout) - if err := bs.network.Provide(ctx, k); err != nil { - log.Warn(err) - } - cancel() - - log.Debugw("Bitswap provider worker done", "ID", wid, "run", runCount, "cid", k) - } - }(id) - } -} - func (bs *Server) ReceiveMessage(ctx context.Context, p peer.ID, incoming message.BitSwapMessage) { // This call records changes to wantlists, blocks received, // and number of bytes transfered. diff --git a/bitswap/testinstance/testinstance.go b/bitswap/testinstance/testinstance.go index 5a052b831..f09831b65 100644 --- a/bitswap/testinstance/testinstance.go +++ b/bitswap/testinstance/testinstance.go @@ -8,6 +8,7 @@ import ( bsnet "github.com/ipfs/boxo/bitswap/network" tn "github.com/ipfs/boxo/bitswap/testnet" blockstore "github.com/ipfs/boxo/blockstore" + mockrouting "github.com/ipfs/boxo/routing/mock" ds "github.com/ipfs/go-datastore" delayed "github.com/ipfs/go-datastore/delayed" ds_sync "github.com/ipfs/go-datastore/sync" @@ -15,11 +16,12 @@ import ( tnet "github.com/libp2p/go-libp2p-testing/net" p2ptestutil "github.com/libp2p/go-libp2p-testing/netutil" peer "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/core/routing" ) // NewTestInstanceGenerator generates a new InstanceGenerator for the given // testnet -func NewTestInstanceGenerator(net tn.Network, netOptions []bsnet.NetOpt, bsOptions []bitswap.Option) InstanceGenerator { +func NewTestInstanceGenerator(net tn.Network, routing mockrouting.Server, netOptions []bsnet.NetOpt, bsOptions []bitswap.Option) InstanceGenerator { ctx, cancel := context.WithCancel(context.Background()) return InstanceGenerator{ net: net, @@ -28,6 +30,7 @@ func NewTestInstanceGenerator(net tn.Network, netOptions []bsnet.NetOpt, bsOptio cancel: cancel, bsOptions: bsOptions, netOptions: netOptions, + routing: routing, } } @@ -39,6 +42,7 @@ type InstanceGenerator struct { cancel context.CancelFunc bsOptions []bitswap.Option netOptions []bsnet.NetOpt + routing mockrouting.Server } // Close closes the clobal context, shutting down all test instances @@ -54,7 +58,7 @@ func (g *InstanceGenerator) Next() Instance { if err != nil { panic("FIXME") // TODO change signature } - return NewInstance(g.ctx, g.net, p, g.netOptions, g.bsOptions) + return NewInstance(g.ctx, g.net, g.routing.Client(p), p, g.netOptions, g.bsOptions) } // Instances creates N test instances of bitswap + dependencies and connects @@ -74,7 +78,7 @@ func ConnectInstances(instances []Instance) { for i, inst := range instances { for j := i + 1; j < len(instances); j++ { oinst := instances[j] - err := inst.Adapter.ConnectTo(context.Background(), oinst.Peer) + err := inst.Adapter.Connect(context.Background(), peer.AddrInfo{ID: oinst.Identity.ID()}) if err != nil { panic(err.Error()) } @@ -84,18 +88,15 @@ func ConnectInstances(instances []Instance) { // Instance is a test instance of bitswap + dependencies for integration testing type Instance struct { - Peer peer.ID + Identity tnet.Identity + Datastore ds.Batching Exchange *bitswap.Bitswap - blockstore blockstore.Blockstore + Blockstore blockstore.Blockstore Adapter bsnet.BitSwapNetwork + Routing routing.Routing blockstoreDelay delay.D } -// Blockstore returns the block store for this test instance -func (i *Instance) Blockstore() blockstore.Blockstore { - return i.blockstore -} - // SetBlockstoreLatency customizes the artificial delay on receiving blocks // from a blockstore test instance. func (i *Instance) SetBlockstoreLatency(t time.Duration) time.Duration { @@ -107,26 +108,28 @@ func (i *Instance) SetBlockstoreLatency(t time.Duration) time.Duration { // NB: It's easy make mistakes by providing the same peer ID to two different // instances. To safeguard, use the InstanceGenerator to generate instances. It's // just a much better idea. -func NewInstance(ctx context.Context, net tn.Network, p tnet.Identity, netOptions []bsnet.NetOpt, bsOptions []bitswap.Option) Instance { +func NewInstance(ctx context.Context, net tn.Network, router routing.Routing, p tnet.Identity, netOptions []bsnet.NetOpt, bsOptions []bitswap.Option) Instance { bsdelay := delay.Fixed(0) adapter := net.Adapter(p, netOptions...) dstore := ds_sync.MutexWrap(delayed.New(ds.NewMapDatastore(), bsdelay)) + ds := ds_sync.MutexWrap(dstore) bstore, err := blockstore.CachedBlockstore(ctx, - blockstore.NewBlockstore(ds_sync.MutexWrap(dstore)), + blockstore.NewBlockstore(ds), blockstore.DefaultCacheOpts()) if err != nil { panic(err.Error()) // FIXME perhaps change signature and return error. } - bs := bitswap.New(ctx, adapter, bstore, bsOptions...) - + bs := bitswap.New(ctx, adapter, router, bstore, bsOptions...) return Instance{ + Datastore: ds, Adapter: adapter, - Peer: p.ID(), + Identity: p, Exchange: bs, - blockstore: bstore, + Routing: router, + Blockstore: bstore, blockstoreDelay: bsdelay, } } diff --git a/bitswap/testnet/network_test.go b/bitswap/testnet/network_test.go index 0947eff3e..2d45e09b1 100644 --- a/bitswap/testnet/network_test.go +++ b/bitswap/testnet/network_test.go @@ -8,7 +8,6 @@ import ( bsmsg "github.com/ipfs/boxo/bitswap/message" bsnet "github.com/ipfs/boxo/bitswap/network" - mockrouting "github.com/ipfs/boxo/routing/mock" blocks "github.com/ipfs/go-block-format" delay "github.com/ipfs/go-ipfs-delay" @@ -17,7 +16,7 @@ import ( ) func TestSendMessageAsyncButWaitForResponse(t *testing.T) { - net := VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0)) + net := VirtualNetwork(delay.Fixed(0)) responderPeer := tnet.RandIdentityOrFatal(t) waiter := net.Adapter(tnet.RandIdentityOrFatal(t)) responder := net.Adapter(responderPeer) diff --git a/bitswap/testnet/peernet.go b/bitswap/testnet/peernet.go index e4df19699..84fa70c6e 100644 --- a/bitswap/testnet/peernet.go +++ b/bitswap/testnet/peernet.go @@ -5,9 +5,6 @@ import ( bsnet "github.com/ipfs/boxo/bitswap/network" - mockrouting "github.com/ipfs/boxo/routing/mock" - ds "github.com/ipfs/go-datastore" - tnet "github.com/libp2p/go-libp2p-testing/net" "github.com/libp2p/go-libp2p/core/peer" mockpeernet "github.com/libp2p/go-libp2p/p2p/net/mock" @@ -15,12 +12,11 @@ import ( type peernet struct { mockpeernet.Mocknet - routingserver mockrouting.Server } // StreamNet is a testnet that uses libp2p's MockNet -func StreamNet(ctx context.Context, net mockpeernet.Mocknet, rs mockrouting.Server) (Network, error) { - return &peernet{net, rs}, nil +func StreamNet(ctx context.Context, net mockpeernet.Mocknet) (Network, error) { + return &peernet{net}, nil } func (pn *peernet) Adapter(p tnet.Identity, opts ...bsnet.NetOpt) bsnet.BitSwapNetwork { @@ -28,8 +24,8 @@ func (pn *peernet) Adapter(p tnet.Identity, opts ...bsnet.NetOpt) bsnet.BitSwapN if err != nil { panic(err.Error()) } - routing := pn.routingserver.ClientWithDatastore(context.TODO(), p, ds.NewMapDatastore()) - return bsnet.NewFromIpfsHost(client, routing, opts...) + + return bsnet.NewFromIpfsHost(client, opts...) } func (pn *peernet) HasPeer(p peer.ID) bool { diff --git a/bitswap/testnet/virtual.go b/bitswap/testnet/virtual.go index 914044aed..0acf083a9 100644 --- a/bitswap/testnet/virtual.go +++ b/bitswap/testnet/virtual.go @@ -11,27 +11,23 @@ import ( bsmsg "github.com/ipfs/boxo/bitswap/message" bsnet "github.com/ipfs/boxo/bitswap/network" - mockrouting "github.com/ipfs/boxo/routing/mock" - cid "github.com/ipfs/go-cid" delay "github.com/ipfs/go-ipfs-delay" tnet "github.com/libp2p/go-libp2p-testing/net" "github.com/libp2p/go-libp2p/core/connmgr" "github.com/libp2p/go-libp2p/core/peer" protocol "github.com/libp2p/go-libp2p/core/protocol" - "github.com/libp2p/go-libp2p/core/routing" mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" "github.com/libp2p/go-libp2p/p2p/protocol/ping" ) // VirtualNetwork generates a new testnet instance - a fake network that // is used to simulate sending messages. -func VirtualNetwork(rs mockrouting.Server, d delay.D) Network { +func VirtualNetwork(d delay.D) Network { return &network{ latencies: make(map[peer.ID]map[peer.ID]time.Duration), clients: make(map[peer.ID]*receiverQueue), delay: d, - routingserver: rs, isRateLimited: false, rateLimitGenerator: nil, conns: make(map[string]struct{}), @@ -45,13 +41,12 @@ type RateLimitGenerator interface { // RateLimitedVirtualNetwork generates a testnet instance where nodes are rate // limited in the upload/download speed. -func RateLimitedVirtualNetwork(rs mockrouting.Server, d delay.D, rateLimitGenerator RateLimitGenerator) Network { +func RateLimitedVirtualNetwork(d delay.D, rateLimitGenerator RateLimitGenerator) Network { return &network{ latencies: make(map[peer.ID]map[peer.ID]time.Duration), rateLimiters: make(map[peer.ID]map[peer.ID]*mocknet.RateLimiter), clients: make(map[peer.ID]*receiverQueue), delay: d, - routingserver: rs, isRateLimited: true, rateLimitGenerator: rateLimitGenerator, conns: make(map[string]struct{}), @@ -63,7 +58,6 @@ type network struct { latencies map[peer.ID]map[peer.ID]time.Duration rateLimiters map[peer.ID]map[peer.ID]*mocknet.RateLimiter clients map[peer.ID]*receiverQueue - routingserver mockrouting.Server delay delay.D isRateLimited bool rateLimitGenerator RateLimitGenerator @@ -105,7 +99,6 @@ func (n *network) Adapter(p tnet.Identity, opts ...bsnet.NetOpt) bsnet.BitSwapNe client := &networkClient{ local: p.ID(), network: n, - routing: n.routingserver.Client(p), supportedProtocols: s.SupportedProtocols, } n.clients[p.ID()] = &receiverQueue{receiver: client} @@ -192,7 +185,6 @@ type networkClient struct { local peer.ID receivers []bsnet.Receiver network *network - routing routing.Routing supportedProtocols []protocol.ID } @@ -253,27 +245,6 @@ func (nc *networkClient) Stats() bsnet.Stats { } } -// FindProvidersAsync returns a channel of providers for the given key. -func (nc *networkClient) FindProvidersAsync(ctx context.Context, k cid.Cid, max int) <-chan peer.ID { - // NB: this function duplicates the AddrInfo -> ID transformation in the - // bitswap network adapter. Not to worry. This network client will be - // deprecated once the ipfsnet.Mock is added. The code below is only - // temporary. - - out := make(chan peer.ID) - go func() { - defer close(out) - providers := nc.routing.FindProvidersAsync(ctx, k, max) - for info := range providers { - select { - case <-ctx.Done(): - case out <- info.ID: - } - } - }() - return out -} - func (nc *networkClient) ConnectionManager() connmgr.ConnManager { return &connmgr.NullConnMgr{} } @@ -322,11 +293,6 @@ func (nc *networkClient) NewMessageSender(ctx context.Context, p peer.ID, opts * }, nil } -// Provide provides the key to the network. -func (nc *networkClient) Provide(ctx context.Context, k cid.Cid) error { - return nc.routing.Provide(ctx, k, true) -} - func (nc *networkClient) Start(r ...bsnet.Receiver) { nc.receivers = r } @@ -334,15 +300,15 @@ func (nc *networkClient) Start(r ...bsnet.Receiver) { func (nc *networkClient) Stop() { } -func (nc *networkClient) ConnectTo(_ context.Context, p peer.ID) error { +func (nc *networkClient) Connect(_ context.Context, p peer.AddrInfo) error { nc.network.mu.Lock() - otherClient, ok := nc.network.clients[p] + otherClient, ok := nc.network.clients[p.ID] if !ok { nc.network.mu.Unlock() return errors.New("no such peer in network") } - tag := tagForPeers(nc.local, p) + tag := tagForPeers(nc.local, p.ID) if _, ok := nc.network.conns[tag]; ok { nc.network.mu.Unlock() // log.Warning("ALREADY CONNECTED TO PEER (is this a reconnect? test lib needs fixing)") @@ -352,7 +318,7 @@ func (nc *networkClient) ConnectTo(_ context.Context, p peer.ID) error { nc.network.mu.Unlock() otherClient.receiver.PeerConnected(nc.local) - nc.PeerConnected(p) + nc.PeerConnected(p.ID) return nil } diff --git a/blockservice/test/mock.go b/blockservice/test/mock.go index e32b10b99..77eeed127 100644 --- a/blockservice/test/mock.go +++ b/blockservice/test/mock.go @@ -10,14 +10,15 @@ import ( // Mocks returns |n| connected mock Blockservices func Mocks(n int, opts ...blockservice.Option) []blockservice.BlockService { - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0)) - sg := testinstance.NewTestInstanceGenerator(net, nil, nil) - + net := tn.VirtualNetwork(delay.Fixed(0)) + routing := mockrouting.NewServer() + sg := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) instances := sg.Instances(n) var servs []blockservice.BlockService for _, i := range instances { - servs = append(servs, blockservice.New(i.Blockstore(), i.Exchange, opts...)) + servs = append(servs, blockservice.New(i.Blockstore, + i.Exchange, opts...)) } return servs } diff --git a/examples/bitswap-transfer/main.go b/examples/bitswap-transfer/main.go index 921dca3fa..fc2d5ded3 100644 --- a/examples/bitswap-transfer/main.go +++ b/examples/bitswap-transfer/main.go @@ -32,7 +32,6 @@ import ( unixfile "github.com/ipfs/boxo/ipld/unixfs/file" "github.com/ipfs/boxo/ipld/unixfs/importer/balanced" uih "github.com/ipfs/boxo/ipld/unixfs/importer/helpers" - routinghelpers "github.com/libp2p/go-libp2p-routing-helpers" bsclient "github.com/ipfs/boxo/bitswap/client" bsnet "github.com/ipfs/boxo/bitswap/network" @@ -178,15 +177,15 @@ func startDataServer(ctx context.Context, h host.Host) (cid.Cid, *bsserver.Serve // Start listening on the Bitswap protocol // For this example we're not leveraging any content routing (DHT, IPNI, delegated routing requests, etc.) as we know the peer we are fetching from - n := bsnet.NewFromIpfsHost(h, routinghelpers.Null{}) + n := bsnet.NewFromIpfsHost(h) bswap := bsserver.New(ctx, n, bs) n.Start(bswap) return nd.Cid(), bswap, nil } func runClient(ctx context.Context, h host.Host, c cid.Cid, targetPeer string) ([]byte, error) { - n := bsnet.NewFromIpfsHost(h, routinghelpers.Null{}) - bswap := bsclient.New(ctx, n, blockstore.NewBlockstore(datastore.NewNullDatastore())) + n := bsnet.NewFromIpfsHost(h) + bswap := bsclient.New(ctx, n, nil, blockstore.NewBlockstore(datastore.NewNullDatastore())) n.Start(bswap) defer bswap.Close() diff --git a/examples/go.mod b/examples/go.mod index ac7a5343e..c7c389626 100644 --- a/examples/go.mod +++ b/examples/go.mod @@ -12,7 +12,6 @@ require ( github.com/ipld/go-car/v2 v2.14.2 github.com/ipld/go-ipld-prime v0.21.0 github.com/libp2p/go-libp2p v0.37.0 - github.com/libp2p/go-libp2p-routing-helpers v0.7.4 github.com/multiformats/go-multiaddr v0.13.0 github.com/multiformats/go-multicodec v0.9.0 github.com/prometheus/client_golang v1.20.5 @@ -95,6 +94,7 @@ require ( github.com/libp2p/go-libp2p-kad-dht v0.27.0 // indirect github.com/libp2p/go-libp2p-kbucket v0.6.4 // indirect github.com/libp2p/go-libp2p-record v0.2.0 // indirect + github.com/libp2p/go-libp2p-routing-helpers v0.7.4 // indirect github.com/libp2p/go-msgio v0.3.0 // indirect github.com/libp2p/go-nat v0.2.0 // indirect github.com/libp2p/go-netroute v0.2.1 // indirect diff --git a/exchange/providing/providing.go b/exchange/providing/providing.go new file mode 100644 index 000000000..6b2887858 --- /dev/null +++ b/exchange/providing/providing.go @@ -0,0 +1,46 @@ +// Package providing implements an exchange wrapper which +// does content providing for new blocks. +package providing + +import ( + "context" + + "github.com/ipfs/boxo/exchange" + "github.com/ipfs/boxo/provider" + blocks "github.com/ipfs/go-block-format" +) + +// Exchange is an exchange wrapper that calls Provide for blocks received +// over NotifyNewBlocks. +type Exchange struct { + exchange.Interface + provider provider.Provider +} + +// New creates a new providing Exchange with the given exchange and provider. +// This is a light wrapper. We recommend that the provider supports the +// handling of many concurrent provides etc. as it is called directly for +// every new block. +func New(base exchange.Interface, provider provider.Provider) *Exchange { + return &Exchange{ + Interface: base, + provider: provider, + } +} + +// NotifyNewBlocks calls NotifyNewBlocks on the underlying provider and +// provider.Provide for every block after that. +func (ex *Exchange) NotifyNewBlocks(ctx context.Context, blocks ...blocks.Block) error { + // Notify blocks on the underlying exchange. + err := ex.Interface.NotifyNewBlocks(ctx, blocks...) + if err != nil { + return err + } + + for _, b := range blocks { + if err := ex.provider.Provide(ctx, b.Cid(), true); err != nil { + return err + } + } + return nil +} diff --git a/exchange/providing/providing_test.go b/exchange/providing/providing_test.go new file mode 100644 index 000000000..42a2a0cb9 --- /dev/null +++ b/exchange/providing/providing_test.go @@ -0,0 +1,74 @@ +package providing + +import ( + "context" + "testing" + "time" + + testinstance "github.com/ipfs/boxo/bitswap/testinstance" + tn "github.com/ipfs/boxo/bitswap/testnet" + "github.com/ipfs/boxo/blockservice" + "github.com/ipfs/boxo/provider" + mockrouting "github.com/ipfs/boxo/routing/mock" + delay "github.com/ipfs/go-ipfs-delay" + "github.com/ipfs/go-test/random" +) + +func TestExchange(t *testing.T) { + ctx := context.Background() + net := tn.VirtualNetwork(delay.Fixed(0)) + routing := mockrouting.NewServer() + sg := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) + i := sg.Next() + provFinder := routing.Client(i.Identity) + prov, err := provider.New(i.Datastore, + provider.Online(provFinder), + ) + if err != nil { + t.Fatal(err) + } + provExchange := New(i.Exchange, prov) + // write-through so that we notify when re-adding block + bs := blockservice.New(i.Blockstore, provExchange, + blockservice.WriteThrough()) + block := random.BlocksOfSize(1, 10)[0] + // put it on the blockstore of the first instance + err = i.Blockstore.Put(ctx, block) + if err != nil { + t.Fatal() + } + + // Trigger reproviding, otherwise it's not really provided. + err = prov.Reprovide(ctx) + if err != nil { + t.Fatal(err) + } + + time.Sleep(200 * time.Millisecond) + + providersChan := provFinder.FindProvidersAsync(ctx, block.Cid(), 1) + _, ok := <-providersChan + if ok { + t.Fatal("there should be no providers yet for block") + } + + // Now add it via BlockService. It should trigger NotifyNewBlocks + // on the exchange and thus they should get announced. + err = bs.AddBlock(ctx, block) + if err != nil { + t.Fatal() + } + // Trigger reproviding, otherwise it's not really provided. + err = prov.Reprovide(ctx) + if err != nil { + t.Fatal(err) + } + + time.Sleep(200 * time.Millisecond) + + providersChan = provFinder.FindProvidersAsync(ctx, block.Cid(), 1) + _, ok = <-providersChan + if !ok { + t.Fatal("there should be one provider for the block") + } +} diff --git a/fetcher/helpers/block_visitor_test.go b/fetcher/helpers/block_visitor_test.go index 57d3e11ad..9ea0eacd9 100644 --- a/fetcher/helpers/block_visitor_test.go +++ b/fetcher/helpers/block_visitor_test.go @@ -44,8 +44,9 @@ func TestFetchGraphToBlocks(t *testing.T) { }) })) - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0*time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + routing := mockrouting.NewServer() + net := tn.VirtualNetwork(delay.Fixed(0 * time.Millisecond)) + ig := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) defer ig.Close() peers := ig.Instances(2) @@ -53,7 +54,7 @@ func TestFetchGraphToBlocks(t *testing.T) { defer hasBlock.Exchange.Close() blocks := []blocks.Block{block1, block2, block3, block4} - err := hasBlock.Blockstore().PutMany(bg, blocks) + err := hasBlock.Blockstore.PutMany(bg, blocks) require.NoError(t, err) err = hasBlock.Exchange.NotifyNewBlocks(bg, blocks...) require.NoError(t, err) @@ -61,7 +62,7 @@ func TestFetchGraphToBlocks(t *testing.T) { wantsBlock := peers[1] defer wantsBlock.Exchange.Close() - wantsGetter := blockservice.New(wantsBlock.Blockstore(), wantsBlock.Exchange) + wantsGetter := blockservice.New(wantsBlock.Blockstore, wantsBlock.Exchange) fetcherConfig := bsfetcher.NewFetcherConfig(wantsGetter) session := fetcherConfig.NewSession(context.Background()) ctx, cancel := context.WithTimeout(context.Background(), time.Second) @@ -94,15 +95,16 @@ func TestFetchGraphToUniqueBlocks(t *testing.T) { }) })) - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0*time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + routing := mockrouting.NewServer() + net := tn.VirtualNetwork(delay.Fixed(0 * time.Millisecond)) + ig := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) defer ig.Close() peers := ig.Instances(2) hasBlock := peers[0] defer hasBlock.Exchange.Close() - err := hasBlock.Blockstore().PutMany(bg, []blocks.Block{block1, block2, block3}) + err := hasBlock.Blockstore.PutMany(bg, []blocks.Block{block1, block2, block3}) require.NoError(t, err) err = hasBlock.Exchange.NotifyNewBlocks(bg, block1, block2, block3) @@ -111,7 +113,7 @@ func TestFetchGraphToUniqueBlocks(t *testing.T) { wantsBlock := peers[1] defer wantsBlock.Exchange.Close() - wantsGetter := blockservice.New(wantsBlock.Blockstore(), wantsBlock.Exchange) + wantsGetter := blockservice.New(wantsBlock.Blockstore, wantsBlock.Exchange) fetcherConfig := bsfetcher.NewFetcherConfig(wantsGetter) session := fetcherConfig.NewSession(context.Background()) ctx, cancel := context.WithTimeout(context.Background(), time.Second) diff --git a/fetcher/impl/blockservice/fetcher_test.go b/fetcher/impl/blockservice/fetcher_test.go index 5a0b071f4..55c1d5c21 100644 --- a/fetcher/impl/blockservice/fetcher_test.go +++ b/fetcher/impl/blockservice/fetcher_test.go @@ -38,15 +38,16 @@ func TestFetchIPLDPrimeNode(t *testing.T) { }) })) - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0*time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + routing := mockrouting.NewServer() + net := tn.VirtualNetwork(delay.Fixed(0 * time.Millisecond)) + ig := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) defer ig.Close() peers := ig.Instances(2) hasBlock := peers[0] defer hasBlock.Exchange.Close() - err := hasBlock.Blockstore().Put(bg, block) + err := hasBlock.Blockstore.Put(bg, block) require.NoError(t, err) err = hasBlock.Exchange.NotifyNewBlocks(bg, block) @@ -55,7 +56,7 @@ func TestFetchIPLDPrimeNode(t *testing.T) { wantsBlock := peers[1] defer wantsBlock.Exchange.Close() - wantsGetter := blockservice.New(wantsBlock.Blockstore(), wantsBlock.Exchange) + wantsGetter := blockservice.New(wantsBlock.Blockstore, wantsBlock.Exchange) fetcherConfig := bsfetcher.NewFetcherConfig(wantsGetter) session := fetcherConfig.NewSession(context.Background()) @@ -87,8 +88,9 @@ func TestFetchIPLDGraph(t *testing.T) { }) })) - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0*time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + routing := mockrouting.NewServer() + net := tn.VirtualNetwork(delay.Fixed(0 * time.Millisecond)) + ig := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) defer ig.Close() peers := ig.Instances(2) @@ -96,7 +98,7 @@ func TestFetchIPLDGraph(t *testing.T) { defer hasBlock.Exchange.Close() blocks := []blocks.Block{block1, block2, block3, block4} - err := hasBlock.Blockstore().PutMany(bg, blocks) + err := hasBlock.Blockstore.PutMany(bg, blocks) require.NoError(t, err) err = hasBlock.Exchange.NotifyNewBlocks(bg, blocks...) require.NoError(t, err) @@ -104,7 +106,7 @@ func TestFetchIPLDGraph(t *testing.T) { wantsBlock := peers[1] defer wantsBlock.Exchange.Close() - wantsGetter := blockservice.New(wantsBlock.Blockstore(), wantsBlock.Exchange) + wantsGetter := blockservice.New(wantsBlock.Blockstore, wantsBlock.Exchange) fetcherConfig := bsfetcher.NewFetcherConfig(wantsGetter) session := fetcherConfig.NewSession(context.Background()) ctx, cancel := context.WithTimeout(context.Background(), time.Second) @@ -143,8 +145,9 @@ func TestFetchIPLDPath(t *testing.T) { }) })) - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0*time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + routing := mockrouting.NewServer() + net := tn.VirtualNetwork(delay.Fixed(0 * time.Millisecond)) + ig := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) defer ig.Close() peers := ig.Instances(2) @@ -152,7 +155,7 @@ func TestFetchIPLDPath(t *testing.T) { defer hasBlock.Exchange.Close() blocks := []blocks.Block{block1, block2, block3, block4, block5} - err := hasBlock.Blockstore().PutMany(bg, blocks) + err := hasBlock.Blockstore.PutMany(bg, blocks) require.NoError(t, err) err = hasBlock.Exchange.NotifyNewBlocks(bg, blocks...) require.NoError(t, err) @@ -160,7 +163,7 @@ func TestFetchIPLDPath(t *testing.T) { wantsBlock := peers[1] defer wantsBlock.Exchange.Close() - wantsGetter := blockservice.New(wantsBlock.Blockstore(), wantsBlock.Exchange) + wantsGetter := blockservice.New(wantsBlock.Blockstore, wantsBlock.Exchange) fetcherConfig := bsfetcher.NewFetcherConfig(wantsGetter) session := fetcherConfig.NewSession(context.Background()) ctx, cancel := context.WithTimeout(context.Background(), time.Second) @@ -206,9 +209,9 @@ func TestHelpers(t *testing.T) { na.AssembleEntry("nonlink").AssignString("zoo") }) })) - - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0*time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + routing := mockrouting.NewServer() + net := tn.VirtualNetwork(delay.Fixed(0 * time.Millisecond)) + ig := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) defer ig.Close() peers := ig.Instances(2) @@ -216,7 +219,7 @@ func TestHelpers(t *testing.T) { defer hasBlock.Exchange.Close() blocks := []blocks.Block{block1, block2, block3, block4} - err := hasBlock.Blockstore().PutMany(bg, blocks) + err := hasBlock.Blockstore.PutMany(bg, blocks) require.NoError(t, err) err = hasBlock.Exchange.NotifyNewBlocks(bg, blocks...) require.NoError(t, err) @@ -224,7 +227,7 @@ func TestHelpers(t *testing.T) { wantsBlock := peers[1] defer wantsBlock.Exchange.Close() - wantsGetter := blockservice.New(wantsBlock.Blockstore(), wantsBlock.Exchange) + wantsGetter := blockservice.New(wantsBlock.Blockstore, wantsBlock.Exchange) t.Run("Block retrieves node", func(t *testing.T) { fetcherConfig := bsfetcher.NewFetcherConfig(wantsGetter) @@ -321,8 +324,9 @@ func TestNodeReification(t *testing.T) { na.AssembleEntry("link4").AssignLink(link4) })) - net := tn.VirtualNetwork(mockrouting.NewServer(), delay.Fixed(0*time.Millisecond)) - ig := testinstance.NewTestInstanceGenerator(net, nil, nil) + routing := mockrouting.NewServer() + net := tn.VirtualNetwork(delay.Fixed(0 * time.Millisecond)) + ig := testinstance.NewTestInstanceGenerator(net, routing, nil, nil) defer ig.Close() peers := ig.Instances(2) @@ -330,7 +334,7 @@ func TestNodeReification(t *testing.T) { defer hasBlock.Exchange.Close() blocks := []blocks.Block{block2, block3, block4} - err := hasBlock.Blockstore().PutMany(bg, blocks) + err := hasBlock.Blockstore.PutMany(bg, blocks) require.NoError(t, err) err = hasBlock.Exchange.NotifyNewBlocks(bg, blocks...) require.NoError(t, err) @@ -338,7 +342,7 @@ func TestNodeReification(t *testing.T) { wantsBlock := peers[1] defer wantsBlock.Exchange.Close() - wantsGetter := blockservice.New(wantsBlock.Blockstore(), wantsBlock.Exchange) + wantsGetter := blockservice.New(wantsBlock.Blockstore, wantsBlock.Exchange) fetcherConfig := bsfetcher.NewFetcherConfig(wantsGetter) nodeReifier := func(lnkCtx ipld.LinkContext, nd ipld.Node, ls *ipld.LinkSystem) (ipld.Node, error) { return &selfLoader{Node: nd, ctx: lnkCtx.Ctx, ls: ls}, nil diff --git a/provider/noop.go b/provider/noop.go index 5367ccb30..50c3e3502 100644 --- a/provider/noop.go +++ b/provider/noop.go @@ -19,7 +19,7 @@ func (op *noopProvider) Close() error { return nil } -func (op *noopProvider) Provide(cid.Cid) error { +func (op *noopProvider) Provide(context.Context, cid.Cid, bool) error { return nil } diff --git a/provider/provider.go b/provider/provider.go index a20a805cb..4197f3dae 100644 --- a/provider/provider.go +++ b/provider/provider.go @@ -18,7 +18,7 @@ var logR = logging.Logger("reprovider.simple") // Provider announces blocks to the network type Provider interface { // Provide takes a cid and makes an attempt to announce it to the network - Provide(cid.Cid) error + Provide(context.Context, cid.Cid, bool) error } // Reprovider reannounces blocks to the network diff --git a/provider/reprovider.go b/provider/reprovider.go index 219bacc75..048a2067d 100644 --- a/provider/reprovider.go +++ b/provider/reprovider.go @@ -455,7 +455,7 @@ func (s *reprovider) Close() error { return err } -func (s *reprovider) Provide(cid cid.Cid) error { +func (s *reprovider) Provide(ctx context.Context, cid cid.Cid, announce bool) error { return s.q.Enqueue(cid) } diff --git a/provider/reprovider_test.go b/provider/reprovider_test.go index 4ae58148e..ceb72f97b 100644 --- a/provider/reprovider_test.go +++ b/provider/reprovider_test.go @@ -198,7 +198,7 @@ func TestOfflineRecordsThenOnlineRepublish(t *testing.T) { sys, err := New(ds) assert.NoError(t, err) - err = sys.Provide(c) + err = sys.Provide(context.Background(), c, true) assert.NoError(t, err) err = sys.Close() diff --git a/routing/mock/centralized_client.go b/routing/mock/centralized_client.go index 02c68d100..2c2135bb8 100644 --- a/routing/mock/centralized_client.go +++ b/routing/mock/centralized_client.go @@ -47,11 +47,12 @@ func (c *client) FindPeer(ctx context.Context, pid peer.ID) (peer.AddrInfo, erro } func (c *client) FindProvidersAsync(ctx context.Context, k cid.Cid, max int) <-chan peer.AddrInfo { + log.Debugf("FindProvidersAsync: %s %d", k, max) out := make(chan peer.AddrInfo) go func() { defer close(out) for i, p := range c.server.Providers(k) { - if max <= i { + if max > 0 && max <= i { return } select { diff --git a/routing/mock/centralized_server.go b/routing/mock/centralized_server.go index d55de7081..85c768814 100644 --- a/routing/mock/centralized_server.go +++ b/routing/mock/centralized_server.go @@ -39,7 +39,7 @@ func (rs *s) Announce(p peer.AddrInfo, c cid.Cid) error { rs.lock.Lock() defer rs.lock.Unlock() - k := c.KeyString() + k := c.Hash().String() _, ok := rs.providers[k] if !ok { @@ -54,16 +54,16 @@ func (rs *s) Announce(p peer.AddrInfo, c cid.Cid) error { func (rs *s) Providers(c cid.Cid) []peer.AddrInfo { rs.delayConf.Query.Wait() // before locking - rs.lock.RLock() defer rs.lock.RUnlock() - k := c.KeyString() + k := c.Hash().String() var ret []peer.AddrInfo records, ok := rs.providers[k] if !ok { return ret } + for _, r := range records { if time.Since(r.Created) > rs.delayConf.ValueVisibility.Get() { ret = append(ret, r.Peer) @@ -74,7 +74,6 @@ func (rs *s) Providers(c cid.Cid) []peer.AddrInfo { j := rand.Intn(i + 1) ret[i], ret[j] = ret[j], ret[i] } - return ret } diff --git a/bitswap/client/internal/providerquerymanager/providerquerymanager.go b/routing/providerquerymanager/providerquerymanager.go similarity index 69% rename from bitswap/client/internal/providerquerymanager/providerquerymanager.go rename to routing/providerquerymanager/providerquerymanager.go index c85efe737..d9005020e 100644 --- a/bitswap/client/internal/providerquerymanager/providerquerymanager.go +++ b/routing/providerquerymanager/providerquerymanager.go @@ -5,27 +5,28 @@ import ( "sync" "time" - "github.com/ipfs/boxo/bitswap/client/internal" "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" peer "github.com/libp2p/go-libp2p/core/peer" + swarm "github.com/libp2p/go-libp2p/p2p/net/swarm" + "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" ) -var log = logging.Logger("bitswap/client/provqrymgr") +var log = logging.Logger("routing/provqrymgr") const ( - maxProviders = 10 - maxInProcessRequests = 6 - defaultTimeout = 10 * time.Second + defaultMaxInProcessRequests = 6 + defaultMaxProviders = 0 + defaultTimeout = 10 * time.Second ) type inProgressRequestStatus struct { ctx context.Context cancelFn func() - providersSoFar []peer.ID - listeners map[chan peer.ID]struct{} + providersSoFar []peer.AddrInfo + listeners map[chan peer.AddrInfo]struct{} } type findProviderRequest struct { @@ -33,11 +34,16 @@ type findProviderRequest struct { ctx context.Context } -// ProviderQueryNetwork is an interface for finding providers and connecting to -// peers. -type ProviderQueryNetwork interface { - ConnectTo(context.Context, peer.ID) error - FindProvidersAsync(context.Context, cid.Cid, int) <-chan peer.ID +// ProviderQueryDialer is an interface for connecting to peers. Usually a +// libp2p.Host +type ProviderQueryDialer interface { + Connect(context.Context, peer.AddrInfo) error +} + +// ProviderQueryRouter is an interface for finding providers. Usually a libp2p +// ContentRouter. +type ProviderQueryRouter interface { + FindProvidersAsync(context.Context, cid.Cid, int) <-chan peer.AddrInfo } type providerQueryMessage interface { @@ -48,7 +54,7 @@ type providerQueryMessage interface { type receivedProviderMessage struct { ctx context.Context k cid.Cid - p peer.ID + p peer.AddrInfo } type finishedProviderQueryMessage struct { @@ -64,7 +70,7 @@ type newProvideQueryMessage struct { type cancelRequestMessage struct { ctx context.Context - incomingProviders chan peer.ID + incomingProviders chan peer.AddrInfo k cid.Cid } @@ -77,7 +83,8 @@ type cancelRequestMessage struct { // - manage timeouts type ProviderQueryManager struct { ctx context.Context - network ProviderQueryNetwork + dialer ProviderQueryDialer + router ProviderQueryRouter providerQueryMessages chan providerQueryMessage providerRequestsProcessing chan *findProviderRequest incomingFindProviderRequests chan *findProviderRequest @@ -85,22 +92,63 @@ type ProviderQueryManager struct { findProviderTimeout time.Duration timeoutMutex sync.RWMutex + maxProviders int + maxInProcessRequests int + // do not touch outside the run loop inProgressRequestStatuses map[cid.Cid]*inProgressRequestStatus } +type Option func(*ProviderQueryManager) error + +func WithMaxTimeout(timeout time.Duration) Option { + return func(mgr *ProviderQueryManager) error { + mgr.findProviderTimeout = timeout + return nil + } +} + +// WithMaxInProcessRequests is the maximum number of requests that can be processed in parallel +func WithMaxInProcessRequests(count int) Option { + return func(mgr *ProviderQueryManager) error { + mgr.maxInProcessRequests = count + return nil + } +} + +// WithMaxProviders is the maximum number of providers that will be looked up +// per query. We only return providers that we can connect to. Defaults to 0, +// which means unbounded. +func WithMaxProviders(count int) Option { + return func(mgr *ProviderQueryManager) error { + mgr.maxProviders = count + return nil + } +} + // New initializes a new ProviderQueryManager for a given context and a given // network provider. -func New(ctx context.Context, network ProviderQueryNetwork) *ProviderQueryManager { - return &ProviderQueryManager{ +func New(ctx context.Context, dialer ProviderQueryDialer, router ProviderQueryRouter, opts ...Option) (*ProviderQueryManager, error) { + pqm := &ProviderQueryManager{ ctx: ctx, - network: network, + dialer: dialer, + router: router, providerQueryMessages: make(chan providerQueryMessage, 16), providerRequestsProcessing: make(chan *findProviderRequest), incomingFindProviderRequests: make(chan *findProviderRequest), inProgressRequestStatuses: make(map[cid.Cid]*inProgressRequestStatus), findProviderTimeout: defaultTimeout, + maxInProcessRequests: defaultMaxInProcessRequests, + maxProviders: defaultMaxProviders, + } + + for _, o := range opts { + if err := o(pqm); err != nil { + return nil, err + } } + + return pqm, nil } // Startup starts processing for the ProviderQueryManager. @@ -109,23 +157,30 @@ func (pqm *ProviderQueryManager) Startup() { } type inProgressRequest struct { - providersSoFar []peer.ID - incoming chan peer.ID + providersSoFar []peer.AddrInfo + incoming chan peer.AddrInfo } -// SetFindProviderTimeout changes the timeout for finding providers -func (pqm *ProviderQueryManager) SetFindProviderTimeout(findProviderTimeout time.Duration) { +// setFindProviderTimeout changes the timeout for finding providers +func (pqm *ProviderQueryManager) setFindProviderTimeout(findProviderTimeout time.Duration) { pqm.timeoutMutex.Lock() pqm.findProviderTimeout = findProviderTimeout pqm.timeoutMutex.Unlock() } -// FindProvidersAsync finds providers for the given block. -func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, k cid.Cid) <-chan peer.ID { +// FindProvidersAsync finds providers for the given block. The max parameter +// controls how many will be returned at most. For a provider to be returned, +// we must have successfully connected to it. Setting max to 0 will use the +// configured MaxProviders which defaults to 0 (unbounded). +func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, k cid.Cid, max int) <-chan peer.AddrInfo { + if max == 0 { + max = pqm.maxProviders + } + inProgressRequestChan := make(chan inProgressRequest) var span trace.Span - sessionCtx, span = internal.StartSpan(sessionCtx, "ProviderQueryManager.FindProvidersAsync", trace.WithAttributes(attribute.Stringer("cid", k))) + sessionCtx, span = otel.Tracer("routing").Start(sessionCtx, "ProviderQueryManager.FindProvidersAsync", trace.WithAttributes(attribute.Stringer("cid", k))) select { case pqm.providerQueryMessages <- &newProvideQueryMessage{ @@ -134,12 +189,12 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, inProgressRequestChan: inProgressRequestChan, }: case <-pqm.ctx.Done(): - ch := make(chan peer.ID) + ch := make(chan peer.AddrInfo) close(ch) span.End() return ch case <-sessionCtx.Done(): - ch := make(chan peer.ID) + ch := make(chan peer.AddrInfo) close(ch) return ch } @@ -150,41 +205,59 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, var receivedInProgressRequest inProgressRequest select { case <-pqm.ctx.Done(): - ch := make(chan peer.ID) + ch := make(chan peer.AddrInfo) close(ch) span.End() return ch case receivedInProgressRequest = <-inProgressRequestChan: } - return pqm.receiveProviders(sessionCtx, k, receivedInProgressRequest, func() { span.End() }) + return pqm.receiveProviders(sessionCtx, k, max, receivedInProgressRequest, func() { span.End() }) } -func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k cid.Cid, receivedInProgressRequest inProgressRequest, onCloseFn func()) <-chan peer.ID { +func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k cid.Cid, max int, receivedInProgressRequest inProgressRequest, onCloseFn func()) <-chan peer.AddrInfo { // maintains an unbuffered queue for incoming providers for given request for a given session // essentially, as a provider comes in, for a given CID, we want to immediately broadcast to all // sessions that queried that CID, without worrying about whether the client code is actually // reading from the returned channel -- so that the broadcast never blocks // based on: https://medium.com/capital-one-tech/building-an-unbounded-channel-in-go-789e175cd2cd - returnedProviders := make(chan peer.ID) - receivedProviders := append([]peer.ID(nil), receivedInProgressRequest.providersSoFar[0:]...) + returnedProviders := make(chan peer.AddrInfo) + receivedProviders := append([]peer.AddrInfo(nil), receivedInProgressRequest.providersSoFar[0:]...) incomingProviders := receivedInProgressRequest.incoming + // count how many providers we received from our workers etc. + // these providers should be peers we managed to connect to. + total := len(receivedProviders) go func() { defer close(returnedProviders) defer onCloseFn() - outgoingProviders := func() chan<- peer.ID { + outgoingProviders := func() chan<- peer.AddrInfo { if len(receivedProviders) == 0 { return nil } return returnedProviders } - nextProvider := func() peer.ID { + nextProvider := func() peer.AddrInfo { if len(receivedProviders) == 0 { - return "" + return peer.AddrInfo{} } return receivedProviders[0] } + + stopWhenMaxReached := func() { + if max > 0 && total >= max { + if incomingProviders != nil { + // drains incomingProviders. + pqm.cancelProviderRequest(sessionCtx, k, incomingProviders) + incomingProviders = nil + } + } + } + + // Handle the case when providersSoFar already is more than we + // need. + stopWhenMaxReached() + for len(receivedProviders) > 0 || incomingProviders != nil { select { case <-pqm.ctx.Done(): @@ -199,6 +272,13 @@ func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k incomingProviders = nil } else { receivedProviders = append(receivedProviders, provider) + total++ + stopWhenMaxReached() + // we do not return, we will loop on + // the case below until + // len(receivedProviders) == 0, which + // means they have all been sent out + // via returnedProviders } case outgoingProviders() <- nextProvider(): receivedProviders = receivedProviders[1:] @@ -208,7 +288,7 @@ func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k return returnedProviders } -func (pqm *ProviderQueryManager) cancelProviderRequest(ctx context.Context, k cid.Cid, incomingProviders chan peer.ID) { +func (pqm *ProviderQueryManager) cancelProviderRequest(ctx context.Context, k cid.Cid, incomingProviders chan peer.AddrInfo) { cancelMessageChannel := pqm.providerQueryMessages for { select { @@ -247,20 +327,24 @@ func (pqm *ProviderQueryManager) findProviderWorker() { pqm.timeoutMutex.RUnlock() span := trace.SpanFromContext(findProviderCtx) span.AddEvent("StartFindProvidersAsync") - providers := pqm.network.FindProvidersAsync(findProviderCtx, k, maxProviders) + // We set count == 0. We will cancel the query + // manually once we have enough. This assumes the + // ContentDiscovery implementation does that, which a + // requirement per the libp2p/core/routing interface. + providers := pqm.router.FindProvidersAsync(findProviderCtx, k, 0) wg := &sync.WaitGroup{} for p := range providers { wg.Add(1) - go func(p peer.ID) { + go func(p peer.AddrInfo) { defer wg.Done() - span.AddEvent("FoundProvider", trace.WithAttributes(attribute.Stringer("peer", p))) - err := pqm.network.ConnectTo(findProviderCtx, p) - if err != nil { - span.RecordError(err, trace.WithAttributes(attribute.Stringer("peer", p))) - log.Debugf("failed to connect to provider %s: %s", p, err) + span.AddEvent("FoundProvider", trace.WithAttributes(attribute.Stringer("peer", p.ID))) + err := pqm.dialer.Connect(findProviderCtx, p) + if err != nil && err != swarm.ErrDialToSelf { + span.RecordError(err, trace.WithAttributes(attribute.Stringer("peer", p.ID))) + log.Debugf("failed to connect to provider %s: %s", p.ID, err) return } - span.AddEvent("ConnectedToProvider", trace.WithAttributes(attribute.Stringer("peer", p))) + span.AddEvent("ConnectedToProvider", trace.WithAttributes(attribute.Stringer("peer", p.ID))) select { case pqm.providerQueryMessages <- &receivedProviderMessage{ ctx: fpr.ctx, @@ -334,7 +418,7 @@ func (pqm *ProviderQueryManager) run() { defer pqm.cleanupInProcessRequests() go pqm.providerRequestBufferWorker() - for i := 0; i < maxInProcessRequests; i++ { + for i := 0; i < pqm.maxInProcessRequests; i++ { go pqm.findProviderWorker() } @@ -403,7 +487,7 @@ func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { ctx = trace.ContextWithSpan(ctx, span) requestStatus = &inProgressRequestStatus{ - listeners: make(map[chan peer.ID]struct{}), + listeners: make(map[chan peer.AddrInfo]struct{}), ctx: ctx, cancelFn: cancelFn, } @@ -421,7 +505,7 @@ func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { } else { trace.SpanFromContext(npqm.ctx).AddEvent("JoinQuery", trace.WithAttributes(attribute.Stringer("cid", npqm.k))) } - inProgressChan := make(chan peer.ID) + inProgressChan := make(chan peer.AddrInfo) requestStatus.listeners[inProgressChan] = struct{}{} select { case npqm.inProgressRequestChan <- inProgressRequest{ diff --git a/bitswap/client/internal/providerquerymanager/providerquerymanager_test.go b/routing/providerquerymanager/providerquerymanager_test.go similarity index 69% rename from bitswap/client/internal/providerquerymanager/providerquerymanager_test.go rename to routing/providerquerymanager/providerquerymanager_test.go index 9deb77f99..b55c1debc 100644 --- a/bitswap/client/internal/providerquerymanager/providerquerymanager_test.go +++ b/routing/providerquerymanager/providerquerymanager_test.go @@ -13,27 +13,30 @@ import ( "github.com/libp2p/go-libp2p/core/peer" ) -type fakeProviderNetwork struct { +type fakeProviderDialer struct { + connectError error + connectDelay time.Duration +} + +type fakeProviderDiscovery struct { peersFound []peer.ID - connectError error delay time.Duration - connectDelay time.Duration queriesMadeMutex sync.RWMutex queriesMade int liveQueries int } -func (fpn *fakeProviderNetwork) ConnectTo(context.Context, peer.ID) error { - time.Sleep(fpn.connectDelay) - return fpn.connectError +func (fpd *fakeProviderDialer) Connect(context.Context, peer.AddrInfo) error { + time.Sleep(fpd.connectDelay) + return fpd.connectError } -func (fpn *fakeProviderNetwork) FindProvidersAsync(ctx context.Context, k cid.Cid, max int) <-chan peer.ID { +func (fpn *fakeProviderDiscovery) FindProvidersAsync(ctx context.Context, k cid.Cid, max int) <-chan peer.AddrInfo { fpn.queriesMadeMutex.Lock() fpn.queriesMade++ fpn.liveQueries++ fpn.queriesMadeMutex.Unlock() - incomingPeers := make(chan peer.ID) + incomingPeers := make(chan peer.AddrInfo) go func() { defer close(incomingPeers) for _, p := range fpn.peersFound { @@ -44,7 +47,7 @@ func (fpn *fakeProviderNetwork) FindProvidersAsync(ctx context.Context, k cid.Ci default: } select { - case incomingPeers <- p: + case incomingPeers <- peer.AddrInfo{ID: p}: case <-ctx.Done(): return } @@ -57,28 +60,36 @@ func (fpn *fakeProviderNetwork) FindProvidersAsync(ctx context.Context, k cid.Ci return incomingPeers } +func mustNotErr[T any](out T, err error) T { + if err != nil { + panic(err) + } + return out +} + func TestNormalSimultaneousFetch(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() keys := random.Cids(2) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() - firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0]) - secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[1]) + firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) + secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[1], 0) - var firstPeersReceived []peer.ID + var firstPeersReceived []peer.AddrInfo for p := range firstRequestChan { firstPeersReceived = append(firstPeersReceived, p) } - var secondPeersReceived []peer.ID + var secondPeersReceived []peer.AddrInfo for p := range secondRequestChan { secondPeersReceived = append(secondPeersReceived, p) } @@ -96,26 +107,27 @@ func TestNormalSimultaneousFetch(t *testing.T) { func TestDedupingProviderRequests(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() key := random.Cids(1)[0] sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() - firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key) - secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key) + firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) + secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) - var firstPeersReceived []peer.ID + var firstPeersReceived []peer.AddrInfo for p := range firstRequestChan { firstPeersReceived = append(firstPeersReceived, p) } - var secondPeersReceived []peer.ID + var secondPeersReceived []peer.AddrInfo for p := range secondRequestChan { secondPeersReceived = append(secondPeersReceived, p) } @@ -136,12 +148,13 @@ func TestDedupingProviderRequests(t *testing.T) { func TestCancelOneRequestDoesNotTerminateAnother(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() key := random.Cids(1)[0] @@ -149,17 +162,17 @@ func TestCancelOneRequestDoesNotTerminateAnother(t *testing.T) { // first session will cancel before done firstSessionCtx, firstCancel := context.WithTimeout(ctx, 3*time.Millisecond) defer firstCancel() - firstRequestChan := providerQueryManager.FindProvidersAsync(firstSessionCtx, key) + firstRequestChan := providerQueryManager.FindProvidersAsync(firstSessionCtx, key, 0) secondSessionCtx, secondCancel := context.WithTimeout(ctx, 5*time.Second) defer secondCancel() - secondRequestChan := providerQueryManager.FindProvidersAsync(secondSessionCtx, key) + secondRequestChan := providerQueryManager.FindProvidersAsync(secondSessionCtx, key, 0) - var firstPeersReceived []peer.ID + var firstPeersReceived []peer.AddrInfo for p := range firstRequestChan { firstPeersReceived = append(firstPeersReceived, p) } - var secondPeersReceived []peer.ID + var secondPeersReceived []peer.AddrInfo for p := range secondRequestChan { secondPeersReceived = append(secondPeersReceived, p) } @@ -180,29 +193,30 @@ func TestCancelOneRequestDoesNotTerminateAnother(t *testing.T) { func TestCancelManagerExitsGracefully(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 1 * time.Millisecond, } ctx := context.Background() managerCtx, managerCancel := context.WithTimeout(ctx, 5*time.Millisecond) defer managerCancel() - providerQueryManager := New(managerCtx, fpn) + providerQueryManager := mustNotErr(New(managerCtx, fpd, fpn)) providerQueryManager.Startup() key := random.Cids(1)[0] sessionCtx, cancel := context.WithTimeout(ctx, 20*time.Millisecond) defer cancel() - firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key) - secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key) + firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) + secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) - var firstPeersReceived []peer.ID + var firstPeersReceived []peer.AddrInfo for p := range firstRequestChan { firstPeersReceived = append(firstPeersReceived, p) } - var secondPeersReceived []peer.ID + var secondPeersReceived []peer.AddrInfo for p := range secondRequestChan { secondPeersReceived = append(secondPeersReceived, p) } @@ -215,28 +229,30 @@ func TestCancelManagerExitsGracefully(t *testing.T) { func TestPeersWithConnectionErrorsNotAddedToPeerList(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ - peersFound: peers, + fpd := &fakeProviderDialer{ connectError: errors.New("not able to connect"), - delay: 1 * time.Millisecond, + } + fpn := &fakeProviderDiscovery{ + peersFound: peers, + delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() key := random.Cids(1)[0] sessionCtx, cancel := context.WithTimeout(ctx, 20*time.Millisecond) defer cancel() - firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key) - secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key) + firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) + secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) - var firstPeersReceived []peer.ID + var firstPeersReceived []peer.AddrInfo for p := range firstRequestChan { firstPeersReceived = append(firstPeersReceived, p) } - var secondPeersReceived []peer.ID + var secondPeersReceived []peer.AddrInfo for p := range secondRequestChan { secondPeersReceived = append(secondPeersReceived, p) } @@ -248,38 +264,39 @@ func TestPeersWithConnectionErrorsNotAddedToPeerList(t *testing.T) { func TestRateLimitingRequests(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 5 * time.Millisecond, } ctx := context.Background() ctx, cancel := context.WithCancel(ctx) defer cancel() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() - keys := random.Cids(maxInProcessRequests + 1) + keys := random.Cids(providerQueryManager.maxInProcessRequests + 1) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() - var requestChannels []<-chan peer.ID - for i := 0; i < maxInProcessRequests+1; i++ { - requestChannels = append(requestChannels, providerQueryManager.FindProvidersAsync(sessionCtx, keys[i])) + var requestChannels []<-chan peer.AddrInfo + for i := 0; i < providerQueryManager.maxInProcessRequests+1; i++ { + requestChannels = append(requestChannels, providerQueryManager.FindProvidersAsync(sessionCtx, keys[i], 0)) } time.Sleep(20 * time.Millisecond) fpn.queriesMadeMutex.Lock() - if fpn.liveQueries != maxInProcessRequests { + if fpn.liveQueries != providerQueryManager.maxInProcessRequests { t.Logf("Queries made: %d\n", fpn.liveQueries) t.Fatal("Did not limit parallel requests to rate limit") } fpn.queriesMadeMutex.Unlock() - for i := 0; i < maxInProcessRequests+1; i++ { + for i := 0; i < providerQueryManager.maxInProcessRequests+1; i++ { for range requestChannels[i] { } } fpn.queriesMadeMutex.Lock() defer fpn.queriesMadeMutex.Unlock() - if fpn.queriesMade != maxInProcessRequests+1 { + if fpn.queriesMade != providerQueryManager.maxInProcessRequests+1 { t.Logf("Queries made: %d\n", fpn.queriesMade) t.Fatal("Did not make all separate requests") } @@ -287,20 +304,21 @@ func TestRateLimitingRequests(t *testing.T) { func TestFindProviderTimeout(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 10 * time.Millisecond, } ctx := context.Background() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() - providerQueryManager.SetFindProviderTimeout(2 * time.Millisecond) + providerQueryManager.setFindProviderTimeout(2 * time.Millisecond) keys := random.Cids(1) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() - firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0]) - var firstPeersReceived []peer.ID + firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) + var firstPeersReceived []peer.AddrInfo for p := range firstRequestChan { firstPeersReceived = append(firstPeersReceived, p) } @@ -311,19 +329,20 @@ func TestFindProviderTimeout(t *testing.T) { func TestFindProviderPreCanceled(t *testing.T) { peers := random.Peers(10) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() - providerQueryManager.SetFindProviderTimeout(100 * time.Millisecond) + providerQueryManager.setFindProviderTimeout(100 * time.Millisecond) keys := random.Cids(1) sessionCtx, cancel := context.WithCancel(ctx) cancel() - firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0]) + firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) if firstRequestChan == nil { t.Fatal("expected non-nil channel") } @@ -336,18 +355,19 @@ func TestFindProviderPreCanceled(t *testing.T) { func TestCancelFindProvidersAfterCompletion(t *testing.T) { peers := random.Peers(2) - fpn := &fakeProviderNetwork{ + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ peersFound: peers, delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := New(ctx, fpn) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) providerQueryManager.Startup() - providerQueryManager.SetFindProviderTimeout(100 * time.Millisecond) + providerQueryManager.setFindProviderTimeout(100 * time.Millisecond) keys := random.Cids(1) sessionCtx, cancel := context.WithCancel(ctx) - firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0]) + firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) <-firstRequestChan // wait for everything to start. time.Sleep(10 * time.Millisecond) // wait for the incoming providres to stop. cancel() // cancel the context. @@ -365,3 +385,27 @@ func TestCancelFindProvidersAfterCompletion(t *testing.T) { } } } + +func TestLimitedProviders(t *testing.T) { + max := 5 + peers := random.Peers(10) + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ + peersFound: peers, + delay: 1 * time.Millisecond, + } + ctx := context.Background() + providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxProviders(max))) + providerQueryManager.Startup() + providerQueryManager.setFindProviderTimeout(100 * time.Millisecond) + keys := random.Cids(1) + + providersChan := providerQueryManager.FindProvidersAsync(ctx, keys[0], 0) + total := 0 + for range providersChan { + total++ + } + if total != max { + t.Fatal("returned more providers than requested") + } +} From e2d2f36bfb12e7cf7550760f9099baa3937ae34b Mon Sep 17 00:00:00 2001 From: Hlib Kanunnikov Date: Tue, 26 Nov 2024 00:18:53 +0100 Subject: [PATCH 14/36] fix(bitswap/client/msgq): prevent duplicate requests (#691) * fix(bitswap/client/msgq): prevent duplicate requests Previously, in-progress requests could be re-requested again during periodic rebroadcast. The queue requests, and while awaiting response, the rebroadcast event happens. Rebroadcast event changes previosly sent WANTs to pending and sends them again in a new message. The solution here is to ensure WANT was in sent status for long enough, before bringing it back to pending. This utilizes existing `sendAt` map which tracks when every CID was sent. * Attempt rebroadcast more frequently than once per rebroadcast interval --------- Co-authored-by: gammazero <11790789+gammazero@users.noreply.github.com> --- CHANGELOG.md | 2 + .../internal/messagequeue/messagequeue.go | 41 +++++++++++++---- bitswap/client/wantlist/wantlist.go | 10 ---- bitswap/client/wantlist/wantlist_test.go | 46 ------------------- 4 files changed, 34 insertions(+), 65 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 84e8558b5..87a33f260 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -67,6 +67,8 @@ The following emojis are used to highlight certain changes: ### Changed +- `routing/http/client`: creating delegated routing client with `New` now defaults to querying delegated routing server with `DefaultProtocolFilter` ([IPIP-484](https://github.com/ipfs/specs/pull/484)) [#689](https://github.com/ipfs/boxo/pull/689) +- `bitswap/client`: Wait at lease one broadcast interval before resending wants to a peer. Check for peers to rebroadcast to more often than one broadcast interval. - No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. [#710](https://github.com/ipfs/boxo/pull/710) ### Removed diff --git a/bitswap/client/internal/messagequeue/messagequeue.go b/bitswap/client/internal/messagequeue/messagequeue.go index 11c25089e..e8d8a4818 100644 --- a/bitswap/client/internal/messagequeue/messagequeue.go +++ b/bitswap/client/internal/messagequeue/messagequeue.go @@ -175,6 +175,23 @@ func (r *recallWantlist) ClearSentAt(c cid.Cid) { delete(r.sentAt, c) } +// Refresh moves wants from the sent list back to the pending list. +// If a want has been sent for longer than the interval, it is moved back to the pending list. +// Returns the number of wants that were refreshed. +func (r *recallWantlist) Refresh(now time.Time, interval time.Duration) int { + var refreshed int + for _, want := range r.sent.Entries() { + sentAt, ok := r.sentAt[want.Cid] + if ok && now.Sub(sentAt) >= interval { + r.pending.Add(want.Cid, want.Priority, want.WantType) + r.sent.Remove(want.Cid) + refreshed++ + } + } + + return refreshed +} + type peerConn struct { p peer.ID network MessageNetwork @@ -389,8 +406,10 @@ func (mq *MessageQueue) SetRebroadcastInterval(delay time.Duration) { // Startup starts the processing of messages and rebroadcasting. func (mq *MessageQueue) Startup() { + const checksPerInterval = 2 + mq.rebroadcastIntervalLk.Lock() - mq.rebroadcastTimer = mq.clock.Timer(mq.rebroadcastInterval) + mq.rebroadcastTimer = mq.clock.Timer(mq.rebroadcastInterval / checksPerInterval) mq.rebroadcastIntervalLk.Unlock() go mq.runQueue() } @@ -476,27 +495,31 @@ func (mq *MessageQueue) rebroadcastWantlist() { mq.rebroadcastIntervalLk.Unlock() // If some wants were transferred from the rebroadcast list - if mq.transferRebroadcastWants() { + if toRebroadcast := mq.transferRebroadcastWants(); toRebroadcast > 0 { // Send them out mq.sendMessage() + log.Infow("Rebroadcasting wants", "amount", toRebroadcast, "peer", mq.p) } } // Transfer wants from the rebroadcast lists into the pending lists. -func (mq *MessageQueue) transferRebroadcastWants() bool { +func (mq *MessageQueue) transferRebroadcastWants() int { mq.wllock.Lock() defer mq.wllock.Unlock() - // Check if there are any wants to rebroadcast if mq.bcstWants.sent.Len() == 0 && mq.peerWants.sent.Len() == 0 { - return false + return 0 } - // Copy sent wants into pending wants lists - mq.bcstWants.pending.Absorb(mq.bcstWants.sent) - mq.peerWants.pending.Absorb(mq.peerWants.sent) + mq.rebroadcastIntervalLk.Lock() + rebroadcastInterval := mq.rebroadcastInterval + mq.rebroadcastIntervalLk.Unlock() - return true + now := mq.clock.Now() + // Transfer sent wants into pending wants lists + transferred := mq.bcstWants.Refresh(now, rebroadcastInterval) + transferred += mq.peerWants.Refresh(now, rebroadcastInterval) + return transferred } func (mq *MessageQueue) signalWorkReady() { diff --git a/bitswap/client/wantlist/wantlist.go b/bitswap/client/wantlist/wantlist.go index 6cb71eecc..245085af9 100644 --- a/bitswap/client/wantlist/wantlist.go +++ b/bitswap/client/wantlist/wantlist.go @@ -130,13 +130,3 @@ func (w *Wantlist) Entries() []Entry { w.cached = es return es[0:len(es):len(es)] } - -// Absorb all the entries in other into this want list -func (w *Wantlist) Absorb(other *Wantlist) { - // Invalidate the cache up-front to avoid doing any work trying to keep it up-to-date. - w.cached = nil - - for _, e := range other.Entries() { - w.Add(e.Cid, e.Priority, e.WantType) - } -} diff --git a/bitswap/client/wantlist/wantlist_test.go b/bitswap/client/wantlist/wantlist_test.go index 07d4ce415..901fe0d67 100644 --- a/bitswap/client/wantlist/wantlist_test.go +++ b/bitswap/client/wantlist/wantlist_test.go @@ -157,52 +157,6 @@ func TestAddBlockThenRemoveAny(t *testing.T) { } } -func TestAbsort(t *testing.T) { - wl := New() - wl.Add(testcids[0], 5, pb.Message_Wantlist_Block) - wl.Add(testcids[1], 4, pb.Message_Wantlist_Have) - wl.Add(testcids[2], 3, pb.Message_Wantlist_Have) - - wl2 := New() - wl2.Add(testcids[0], 2, pb.Message_Wantlist_Have) - wl2.Add(testcids[1], 1, pb.Message_Wantlist_Block) - - wl.Absorb(wl2) - - e, ok := wl.Contains(testcids[0]) - if !ok { - t.Fatal("expected to have ", testcids[0]) - } - if e.Priority != 5 { - t.Fatal("expected priority 5") - } - if e.WantType != pb.Message_Wantlist_Block { - t.Fatal("expected type ", pb.Message_Wantlist_Block) - } - - e, ok = wl.Contains(testcids[1]) - if !ok { - t.Fatal("expected to have ", testcids[1]) - } - if e.Priority != 1 { - t.Fatal("expected priority 1") - } - if e.WantType != pb.Message_Wantlist_Block { - t.Fatal("expected type ", pb.Message_Wantlist_Block) - } - - e, ok = wl.Contains(testcids[2]) - if !ok { - t.Fatal("expected to have ", testcids[2]) - } - if e.Priority != 3 { - t.Fatal("expected priority 3") - } - if e.WantType != pb.Message_Wantlist_Have { - t.Fatal("expected type ", pb.Message_Wantlist_Have) - } -} - func TestSortEntries(t *testing.T) { wl := New() From 91c4d505780bc94877acf3832c22ee13205900b1 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Mon, 25 Nov 2024 19:25:47 -1000 Subject: [PATCH 15/36] Tests can signal immediate rebroadcast (#726) * Tests can signal immediate rebroadcast - No more need to update a synchronizd timer. * Use zero interval for immediate rebroadcast * simplify logic --- .../internal/messagequeue/messagequeue.go | 122 ++++++++---------- .../messagequeue/messagequeue_test.go | 18 +-- 2 files changed, 56 insertions(+), 84 deletions(-) diff --git a/bitswap/client/internal/messagequeue/messagequeue.go b/bitswap/client/internal/messagequeue/messagequeue.go index e8d8a4818..0b9dc249e 100644 --- a/bitswap/client/internal/messagequeue/messagequeue.go +++ b/bitswap/client/internal/messagequeue/messagequeue.go @@ -24,28 +24,30 @@ var ( ) const ( - defaultRebroadcastInterval = 30 * time.Second - // maxRetries is the number of times to attempt to send a message before - // giving up - maxRetries = 3 - sendTimeout = 30 * time.Second // maxMessageSize is the maximum message size in bytes maxMessageSize = 1024 * 1024 * 2 + // maxPriority is the max priority as defined by the bitswap protocol + maxPriority = math.MaxInt32 + // maxRetries is the number of times to attempt to send a message before + // giving up + maxRetries = 3 + // The maximum amount of time in which to accept a response as being valid + // for latency calculation (as opposed to discarding it as an outlier) + maxValidLatency = 30 * time.Second + // rebroadcastInterval is the minimum amount of time that must elapse before + // resending wants to a peer + rebroadcastInterval = 30 * time.Second // sendErrorBackoff is the time to wait before retrying to connect after // an error when trying to send a message sendErrorBackoff = 100 * time.Millisecond - // maxPriority is the max priority as defined by the bitswap protocol - maxPriority = math.MaxInt32 - // sendMessageDebounce is the debounce duration when calling sendMessage() - sendMessageDebounce = time.Millisecond // when we reach sendMessageCutoff wants/cancels, we'll send the message immediately. sendMessageCutoff = 256 + // sendMessageDebounce is the debounce duration when calling sendMessage() + sendMessageDebounce = time.Millisecond // when we debounce for more than sendMessageMaxDelay, we'll send the // message immediately. sendMessageMaxDelay = 20 * time.Millisecond - // The maximum amount of time in which to accept a response as being valid - // for latency calculation (as opposed to discarding it as an outlier) - maxValidLatency = 30 * time.Second + sendTimeout = 30 * time.Second ) // MessageNetwork is any network that can connect peers and generate a message @@ -92,10 +94,8 @@ type MessageQueue struct { priority int32 // Dont touch any of these variables outside of run loop - sender bsnet.MessageSender - rebroadcastIntervalLk sync.Mutex - rebroadcastInterval time.Duration - rebroadcastTimer *clock.Timer + sender bsnet.MessageSender + rebroadcastNow chan struct{} // For performance reasons we just clear out the fields of the message // instead of creating a new one every time. msg bsmsg.BitSwapMessage @@ -263,21 +263,21 @@ func newMessageQueue( ) *MessageQueue { ctx, cancel := context.WithCancel(ctx) return &MessageQueue{ - ctx: ctx, - shutdown: cancel, - p: p, - network: network, - dhTimeoutMgr: dhTimeoutMgr, - maxMessageSize: maxMsgSize, - bcstWants: newRecallWantList(), - peerWants: newRecallWantList(), - cancels: cid.NewSet(), - outgoingWork: make(chan time.Time, 1), - responses: make(chan []cid.Cid, 8), - rebroadcastInterval: defaultRebroadcastInterval, - sendErrorBackoff: sendErrorBackoff, - maxValidLatency: maxValidLatency, - priority: maxPriority, + ctx: ctx, + shutdown: cancel, + p: p, + network: network, + dhTimeoutMgr: dhTimeoutMgr, + maxMessageSize: maxMsgSize, + bcstWants: newRecallWantList(), + peerWants: newRecallWantList(), + cancels: cid.NewSet(), + outgoingWork: make(chan time.Time, 1), + responses: make(chan []cid.Cid, 8), + rebroadcastNow: make(chan struct{}), + sendErrorBackoff: sendErrorBackoff, + maxValidLatency: maxValidLatency, + priority: maxPriority, // For performance reasons we just clear out the fields of the message // after using it, instead of creating a new one every time. msg: bsmsg.New(false), @@ -394,23 +394,15 @@ func (mq *MessageQueue) ResponseReceived(ks []cid.Cid) { } } -// SetRebroadcastInterval sets a new interval on which to rebroadcast the full wantlist -func (mq *MessageQueue) SetRebroadcastInterval(delay time.Duration) { - mq.rebroadcastIntervalLk.Lock() - mq.rebroadcastInterval = delay - if mq.rebroadcastTimer != nil { - mq.rebroadcastTimer.Reset(delay) +func (mq *MessageQueue) RebroadcastNow() { + select { + case mq.rebroadcastNow <- struct{}{}: + case <-mq.ctx.Done(): } - mq.rebroadcastIntervalLk.Unlock() } // Startup starts the processing of messages and rebroadcasting. func (mq *MessageQueue) Startup() { - const checksPerInterval = 2 - - mq.rebroadcastIntervalLk.Lock() - mq.rebroadcastTimer = mq.clock.Timer(mq.rebroadcastInterval / checksPerInterval) - mq.rebroadcastIntervalLk.Unlock() go mq.runQueue() } @@ -430,6 +422,8 @@ func (mq *MessageQueue) onShutdown() { } func (mq *MessageQueue) runQueue() { + const runRebroadcastsInterval = rebroadcastInterval / 2 + defer mq.onShutdown() // Create a timer for debouncing scheduled work. @@ -440,11 +434,18 @@ func (mq *MessageQueue) runQueue() { <-scheduleWork.C } + rebroadcastTimer := mq.clock.Timer(runRebroadcastsInterval) + defer rebroadcastTimer.Stop() + var workScheduled time.Time for { select { - case <-mq.rebroadcastTimer.C: - mq.rebroadcastWantlist() + case now := <-rebroadcastTimer.C: + mq.rebroadcastWantlist(now, rebroadcastInterval) + rebroadcastTimer.Reset(runRebroadcastsInterval) + + case <-mq.rebroadcastNow: + mq.rebroadcastWantlist(mq.clock.Now(), 0) case when := <-mq.outgoingWork: // If we have work scheduled, cancel the timer. If we @@ -489,39 +490,20 @@ func (mq *MessageQueue) runQueue() { } // Periodically resend the list of wants to the peer -func (mq *MessageQueue) rebroadcastWantlist() { - mq.rebroadcastIntervalLk.Lock() - mq.rebroadcastTimer.Reset(mq.rebroadcastInterval) - mq.rebroadcastIntervalLk.Unlock() +func (mq *MessageQueue) rebroadcastWantlist(now time.Time, interval time.Duration) { + mq.wllock.Lock() + // Transfer wants from the rebroadcast lists into the pending lists. + toRebroadcast := mq.bcstWants.Refresh(now, interval) + mq.peerWants.Refresh(now, interval) + mq.wllock.Unlock() // If some wants were transferred from the rebroadcast list - if toRebroadcast := mq.transferRebroadcastWants(); toRebroadcast > 0 { + if toRebroadcast > 0 { // Send them out mq.sendMessage() log.Infow("Rebroadcasting wants", "amount", toRebroadcast, "peer", mq.p) } } -// Transfer wants from the rebroadcast lists into the pending lists. -func (mq *MessageQueue) transferRebroadcastWants() int { - mq.wllock.Lock() - defer mq.wllock.Unlock() - - if mq.bcstWants.sent.Len() == 0 && mq.peerWants.sent.Len() == 0 { - return 0 - } - - mq.rebroadcastIntervalLk.Lock() - rebroadcastInterval := mq.rebroadcastInterval - mq.rebroadcastIntervalLk.Unlock() - - now := mq.clock.Now() - // Transfer sent wants into pending wants lists - transferred := mq.bcstWants.Refresh(now, rebroadcastInterval) - transferred += mq.peerWants.Refresh(now, rebroadcastInterval) - return transferred -} - func (mq *MessageQueue) signalWorkReady() { select { case mq.outgoingWork <- mq.clock.Now(): diff --git a/bitswap/client/internal/messagequeue/messagequeue_test.go b/bitswap/client/internal/messagequeue/messagequeue_test.go index 1073a9f74..dee2aa58c 100644 --- a/bitswap/client/internal/messagequeue/messagequeue_test.go +++ b/bitswap/client/internal/messagequeue/messagequeue_test.go @@ -432,9 +432,7 @@ func TestWantlistRebroadcast(t *testing.T) { t.Fatal("wrong number of wants") } - // Tell message queue to rebroadcast after 5ms, then wait 8ms - messageQueue.SetRebroadcastInterval(5 * time.Millisecond) - clock.Add(8 * time.Millisecond) + messageQueue.RebroadcastNow() message = <-messagesSent expectEvent(t, events, messageFinishedSending) @@ -443,10 +441,7 @@ func TestWantlistRebroadcast(t *testing.T) { t.Fatal("did not rebroadcast all wants") } - // Tell message queue to rebroadcast after a long time (so it doesn't - // interfere with the next message collection), then send out some - // regular wants and collect them - messageQueue.SetRebroadcastInterval(1 * time.Second) + // Send out some regular wants and collect them messageQueue.AddWants(wantBlocks, wantHaves) expectEvent(t, events, messageQueued) clock.Add(10 * time.Millisecond) @@ -464,9 +459,7 @@ func TestWantlistRebroadcast(t *testing.T) { default: } - // Tell message queue to rebroadcast after 10ms, then wait 15ms - messageQueue.SetRebroadcastInterval(10 * time.Millisecond) - clock.Add(15 * time.Millisecond) + messageQueue.RebroadcastNow() message = <-messagesSent expectEvent(t, events, messageFinishedSending) @@ -477,7 +470,6 @@ func TestWantlistRebroadcast(t *testing.T) { } // Cancel some of the wants - messageQueue.SetRebroadcastInterval(1 * time.Second) cancels := append([]cid.Cid{bcstwh[0]}, wantHaves[0], wantBlocks[0]) messageQueue.AddCancels(cancels) expectEvent(t, events, messageQueued) @@ -501,9 +493,7 @@ func TestWantlistRebroadcast(t *testing.T) { } } - // Tell message queue to rebroadcast after 10ms, then wait 15ms - messageQueue.SetRebroadcastInterval(10 * time.Millisecond) - clock.Add(15 * time.Millisecond) + messageQueue.RebroadcastNow() message = <-messagesSent expectEvent(t, events, messageFinishedSending) From 86735605944be46d97ca0c8e1d8e326069f22a23 Mon Sep 17 00:00:00 2001 From: Hector Sanjuan Date: Tue, 26 Nov 2024 16:31:54 +0100 Subject: [PATCH 16/36] dspinner: RecursiveKeys(): do not hang on cancellations (#727) Per https://github.com/ipfs/kubo/issues/10593, if no one is reading from the channel returned by RecursiveKeys() and the context is cancelled, streamIndex will hang indefinitely. Proposed fix is to always select when attempting to write to the `out` channel. If the context is done and there is no one to read, we can abort. Co-authored-by: Andrew Gillis <11790789+gammazero@users.noreply.github.com> --- CHANGELOG.md | 3 ++- pinning/pinner/dspinner/pin.go | 19 +++++++++++++------ 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 87a33f260..a7951b656 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -129,7 +129,8 @@ The following emojis are used to highlight certain changes: ### Fixed -- `unixfs/hamt` Log error instead of panic if both link and shard are nil [#393](https://github.com/ipfs/boxo/pull/393) +- `unixfs/hamt`: Log error instead of panic if both link and shard are nil [#393](https://github.com/ipfs/boxo/pull/393) +- `pinner/dspinner`: do not hang when listing keys and the `out` channel is no longer read [#727](https://github.com/ipfs/boxo/pull/727) ### Security diff --git a/pinning/pinner/dspinner/pin.go b/pinning/pinner/dspinner/pin.go index bc1f61902..ddc93c2c5 100644 --- a/pinning/pinner/dspinner/pin.go +++ b/pinning/pinner/dspinner/pin.go @@ -707,11 +707,19 @@ func (p *pinner) streamIndex(ctx context.Context, index dsindex.Indexer, detaile defer p.lock.RUnlock() cidSet := cid.NewSet() + send := func(sp ipfspinner.StreamedPin) (ok bool) { + select { + case <-ctx.Done(): + return false + case out <- sp: + return true + } + } err := index.ForEach(ctx, "", func(key, value string) bool { c, err := cid.Cast([]byte(key)) if err != nil { - out <- ipfspinner.StreamedPin{Err: err} + send(ipfspinner.StreamedPin{Err: err}) return false } @@ -719,7 +727,7 @@ func (p *pinner) streamIndex(ctx context.Context, index dsindex.Indexer, detaile if detailed { pp, err := p.loadPin(ctx, value) if err != nil { - out <- ipfspinner.StreamedPin{Err: err} + send(ipfspinner.StreamedPin{Err: err}) return false } @@ -731,17 +739,16 @@ func (p *pinner) streamIndex(ctx context.Context, index dsindex.Indexer, detaile } if !cidSet.Has(c) { - select { - case <-ctx.Done(): + if !send(ipfspinner.StreamedPin{Pin: pin}) { return false - case out <- ipfspinner.StreamedPin{Pin: pin}: } cidSet.Add(c) } return true }) if err != nil { - out <- ipfspinner.StreamedPin{Err: err} + send(ipfspinner.StreamedPin{Err: err}) + return } }() From 821f53924d45706f0a686f7b4d6f3667629926ad Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Wed, 27 Nov 2024 02:24:15 -1000 Subject: [PATCH 17/36] chore: minor Improvements to providerquerymanager (#728) * Replace providerRequestBufferWorker with ChanQueue * Set inProgressRequestStatuses map to nil when empty * Remove setFindProviderTimeout and associated mutex. This can be set at creation time using WithMaxTimeout option. * Replace forever-appended-to slice with circular buffer * Increase default number of concurrent finds * Optionally allow an unlimited number of concurrent find requests --- examples/go.mod | 2 + examples/go.sum | 4 + go.mod | 2 + go.sum | 4 + .../providerquerymanager.go | 183 ++++++++---------- .../providerquerymanager_test.go | 68 +++++-- 6 files changed, 149 insertions(+), 114 deletions(-) diff --git a/examples/go.mod b/examples/go.mod index c7c389626..16b6faa3e 100644 --- a/examples/go.mod +++ b/examples/go.mod @@ -43,6 +43,8 @@ require ( github.com/flynn/noise v1.1.0 // indirect github.com/francoispqt/gojay v1.2.13 // indirect github.com/gabriel-vasile/mimetype v1.4.6 // indirect + github.com/gammazero/chanqueue v1.0.0 // indirect + github.com/gammazero/deque v1.0.0 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-task/slim-sprig/v3 v3.0.0 // indirect diff --git a/examples/go.sum b/examples/go.sum index 8d7be4a96..401e43184 100644 --- a/examples/go.sum +++ b/examples/go.sum @@ -77,6 +77,10 @@ github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7z github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/gabriel-vasile/mimetype v1.4.6 h1:3+PzJTKLkvgjeTbts6msPJt4DixhT4YtFNf1gtGe3zc= github.com/gabriel-vasile/mimetype v1.4.6/go.mod h1:JX1qVKqZd40hUPpAfiNTe0Sne7hdfKSbOqqmkq8GCXc= +github.com/gammazero/chanqueue v1.0.0 h1:FER/sMailGFA3DDvFooEkipAMU+3c9Bg3bheloPSz6o= +github.com/gammazero/chanqueue v1.0.0/go.mod h1:fMwpwEiuUgpab0sH4VHiVcEoji1pSi+EIzeG4TPeKPc= +github.com/gammazero/deque v1.0.0 h1:LTmimT8H7bXkkCy6gZX7zNLtkbz4NdS2z8LZuor3j34= +github.com/gammazero/deque v1.0.0/go.mod h1:iflpYvtGfM3U8S8j+sZEKIak3SAKYpA5/SQewgfXDKo= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= diff --git a/go.mod b/go.mod index db188a203..292fbb95a 100644 --- a/go.mod +++ b/go.mod @@ -12,6 +12,8 @@ require ( github.com/cskr/pubsub v1.0.2 github.com/dustin/go-humanize v1.0.1 github.com/gabriel-vasile/mimetype v1.4.6 + github.com/gammazero/chanqueue v1.0.0 + github.com/gammazero/deque v1.0.0 github.com/gogo/protobuf v1.3.2 github.com/google/uuid v1.6.0 github.com/gorilla/mux v1.8.1 diff --git a/go.sum b/go.sum index f3c5da72b..2e41a5014 100644 --- a/go.sum +++ b/go.sum @@ -77,6 +77,10 @@ github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7z github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/gabriel-vasile/mimetype v1.4.6 h1:3+PzJTKLkvgjeTbts6msPJt4DixhT4YtFNf1gtGe3zc= github.com/gabriel-vasile/mimetype v1.4.6/go.mod h1:JX1qVKqZd40hUPpAfiNTe0Sne7hdfKSbOqqmkq8GCXc= +github.com/gammazero/chanqueue v1.0.0 h1:FER/sMailGFA3DDvFooEkipAMU+3c9Bg3bheloPSz6o= +github.com/gammazero/chanqueue v1.0.0/go.mod h1:fMwpwEiuUgpab0sH4VHiVcEoji1pSi+EIzeG4TPeKPc= +github.com/gammazero/deque v1.0.0 h1:LTmimT8H7bXkkCy6gZX7zNLtkbz4NdS2z8LZuor3j34= +github.com/gammazero/deque v1.0.0/go.mod h1:iflpYvtGfM3U8S8j+sZEKIak3SAKYpA5/SQewgfXDKo= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= diff --git a/routing/providerquerymanager/providerquerymanager.go b/routing/providerquerymanager/providerquerymanager.go index d9005020e..55880ecd9 100644 --- a/routing/providerquerymanager/providerquerymanager.go +++ b/routing/providerquerymanager/providerquerymanager.go @@ -5,6 +5,8 @@ import ( "sync" "time" + "github.com/gammazero/chanqueue" + "github.com/gammazero/deque" "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" peer "github.com/libp2p/go-libp2p/core/peer" @@ -17,7 +19,7 @@ import ( var log = logging.Logger("routing/provqrymgr") const ( - defaultMaxInProcessRequests = 6 + defaultMaxInProcessRequests = 16 defaultMaxProviders = 0 defaultTimeout = 10 * time.Second ) @@ -82,15 +84,13 @@ type cancelRequestMessage struct { // - ensure two findprovider calls for the same block don't run concurrently // - manage timeouts type ProviderQueryManager struct { - ctx context.Context - dialer ProviderQueryDialer - router ProviderQueryRouter - providerQueryMessages chan providerQueryMessage - providerRequestsProcessing chan *findProviderRequest - incomingFindProviderRequests chan *findProviderRequest + ctx context.Context + dialer ProviderQueryDialer + router ProviderQueryRouter + providerQueryMessages chan providerQueryMessage + providerRequestsProcessing *chanqueue.ChanQueue[*findProviderRequest] findProviderTimeout time.Duration - timeoutMutex sync.RWMutex maxProviders int maxInProcessRequests int @@ -108,7 +108,9 @@ func WithMaxTimeout(timeout time.Duration) Option { } } -// WithMaxInProcessRequests is the maximum number of requests that can be processed in parallel +// WithMaxInProcessRequests is the maximum number of requests that can be +// processed in parallel. If this is 0, then the number is unlimited. Default +// is defaultMaxInProcessRequests (16). func WithMaxInProcessRequests(count int) Option { return func(mgr *ProviderQueryManager) error { mgr.maxInProcessRequests = count @@ -117,7 +119,7 @@ func WithMaxInProcessRequests(count int) Option { } // WithMaxProviders is the maximum number of providers that will be looked up -// per query. We only return providers that we can connect to. Defaults to 0, +// per query. We only return providers that we can connect to. Defaults to 0, // which means unbounded. func WithMaxProviders(count int) Option { return func(mgr *ProviderQueryManager) error { @@ -130,16 +132,13 @@ func WithMaxProviders(count int) Option { // network provider. func New(ctx context.Context, dialer ProviderQueryDialer, router ProviderQueryRouter, opts ...Option) (*ProviderQueryManager, error) { pqm := &ProviderQueryManager{ - ctx: ctx, - dialer: dialer, - router: router, - providerQueryMessages: make(chan providerQueryMessage, 16), - providerRequestsProcessing: make(chan *findProviderRequest), - incomingFindProviderRequests: make(chan *findProviderRequest), - inProgressRequestStatuses: make(map[cid.Cid]*inProgressRequestStatus), - findProviderTimeout: defaultTimeout, - maxInProcessRequests: defaultMaxInProcessRequests, - maxProviders: defaultMaxProviders, + ctx: ctx, + dialer: dialer, + router: router, + providerQueryMessages: make(chan providerQueryMessage), + findProviderTimeout: defaultTimeout, + maxInProcessRequests: defaultMaxInProcessRequests, + maxProviders: defaultMaxProviders, } for _, o := range opts { @@ -161,13 +160,6 @@ type inProgressRequest struct { incoming chan peer.AddrInfo } -// setFindProviderTimeout changes the timeout for finding providers -func (pqm *ProviderQueryManager) setFindProviderTimeout(findProviderTimeout time.Duration) { - pqm.timeoutMutex.Lock() - pqm.findProviderTimeout = findProviderTimeout - pqm.timeoutMutex.Unlock() -} - // FindProvidersAsync finds providers for the given block. The max parameter // controls how many will be returned at most. For a provider to be returned, // we must have successfully connected to it. Setting max to 0 will use the @@ -216,32 +208,36 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, } func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k cid.Cid, max int, receivedInProgressRequest inProgressRequest, onCloseFn func()) <-chan peer.AddrInfo { - // maintains an unbuffered queue for incoming providers for given request for a given session - // essentially, as a provider comes in, for a given CID, we want to immediately broadcast to all - // sessions that queried that CID, without worrying about whether the client code is actually - // reading from the returned channel -- so that the broadcast never blocks - // based on: https://medium.com/capital-one-tech/building-an-unbounded-channel-in-go-789e175cd2cd + // maintains an unbuffered queue for incoming providers for given request + // for a given session. Essentially, as a provider comes in, for a given + // CID, immediately broadcast to all sessions that queried that CID, + // without worrying about whether the client code is actually reading from + // the returned channel -- so that the broadcast never blocks. returnedProviders := make(chan peer.AddrInfo) - receivedProviders := append([]peer.AddrInfo(nil), receivedInProgressRequest.providersSoFar[0:]...) + var receivedProviders deque.Deque[peer.AddrInfo] + receivedProviders.Grow(len(receivedInProgressRequest.providersSoFar)) + for _, addrInfo := range receivedInProgressRequest.providersSoFar { + receivedProviders.PushBack(addrInfo) + } incomingProviders := receivedInProgressRequest.incoming // count how many providers we received from our workers etc. // these providers should be peers we managed to connect to. - total := len(receivedProviders) + total := receivedProviders.Len() go func() { defer close(returnedProviders) defer onCloseFn() outgoingProviders := func() chan<- peer.AddrInfo { - if len(receivedProviders) == 0 { + if receivedProviders.Len() == 0 { return nil } return returnedProviders } nextProvider := func() peer.AddrInfo { - if len(receivedProviders) == 0 { + if receivedProviders.Len() == 0 { return peer.AddrInfo{} } - return receivedProviders[0] + return receivedProviders.Front() } stopWhenMaxReached := func() { @@ -258,7 +254,7 @@ func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k // need. stopWhenMaxReached() - for len(receivedProviders) > 0 || incomingProviders != nil { + for receivedProviders.Len() > 0 || incomingProviders != nil { select { case <-pqm.ctx.Done(): return @@ -271,7 +267,7 @@ func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k if !ok { incomingProviders = nil } else { - receivedProviders = append(receivedProviders, provider) + receivedProviders.PushBack(provider) total++ stopWhenMaxReached() // we do not return, we will loop on @@ -281,7 +277,7 @@ func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k // via returnedProviders } case outgoingProviders() <- nextProvider(): - receivedProviders = receivedProviders[1:] + receivedProviders.PopFront() } } }() @@ -310,27 +306,42 @@ func (pqm *ProviderQueryManager) cancelProviderRequest(ctx context.Context, k ci } } +// findProviderWorker cycles through incoming provider queries one at a time. func (pqm *ProviderQueryManager) findProviderWorker() { - // findProviderWorker just cycles through incoming provider queries one - // at a time. We have six of these workers running at once - // to let requests go in parallel but keep them rate limited - for { - select { - case fpr, ok := <-pqm.providerRequestsProcessing: - if !ok { + var findSem chan struct{} + // If limiting the number of concurrent requests, create a counting + // semaphore to enforce this limit. + if pqm.maxInProcessRequests > 0 { + findSem = make(chan struct{}, pqm.maxInProcessRequests) + } + + // Read find provider requests until channel is closed. The channl is + // closed as soon as pqm.ctx is canceled, so there is no need to select on + // that context here. + for fpr := range pqm.providerRequestsProcessing.Out() { + if findSem != nil { + select { + case findSem <- struct{}{}: + case <-pqm.ctx.Done(): return } - k := fpr.k + } + + go func(ctx context.Context, k cid.Cid) { + if findSem != nil { + defer func() { + <-findSem + }() + } + log.Debugf("Beginning Find Provider Request for cid: %s", k.String()) - pqm.timeoutMutex.RLock() - findProviderCtx, cancel := context.WithTimeout(fpr.ctx, pqm.findProviderTimeout) - pqm.timeoutMutex.RUnlock() + findProviderCtx, cancel := context.WithTimeout(ctx, pqm.findProviderTimeout) span := trace.SpanFromContext(findProviderCtx) span.AddEvent("StartFindProvidersAsync") - // We set count == 0. We will cancel the query - // manually once we have enough. This assumes the - // ContentDiscovery implementation does that, which a - // requirement per the libp2p/core/routing interface. + // We set count == 0. We will cancel the query manually once we + // have enough. This assumes the ContentDiscovery + // implementation does that, which a requirement per the + // libp2p/core/routing interface. providers := pqm.router.FindProvidersAsync(findProviderCtx, k, 0) wg := &sync.WaitGroup{} for p := range providers { @@ -347,7 +358,7 @@ func (pqm *ProviderQueryManager) findProviderWorker() { span.AddEvent("ConnectedToProvider", trace.WithAttributes(attribute.Stringer("peer", p.ID))) select { case pqm.providerQueryMessages <- &receivedProviderMessage{ - ctx: fpr.ctx, + ctx: ctx, k: k, p: p, }: @@ -360,48 +371,12 @@ func (pqm *ProviderQueryManager) findProviderWorker() { cancel() select { case pqm.providerQueryMessages <- &finishedProviderQueryMessage{ - ctx: fpr.ctx, + ctx: ctx, k: k, }: case <-pqm.ctx.Done(): } - case <-pqm.ctx.Done(): - return - } - } -} - -func (pqm *ProviderQueryManager) providerRequestBufferWorker() { - // the provider request buffer worker just maintains an unbounded - // buffer for incoming provider queries and dispatches to the find - // provider workers as they become available - // based on: https://medium.com/capital-one-tech/building-an-unbounded-channel-in-go-789e175cd2cd - var providerQueryRequestBuffer []*findProviderRequest - nextProviderQuery := func() *findProviderRequest { - if len(providerQueryRequestBuffer) == 0 { - return nil - } - return providerQueryRequestBuffer[0] - } - outgoingRequests := func() chan<- *findProviderRequest { - if len(providerQueryRequestBuffer) == 0 { - return nil - } - return pqm.providerRequestsProcessing - } - - for { - select { - case incomingRequest, ok := <-pqm.incomingFindProviderRequests: - if !ok { - return - } - providerQueryRequestBuffer = append(providerQueryRequestBuffer, incomingRequest) - case outgoingRequests() <- nextProviderQuery(): - providerQueryRequestBuffer = providerQueryRequestBuffer[1:] - case <-pqm.ctx.Done(): - return - } + }(fpr.ctx, fpr.k) } } @@ -417,10 +392,10 @@ func (pqm *ProviderQueryManager) cleanupInProcessRequests() { func (pqm *ProviderQueryManager) run() { defer pqm.cleanupInProcessRequests() - go pqm.providerRequestBufferWorker() - for i := 0; i < pqm.maxInProcessRequests; i++ { - go pqm.findProviderWorker() - } + pqm.providerRequestsProcessing = chanqueue.New[*findProviderRequest]() + defer pqm.providerRequestsProcessing.Shutdown() + + go pqm.findProviderWorker() for { select { @@ -469,6 +444,9 @@ func (fpqm *finishedProviderQueryMessage) handle(pqm *ProviderQueryManager) { close(listener) } delete(pqm.inProgressRequestStatuses, fpqm.k) + if len(pqm.inProgressRequestStatuses) == 0 { + pqm.inProgressRequestStatuses = nil + } requestStatus.cancelFn() } @@ -480,7 +458,6 @@ func (npqm *newProvideQueryMessage) debugMessage() { func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { requestStatus, ok := pqm.inProgressRequestStatuses[npqm.k] if !ok { - ctx, cancelFn := context.WithCancel(pqm.ctx) span := trace.SpanFromContext(npqm.ctx) span.AddEvent("NewQuery", trace.WithAttributes(attribute.Stringer("cid", npqm.k))) @@ -492,10 +469,13 @@ func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { cancelFn: cancelFn, } + if pqm.inProgressRequestStatuses == nil { + pqm.inProgressRequestStatuses = make(map[cid.Cid]*inProgressRequestStatus) + } pqm.inProgressRequestStatuses[npqm.k] = requestStatus select { - case pqm.incomingFindProviderRequests <- &findProviderRequest{ + case pqm.providerRequestsProcessing.In() <- &findProviderRequest{ k: npqm.k, ctx: ctx, }: @@ -536,6 +516,9 @@ func (crm *cancelRequestMessage) handle(pqm *ProviderQueryManager) { close(crm.incomingProviders) if len(requestStatus.listeners) == 0 { delete(pqm.inProgressRequestStatuses, crm.k) + if len(pqm.inProgressRequestStatuses) == 0 { + pqm.inProgressRequestStatuses = nil + } requestStatus.cancelFn() } } diff --git a/routing/providerquerymanager/providerquerymanager_test.go b/routing/providerquerymanager/providerquerymanager_test.go index b55c1debc..7369231de 100644 --- a/routing/providerquerymanager/providerquerymanager_test.go +++ b/routing/providerquerymanager/providerquerymanager_test.go @@ -263,6 +263,8 @@ func TestPeersWithConnectionErrorsNotAddedToPeerList(t *testing.T) { } func TestRateLimitingRequests(t *testing.T) { + const maxInProcessRequests = 6 + peers := random.Peers(10) fpd := &fakeProviderDialer{} fpn := &fakeProviderDiscovery{ @@ -272,31 +274,73 @@ func TestRateLimitingRequests(t *testing.T) { ctx := context.Background() ctx, cancel := context.WithCancel(ctx) defer cancel() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxInProcessRequests(maxInProcessRequests))) providerQueryManager.Startup() - keys := random.Cids(providerQueryManager.maxInProcessRequests + 1) + keys := random.Cids(maxInProcessRequests + 1) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() var requestChannels []<-chan peer.AddrInfo - for i := 0; i < providerQueryManager.maxInProcessRequests+1; i++ { + for i := 0; i < maxInProcessRequests+1; i++ { requestChannels = append(requestChannels, providerQueryManager.FindProvidersAsync(sessionCtx, keys[i], 0)) } time.Sleep(20 * time.Millisecond) fpn.queriesMadeMutex.Lock() - if fpn.liveQueries != providerQueryManager.maxInProcessRequests { + if fpn.liveQueries != maxInProcessRequests { t.Logf("Queries made: %d\n", fpn.liveQueries) t.Fatal("Did not limit parallel requests to rate limit") } fpn.queriesMadeMutex.Unlock() - for i := 0; i < providerQueryManager.maxInProcessRequests+1; i++ { + for i := 0; i < maxInProcessRequests+1; i++ { + for range requestChannels[i] { + } + } + + fpn.queriesMadeMutex.Lock() + defer fpn.queriesMadeMutex.Unlock() + if fpn.queriesMade != maxInProcessRequests+1 { + t.Logf("Queries made: %d\n", fpn.queriesMade) + t.Fatal("Did not make all separate requests") + } +} + +func TestUnlimitedRequests(t *testing.T) { + const inProcessRequests = 11 + + peers := random.Peers(10) + fpd := &fakeProviderDialer{} + fpn := &fakeProviderDiscovery{ + peersFound: peers, + delay: 5 * time.Millisecond, + } + ctx := context.Background() + ctx, cancel := context.WithCancel(ctx) + defer cancel() + providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxInProcessRequests(0))) + providerQueryManager.Startup() + + keys := random.Cids(inProcessRequests) + sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + var requestChannels []<-chan peer.AddrInfo + for i := 0; i < inProcessRequests; i++ { + requestChannels = append(requestChannels, providerQueryManager.FindProvidersAsync(sessionCtx, keys[i], 0)) + } + time.Sleep(20 * time.Millisecond) + fpn.queriesMadeMutex.Lock() + if fpn.liveQueries != inProcessRequests { + t.Logf("Queries made: %d\n", fpn.liveQueries) + t.Fatal("Parallel requests appear to be rate limited") + } + fpn.queriesMadeMutex.Unlock() + for i := 0; i < inProcessRequests; i++ { for range requestChannels[i] { } } fpn.queriesMadeMutex.Lock() defer fpn.queriesMadeMutex.Unlock() - if fpn.queriesMade != providerQueryManager.maxInProcessRequests+1 { + if fpn.queriesMade != inProcessRequests { t.Logf("Queries made: %d\n", fpn.queriesMade) t.Fatal("Did not make all separate requests") } @@ -310,9 +354,8 @@ func TestFindProviderTimeout(t *testing.T) { delay: 10 * time.Millisecond, } ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(2*time.Millisecond))) providerQueryManager.Startup() - providerQueryManager.setFindProviderTimeout(2 * time.Millisecond) keys := random.Cids(1) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) @@ -335,9 +378,8 @@ func TestFindProviderPreCanceled(t *testing.T) { delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(100*time.Millisecond))) providerQueryManager.Startup() - providerQueryManager.setFindProviderTimeout(100 * time.Millisecond) keys := random.Cids(1) sessionCtx, cancel := context.WithCancel(ctx) @@ -361,9 +403,8 @@ func TestCancelFindProvidersAfterCompletion(t *testing.T) { delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(100*time.Millisecond))) providerQueryManager.Startup() - providerQueryManager.setFindProviderTimeout(100 * time.Millisecond) keys := random.Cids(1) sessionCtx, cancel := context.WithCancel(ctx) @@ -395,9 +436,8 @@ func TestLimitedProviders(t *testing.T) { delay: 1 * time.Millisecond, } ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxProviders(max))) + providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxProviders(max), WithMaxTimeout(100*time.Millisecond))) providerQueryManager.Startup() - providerQueryManager.setFindProviderTimeout(100 * time.Millisecond) keys := random.Cids(1) providersChan := providerQueryManager.FindProvidersAsync(ctx, keys[0], 0) From cdefbf203afc4da545284c0f36731c3f97630e2a Mon Sep 17 00:00:00 2001 From: Marcin Rataj Date: Wed, 27 Nov 2024 19:45:35 +0100 Subject: [PATCH 18/36] refactor: default to prometheus.DefaultRegisterer (#722) - removes calls to prometheus.NewRegistry() - replaces NewRegistry() call with global `prometheus.DefaultRegisterer` so by default boxo users who did not specify custom registry are not missing any metrics. - ensures we don't panic if tests run in parallel and DefaultRegisterer is used --- CHANGELOG.md | 4 +++- gateway/backend_car.go | 19 ++++++++++++------- gateway/blockstore.go | 16 ++++------------ gateway/metrics.go | 11 +++++++++++ routing/http/server/server.go | 18 ++++++++++++++++-- 5 files changed, 46 insertions(+), 22 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a7951b656..fa0b9de5b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -49,7 +49,7 @@ The following emojis are used to highlight certain changes: - The above is only necessary if content routing is needed. Otherwise: -``` +```go // Create network: no contentRouter anymore bswapnet := network.NewFromIpfsHost(host) // Create Bitswap: a new "discovery" parameter set to nil (disable content discovery) @@ -64,6 +64,8 @@ The following emojis are used to highlight certain changes: - `routing/http/server`: added built-in Prometheus instrumentation to http delegated `/routing/v1/` endpoints, with custom buckets for response size and duration to match real world data observed at [the `delegated-ipfs.dev` instance](https://docs.ipfs.tech/concepts/public-utilities/#delegated-routing). [#718](https://github.com/ipfs/boxo/pull/718) [#724](https://github.com/ipfs/boxo/pull/724) - `routing/http/server`: added configurable routing timeout (`DefaultRoutingTimeout` being 30s) to prevent indefinite hangs during content/peer routing. Set custom duration via `WithRoutingTimeout`. [#720](https://github.com/ipfs/boxo/pull/720) +- `routing/http/server`: exposes Prometheus metrics on `prometheus.DefaultRegisterer` and a custom one can be provided via `WithPrometheusRegistry` [#722](https://github.com/ipfs/boxo/pull/722) +- `gateway`: `NewCacheBlockStore` and `NewCarBackend` will use `prometheus.DefaultRegisterer` when a custom one is not specified via `WithPrometheusRegistry` [#722](https://github.com/ipfs/boxo/pull/722) ### Changed diff --git a/gateway/backend_car.go b/gateway/backend_car.go index d2b33a0fc..a0a9eca2d 100644 --- a/gateway/backend_car.go +++ b/gateway/backend_car.go @@ -73,7 +73,7 @@ func NewCarBackend(f CarFetcher, opts ...BackendOption) (*CarBackend, error) { return nil, err } - var promReg prometheus.Registerer = prometheus.NewRegistry() + var promReg prometheus.Registerer = prometheus.DefaultRegisterer if compiledOptions.promRegistry != nil { promReg = compiledOptions.promRegistry } @@ -117,6 +117,11 @@ func NewRemoteCarBackend(gatewayURL []string, httpClient *http.Client, opts ...B } func registerCarBackendMetrics(promReg prometheus.Registerer) *CarBackendMetrics { + // make sure we have functional registry + if promReg == nil { + promReg = prometheus.DefaultRegisterer + } + // How many CAR Fetch attempts we had? Need this to calculate % of various car request types. // We only count attempts here, because success/failure with/without retries are provided by caboose: // - ipfs_caboose_fetch_duration_car_success_count @@ -129,7 +134,7 @@ func registerCarBackendMetrics(promReg prometheus.Registerer) *CarBackendMetrics Name: "car_fetch_attempts", Help: "The number of times a CAR fetch was attempted by IPFSBackend.", }) - promReg.MustRegister(carFetchAttemptMetric) + registerMetric(promReg, carFetchAttemptMetric) contextAlreadyCancelledMetric := prometheus.NewCounter(prometheus.CounterOpts{ Namespace: "ipfs", @@ -137,7 +142,7 @@ func registerCarBackendMetrics(promReg prometheus.Registerer) *CarBackendMetrics Name: "car_fetch_context_already_cancelled", Help: "The number of times context is already cancelled when a CAR fetch was attempted by IPFSBackend.", }) - promReg.MustRegister(contextAlreadyCancelledMetric) + registerMetric(promReg, contextAlreadyCancelledMetric) // How many blocks were read via CARs? // Need this as a baseline to reason about error ratio vs raw_block_recovery_attempts. @@ -147,7 +152,7 @@ func registerCarBackendMetrics(promReg prometheus.Registerer) *CarBackendMetrics Name: "car_blocks_fetched", Help: "The number of blocks successfully read via CAR fetch.", }) - promReg.MustRegister(carBlocksFetchedMetric) + registerMetric(promReg, carBlocksFetchedMetric) carParamsMetric := prometheus.NewCounterVec(prometheus.CounterOpts{ Namespace: "ipfs", @@ -155,7 +160,7 @@ func registerCarBackendMetrics(promReg prometheus.Registerer) *CarBackendMetrics Name: "car_fetch_params", Help: "How many times specific CAR parameter was used during CAR data fetch.", }, []string{"dagScope", "entityRanges"}) // we use 'ranges' instead of 'bytes' here because we only count the number of ranges present - promReg.MustRegister(carParamsMetric) + registerMetric(promReg, carParamsMetric) bytesRangeStartMetric := prometheus.NewHistogram(prometheus.HistogramOpts{ Namespace: "ipfs", @@ -164,7 +169,7 @@ func registerCarBackendMetrics(promReg prometheus.Registerer) *CarBackendMetrics Help: "Tracks where did the range request start.", Buckets: prometheus.ExponentialBuckets(1024, 2, 24), // 1024 bytes to 8 GiB }) - promReg.MustRegister(bytesRangeStartMetric) + registerMetric(promReg, bytesRangeStartMetric) bytesRangeSizeMetric := prometheus.NewHistogram(prometheus.HistogramOpts{ Namespace: "ipfs", @@ -173,7 +178,7 @@ func registerCarBackendMetrics(promReg prometheus.Registerer) *CarBackendMetrics Help: "Tracks the size of range requests.", Buckets: prometheus.ExponentialBuckets(256*1024, 2, 10), // From 256KiB to 100MiB }) - promReg.MustRegister(bytesRangeSizeMetric) + registerMetric(promReg, bytesRangeSizeMetric) return &CarBackendMetrics{ contextAlreadyCancelledMetric, diff --git a/gateway/blockstore.go b/gateway/blockstore.go index 68cd729ac..07c758d05 100644 --- a/gateway/blockstore.go +++ b/gateway/blockstore.go @@ -35,8 +35,7 @@ var _ blockstore.Blockstore = (*cacheBlockStore)(nil) // in memory using a two queue cache. It can be useful, for example, when paired // with a proxy blockstore (see [NewRemoteBlockstore]). // -// If the given [prometheus.Registerer] is nil, a new one will be created using -// [prometheus.NewRegistry]. +// If the given [prometheus.Registerer] is nil, a [prometheus.DefaultRegisterer] will be used. func NewCacheBlockStore(size int, reg prometheus.Registerer) (blockstore.Blockstore, error) { c, err := lru.New2Q[string, []byte](size) if err != nil { @@ -44,7 +43,7 @@ func NewCacheBlockStore(size int, reg prometheus.Registerer) (blockstore.Blockst } if reg == nil { - reg = prometheus.NewRegistry() + reg = prometheus.DefaultRegisterer } cacheHitsMetric := prometheus.NewCounter(prometheus.CounterOpts{ @@ -61,15 +60,8 @@ func NewCacheBlockStore(size int, reg prometheus.Registerer) (blockstore.Blockst Help: "The number of global block cache requests.", }) - err = reg.Register(cacheHitsMetric) - if err != nil { - return nil, err - } - - err = reg.Register(cacheRequestsMetric) - if err != nil { - return nil, err - } + registerMetric(reg, cacheHitsMetric) + registerMetric(reg, cacheRequestsMetric) return &cacheBlockStore{ cache: c, diff --git a/gateway/metrics.go b/gateway/metrics.go index 32bb66568..29df1272d 100644 --- a/gateway/metrics.go +++ b/gateway/metrics.go @@ -307,3 +307,14 @@ var tracer = otel.Tracer("boxo/gateway") func spanTrace(ctx context.Context, spanName string, opts ...trace.SpanStartOption) (context.Context, trace.Span) { return tracer.Start(ctx, "Gateway."+spanName, opts...) } + +// registerMetric registers metrics in registry or logs an error. +// +// Registration may error if metric is alreadyregistered. we are not using +// MustRegister here to allow people to run tests in parallel without having to +// write tedious glue code that creates unique registry for each unit test +func registerMetric(registry prometheus.Registerer, metric prometheus.Collector) { + if err := registry.Register(metric); err != nil { + log.Errorf("failed to register %v: %v", metric, err) + } +} diff --git a/routing/http/server/server.go b/routing/http/server/server.go index fd964cbab..c951701c3 100644 --- a/routing/http/server/server.go +++ b/routing/http/server/server.go @@ -10,6 +10,7 @@ import ( "mime" "net/http" "strings" + "sync/atomic" "time" "github.com/cespare/xxhash/v2" @@ -152,14 +153,25 @@ func Handler(svc ContentRouter, opts ...Option) http.Handler { } if server.promRegistry == nil { - server.promRegistry = prometheus.NewRegistry() + server.promRegistry = prometheus.DefaultRegisterer + } + + // Workaround due to https://github.com/slok/go-http-metrics + // using egistry.MustRegister internally. + // In production there will be only one handler, however we append counter + // to ensure duplicate metric registration will not panic in parallel tests + // when global prometheus.DefaultRegisterer is used. + metricsPrefix := "delegated_routing_server" + c := handlerCount.Add(1) + if c > 1 { + metricsPrefix = fmt.Sprintf("%s_%d", metricsPrefix, c) } // Create middleware with prometheus recorder mdlw := middleware.New(middleware.Config{ Recorder: metrics.NewRecorder(metrics.Config{ Registry: server.promRegistry, - Prefix: "delegated_routing_server", + Prefix: metricsPrefix, SizeBuckets: prometheus.ExponentialBuckets(100, 4, 8), // [100 400 1600 6400 25600 102400 409600 1.6384e+06] DurationBuckets: []float64{0.1, 0.5, 1, 2, 5, 8, 10, 20, 30}, }), @@ -176,6 +188,8 @@ func Handler(svc ContentRouter, opts ...Option) http.Handler { return r } +var handlerCount atomic.Int32 + type server struct { svc ContentRouter disableNDJSON bool From 970dcb3e013064d51fbfa18978fbaecb8239f4da Mon Sep 17 00:00:00 2001 From: Hector Sanjuan Date: Wed, 27 Nov 2024 21:23:32 +0100 Subject: [PATCH 19/36] [skip changelog] staticcheck fixes / remove ununsed variables (#730) staticcheck fixes / remove ununsed variables --- bitswap/internal/defaults/defaults.go | 4 ++-- chunker/parse_test.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/bitswap/internal/defaults/defaults.go b/bitswap/internal/defaults/defaults.go index dbcd62a31..646b56b0d 100644 --- a/bitswap/internal/defaults/defaults.go +++ b/bitswap/internal/defaults/defaults.go @@ -6,8 +6,8 @@ import ( ) const ( - // these requests take at _least_ two minutes at the moment. - ProvideTimeout = time.Minute * 3 + // ProvSearchDelay specifies how long to wait before we start + // broadcasting outstanding wants for the first time. ProvSearchDelay = time.Second // Number of concurrent workers in decision engine that process requests to the blockstore diff --git a/chunker/parse_test.go b/chunker/parse_test.go index 6809476e9..108b96729 100644 --- a/chunker/parse_test.go +++ b/chunker/parse_test.go @@ -17,7 +17,7 @@ func TestParseRabin(t *testing.T) { _, err := FromString(r, "rabin-18-25-32") if err != nil { - t.Errorf(err.Error()) + t.Error(err.Error()) } _, err = FromString(r, "rabin-15-23-31") From 33c7b9c236f9d8fcf1c7dea1a263be05eeb7b493 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Wed, 27 Nov 2024 10:24:18 -1000 Subject: [PATCH 20/36] Add debug logging for deduplicated queries (#729) * Add debug logging for deduplicated queries --- routing/providerquerymanager/providerquerymanager.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/routing/providerquerymanager/providerquerymanager.go b/routing/providerquerymanager/providerquerymanager.go index 55880ecd9..025fff39d 100644 --- a/routing/providerquerymanager/providerquerymanager.go +++ b/routing/providerquerymanager/providerquerymanager.go @@ -14,6 +14,7 @@ import ( "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" + "go.uber.org/zap/zapcore" ) var log = logging.Logger("routing/provqrymgr") @@ -463,6 +464,13 @@ func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { span.AddEvent("NewQuery", trace.WithAttributes(attribute.Stringer("cid", npqm.k))) ctx = trace.ContextWithSpan(ctx, span) + // Use context derived from pqm.ctx here, and not the context from the + // request (npqm.ctx), because this inProgressRequestStatus applies to + // all in-progress requests for the CID (npqm.k). + // + // For tracing, this means that only the span from the first + // request-in-progress for a CID is used, even if there are multiple + // requests for the same CID. requestStatus = &inProgressRequestStatus{ listeners: make(map[chan peer.AddrInfo]struct{}), ctx: ctx, @@ -484,6 +492,9 @@ func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { } } else { trace.SpanFromContext(npqm.ctx).AddEvent("JoinQuery", trace.WithAttributes(attribute.Stringer("cid", npqm.k))) + if log.Level().Enabled(zapcore.DebugLevel) { + log.Debugf("Joined existing query for cid %s which now has %d queries in progress", npqm.k, len(requestStatus.listeners)+1) + } } inProgressChan := make(chan peer.AddrInfo) requestStatus.listeners[inProgressChan] = struct{}{} From b5656aa69b3657b4b744dbf4d0b81c1cc7fb069d Mon Sep 17 00:00:00 2001 From: Hector Sanjuan Date: Thu, 28 Nov 2024 17:23:11 +0100 Subject: [PATCH 21/36] bitswap/client: fix wiring when passing custom providerFinder --- bitswap/client/client.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/bitswap/client/client.go b/bitswap/client/client.go index 13d8a006a..08c9c7fa7 100644 --- a/bitswap/client/client.go +++ b/bitswap/client/client.go @@ -202,11 +202,13 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, providerFinder Pr // into session.ProviderFinder when passing it, it will become // not nil. Related: // https://groups.google.com/g/golang-nuts/c/wnH302gBa4I?pli=1 - var pqm bssession.ProviderFinder + var sessionProvFinder bssession.ProviderFinder if bs.pqm != nil { - pqm = bs.pqm + sessionProvFinder = bs.pqm + } else if providerFinder != nil { + sessionProvFinder = providerFinder } - return bssession.New(sessctx, sessmgr, id, spm, pqm, sim, pm, bpm, notif, provSearchDelay, rebroadcastDelay, self) + return bssession.New(sessctx, sessmgr, id, spm, sessionProvFinder, sim, pm, bpm, notif, provSearchDelay, rebroadcastDelay, self) } sessionPeerManagerFactory := func(ctx context.Context, id uint64) bssession.SessionPeerManager { return bsspm.New(id, network.ConnectionManager()) From a6723bf9994ba397a8520e44751c5990921a2b1b Mon Sep 17 00:00:00 2001 From: Hector Sanjuan Date: Fri, 29 Nov 2024 11:59:56 +0100 Subject: [PATCH 22/36] bitswap/client: add test using custom providerQueryManager --- bitswap/client/bitswap_with_sessions_test.go | 58 ++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/bitswap/client/bitswap_with_sessions_test.go b/bitswap/client/bitswap_with_sessions_test.go index 62695de8b..110c43f5c 100644 --- a/bitswap/client/bitswap_with_sessions_test.go +++ b/bitswap/client/bitswap_with_sessions_test.go @@ -8,11 +8,13 @@ import ( "time" "github.com/ipfs/boxo/bitswap" + "github.com/ipfs/boxo/bitswap/client" "github.com/ipfs/boxo/bitswap/client/internal/session" "github.com/ipfs/boxo/bitswap/client/traceability" testinstance "github.com/ipfs/boxo/bitswap/testinstance" tn "github.com/ipfs/boxo/bitswap/testnet" mockrouting "github.com/ipfs/boxo/routing/mock" + "github.com/ipfs/boxo/routing/providerquerymanager" blocks "github.com/ipfs/go-block-format" cid "github.com/ipfs/go-cid" delay "github.com/ipfs/go-ipfs-delay" @@ -112,6 +114,62 @@ func assertBlockListsFrom(from peer.ID, got, exp []blocks.Block) error { return nil } +// TestCustomProviderQueryManager tests that nothing breaks if we use a custom +// PQM when creating bitswap. +func TestCustomProviderQueryManager(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + vnet := getVirtualNetwork() + router := mockrouting.NewServer() + ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) + defer ig.Close() + + block := random.BlocksOfSize(1, blockSize)[0] + a := ig.Next() + b := ig.Next() + + // Replace bitswap in instance a with our customized one. + pqm, err := providerquerymanager.New(ctx, a.Adapter, router.Client(a.Identity)) + if err != nil { + t.Fatal(err) + } + pqm.Startup() + bs := bitswap.New(ctx, a.Adapter, pqm, a.Blockstore, + bitswap.WithClientOption(client.WithDefaultProviderQueryManager(false))) + a.Exchange.Close() // close old to be sure. + a.Exchange = bs + // Connect instances only after bitswap exists. + testinstance.ConnectInstances([]testinstance.Instance{a, b}) + + // Add a block to Peer B + if err := b.Blockstore.Put(ctx, block); err != nil { + t.Fatal(err) + } + + // Create a session on Peer A + sesa := a.Exchange.NewSession(ctx) + + // Get the block + blkout, err := sesa.GetBlock(ctx, block.Cid()) + if err != nil { + t.Fatal(err) + } + + if !blkout.Cid().Equals(block.Cid()) { + t.Fatal("got wrong block") + } + + traceBlock, ok := blkout.(traceability.Block) + if !ok { + t.Fatal("did not get tracable block") + } + + if traceBlock.From != b.Identity.ID() { + t.Fatal("should have received block from peer B, did not") + } +} + func TestSessionBetweenPeers(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() From 59fbac950378d7694120e3cd30bf3774b045760f Mon Sep 17 00:00:00 2001 From: John <153272819+hishope@users.noreply.github.com> Date: Sat, 30 Nov 2024 05:48:03 +0800 Subject: [PATCH 23/36] chore: fix invalid url in docs (#733) chore: fix 404 status URL Signed-off-by: hishope --- docs/tracing.md | 4 ++-- examples/gateway/common/tracing.go | 2 +- tracing/exporters.go | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/tracing.md b/docs/tracing.md index dae37024f..a620fa655 100644 --- a/docs/tracing.md +++ b/docs/tracing.md @@ -54,7 +54,7 @@ Default: `"grpc"` ### `Zipkin Exporter` -See [Zipkin Exporter](https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/sdk-environment-variables.md#zipkin-exporter). +See [Zipkin Exporter](https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/configuration/sdk-environment-variables.md#zipkin-exporter). ### `File Exporter` @@ -66,7 +66,7 @@ Default: `"$PWD/traces.json"` ### `OTEL_PROPAGATORS` -See [General SDK Configuration](https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/sdk-environment-variables.md#general-sdk-configuration). +See [General SDK Configuration](https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/configuration/sdk-environment-variables.md#general-sdk-configuration). ## Using Jaeger UI diff --git a/examples/gateway/common/tracing.go b/examples/gateway/common/tracing.go index 403c8cf50..e7255520c 100644 --- a/examples/gateway/common/tracing.go +++ b/examples/gateway/common/tracing.go @@ -28,7 +28,7 @@ func SetupTracing(ctx context.Context, serviceName string) (*trace.TracerProvide // using autoprop.NewTextMapPropagator, we ensure the value of the environmental // variable OTEL_PROPAGATORS is respected, if set. By default, Trace Context // and Baggage are used. More details on: - // https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/sdk-environment-variables.md + // https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/configuration/sdk-environment-variables.md otel.SetTextMapPropagator(autoprop.NewTextMapPropagator()) return tp, nil diff --git a/tracing/exporters.go b/tracing/exporters.go index 6c57f8456..71a564c7c 100644 --- a/tracing/exporters.go +++ b/tracing/exporters.go @@ -20,7 +20,7 @@ import ( // most of this code. // // Specs: -// - https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/sdk-environment-variables.md#exporter-selection +// - https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/configuration/sdk-environment-variables.md#exporter-selection // - https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/protocol/exporter.md func NewSpanExporters(ctx context.Context) ([]trace.SpanExporter, error) { var exporters []trace.SpanExporter From caa6763962145eb1d78ff1a370dbc96218e8b457 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Mon, 2 Dec 2024 11:00:34 -1000 Subject: [PATCH 24/36] misc comments and spelling (#735) --- examples/README.md | 2 +- pinning/remote/client/openapi/README.md | 17 ++++------------- .../providerquerymanager.go | 4 +++- 3 files changed, 8 insertions(+), 15 deletions(-) diff --git a/examples/README.md b/examples/README.md index 19313caf4..aeee689eb 100644 --- a/examples/README.md +++ b/examples/README.md @@ -26,7 +26,7 @@ Once you have your example finished, do not forget to run `go mod tidy` and addi ## Examples and Tutorials -- [Transfering UnixFS file data with Bitswap](./bitswap-transfer) +- [Transferring UnixFS file data with Bitswap](./bitswap-transfer) - [Gateway backed by a local blockstore in form of a CAR file](./gateway/car-file) - [Gateway backed by a remote (HTTP) blockstore and IPNS resolver](./gateway/proxy-blocks) - [Gateway backed by a remote (HTTP) CAR Gateway](./gateway/proxy-car) diff --git a/pinning/remote/client/openapi/README.md b/pinning/remote/client/openapi/README.md index eddae08b7..7c5c3ae0f 100644 --- a/pinning/remote/client/openapi/README.md +++ b/pinning/remote/client/openapi/README.md @@ -1,7 +1,5 @@ # Go API client for openapi - - ## About this spec The IPFS Pinning Service API is intended to be an implementation-agnostic API: - For use and implementation by pinning service providers @@ -15,8 +13,10 @@ This section describes the most important object types and conventions. A full list of fields and schemas can be found in the `schemas` section of the [YAML file](https://github.com/ipfs/pinning-services-api-spec/blob/master/ipfs-pinning-service.yaml). ## Identifiers + ### cid [Content Identifier (CID)](https://docs.ipfs.io/concepts/content-addressing/) points at the root of a DAG that is pinned recursively. + ### requestid Unique identifier of a pin request. @@ -25,6 +25,7 @@ When a pin is created, the service responds with unique `requestid` that can be Service implementation should use UUID, `hash(accessToken,Pin,PinStatus.created)`, or any other opaque identifier that provides equally strong protection against race conditions. ## Objects + ### Pin object ![pin object](https://bafybeideck2fchyxna4wqwc2mo67yriokehw3yujboc5redjdaajrk2fjq.ipfs.dweb.link/pin.png) @@ -46,7 +47,7 @@ It includes the original `pin` object, along with the current `status` and globa ### Creating a new pin object The user sends a `Pin` object to `POST /pins` and receives a `PinStatus` response: -- `requestid` in `PinStatus` is the identifier of the pin operation, which can can be used for checking status, and removing the pin in the future +- `requestid` in `PinStatus` is the identifier of the pin operation, which can be used for checking status, and removing the pin in the future - `status` in `PinStatus` indicates the current state of a pin ### Checking status of in-progress pinning @@ -60,7 +61,6 @@ The user can replace an existing pin object via `POST /pins/{requestid}`. This i ### Removing a pin object A pin object can be removed via `DELETE /pins/{requestid}`. - ## Provider hints Pinning of new data can be accelerated by providing a list of known data sources in `Pin.origins`, and connecting at least one of them to pinning service nodes at `PinStatus.delegates`. @@ -99,8 +99,6 @@ Pin objects can be listed by executing `GET /pins` with optional parameters: > **Note**: pagination by the `created` timestamp requires each value to be globally unique. Any future considerations to add support for bulk creation must account for this. - - ## Overview This API client was generated by the [OpenAPI Generator](https://openapi-generator.tech) project. By using the [OpenAPI-spec](https://www.openapis.org/) from a remote server, you can easily generate an API client. @@ -177,7 +175,6 @@ Class | Method | HTTP request | Description *PinsApi* | [**PinsRequestidGet**](docs/PinsApi.md#pinsrequestidget) | **Get** /pins/{requestid} | Get pin object *PinsApi* | [**PinsRequestidPost**](docs/PinsApi.md#pinsrequestidpost) | **Post** /pins/{requestid} | Replace pin object - ## Documentation For Models - [Failure](docs/Failure.md) @@ -190,11 +187,8 @@ Class | Method | HTTP request | Description ## Documentation For Authorization - - ### accessToken - ## Documentation for Utility Methods Due to the fact that model structure members are all pointers, this package contains @@ -212,6 +206,3 @@ Each of these functions takes a value of the given basic type and returns a poin * `PtrTime` ## Author - - - diff --git a/routing/providerquerymanager/providerquerymanager.go b/routing/providerquerymanager/providerquerymanager.go index 025fff39d..b5a5b3fa0 100644 --- a/routing/providerquerymanager/providerquerymanager.go +++ b/routing/providerquerymanager/providerquerymanager.go @@ -102,6 +102,8 @@ type ProviderQueryManager struct { type Option func(*ProviderQueryManager) error +// WithMaxTimeout sets the limit on the amount of time to spend waiting for the +// maximum number of providers from a find request. func WithMaxTimeout(timeout time.Duration) Option { return func(mgr *ProviderQueryManager) error { mgr.findProviderTimeout = timeout @@ -316,7 +318,7 @@ func (pqm *ProviderQueryManager) findProviderWorker() { findSem = make(chan struct{}, pqm.maxInProcessRequests) } - // Read find provider requests until channel is closed. The channl is + // Read find provider requests until channel is closed. The channel is // closed as soon as pqm.ctx is canceled, so there is no need to select on // that context here. for fpr := range pqm.providerRequestsProcessing.Out() { From 984fa897fd69473693bff3ff95a644badd71f915 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Mon, 2 Dec 2024 12:58:26 -1000 Subject: [PATCH 25/36] chore: minor examples cleanup (#736) - Pass time.Duration instead of passing int that gets converted to time.Duration. - Remove unneeded else clauses - spelling --- examples/bitswap-transfer/README.md | 2 +- examples/routing/delegated-routing-client/main.go | 15 ++++++++------- .../routing/delegated-routing-client/main_test.go | 6 +++--- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/examples/bitswap-transfer/README.md b/examples/bitswap-transfer/README.md index 55d69e9d6..3a42b3beb 100644 --- a/examples/bitswap-transfer/README.md +++ b/examples/bitswap-transfer/README.md @@ -1,4 +1,4 @@ -# Transfering UnixFS file with Bitswap +# Transferring UnixFS file with Bitswap This is an example that quickly shows how to use IPFS tooling to move around a file. diff --git a/examples/routing/delegated-routing-client/main.go b/examples/routing/delegated-routing-client/main.go index 8fac342ac..d60d08675 100644 --- a/examples/routing/delegated-routing-client/main.go +++ b/examples/routing/delegated-routing-client/main.go @@ -26,31 +26,32 @@ func main() { namePtr := flag.String("ipns", "", "ipns name to retrieve record for") flag.Parse() - if err := run(os.Stdout, *gatewayUrlPtr, *cidPtr, *pidPtr, *namePtr, *timeoutPtr); err != nil { + timeout := time.Duration(*timeoutPtr) * time.Second + if err := run(os.Stdout, *gatewayUrlPtr, *cidPtr, *pidPtr, *namePtr, timeout); err != nil { log.Fatal(err) } } -func run(w io.Writer, gatewayURL, cidStr, pidStr, nameStr string, timeoutSeconds int) error { +func run(w io.Writer, gatewayURL, cidStr, pidStr, nameStr string, timeout time.Duration) error { // Creates a new Delegated Routing V1 client. client, err := client.New(gatewayURL) if err != nil { return err } - timeout := time.Duration(timeoutSeconds) * time.Second ctx, cancel := context.WithTimeout(context.Background(), timeout) defer cancel() if cidStr != "" { return findProviders(w, ctx, client, cidStr) - } else if pidStr != "" { + } + if pidStr != "" { return findPeers(w, ctx, client, pidStr) - } else if nameStr != "" { + } + if nameStr != "" { return findIPNS(w, ctx, client, nameStr) - } else { - return errors.New("cid or peer must be provided") } + return errors.New("cid or peer must be provided") } func findProviders(w io.Writer, ctx context.Context, client *client.Client, cidStr string) error { diff --git a/examples/routing/delegated-routing-client/main_test.go b/examples/routing/delegated-routing-client/main_test.go index 1c1da193f..625735efc 100644 --- a/examples/routing/delegated-routing-client/main_test.go +++ b/examples/routing/delegated-routing-client/main_test.go @@ -32,7 +32,7 @@ func TestFindProviders(t *testing.T) { t.Cleanup(ts.Close) out := &bytes.Buffer{} - err := run(out, ts.URL, cidStr, "", "", 1) + err := run(out, ts.URL, cidStr, "", "", time.Second) assert.Contains(t, out.String(), "12D3KooWM8sovaEGU1bmiWGWAzvs47DEcXKZZTuJnpQyVTkRs2Vn\n\tProtocols: [transport-bitswap]\n\tAddresses: [/ip4/111.222.222.111/tcp/5734]\n") assert.Contains(t, out.String(), "12D3KooWB6RAWgcmHAP7TGEGK7utV2ZuqSzX1DNjRa97TtJ7139n\n\tProtocols: []\n\tAddresses: [/ip4/127.0.0.1/tcp/5734]\n") assert.NoError(t, err) @@ -50,7 +50,7 @@ func TestFindPeers(t *testing.T) { t.Cleanup(ts.Close) out := &bytes.Buffer{} - err := run(out, ts.URL, "", pidStr, "", 1) + err := run(out, ts.URL, "", pidStr, "", time.Second) assert.Contains(t, out.String(), "12D3KooWM8sovaEGU1bmiWGWAzvs47DEcXKZZTuJnpQyVTkRs2Vn\n\tProtocols: [transport-bitswap]\n\tAddresses: [/ip4/111.222.222.111/tcp/5734]\n") assert.NoError(t, err) } @@ -67,7 +67,7 @@ func TestGetIPNS(t *testing.T) { t.Cleanup(ts.Close) out := &bytes.Buffer{} - err := run(out, ts.URL, "", "", name.String(), 1) + err := run(out, ts.URL, "", "", name.String(), time.Second) assert.Contains(t, out.String(), fmt.Sprintf("/ipns/%s\n\tSignature: VALID\n\tValue: /ipfs/bafkreifjjcie6lypi6ny7amxnfftagclbuxndqonfipmb64f2km2devei4\n", name.String())) assert.NoError(t, err) } From 9069a292c3d49877b70b1d569082343454601b5f Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Tue, 3 Dec 2024 08:45:09 -1000 Subject: [PATCH 26/36] Bitswap default ProviderQueryManager uses explicit options (#737) Set explicit options for bitswap default PQM, to make it more conservative and isolate it from a PQM without options --- bitswap/client/client.go | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/bitswap/client/client.go b/bitswap/client/client.go index 08c9c7fa7..4523962c1 100644 --- a/bitswap/client/client.go +++ b/bitswap/client/client.go @@ -97,13 +97,19 @@ func WithoutDuplicatedBlockStats() Option { } } -// WithDefaultProviderQueryManager indicates wether we should use a the -// default ProviderQueryManager, a wrapper of the content Router which -// provides bounded paralelism and limits for these lookups. The -// ProviderQueryManager setup by default uses maxInProcessRequests = 6 and -// maxProviders = 10. To use a custom ProviderQueryManager, set to false and -// wrap directly the content router provided with the WithContentRouting() -// option. Only takes effect if WithContentRouting is set. +// WithDefaultProviderQueryManager indicates whether to use the default +// ProviderQueryManager as a wrapper of the content Router. The default bitswap +// ProviderQueryManager provides bounded parallelism and limits for these +// lookups. The bitswap default ProviderQueryManager uses these options, which +// may be more conservative than the ProviderQueryManager defaults: +// +// - WithMaxInProcessRequests(16) +// - WithMaxProviders(10) +// - WithMaxTimeout(10 *time.Second) +// +// To use a custom ProviderQueryManager, set to false and wrap directly the +// content router provided with the WithContentRouting() option. Only takes +// effect if WithContentRouting is set. func WithDefaultProviderQueryManager(defaultProviderQueryManager bool) Option { return func(bs *Client) { bs.defaultProviderQueryManager = defaultProviderQueryManager @@ -176,7 +182,10 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, providerFinder Pr if bs.providerFinder != nil && bs.defaultProviderQueryManager { // network can do dialing. - pqm, err := rpqm.New(ctx, network, bs.providerFinder, rpqm.WithMaxProviders(10)) + pqm, err := rpqm.New(ctx, network, bs.providerFinder, + rpqm.WithMaxInProcessRequests(16), + rpqm.WithMaxProviders(10), + rpqm.WithMaxTimeout(10*time.Second)) if err != nil { // Should not be possible to hit this panic(err) From 3a3e8afa3492af7692c182b2126f8519f6fa05f0 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Tue, 3 Dec 2024 08:55:33 -1000 Subject: [PATCH 27/36] refactor(remote/pinning): `Ls` to take results channel instead of returning one (#738) --- CHANGELOG.md | 1 + pinning/remote/client/client.go | 130 ++++++++++++++++++-------------- 2 files changed, 73 insertions(+), 58 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fa0b9de5b..b29cabedd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -72,6 +72,7 @@ The following emojis are used to highlight certain changes: - `routing/http/client`: creating delegated routing client with `New` now defaults to querying delegated routing server with `DefaultProtocolFilter` ([IPIP-484](https://github.com/ipfs/specs/pull/484)) [#689](https://github.com/ipfs/boxo/pull/689) - `bitswap/client`: Wait at lease one broadcast interval before resending wants to a peer. Check for peers to rebroadcast to more often than one broadcast interval. - No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. [#710](https://github.com/ipfs/boxo/pull/710) +- `pinning/remote/client`: Refactor remote pinning `Ls` to take results channel instead of returning one. The previous `Ls` behavior is implemented by the GoLs function, which creates the channels, starts the goroutine that calls Ls, and returns the channels to the caller [#738](https://github.com/ipfs/boxo/pull/738) ### Removed diff --git a/pinning/remote/client/client.go b/pinning/remote/client/client.go index b3cad4071..3b6270093 100644 --- a/pinning/remote/client/client.go +++ b/pinning/remote/client/client.go @@ -137,77 +137,90 @@ func (pinLsOpts) LsMeta(meta map[string]string) LsOption { type pinResults = openapi.PinResults -func (c *Client) Ls(ctx context.Context, opts ...LsOption) (chan PinStatusGetter, chan error) { - res := make(chan PinStatusGetter, 1) - errs := make(chan error, 1) - +// Ls writes pin statuses to the PinStatusGetter channel. The channel is +// closed when there are no more pins. If an error occurs or ctx is canceled, +// then the channel is closed and an error is returned. +// +// Example: +// +// res := make(chan PinStatusGetter, 1) +// lsErr := make(chan error, 1) +// go func() { +// lsErr <- c.Ls(ctx, res, opts...) +// }() +// for r := range res { +// processPin(r) +// } +// return <-lsErr +func (c *Client) Ls(ctx context.Context, res chan<- PinStatusGetter, opts ...LsOption) (err error) { settings := new(lsSettings) for _, o := range opts { - if err := o(settings); err != nil { + if err = o(settings); err != nil { close(res) - errs <- err - close(errs) - return res, errs + return err } } - go func() { - defer func() { - if r := recover(); r != nil { - var err error - switch x := r.(type) { - case string: - err = fmt.Errorf("unexpected error while listing remote pins: %s", x) - case error: - err = fmt.Errorf("unexpected error while listing remote pins: %w", x) - default: - err = errors.New("unknown panic while listing remote pins") - } - errs <- err - } - close(errs) - close(res) - }() - - for { - pinRes, err := c.lsInternal(ctx, settings) - if err != nil { - errs <- err - return + defer func() { + if r := recover(); r != nil { + switch x := r.(type) { + case string: + err = fmt.Errorf("unexpected error while listing remote pins: %s", x) + case error: + err = fmt.Errorf("unexpected error while listing remote pins: %w", x) + default: + err = errors.New("unknown panic while listing remote pins") } + } + close(res) + }() - results := pinRes.GetResults() - for _, r := range results { - select { - case res <- &pinStatusObject{r}: - case <-ctx.Done(): - errs <- ctx.Err() - return - } - } + for { + pinRes, err := c.lsInternal(ctx, settings) + if err != nil { + return err + } - batchSize := len(results) - if int(pinRes.Count) == batchSize { - // no more batches - return + results := pinRes.GetResults() + for _, r := range results { + select { + case res <- &pinStatusObject{r}: + case <-ctx.Done(): + return ctx.Err() } + } - // Better DX/UX for cases like https://github.com/application-research/estuary/issues/124 - if batchSize == 0 && int(pinRes.Count) != 0 { - errs <- fmt.Errorf("invalid pinning service response: PinResults.count=%d but no PinResults.results", int(pinRes.Count)) - return - } + batchSize := len(results) + if int(pinRes.Count) == batchSize { + // no more batches + return nil + } - oldestResult := results[batchSize-1] - settings.before = &oldestResult.Created + // Better DX/UX for cases like https://github.com/application-research/estuary/issues/124 + if batchSize == 0 && int(pinRes.Count) != 0 { + return fmt.Errorf("invalid pinning service response: PinResults.count=%d but no PinResults.results", int(pinRes.Count)) } + + oldestResult := results[batchSize-1] + settings.before = &oldestResult.Created + } +} + +// GoLs creates the results and error channels, starts the goroutine that calls +// Ls, and returns the channels to the caller. +func (c *Client) GoLs(ctx context.Context, opts ...LsOption) (<-chan PinStatusGetter, <-chan error) { + res := make(chan PinStatusGetter) + errs := make(chan error, 1) + + go func() { + errs <- c.Ls(ctx, res, opts...) }() return res, errs } func (c *Client) LsSync(ctx context.Context, opts ...LsOption) ([]PinStatusGetter, error) { - resCh, errCh := c.Ls(ctx, opts...) + resCh, errCh := c.GoLs(ctx, opts...) var res []PinStatusGetter for r := range resCh { @@ -219,8 +232,6 @@ func (c *Client) LsSync(ctx context.Context, opts ...LsOption) ([]PinStatusGette // Manual version of Ls that returns a single batch of results and int with total count func (c *Client) LsBatchSync(ctx context.Context, opts ...LsOption) ([]PinStatusGetter, int, error) { - var res []PinStatusGetter - settings := new(lsSettings) for _, o := range opts { if err := o(settings); err != nil { @@ -233,9 +244,13 @@ func (c *Client) LsBatchSync(ctx context.Context, opts ...LsOption) ([]PinStatus return nil, 0, err } + var res []PinStatusGetter results := pinRes.GetResults() - for _, r := range results { - res = append(res, &pinStatusObject{r}) + if len(results) != 0 { + res = make([]PinStatusGetter, len(results)) + for i, r := range results { + res[i] = &pinStatusObject{r} + } } return res, int(pinRes.Count), nil @@ -274,8 +289,7 @@ func (c *Client) lsInternal(ctx context.Context, settings *lsSettings) (pinResul // TODO: Ignoring HTTP Response OK? results, httpresp, err := getter.Execute() if err != nil { - err := httperr(httpresp, err) - return pinResults{}, err + return pinResults{}, httperr(httpresp, err) } return results, nil From 6c7f2b728b46ceaa73ac9b620951082a2584eaea Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Tue, 3 Dec 2024 10:24:37 -1000 Subject: [PATCH 28/36] chore: update to latest go-libp2p (#739) --- CHANGELOG.md | 1 + examples/go.mod | 8 ++++---- examples/go.sum | 16 ++++++++-------- go.mod | 8 ++++---- go.sum | 16 ++++++++-------- 5 files changed, 25 insertions(+), 24 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b29cabedd..da6de76b2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -73,6 +73,7 @@ The following emojis are used to highlight certain changes: - `bitswap/client`: Wait at lease one broadcast interval before resending wants to a peer. Check for peers to rebroadcast to more often than one broadcast interval. - No longer using `github.com/jbenet/goprocess` to avoid requiring in dependents. [#710](https://github.com/ipfs/boxo/pull/710) - `pinning/remote/client`: Refactor remote pinning `Ls` to take results channel instead of returning one. The previous `Ls` behavior is implemented by the GoLs function, which creates the channels, starts the goroutine that calls Ls, and returns the channels to the caller [#738](https://github.com/ipfs/boxo/pull/738) +- updated to go-libp2p to [v0.37.2](https://github.com/libp2p/go-libp2p/releases/tag/v0.37.2) ### Removed diff --git a/examples/go.mod b/examples/go.mod index 16b6faa3e..b5856dee3 100644 --- a/examples/go.mod +++ b/examples/go.mod @@ -11,7 +11,7 @@ require ( github.com/ipfs/go-datastore v0.6.0 github.com/ipld/go-car/v2 v2.14.2 github.com/ipld/go-ipld-prime v0.21.0 - github.com/libp2p/go-libp2p v0.37.0 + github.com/libp2p/go-libp2p v0.37.2 github.com/multiformats/go-multiaddr v0.13.0 github.com/multiformats/go-multicodec v0.9.0 github.com/prometheus/client_golang v1.20.5 @@ -111,11 +111,11 @@ require ( github.com/mr-tron/base58 v1.2.0 // indirect github.com/multiformats/go-base32 v0.1.0 // indirect github.com/multiformats/go-base36 v0.2.0 // indirect - github.com/multiformats/go-multiaddr-dns v0.4.0 // indirect + github.com/multiformats/go-multiaddr-dns v0.4.1 // indirect github.com/multiformats/go-multiaddr-fmt v0.1.0 // indirect github.com/multiformats/go-multibase v0.2.0 // indirect github.com/multiformats/go-multihash v0.2.3 // indirect - github.com/multiformats/go-multistream v0.5.0 // indirect + github.com/multiformats/go-multistream v0.6.0 // indirect github.com/multiformats/go-varint v0.0.7 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/onsi/ginkgo/v2 v2.20.2 // indirect @@ -147,7 +147,7 @@ require ( github.com/prometheus/common v0.60.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/quic-go/qpack v0.5.1 // indirect - github.com/quic-go/quic-go v0.48.1 // indirect + github.com/quic-go/quic-go v0.48.2 // indirect github.com/quic-go/webtransport-go v0.8.1-0.20241018022711-4ac2c9250e66 // indirect github.com/raulk/go-watchdog v1.3.0 // indirect github.com/samber/lo v1.47.0 // indirect diff --git a/examples/go.sum b/examples/go.sum index 401e43184..fce1aab4a 100644 --- a/examples/go.sum +++ b/examples/go.sum @@ -272,8 +272,8 @@ github.com/libp2p/go-doh-resolver v0.4.0 h1:gUBa1f1XsPwtpE1du0O+nnZCUqtG7oYi7Bb+ github.com/libp2p/go-doh-resolver v0.4.0/go.mod h1:v1/jwsFusgsWIGX/c6vCRrnJ60x7bhTiq/fs2qt0cAg= github.com/libp2p/go-flow-metrics v0.2.0 h1:EIZzjmeOE6c8Dav0sNv35vhZxATIXWZg6j/C08XmmDw= github.com/libp2p/go-flow-metrics v0.2.0/go.mod h1:st3qqfu8+pMfh+9Mzqb2GTiwrAGjIPszEjZmtksN8Jc= -github.com/libp2p/go-libp2p v0.37.0 h1:8K3mcZgwTldydMCNOiNi/ZJrOB9BY+GlI3UxYzxBi9A= -github.com/libp2p/go-libp2p v0.37.0/go.mod h1:GOKmSN99scDuYGTwaTbQPR8Nt6dxrK3ue7OjW2NGDg4= +github.com/libp2p/go-libp2p v0.37.2 h1:Irh+n9aDPTLt9wJYwtlHu6AhMUipbC1cGoJtOiBqI9c= +github.com/libp2p/go-libp2p v0.37.2/go.mod h1:M8CRRywYkqC6xKHdZ45hmqVckBj5z4mRLIMLWReypz8= github.com/libp2p/go-libp2p-asn-util v0.4.1 h1:xqL7++IKD9TBFMgnLPZR6/6iYhawHKHl950SO9L6n94= github.com/libp2p/go-libp2p-asn-util v0.4.1/go.mod h1:d/NI6XZ9qxw67b4e+NgpQexCIiFYJjErASrYW4PFDN8= github.com/libp2p/go-libp2p-kad-dht v0.27.0 h1:1Ea32tVTPiAfaLpPMbaBWFJgbsi/JpMqC2YBuFdf32o= @@ -332,8 +332,8 @@ github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y9 github.com/multiformats/go-multiaddr v0.13.0 h1:BCBzs61E3AGHcYYTv8dqRH43ZfyrqM8RXVPT8t13tLQ= github.com/multiformats/go-multiaddr v0.13.0/go.mod h1:sBXrNzucqkFJhvKOiwwLyqamGa/P5EIXNPLovyhQCII= github.com/multiformats/go-multiaddr-dns v0.3.0/go.mod h1:mNzQ4eTGDg0ll1N9jKPOUogZPoJ30W8a7zk66FQPpdQ= -github.com/multiformats/go-multiaddr-dns v0.4.0 h1:P76EJ3qzBXpUXZ3twdCDx/kvagMsNo0LMFXpyms/zgU= -github.com/multiformats/go-multiaddr-dns v0.4.0/go.mod h1:7hfthtB4E4pQwirrz+J0CcDUfbWzTqEzVyYKKIKpgkc= +github.com/multiformats/go-multiaddr-dns v0.4.1 h1:whi/uCLbDS3mSEUMb1MsoT4uzUeZB0N32yzufqS0i5M= +github.com/multiformats/go-multiaddr-dns v0.4.1/go.mod h1:7hfthtB4E4pQwirrz+J0CcDUfbWzTqEzVyYKKIKpgkc= github.com/multiformats/go-multiaddr-fmt v0.1.0 h1:WLEFClPycPkp4fnIzoFoV9FVd49/eQsuaL3/CWe167E= github.com/multiformats/go-multiaddr-fmt v0.1.0/go.mod h1:hGtDIW4PU4BqJ50gW2quDuPVjyWNZxToGUh/HwTZYJo= github.com/multiformats/go-multibase v0.2.0 h1:isdYCVLvksgWlMW9OZRYJEa9pZETFivncJHmHnnd87g= @@ -343,8 +343,8 @@ github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI1 github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.2.3 h1:7Lyc8XfX/IY2jWb/gI7JP+o7JEq9hOa7BFvVU9RSh+U= github.com/multiformats/go-multihash v0.2.3/go.mod h1:dXgKXCXjBzdscBLk9JkjINiEsCKRVch90MdaGiKsvSM= -github.com/multiformats/go-multistream v0.5.0 h1:5htLSLl7lvJk3xx3qT/8Zm9J4K8vEOf/QGkvOGQAyiE= -github.com/multiformats/go-multistream v0.5.0/go.mod h1:n6tMZiwiP2wUsR8DgfDWw1dydlEqV3l6N3/GBsX6ILA= +github.com/multiformats/go-multistream v0.6.0 h1:ZaHKbsL404720283o4c/IHQXiS6gb8qAN5EIJ4PN5EA= +github.com/multiformats/go-multistream v0.6.0/go.mod h1:MOyoG5otO24cHIg8kf9QW2/NozURlkP/rvi2FQJyCPg= github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8= github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU= @@ -432,8 +432,8 @@ github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0leargg github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/quic-go/qpack v0.5.1 h1:giqksBPnT/HDtZ6VhtFKgoLOWmlyo9Ei6u9PqzIMbhI= github.com/quic-go/qpack v0.5.1/go.mod h1:+PC4XFrEskIVkcLzpEkbLqq1uCoxPhQuvK5rH1ZgaEg= -github.com/quic-go/quic-go v0.48.1 h1:y/8xmfWI9qmGTc+lBr4jKRUWLGSlSigv847ULJ4hYXA= -github.com/quic-go/quic-go v0.48.1/go.mod h1:yBgs3rWBOADpga7F+jJsb6Ybg1LSYiQvwWlLX+/6HMs= +github.com/quic-go/quic-go v0.48.2 h1:wsKXZPeGWpMpCGSWqOcqpW2wZYic/8T3aqiOID0/KWE= +github.com/quic-go/quic-go v0.48.2/go.mod h1:yBgs3rWBOADpga7F+jJsb6Ybg1LSYiQvwWlLX+/6HMs= github.com/quic-go/webtransport-go v0.8.1-0.20241018022711-4ac2c9250e66 h1:4WFk6u3sOT6pLa1kQ50ZVdm8BQFgJNA117cepZxtLIg= github.com/quic-go/webtransport-go v0.8.1-0.20241018022711-4ac2c9250e66/go.mod h1:Vp72IJajgeOL6ddqrAhmp7IM9zbTcgkQxD/YdxrVwMw= github.com/raulk/go-watchdog v1.3.0 h1:oUmdlHxdkXRJlwfG0O9omj8ukerm8MEQavSiDTEtBsk= diff --git a/go.mod b/go.mod index 292fbb95a..609bff982 100644 --- a/go.mod +++ b/go.mod @@ -41,7 +41,7 @@ require ( github.com/ipld/go-ipld-prime v0.21.0 github.com/libp2p/go-buffer-pool v0.1.0 github.com/libp2p/go-doh-resolver v0.4.0 - github.com/libp2p/go-libp2p v0.37.0 + github.com/libp2p/go-libp2p v0.37.2 github.com/libp2p/go-libp2p-kad-dht v0.27.0 github.com/libp2p/go-libp2p-record v0.2.0 github.com/libp2p/go-libp2p-routing-helpers v0.7.4 @@ -51,11 +51,11 @@ require ( github.com/mr-tron/base58 v1.2.0 github.com/multiformats/go-base32 v0.1.0 github.com/multiformats/go-multiaddr v0.13.0 - github.com/multiformats/go-multiaddr-dns v0.4.0 + github.com/multiformats/go-multiaddr-dns v0.4.1 github.com/multiformats/go-multibase v0.2.0 github.com/multiformats/go-multicodec v0.9.0 github.com/multiformats/go-multihash v0.2.3 - github.com/multiformats/go-multistream v0.5.0 + github.com/multiformats/go-multistream v0.6.0 github.com/polydawn/refmt v0.89.0 github.com/prometheus/client_golang v1.20.5 github.com/samber/lo v1.47.0 @@ -168,7 +168,7 @@ require ( github.com/prometheus/common v0.60.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/quic-go/qpack v0.5.1 // indirect - github.com/quic-go/quic-go v0.48.1 // indirect + github.com/quic-go/quic-go v0.48.2 // indirect github.com/quic-go/webtransport-go v0.8.1-0.20241018022711-4ac2c9250e66 // indirect github.com/raulk/go-watchdog v1.3.0 // indirect github.com/stretchr/objx v0.5.2 // indirect diff --git a/go.sum b/go.sum index 2e41a5014..c92125e8f 100644 --- a/go.sum +++ b/go.sum @@ -273,8 +273,8 @@ github.com/libp2p/go-doh-resolver v0.4.0 h1:gUBa1f1XsPwtpE1du0O+nnZCUqtG7oYi7Bb+ github.com/libp2p/go-doh-resolver v0.4.0/go.mod h1:v1/jwsFusgsWIGX/c6vCRrnJ60x7bhTiq/fs2qt0cAg= github.com/libp2p/go-flow-metrics v0.2.0 h1:EIZzjmeOE6c8Dav0sNv35vhZxATIXWZg6j/C08XmmDw= github.com/libp2p/go-flow-metrics v0.2.0/go.mod h1:st3qqfu8+pMfh+9Mzqb2GTiwrAGjIPszEjZmtksN8Jc= -github.com/libp2p/go-libp2p v0.37.0 h1:8K3mcZgwTldydMCNOiNi/ZJrOB9BY+GlI3UxYzxBi9A= -github.com/libp2p/go-libp2p v0.37.0/go.mod h1:GOKmSN99scDuYGTwaTbQPR8Nt6dxrK3ue7OjW2NGDg4= +github.com/libp2p/go-libp2p v0.37.2 h1:Irh+n9aDPTLt9wJYwtlHu6AhMUipbC1cGoJtOiBqI9c= +github.com/libp2p/go-libp2p v0.37.2/go.mod h1:M8CRRywYkqC6xKHdZ45hmqVckBj5z4mRLIMLWReypz8= github.com/libp2p/go-libp2p-asn-util v0.4.1 h1:xqL7++IKD9TBFMgnLPZR6/6iYhawHKHl950SO9L6n94= github.com/libp2p/go-libp2p-asn-util v0.4.1/go.mod h1:d/NI6XZ9qxw67b4e+NgpQexCIiFYJjErASrYW4PFDN8= github.com/libp2p/go-libp2p-kad-dht v0.27.0 h1:1Ea32tVTPiAfaLpPMbaBWFJgbsi/JpMqC2YBuFdf32o= @@ -333,8 +333,8 @@ github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y9 github.com/multiformats/go-multiaddr v0.13.0 h1:BCBzs61E3AGHcYYTv8dqRH43ZfyrqM8RXVPT8t13tLQ= github.com/multiformats/go-multiaddr v0.13.0/go.mod h1:sBXrNzucqkFJhvKOiwwLyqamGa/P5EIXNPLovyhQCII= github.com/multiformats/go-multiaddr-dns v0.3.0/go.mod h1:mNzQ4eTGDg0ll1N9jKPOUogZPoJ30W8a7zk66FQPpdQ= -github.com/multiformats/go-multiaddr-dns v0.4.0 h1:P76EJ3qzBXpUXZ3twdCDx/kvagMsNo0LMFXpyms/zgU= -github.com/multiformats/go-multiaddr-dns v0.4.0/go.mod h1:7hfthtB4E4pQwirrz+J0CcDUfbWzTqEzVyYKKIKpgkc= +github.com/multiformats/go-multiaddr-dns v0.4.1 h1:whi/uCLbDS3mSEUMb1MsoT4uzUeZB0N32yzufqS0i5M= +github.com/multiformats/go-multiaddr-dns v0.4.1/go.mod h1:7hfthtB4E4pQwirrz+J0CcDUfbWzTqEzVyYKKIKpgkc= github.com/multiformats/go-multiaddr-fmt v0.1.0 h1:WLEFClPycPkp4fnIzoFoV9FVd49/eQsuaL3/CWe167E= github.com/multiformats/go-multiaddr-fmt v0.1.0/go.mod h1:hGtDIW4PU4BqJ50gW2quDuPVjyWNZxToGUh/HwTZYJo= github.com/multiformats/go-multibase v0.2.0 h1:isdYCVLvksgWlMW9OZRYJEa9pZETFivncJHmHnnd87g= @@ -344,8 +344,8 @@ github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI1 github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.2.3 h1:7Lyc8XfX/IY2jWb/gI7JP+o7JEq9hOa7BFvVU9RSh+U= github.com/multiformats/go-multihash v0.2.3/go.mod h1:dXgKXCXjBzdscBLk9JkjINiEsCKRVch90MdaGiKsvSM= -github.com/multiformats/go-multistream v0.5.0 h1:5htLSLl7lvJk3xx3qT/8Zm9J4K8vEOf/QGkvOGQAyiE= -github.com/multiformats/go-multistream v0.5.0/go.mod h1:n6tMZiwiP2wUsR8DgfDWw1dydlEqV3l6N3/GBsX6ILA= +github.com/multiformats/go-multistream v0.6.0 h1:ZaHKbsL404720283o4c/IHQXiS6gb8qAN5EIJ4PN5EA= +github.com/multiformats/go-multistream v0.6.0/go.mod h1:MOyoG5otO24cHIg8kf9QW2/NozURlkP/rvi2FQJyCPg= github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8= github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU= @@ -433,8 +433,8 @@ github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0leargg github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/quic-go/qpack v0.5.1 h1:giqksBPnT/HDtZ6VhtFKgoLOWmlyo9Ei6u9PqzIMbhI= github.com/quic-go/qpack v0.5.1/go.mod h1:+PC4XFrEskIVkcLzpEkbLqq1uCoxPhQuvK5rH1ZgaEg= -github.com/quic-go/quic-go v0.48.1 h1:y/8xmfWI9qmGTc+lBr4jKRUWLGSlSigv847ULJ4hYXA= -github.com/quic-go/quic-go v0.48.1/go.mod h1:yBgs3rWBOADpga7F+jJsb6Ybg1LSYiQvwWlLX+/6HMs= +github.com/quic-go/quic-go v0.48.2 h1:wsKXZPeGWpMpCGSWqOcqpW2wZYic/8T3aqiOID0/KWE= +github.com/quic-go/quic-go v0.48.2/go.mod h1:yBgs3rWBOADpga7F+jJsb6Ybg1LSYiQvwWlLX+/6HMs= github.com/quic-go/webtransport-go v0.8.1-0.20241018022711-4ac2c9250e66 h1:4WFk6u3sOT6pLa1kQ50ZVdm8BQFgJNA117cepZxtLIg= github.com/quic-go/webtransport-go v0.8.1-0.20241018022711-4ac2c9250e66/go.mod h1:Vp72IJajgeOL6ddqrAhmp7IM9zbTcgkQxD/YdxrVwMw= github.com/raulk/go-watchdog v1.3.0 h1:oUmdlHxdkXRJlwfG0O9omj8ukerm8MEQavSiDTEtBsk= From 596563704f67bd78dcdd56fd71ad939be7e68af9 Mon Sep 17 00:00:00 2001 From: Dreamacro Date: Wed, 4 Dec 2024 05:21:56 +0800 Subject: [PATCH 29/36] feat(filestore): add mmap reader option (#665) --- CHANGELOG.md | 1 + filestore/filereader.go | 61 +++++++++++++++++ filestore/filestore_test.go | 128 ++++++++++++++++++++---------------- filestore/fsrefstore.go | 37 ++++++++--- go.mod | 2 +- 5 files changed, 161 insertions(+), 68 deletions(-) create mode 100644 filestore/filereader.go diff --git a/CHANGELOG.md b/CHANGELOG.md index da6de76b2..bf21168fb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -66,6 +66,7 @@ The following emojis are used to highlight certain changes: - `routing/http/server`: added configurable routing timeout (`DefaultRoutingTimeout` being 30s) to prevent indefinite hangs during content/peer routing. Set custom duration via `WithRoutingTimeout`. [#720](https://github.com/ipfs/boxo/pull/720) - `routing/http/server`: exposes Prometheus metrics on `prometheus.DefaultRegisterer` and a custom one can be provided via `WithPrometheusRegistry` [#722](https://github.com/ipfs/boxo/pull/722) - `gateway`: `NewCacheBlockStore` and `NewCarBackend` will use `prometheus.DefaultRegisterer` when a custom one is not specified via `WithPrometheusRegistry` [#722](https://github.com/ipfs/boxo/pull/722) +- `filestore`: added opt-in `WithMMapReader` option to `FileManager` to enable memory-mapped file reads [#665](https://github.com/ipfs/boxo/pull/665) ### Changed diff --git a/filestore/filereader.go b/filestore/filereader.go new file mode 100644 index 000000000..fba3cc942 --- /dev/null +++ b/filestore/filereader.go @@ -0,0 +1,61 @@ +package filestore + +import ( + "io" + "os" + + "golang.org/x/exp/mmap" +) + +type FileReader interface { + io.ReaderAt + io.Closer +} + +var _ FileReader = (*stdReader)(nil) + +type stdReader struct { + f *os.File +} + +// ReadAt implements the FileReader interface. +func (r *stdReader) ReadAt(p []byte, off int64) (n int, err error) { + return r.f.ReadAt(p, off) +} + +// Close implements the FileReader interface. +func (r *stdReader) Close() error { + return r.f.Close() +} + +func newStdReader(path string) (FileReader, error) { + f, err := os.Open(path) + if err != nil { + return nil, err + } + return &stdReader{f: f}, nil +} + +var _ FileReader = (*mmapReader)(nil) + +type mmapReader struct { + m *mmap.ReaderAt +} + +// ReadAt implements the FileReader interface. +func (r *mmapReader) ReadAt(p []byte, off int64) (n int, err error) { + return r.m.ReadAt(p, off) +} + +// Close implements the FileReader interface. +func (r *mmapReader) Close() error { + return r.m.Close() +} + +func newMmapReader(path string) (FileReader, error) { + m, err := mmap.Open(path) + if err != nil { + return nil, err + } + return &mmapReader{m: m}, nil +} diff --git a/filestore/filestore_test.go b/filestore/filestore_test.go index 9d455193a..4d17adbe7 100644 --- a/filestore/filestore_test.go +++ b/filestore/filestore_test.go @@ -18,14 +18,14 @@ import ( var bg = context.Background() -func newTestFilestore(t *testing.T) (string, *Filestore) { +func newTestFilestore(t *testing.T, option ...Option) (string, *Filestore) { mds := ds.NewMapDatastore() testdir, err := os.MkdirTemp("", "filestore-test") if err != nil { t.Fatal(err) } - fm := NewFileManager(mds, testdir) + fm := NewFileManager(mds, testdir, option...) fm.AllowFiles = true bs := blockstore.NewBlockstore(mds) @@ -48,62 +48,74 @@ func makeFile(dir string, data []byte) (string, error) { } func TestBasicFilestore(t *testing.T) { - dir, fs := newTestFilestore(t) - - buf := make([]byte, 1000) - rand.Read(buf) - - fname, err := makeFile(dir, buf) - if err != nil { - t.Fatal(err) - } - - var cids []cid.Cid - for i := 0; i < 100; i++ { - n := &posinfo.FilestoreNode{ - PosInfo: &posinfo.PosInfo{ - FullPath: fname, - Offset: uint64(i * 10), - }, - Node: dag.NewRawNode(buf[i*10 : (i+1)*10]), - } - - err := fs.Put(bg, n) - if err != nil { - t.Fatal(err) - } - cids = append(cids, n.Node.Cid()) - } - - for i, c := range cids { - blk, err := fs.Get(bg, c) - if err != nil { - t.Fatal(err) - } - - if !bytes.Equal(blk.RawData(), buf[i*10:(i+1)*10]) { - t.Fatal("data didnt match on the way out") - } - } - - kch, err := fs.AllKeysChan(context.Background()) - if err != nil { - t.Fatal(err) - } - - out := make(map[string]struct{}) - for c := range kch { - out[c.KeyString()] = struct{}{} - } - - if len(out) != len(cids) { - t.Fatal("mismatch in number of entries") - } - - for _, c := range cids { - if _, ok := out[c.KeyString()]; !ok { - t.Fatal("missing cid: ", c) - } + cases := []struct { + name string + options []Option + }{ + {"default", nil}, + {"mmap", []Option{WithMMapReader()}}, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + dir, fs := newTestFilestore(t, c.options...) + + buf := make([]byte, 1000) + rand.Read(buf) + + fname, err := makeFile(dir, buf) + if err != nil { + t.Fatal(err) + } + + var cids []cid.Cid + for i := 0; i < 100; i++ { + n := &posinfo.FilestoreNode{ + PosInfo: &posinfo.PosInfo{ + FullPath: fname, + Offset: uint64(i * 10), + }, + Node: dag.NewRawNode(buf[i*10 : (i+1)*10]), + } + + err := fs.Put(bg, n) + if err != nil { + t.Fatal(err) + } + cids = append(cids, n.Node.Cid()) + } + + for i, c := range cids { + blk, err := fs.Get(bg, c) + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(blk.RawData(), buf[i*10:(i+1)*10]) { + t.Fatal("data didnt match on the way out") + } + } + + kch, err := fs.AllKeysChan(context.Background()) + if err != nil { + t.Fatal(err) + } + + out := make(map[string]struct{}) + for c := range kch { + out[c.KeyString()] = struct{}{} + } + + if len(out) != len(cids) { + t.Fatal("mismatch in number of entries") + } + + for _, c := range cids { + if _, ok := out[c.KeyString()]; !ok { + t.Fatal("missing cid: ", c) + } + } + }) } } diff --git a/filestore/fsrefstore.go b/filestore/fsrefstore.go index 158eadf7a..eb7f190d0 100644 --- a/filestore/fsrefstore.go +++ b/filestore/fsrefstore.go @@ -25,6 +25,8 @@ import ( // FilestorePrefix identifies the key prefix for FileManager blocks. var FilestorePrefix = ds.NewKey("filestore") +type Option func(*FileManager) + // FileManager is a blockstore implementation which stores special // blocks FilestoreNode type. These nodes only contain a reference // to the actual location of the block data in the filesystem @@ -34,6 +36,7 @@ type FileManager struct { AllowUrls bool ds ds.Batching root string + makeReader func(path string) (FileReader, error) } // CorruptReferenceError implements the error interface. @@ -51,11 +54,32 @@ func (c CorruptReferenceError) Error() string { return c.Err.Error() } +// WithMMapReader sets the FileManager's reader factory to use memory-mapped file I/O. +// On Windows, when reading and writing to a file simultaneously, the system would consume +// a significant amount of memory due to caching. This memory usage is not reflected in +// the application but in the system. Using memory-mapped files (implemented with +// CreateFileMapping on Windows) avoids this issue. +func WithMMapReader() Option { + return func(f *FileManager) { + f.makeReader = newMmapReader + } +} + // NewFileManager initializes a new file manager with the given // datastore and root. All FilestoreNodes paths are relative to the // root path given here, which is prepended for any operations. -func NewFileManager(ds ds.Batching, root string) *FileManager { - return &FileManager{ds: dsns.Wrap(ds, FilestorePrefix), root: root} +func NewFileManager(ds ds.Batching, root string, options ...Option) *FileManager { + f := &FileManager{ + ds: dsns.Wrap(ds, FilestorePrefix), + root: root, + makeReader: newStdReader, + } + + for _, option := range options { + option(f) + } + + return f } // AllKeysChan returns a channel from which to read the keys stored in @@ -175,7 +199,7 @@ func (f *FileManager) readFileDataObj(m mh.Multihash, d *pb.DataObj) ([]byte, er p := filepath.FromSlash(d.GetFilePath()) abspath := filepath.Join(f.root, p) - fi, err := os.Open(abspath) + fi, err := f.makeReader(abspath) if os.IsNotExist(err) { return nil, &CorruptReferenceError{StatusFileNotFound, err} } else if err != nil { @@ -183,13 +207,8 @@ func (f *FileManager) readFileDataObj(m mh.Multihash, d *pb.DataObj) ([]byte, er } defer fi.Close() - _, err = fi.Seek(int64(d.GetOffset()), io.SeekStart) - if err != nil { - return nil, &CorruptReferenceError{StatusFileError, err} - } - outbuf := make([]byte, d.GetSize_()) - _, err = io.ReadFull(fi, outbuf) + _, err = fi.ReadAt(outbuf, int64(d.GetOffset())) if err == io.EOF || err == io.ErrUnexpectedEOF { return nil, &CorruptReferenceError{StatusFileChanged, err} } else if err != nil { diff --git a/go.mod b/go.mod index 609bff982..3cb9d498d 100644 --- a/go.mod +++ b/go.mod @@ -75,6 +75,7 @@ require ( go.opentelemetry.io/otel/trace v1.31.0 go.uber.org/multierr v1.11.0 go.uber.org/zap v1.27.0 + golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c golang.org/x/oauth2 v0.23.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.26.0 @@ -185,7 +186,6 @@ require ( go.uber.org/fx v1.23.0 // indirect go.uber.org/mock v0.5.0 // indirect golang.org/x/crypto v0.28.0 // indirect - golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c // indirect golang.org/x/mod v0.21.0 // indirect golang.org/x/net v0.30.0 // indirect golang.org/x/text v0.19.0 // indirect From 86120e2731fee1d626c46e149254813f142bcb73 Mon Sep 17 00:00:00 2001 From: Hannah Howard Date: Wed, 4 Dec 2024 10:31:07 -0800 Subject: [PATCH 30/36] feat(session): do not record erroneous session want sends (#452) * feat(session): do not record erroneous session want sends Co-authored-by: gammazero <11790789+gammazero@users.noreply.github.com> --- CHANGELOG.md | 2 ++ .../internal/peermanager/peermanager.go | 8 ++++--- bitswap/client/internal/session/session.go | 2 +- .../client/internal/session/session_test.go | 8 ++++--- .../internal/session/sessionwantsender.go | 21 ++++++++++++++----- .../session/sessionwantsender_test.go | 3 ++- .../sessionmanager/sessionmanager_test.go | 8 +++---- 7 files changed, 35 insertions(+), 17 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index bf21168fb..d4244b4b9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -80,6 +80,8 @@ The following emojis are used to highlight certain changes: ### Fixed +- Do not erroneously update the state of sent wants when a send a peer disconnected and the send did not happen. [#452](https://github.com/ipfs/boxo/pull/452) + ### Security ## [v0.24.3] diff --git a/bitswap/client/internal/peermanager/peermanager.go b/bitswap/client/internal/peermanager/peermanager.go index 78a1d4c88..76c636062 100644 --- a/bitswap/client/internal/peermanager/peermanager.go +++ b/bitswap/client/internal/peermanager/peermanager.go @@ -143,13 +143,15 @@ func (pm *PeerManager) BroadcastWantHaves(ctx context.Context, wantHaves []cid.C // SendWants sends the given want-blocks and want-haves to the given peer. // It filters out wants that have previously been sent to the peer. -func (pm *PeerManager) SendWants(ctx context.Context, p peer.ID, wantBlocks []cid.Cid, wantHaves []cid.Cid) { +func (pm *PeerManager) SendWants(ctx context.Context, p peer.ID, wantBlocks []cid.Cid, wantHaves []cid.Cid) bool { pm.pqLk.Lock() defer pm.pqLk.Unlock() - if _, ok := pm.peerQueues[p]; ok { - pm.pwm.sendWants(p, wantBlocks, wantHaves) + if _, ok := pm.peerQueues[p]; !ok { + return false } + pm.pwm.sendWants(p, wantBlocks, wantHaves) + return true } // SendCancels sends cancels for the given keys to all peers who had previously diff --git a/bitswap/client/internal/session/session.go b/bitswap/client/internal/session/session.go index 3e2a9b53d..f11d7d83e 100644 --- a/bitswap/client/internal/session/session.go +++ b/bitswap/client/internal/session/session.go @@ -38,7 +38,7 @@ type PeerManager interface { // interested in a peer's connection state UnregisterSession(uint64) // SendWants tells the PeerManager to send wants to the given peer - SendWants(ctx context.Context, peerId peer.ID, wantBlocks []cid.Cid, wantHaves []cid.Cid) + SendWants(ctx context.Context, peerId peer.ID, wantBlocks []cid.Cid, wantHaves []cid.Cid) bool // BroadcastWantHaves sends want-haves to all connected peers (used for // session discovery) BroadcastWantHaves(context.Context, []cid.Cid) diff --git a/bitswap/client/internal/session/session_test.go b/bitswap/client/internal/session/session_test.go index 061e298e5..2dd1e1be6 100644 --- a/bitswap/client/internal/session/session_test.go +++ b/bitswap/client/internal/session/session_test.go @@ -141,9 +141,11 @@ func newFakePeerManager() *fakePeerManager { } } -func (pm *fakePeerManager) RegisterSession(peer.ID, bspm.Session) {} -func (pm *fakePeerManager) UnregisterSession(uint64) {} -func (pm *fakePeerManager) SendWants(context.Context, peer.ID, []cid.Cid, []cid.Cid) {} +func (pm *fakePeerManager) RegisterSession(peer.ID, bspm.Session) {} +func (pm *fakePeerManager) UnregisterSession(uint64) {} +func (pm *fakePeerManager) SendWants(context.Context, peer.ID, []cid.Cid, []cid.Cid) bool { + return true +} func (pm *fakePeerManager) BroadcastWantHaves(ctx context.Context, cids []cid.Cid) { select { case pm.wantReqs <- wantReq{cids}: diff --git a/bitswap/client/internal/session/sessionwantsender.go b/bitswap/client/internal/session/sessionwantsender.go index 338150ec3..09a9381ea 100644 --- a/bitswap/client/internal/session/sessionwantsender.go +++ b/bitswap/client/internal/session/sessionwantsender.go @@ -513,6 +513,7 @@ func (sws *sessionWantSender) processExhaustedWants(exhausted []cid.Cid) { type wantSets struct { wantBlocks *cid.Set wantHaves *cid.Set + sent bool } type allWants map[peer.ID]*wantSets @@ -551,9 +552,6 @@ func (sws *sessionWantSender) sendNextWants(newlyAvailable []peer.ID) { continue } - // Record that we are sending a want-block for this want to the peer - sws.setWantSentTo(c, wi.bestPeer) - // Send a want-block to the chosen peer toSend.forPeer(wi.bestPeer).wantBlocks.Add(c) @@ -567,6 +565,16 @@ func (sws *sessionWantSender) sendNextWants(newlyAvailable []peer.ID) { // Send any wants we've collected sws.sendWants(toSend) + + for c, wi := range sws.wants { + if wi.bestPeer != "" && wi.sentTo == "" { + // check if a want block was successfully sent to the best peer + if toSend.forPeer(wi.bestPeer).sent { + // Record that we are sending a want-block for this want to the peer + sws.setWantSentTo(c, wi.bestPeer) + } + } + } } // sendWants sends want-have and want-blocks to the appropriate peers @@ -584,8 +592,11 @@ func (sws *sessionWantSender) sendWants(sends allWants) { // precedence over want-haves. wblks := snd.wantBlocks.Keys() whaves := snd.wantHaves.Keys() - sws.pm.SendWants(sws.ctx, p, wblks, whaves) - + snd.sent = sws.pm.SendWants(sws.ctx, p, wblks, whaves) + if !snd.sent { + // Do not update state if the wants not sent. + continue + } // Inform the session that we've sent the wants sws.onSend(p, wblks, whaves) diff --git a/bitswap/client/internal/session/sessionwantsender_test.go b/bitswap/client/internal/session/sessionwantsender_test.go index ac094ac06..e5589dd58 100644 --- a/bitswap/client/internal/session/sessionwantsender_test.go +++ b/bitswap/client/internal/session/sessionwantsender_test.go @@ -82,7 +82,7 @@ func (*mockPeerManager) UnregisterSession(uint64) {} func (*mockPeerManager) BroadcastWantHaves(context.Context, []cid.Cid) {} func (*mockPeerManager) SendCancels(context.Context, []cid.Cid) {} -func (pm *mockPeerManager) SendWants(ctx context.Context, p peer.ID, wantBlocks []cid.Cid, wantHaves []cid.Cid) { +func (pm *mockPeerManager) SendWants(ctx context.Context, p peer.ID, wantBlocks []cid.Cid, wantHaves []cid.Cid) bool { pm.lk.Lock() defer pm.lk.Unlock() @@ -92,6 +92,7 @@ func (pm *mockPeerManager) SendWants(ctx context.Context, p peer.ID, wantBlocks pm.peerSends[p] = sw } sw.add(wantBlocks, wantHaves) + return true } func (pm *mockPeerManager) waitNextWants() map[peer.ID]*sentWants { diff --git a/bitswap/client/internal/sessionmanager/sessionmanager_test.go b/bitswap/client/internal/sessionmanager/sessionmanager_test.go index e8259b1d8..bad26ad90 100644 --- a/bitswap/client/internal/sessionmanager/sessionmanager_test.go +++ b/bitswap/client/internal/sessionmanager/sessionmanager_test.go @@ -66,10 +66,10 @@ type fakePeerManager struct { cancels []cid.Cid } -func (*fakePeerManager) RegisterSession(peer.ID, bspm.Session) {} -func (*fakePeerManager) UnregisterSession(uint64) {} -func (*fakePeerManager) SendWants(context.Context, peer.ID, []cid.Cid, []cid.Cid) {} -func (*fakePeerManager) BroadcastWantHaves(context.Context, []cid.Cid) {} +func (*fakePeerManager) RegisterSession(peer.ID, bspm.Session) {} +func (*fakePeerManager) UnregisterSession(uint64) {} +func (*fakePeerManager) SendWants(context.Context, peer.ID, []cid.Cid, []cid.Cid) bool { return true } +func (*fakePeerManager) BroadcastWantHaves(context.Context, []cid.Cid) {} func (fpm *fakePeerManager) SendCancels(ctx context.Context, cancels []cid.Cid) { fpm.lk.Lock() defer fpm.lk.Unlock() From e9446bb594d6dbdbd9e239c1716e089f945aacd4 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Wed, 4 Dec 2024 10:00:22 -1000 Subject: [PATCH 31/36] Reenable flaky bitswap tests (#740) Recent improvements in bitswap should stabalize these tests. Closes #327 --- bitswap/client/internal/session/session_test.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/bitswap/client/internal/session/session_test.go b/bitswap/client/internal/session/session_test.go index 2dd1e1be6..5e6655124 100644 --- a/bitswap/client/internal/session/session_test.go +++ b/bitswap/client/internal/session/session_test.go @@ -12,7 +12,6 @@ import ( bspm "github.com/ipfs/boxo/bitswap/client/internal/peermanager" bssim "github.com/ipfs/boxo/bitswap/client/internal/sessioninterestmanager" bsspm "github.com/ipfs/boxo/bitswap/client/internal/sessionpeermanager" - "github.com/ipfs/boxo/internal/test" blocks "github.com/ipfs/go-block-format" cid "github.com/ipfs/go-cid" delay "github.com/ipfs/go-ipfs-delay" @@ -155,8 +154,6 @@ func (pm *fakePeerManager) BroadcastWantHaves(ctx context.Context, cids []cid.Ci func (pm *fakePeerManager) SendCancels(ctx context.Context, cancels []cid.Cid) {} func TestSessionGetBlocks(t *testing.T) { - test.Flaky(t) - ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) fpm := newFakePeerManager() fspm := newFakeSessionPeerManager() @@ -175,9 +172,7 @@ func TestSessionGetBlocks(t *testing.T) { } _, err := session.GetBlocks(ctx, cids) - if err != nil { - t.Fatal("error getting blocks") - } + require.NoError(t, err, "error getting blocks") // Wait for initial want request receivedWantReq := <-fpm.wantReqs @@ -345,8 +340,6 @@ func TestSessionOnPeersExhausted(t *testing.T) { } func TestSessionFailingToGetFirstBlock(t *testing.T) { - test.Flaky(t) - ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() fpm := newFakePeerManager() From f6befafdbb77a2581381e7ff5e465abded7c9368 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Wed, 4 Dec 2024 17:13:38 -1000 Subject: [PATCH 32/36] removed Startup function from ProviderQueryManager (#741) * removed Startup function from ProviderQueryManager Now `providerquerymanager.New` creates a `ProvicerQueryManager` that is already started. There is no use case for starting PQM at a later time than it is created. Removing the need to call a `Statup` function separately from `New` is more convenient and reduces the opportunity for a problem if calling `Startup` is missed or if called multiple times. * Remove flaky portion of test - requires synchronization to test sucessive timer delays --- CHANGELOG.md | 1 + bitswap/client/bitswap_with_sessions_test.go | 1 - bitswap/client/client.go | 1 - bitswap/client/internal/session/session_test.go | 16 ---------------- .../providerquerymanager/providerquerymanager.go | 7 ++----- .../providerquerymanager_test.go | 11 ----------- 6 files changed, 3 insertions(+), 34 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d4244b4b9..e1379979e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -67,6 +67,7 @@ The following emojis are used to highlight certain changes: - `routing/http/server`: exposes Prometheus metrics on `prometheus.DefaultRegisterer` and a custom one can be provided via `WithPrometheusRegistry` [#722](https://github.com/ipfs/boxo/pull/722) - `gateway`: `NewCacheBlockStore` and `NewCarBackend` will use `prometheus.DefaultRegisterer` when a custom one is not specified via `WithPrometheusRegistry` [#722](https://github.com/ipfs/boxo/pull/722) - `filestore`: added opt-in `WithMMapReader` option to `FileManager` to enable memory-mapped file reads [#665](https://github.com/ipfs/boxo/pull/665) +- `bitswap/routing` `ProviderQueryManager` does not require calling `Startup` separate from `New`. [#741](https://github.com/ipfs/boxo/pull/741) ### Changed diff --git a/bitswap/client/bitswap_with_sessions_test.go b/bitswap/client/bitswap_with_sessions_test.go index 110c43f5c..5d5ac8226 100644 --- a/bitswap/client/bitswap_with_sessions_test.go +++ b/bitswap/client/bitswap_with_sessions_test.go @@ -134,7 +134,6 @@ func TestCustomProviderQueryManager(t *testing.T) { if err != nil { t.Fatal(err) } - pqm.Startup() bs := bitswap.New(ctx, a.Adapter, pqm, a.Blockstore, bitswap.WithClientOption(client.WithDefaultProviderQueryManager(false))) a.Exchange.Close() // close old to be sure. diff --git a/bitswap/client/client.go b/bitswap/client/client.go index 4523962c1..5f950588a 100644 --- a/bitswap/client/client.go +++ b/bitswap/client/client.go @@ -190,7 +190,6 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, providerFinder Pr // Should not be possible to hit this panic(err) } - pqm.Startup() bs.pqm = pqm } diff --git a/bitswap/client/internal/session/session_test.go b/bitswap/client/internal/session/session_test.go index 5e6655124..c0d26a91d 100644 --- a/bitswap/client/internal/session/session_test.go +++ b/bitswap/client/internal/session/session_test.go @@ -357,7 +357,6 @@ func TestSessionFailingToGetFirstBlock(t *testing.T) { for _, block := range blks { cids = append(cids, block.Cid()) } - startTick := time.Now() _, err := session.GetBlocks(ctx, cids) require.NoError(t, err, "error getting blocks") @@ -389,7 +388,6 @@ func TestSessionFailingToGetFirstBlock(t *testing.T) { case <-ctx.Done(): t.Fatal("Did not find more peers") } - firstTickLength := time.Since(startTick) // Wait for another broadcast to occur select { @@ -402,7 +400,6 @@ func TestSessionFailingToGetFirstBlock(t *testing.T) { } // Wait for another broadcast to occur - startTick = time.Now() select { case receivedWantReq := <-fpm.wantReqs: if len(receivedWantReq.cids) < len(cids) { @@ -412,14 +409,7 @@ func TestSessionFailingToGetFirstBlock(t *testing.T) { t.Fatal("Never rebroadcast want list") } - // Tick should take longer - consecutiveTickLength := time.Since(startTick) - if firstTickLength > consecutiveTickLength { - t.Fatal("Should have increased tick length after first consecutive tick") - } - // Wait for another broadcast to occur - startTick = time.Now() select { case receivedWantReq := <-fpm.wantReqs: if len(receivedWantReq.cids) < len(cids) { @@ -429,12 +419,6 @@ func TestSessionFailingToGetFirstBlock(t *testing.T) { t.Fatal("Never rebroadcast want list") } - // Tick should take longer - secondConsecutiveTickLength := time.Since(startTick) - if consecutiveTickLength > secondConsecutiveTickLength { - t.Fatal("Should have increased tick length after first consecutive tick") - } - // Should not have tried to find peers on consecutive ticks select { case <-fpf.findMorePeersRequested: diff --git a/routing/providerquerymanager/providerquerymanager.go b/routing/providerquerymanager/providerquerymanager.go index b5a5b3fa0..98497ee66 100644 --- a/routing/providerquerymanager/providerquerymanager.go +++ b/routing/providerquerymanager/providerquerymanager.go @@ -150,12 +150,9 @@ func New(ctx context.Context, dialer ProviderQueryDialer, router ProviderQueryRo } } - return pqm, nil -} - -// Startup starts processing for the ProviderQueryManager. -func (pqm *ProviderQueryManager) Startup() { go pqm.run() + + return pqm, nil } type inProgressRequest struct { diff --git a/routing/providerquerymanager/providerquerymanager_test.go b/routing/providerquerymanager/providerquerymanager_test.go index 7369231de..1be26c4e3 100644 --- a/routing/providerquerymanager/providerquerymanager_test.go +++ b/routing/providerquerymanager/providerquerymanager_test.go @@ -76,7 +76,6 @@ func TestNormalSimultaneousFetch(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) - providerQueryManager.Startup() keys := random.Cids(2) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) @@ -114,7 +113,6 @@ func TestDedupingProviderRequests(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) - providerQueryManager.Startup() key := random.Cids(1)[0] sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) @@ -155,7 +153,6 @@ func TestCancelOneRequestDoesNotTerminateAnother(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) - providerQueryManager.Startup() key := random.Cids(1)[0] @@ -202,7 +199,6 @@ func TestCancelManagerExitsGracefully(t *testing.T) { managerCtx, managerCancel := context.WithTimeout(ctx, 5*time.Millisecond) defer managerCancel() providerQueryManager := mustNotErr(New(managerCtx, fpd, fpn)) - providerQueryManager.Startup() key := random.Cids(1)[0] @@ -238,7 +234,6 @@ func TestPeersWithConnectionErrorsNotAddedToPeerList(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) - providerQueryManager.Startup() key := random.Cids(1)[0] @@ -275,7 +270,6 @@ func TestRateLimitingRequests(t *testing.T) { ctx, cancel := context.WithCancel(ctx) defer cancel() providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxInProcessRequests(maxInProcessRequests))) - providerQueryManager.Startup() keys := random.Cids(maxInProcessRequests + 1) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) @@ -317,7 +311,6 @@ func TestUnlimitedRequests(t *testing.T) { ctx, cancel := context.WithCancel(ctx) defer cancel() providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxInProcessRequests(0))) - providerQueryManager.Startup() keys := random.Cids(inProcessRequests) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) @@ -355,7 +348,6 @@ func TestFindProviderTimeout(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(2*time.Millisecond))) - providerQueryManager.Startup() keys := random.Cids(1) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) @@ -379,7 +371,6 @@ func TestFindProviderPreCanceled(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(100*time.Millisecond))) - providerQueryManager.Startup() keys := random.Cids(1) sessionCtx, cancel := context.WithCancel(ctx) @@ -404,7 +395,6 @@ func TestCancelFindProvidersAfterCompletion(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(100*time.Millisecond))) - providerQueryManager.Startup() keys := random.Cids(1) sessionCtx, cancel := context.WithCancel(ctx) @@ -437,7 +427,6 @@ func TestLimitedProviders(t *testing.T) { } ctx := context.Background() providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxProviders(max), WithMaxTimeout(100*time.Millisecond))) - providerQueryManager.Startup() keys := random.Cids(1) providersChan := providerQueryManager.FindProvidersAsync(ctx, keys[0], 0) From ef258087623a2c13990edbcbf5172b9f1e630557 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Fri, 6 Dec 2024 10:34:05 -1000 Subject: [PATCH 33/36] chore: no lifecycle context to shutdown ProviderQueryManager (#734) * no lifecycle context to shutdown ProviderQueryManager, use Close function instead. --- CHANGELOG.md | 1 + bitswap/client/bitswap_with_sessions_test.go | 10 +-- bitswap/client/client.go | 5 +- .../providerquerymanager.go | 43 ++++++----- .../providerquerymanager_test.go | 72 +++++++++---------- 5 files changed, 71 insertions(+), 60 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e1379979e..7f4d0c094 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -68,6 +68,7 @@ The following emojis are used to highlight certain changes: - `gateway`: `NewCacheBlockStore` and `NewCarBackend` will use `prometheus.DefaultRegisterer` when a custom one is not specified via `WithPrometheusRegistry` [#722](https://github.com/ipfs/boxo/pull/722) - `filestore`: added opt-in `WithMMapReader` option to `FileManager` to enable memory-mapped file reads [#665](https://github.com/ipfs/boxo/pull/665) - `bitswap/routing` `ProviderQueryManager` does not require calling `Startup` separate from `New`. [#741](https://github.com/ipfs/boxo/pull/741) +- `bitswap/routing` ProviderQueryManager does not use liftcycle context. ### Changed diff --git a/bitswap/client/bitswap_with_sessions_test.go b/bitswap/client/bitswap_with_sessions_test.go index 5d5ac8226..2fee84217 100644 --- a/bitswap/client/bitswap_with_sessions_test.go +++ b/bitswap/client/bitswap_with_sessions_test.go @@ -117,9 +117,6 @@ func assertBlockListsFrom(from peer.ID, got, exp []blocks.Block) error { // TestCustomProviderQueryManager tests that nothing breaks if we use a custom // PQM when creating bitswap. func TestCustomProviderQueryManager(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - vnet := getVirtualNetwork() router := mockrouting.NewServer() ig := testinstance.NewTestInstanceGenerator(vnet, router, nil, nil) @@ -130,10 +127,15 @@ func TestCustomProviderQueryManager(t *testing.T) { b := ig.Next() // Replace bitswap in instance a with our customized one. - pqm, err := providerquerymanager.New(ctx, a.Adapter, router.Client(a.Identity)) + pqm, err := providerquerymanager.New(a.Adapter, router.Client(a.Identity)) if err != nil { t.Fatal(err) } + defer pqm.Close() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + bs := bitswap.New(ctx, a.Adapter, pqm, a.Blockstore, bitswap.WithClientOption(client.WithDefaultProviderQueryManager(false))) a.Exchange.Close() // close old to be sure. diff --git a/bitswap/client/client.go b/bitswap/client/client.go index 5f950588a..a115d07f6 100644 --- a/bitswap/client/client.go +++ b/bitswap/client/client.go @@ -182,7 +182,7 @@ func New(parent context.Context, network bsnet.BitSwapNetwork, providerFinder Pr if bs.providerFinder != nil && bs.defaultProviderQueryManager { // network can do dialing. - pqm, err := rpqm.New(ctx, network, bs.providerFinder, + pqm, err := rpqm.New(network, bs.providerFinder, rpqm.WithMaxInProcessRequests(16), rpqm.WithMaxProviders(10), rpqm.WithMaxTimeout(10*time.Second)) @@ -512,6 +512,9 @@ func (bs *Client) Close() error { close(bs.closing) bs.sm.Shutdown() bs.cancel() + if bs.pqm != nil { + bs.pqm.Close() + } bs.notif.Shutdown() }) return nil diff --git a/routing/providerquerymanager/providerquerymanager.go b/routing/providerquerymanager/providerquerymanager.go index 98497ee66..592f7f814 100644 --- a/routing/providerquerymanager/providerquerymanager.go +++ b/routing/providerquerymanager/providerquerymanager.go @@ -85,7 +85,8 @@ type cancelRequestMessage struct { // - ensure two findprovider calls for the same block don't run concurrently // - manage timeouts type ProviderQueryManager struct { - ctx context.Context + closeOnce sync.Once + closing chan struct{} dialer ProviderQueryDialer router ProviderQueryRouter providerQueryMessages chan providerQueryMessage @@ -133,9 +134,9 @@ func WithMaxProviders(count int) Option { // New initializes a new ProviderQueryManager for a given context and a given // network provider. -func New(ctx context.Context, dialer ProviderQueryDialer, router ProviderQueryRouter, opts ...Option) (*ProviderQueryManager, error) { +func New(dialer ProviderQueryDialer, router ProviderQueryRouter, opts ...Option) (*ProviderQueryManager, error) { pqm := &ProviderQueryManager{ - ctx: ctx, + closing: make(chan struct{}), dialer: dialer, router: router, providerQueryMessages: make(chan providerQueryMessage), @@ -155,6 +156,12 @@ func New(ctx context.Context, dialer ProviderQueryDialer, router ProviderQueryRo return pqm, nil } +func (pqm *ProviderQueryManager) Close() { + pqm.closeOnce.Do(func() { + close(pqm.closing) + }) +} + type inProgressRequest struct { providersSoFar []peer.AddrInfo incoming chan peer.AddrInfo @@ -180,7 +187,7 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, k: k, inProgressRequestChan: inProgressRequestChan, }: - case <-pqm.ctx.Done(): + case <-pqm.closing: ch := make(chan peer.AddrInfo) close(ch) span.End() @@ -196,7 +203,7 @@ func (pqm *ProviderQueryManager) FindProvidersAsync(sessionCtx context.Context, // get to receiveProviders. var receivedInProgressRequest inProgressRequest select { - case <-pqm.ctx.Done(): + case <-pqm.closing: ch := make(chan peer.AddrInfo) close(ch) span.End() @@ -256,7 +263,7 @@ func (pqm *ProviderQueryManager) receiveProviders(sessionCtx context.Context, k for receivedProviders.Len() > 0 || incomingProviders != nil { select { - case <-pqm.ctx.Done(): + case <-pqm.closing: return case <-sessionCtx.Done(): if incomingProviders != nil { @@ -300,7 +307,7 @@ func (pqm *ProviderQueryManager) cancelProviderRequest(ctx context.Context, k ci if !ok { return } - case <-pqm.ctx.Done(): + case <-pqm.closing: return } } @@ -316,13 +323,13 @@ func (pqm *ProviderQueryManager) findProviderWorker() { } // Read find provider requests until channel is closed. The channel is - // closed as soon as pqm.ctx is canceled, so there is no need to select on - // that context here. + // closed as soon as pqm.Close is called, so there is no need to select on + // any other channel to detect shutdown. for fpr := range pqm.providerRequestsProcessing.Out() { if findSem != nil { select { case findSem <- struct{}{}: - case <-pqm.ctx.Done(): + case <-pqm.closing: return } } @@ -362,7 +369,7 @@ func (pqm *ProviderQueryManager) findProviderWorker() { k: k, p: p, }: - case <-pqm.ctx.Done(): + case <-pqm.closing: return } }(p) @@ -374,7 +381,7 @@ func (pqm *ProviderQueryManager) findProviderWorker() { ctx: ctx, k: k, }: - case <-pqm.ctx.Done(): + case <-pqm.closing: } }(fpr.ctx, fpr.k) } @@ -402,7 +409,7 @@ func (pqm *ProviderQueryManager) run() { case nextMessage := <-pqm.providerQueryMessages: nextMessage.debugMessage() nextMessage.handle(pqm) - case <-pqm.ctx.Done(): + case <-pqm.closing: return } } @@ -423,7 +430,7 @@ func (rpm *receivedProviderMessage) handle(pqm *ProviderQueryManager) { for listener := range requestStatus.listeners { select { case listener <- rpm.p: - case <-pqm.ctx.Done(): + case <-pqm.closing: return } } @@ -458,12 +465,12 @@ func (npqm *newProvideQueryMessage) debugMessage() { func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { requestStatus, ok := pqm.inProgressRequestStatuses[npqm.k] if !ok { - ctx, cancelFn := context.WithCancel(pqm.ctx) + ctx, cancelFn := context.WithCancel(context.Background()) span := trace.SpanFromContext(npqm.ctx) span.AddEvent("NewQuery", trace.WithAttributes(attribute.Stringer("cid", npqm.k))) ctx = trace.ContextWithSpan(ctx, span) - // Use context derived from pqm.ctx here, and not the context from the + // Use context derived from background here, and not the context from the // request (npqm.ctx), because this inProgressRequestStatus applies to // all in-progress requests for the CID (npqm.k). // @@ -486,7 +493,7 @@ func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { k: npqm.k, ctx: ctx, }: - case <-pqm.ctx.Done(): + case <-pqm.closing: return } } else { @@ -502,7 +509,7 @@ func (npqm *newProvideQueryMessage) handle(pqm *ProviderQueryManager) { providersSoFar: requestStatus.providersSoFar, incoming: inProgressChan, }: - case <-pqm.ctx.Done(): + case <-pqm.closing: } } diff --git a/routing/providerquerymanager/providerquerymanager_test.go b/routing/providerquerymanager/providerquerymanager_test.go index 1be26c4e3..8026c5364 100644 --- a/routing/providerquerymanager/providerquerymanager_test.go +++ b/routing/providerquerymanager/providerquerymanager_test.go @@ -74,11 +74,11 @@ func TestNormalSimultaneousFetch(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(fpd, fpn)) + defer providerQueryManager.Close() keys := random.Cids(2) - sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + sessionCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[1], 0) @@ -111,11 +111,11 @@ func TestDedupingProviderRequests(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(fpd, fpn)) + defer providerQueryManager.Close() key := random.Cids(1)[0] - sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + sessionCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) @@ -151,12 +151,13 @@ func TestCancelOneRequestDoesNotTerminateAnother(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(fpd, fpn)) + defer providerQueryManager.Close() key := random.Cids(1)[0] // first session will cancel before done + ctx := context.Background() firstSessionCtx, firstCancel := context.WithTimeout(ctx, 3*time.Millisecond) defer firstCancel() firstRequestChan := providerQueryManager.FindProvidersAsync(firstSessionCtx, key, 0) @@ -195,14 +196,13 @@ func TestCancelManagerExitsGracefully(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - managerCtx, managerCancel := context.WithTimeout(ctx, 5*time.Millisecond) - defer managerCancel() - providerQueryManager := mustNotErr(New(managerCtx, fpd, fpn)) + providerQueryManager := mustNotErr(New(fpd, fpn)) + defer providerQueryManager.Close() + time.AfterFunc(5*time.Millisecond, providerQueryManager.Close) key := random.Cids(1)[0] - sessionCtx, cancel := context.WithTimeout(ctx, 20*time.Millisecond) + sessionCtx, cancel := context.WithTimeout(context.Background(), 20*time.Millisecond) defer cancel() firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) @@ -232,12 +232,12 @@ func TestPeersWithConnectionErrorsNotAddedToPeerList(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn)) + providerQueryManager := mustNotErr(New(fpd, fpn)) + defer providerQueryManager.Close() key := random.Cids(1)[0] - sessionCtx, cancel := context.WithTimeout(ctx, 20*time.Millisecond) + sessionCtx, cancel := context.WithTimeout(context.Background(), 20*time.Millisecond) defer cancel() firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) secondRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, key, 0) @@ -266,13 +266,11 @@ func TestRateLimitingRequests(t *testing.T) { peersFound: peers, delay: 5 * time.Millisecond, } - ctx := context.Background() - ctx, cancel := context.WithCancel(ctx) - defer cancel() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxInProcessRequests(maxInProcessRequests))) + providerQueryManager := mustNotErr(New(fpd, fpn, WithMaxInProcessRequests(maxInProcessRequests))) + defer providerQueryManager.Close() keys := random.Cids(maxInProcessRequests + 1) - sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + sessionCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() var requestChannels []<-chan peer.AddrInfo for i := 0; i < maxInProcessRequests+1; i++ { @@ -307,11 +305,11 @@ func TestUnlimitedRequests(t *testing.T) { peersFound: peers, delay: 5 * time.Millisecond, } - ctx := context.Background() - ctx, cancel := context.WithCancel(ctx) - defer cancel() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxInProcessRequests(0))) + providerQueryManager := mustNotErr(New(fpd, fpn, WithMaxInProcessRequests(0))) + defer providerQueryManager.Close() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() keys := random.Cids(inProcessRequests) sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() @@ -346,11 +344,11 @@ func TestFindProviderTimeout(t *testing.T) { peersFound: peers, delay: 10 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(2*time.Millisecond))) + providerQueryManager := mustNotErr(New(fpd, fpn, WithMaxTimeout(2*time.Millisecond))) + defer providerQueryManager.Close() keys := random.Cids(1) - sessionCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + sessionCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) var firstPeersReceived []peer.AddrInfo @@ -369,11 +367,11 @@ func TestFindProviderPreCanceled(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(100*time.Millisecond))) + providerQueryManager := mustNotErr(New(fpd, fpn, WithMaxTimeout(100*time.Millisecond))) + defer providerQueryManager.Close() keys := random.Cids(1) - sessionCtx, cancel := context.WithCancel(ctx) + sessionCtx, cancel := context.WithCancel(context.Background()) cancel() firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) if firstRequestChan == nil { @@ -393,11 +391,11 @@ func TestCancelFindProvidersAfterCompletion(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxTimeout(100*time.Millisecond))) + providerQueryManager := mustNotErr(New(fpd, fpn, WithMaxTimeout(100*time.Millisecond))) + defer providerQueryManager.Close() keys := random.Cids(1) - sessionCtx, cancel := context.WithCancel(ctx) + sessionCtx, cancel := context.WithCancel(context.Background()) firstRequestChan := providerQueryManager.FindProvidersAsync(sessionCtx, keys[0], 0) <-firstRequestChan // wait for everything to start. time.Sleep(10 * time.Millisecond) // wait for the incoming providres to stop. @@ -425,11 +423,11 @@ func TestLimitedProviders(t *testing.T) { peersFound: peers, delay: 1 * time.Millisecond, } - ctx := context.Background() - providerQueryManager := mustNotErr(New(ctx, fpd, fpn, WithMaxProviders(max), WithMaxTimeout(100*time.Millisecond))) + providerQueryManager := mustNotErr(New(fpd, fpn, WithMaxProviders(max), WithMaxTimeout(100*time.Millisecond))) + defer providerQueryManager.Close() keys := random.Cids(1) - providersChan := providerQueryManager.FindProvidersAsync(ctx, keys[0], 0) + providersChan := providerQueryManager.FindProvidersAsync(context.Background(), keys[0], 0) total := 0 for range providersChan { total++ From 97092042b2783ac2efece8e9711fe6d2ac7cf772 Mon Sep 17 00:00:00 2001 From: Andrew Gillis <11790789+gammazero@users.noreply.github.com> Date: Fri, 6 Dec 2024 10:45:08 -1000 Subject: [PATCH 34/36] Use deque instead of slice for queues (#742) * Use deque instead of slice for queues Implementing a queue by appending to a slice will make more GC work as then end of slice's memory is reached. Instead use a deque that maintains a circular buffer that reuses memory from removed items. * Reuse timer instead of creating one each loop iteration --- .../messagequeue/donthavetimeoutmgr.go | 17 +++++----- bitswap/client/internal/session/cidqueue.go | 34 ++++++++++++------- bitswap/network/connecteventmanager.go | 11 +++--- bitswap/network/connecteventmanager_test.go | 2 +- bitswap/network/ipfs_impl.go | 12 +++++-- bitswap/testnet/virtual.go | 21 ++++++------ ipld/merkledag/merkledag.go | 10 +++--- ipld/merkledag/traverse/traverse.go | 32 ++++------------- ipld/unixfs/hamt/hamt.go | 13 ++++--- 9 files changed, 72 insertions(+), 80 deletions(-) diff --git a/bitswap/client/internal/messagequeue/donthavetimeoutmgr.go b/bitswap/client/internal/messagequeue/donthavetimeoutmgr.go index cdeee68ec..a6180a5d8 100644 --- a/bitswap/client/internal/messagequeue/donthavetimeoutmgr.go +++ b/bitswap/client/internal/messagequeue/donthavetimeoutmgr.go @@ -6,6 +6,7 @@ import ( "time" "github.com/benbjohnson/clock" + "github.com/gammazero/deque" cid "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/p2p/protocol/ping" ) @@ -79,7 +80,7 @@ type dontHaveTimeoutMgr struct { // wants that are active (waiting for a response or timeout) activeWants map[cid.Cid]*pendingWant // queue of wants, from oldest to newest - wantQueue []*pendingWant + wantQueue deque.Deque[*pendingWant] // time to wait for a response (depends on latency) timeout time.Duration // ewma of message latency (time from message sent to response received) @@ -222,15 +223,15 @@ func (dhtm *dontHaveTimeoutMgr) measurePingLatency() { // checkForTimeouts checks pending wants to see if any are over the timeout. // Note: this function should only be called within the lock. func (dhtm *dontHaveTimeoutMgr) checkForTimeouts() { - if len(dhtm.wantQueue) == 0 { + if dhtm.wantQueue.Len() == 0 { return } // Figure out which of the blocks that were wanted were not received // within the timeout expired := make([]cid.Cid, 0, len(dhtm.activeWants)) - for len(dhtm.wantQueue) > 0 { - pw := dhtm.wantQueue[0] + for dhtm.wantQueue.Len() > 0 { + pw := dhtm.wantQueue.Front() // If the want is still active if pw.active { @@ -247,7 +248,7 @@ func (dhtm *dontHaveTimeoutMgr) checkForTimeouts() { } // Remove expired or cancelled wants from the want queue - dhtm.wantQueue = dhtm.wantQueue[1:] + dhtm.wantQueue.PopFront() } // Fire the timeout event for the expired wants @@ -255,7 +256,7 @@ func (dhtm *dontHaveTimeoutMgr) checkForTimeouts() { go dhtm.fireTimeout(expired) } - if len(dhtm.wantQueue) == 0 { + if dhtm.wantQueue.Len() == 0 { return } @@ -266,7 +267,7 @@ func (dhtm *dontHaveTimeoutMgr) checkForTimeouts() { // Schedule the next check for the moment when the oldest pending want will // timeout - oldestStart := dhtm.wantQueue[0].sent + oldestStart := dhtm.wantQueue.Front().sent until := oldestStart.Add(dhtm.timeout).Sub(dhtm.clock.Now()) if dhtm.checkForTimeoutsTimer == nil { dhtm.checkForTimeoutsTimer = dhtm.clock.Timer(until) @@ -313,7 +314,7 @@ func (dhtm *dontHaveTimeoutMgr) AddPending(ks []cid.Cid) { active: true, } dhtm.activeWants[c] = &pw - dhtm.wantQueue = append(dhtm.wantQueue, &pw) + dhtm.wantQueue.PushBack(&pw) } } diff --git a/bitswap/client/internal/session/cidqueue.go b/bitswap/client/internal/session/cidqueue.go index aedfa944c..2ecd0f672 100644 --- a/bitswap/client/internal/session/cidqueue.go +++ b/bitswap/client/internal/session/cidqueue.go @@ -1,9 +1,12 @@ package session -import cid "github.com/ipfs/go-cid" +import ( + "github.com/gammazero/deque" + cid "github.com/ipfs/go-cid" +) type cidQueue struct { - elems []cid.Cid + elems deque.Deque[cid.Cid] eset *cid.Set } @@ -13,12 +16,11 @@ func newCidQueue() *cidQueue { func (cq *cidQueue) Pop() cid.Cid { for { - if len(cq.elems) == 0 { + if cq.elems.Len() == 0 { return cid.Cid{} } - out := cq.elems[0] - cq.elems = cq.elems[1:] + out := cq.elems.PopFront() if cq.eset.Has(out) { cq.eset.Remove(out) @@ -29,24 +31,30 @@ func (cq *cidQueue) Pop() cid.Cid { func (cq *cidQueue) Cids() []cid.Cid { // Lazily delete from the list any cids that were removed from the set - if len(cq.elems) > cq.eset.Len() { - i := 0 - for _, c := range cq.elems { + if cq.elems.Len() > cq.eset.Len() { + for i := 0; i < cq.elems.Len(); i++ { + c := cq.elems.PopFront() if cq.eset.Has(c) { - cq.elems[i] = c - i++ + cq.elems.PushBack(c) } } - cq.elems = cq.elems[:i] + } + + if cq.elems.Len() == 0 { + return nil } // Make a copy of the cids - return append([]cid.Cid{}, cq.elems...) + cids := make([]cid.Cid, cq.elems.Len()) + for i := 0; i < cq.elems.Len(); i++ { + cids[i] = cq.elems.At(i) + } + return cids } func (cq *cidQueue) Push(c cid.Cid) { if cq.eset.Visit(c) { - cq.elems = append(cq.elems, c) + cq.elems.PushBack(c) } } diff --git a/bitswap/network/connecteventmanager.go b/bitswap/network/connecteventmanager.go index 88337fce3..bf3766089 100644 --- a/bitswap/network/connecteventmanager.go +++ b/bitswap/network/connecteventmanager.go @@ -3,6 +3,7 @@ package network import ( "sync" + "github.com/gammazero/deque" "github.com/libp2p/go-libp2p/core/peer" ) @@ -25,7 +26,7 @@ type connectEventManager struct { cond sync.Cond peers map[peer.ID]*peerState - changeQueue []peer.ID + changeQueue deque.Deque[peer.ID] stop bool done chan struct{} } @@ -75,7 +76,7 @@ func (c *connectEventManager) setState(p peer.ID, newState state) { state.newState = newState if !state.pending && state.newState != state.curState { state.pending = true - c.changeQueue = append(c.changeQueue, p) + c.changeQueue.PushBack(p) c.cond.Broadcast() } } @@ -83,7 +84,7 @@ func (c *connectEventManager) setState(p peer.ID, newState state) { // Waits for a change to be enqueued, or for the event manager to be stopped. Returns false if the // connect event manager has been stopped. func (c *connectEventManager) waitChange() bool { - for !c.stop && len(c.changeQueue) == 0 { + for !c.stop && c.changeQueue.Len() == 0 { c.cond.Wait() } return !c.stop @@ -95,9 +96,7 @@ func (c *connectEventManager) worker() { defer close(c.done) for c.waitChange() { - pid := c.changeQueue[0] - c.changeQueue[0] = peer.ID("") // free the peer ID (slicing won't do that) - c.changeQueue = c.changeQueue[1:] + pid := c.changeQueue.PopFront() state, ok := c.peers[pid] // If we've disconnected and forgotten, continue. diff --git a/bitswap/network/connecteventmanager_test.go b/bitswap/network/connecteventmanager_test.go index 3107efbcf..5d57fc104 100644 --- a/bitswap/network/connecteventmanager_test.go +++ b/bitswap/network/connecteventmanager_test.go @@ -40,7 +40,7 @@ func wait(t *testing.T, c *connectEventManager) { require.Eventually(t, func() bool { c.lk.RLock() defer c.lk.RUnlock() - return len(c.changeQueue) == 0 + return c.changeQueue.Len() == 0 }, time.Second, time.Millisecond, "connection event manager never processed events") } diff --git a/bitswap/network/ipfs_impl.go b/bitswap/network/ipfs_impl.go index 993b64429..72f86d099 100644 --- a/bitswap/network/ipfs_impl.go +++ b/bitswap/network/ipfs_impl.go @@ -142,8 +142,10 @@ func (s *streamMessageSender) SendMsg(ctx context.Context, msg bsmsg.BitSwapMess // Perform a function with multiple attempts, and a timeout func (s *streamMessageSender) multiAttempt(ctx context.Context, fn func() error) error { - // Try to call the function repeatedly var err error + var timer *time.Timer + + // Try to call the function repeatedly for i := 0; i < s.opts.MaxRetries; i++ { if err = fn(); err == nil { // Attempt was successful @@ -174,8 +176,12 @@ func (s *streamMessageSender) multiAttempt(ctx context.Context, fn func() error) return err } - timer := time.NewTimer(s.opts.SendErrorBackoff) - defer timer.Stop() + if timer == nil { + timer = time.NewTimer(s.opts.SendErrorBackoff) + defer timer.Stop() + } else { + timer.Reset(s.opts.SendErrorBackoff) + } select { case <-ctx.Done(): diff --git a/bitswap/testnet/virtual.go b/bitswap/testnet/virtual.go index 0acf083a9..53e56d67d 100644 --- a/bitswap/testnet/virtual.go +++ b/bitswap/testnet/virtual.go @@ -8,11 +8,10 @@ import ( "sync/atomic" "time" + "github.com/gammazero/deque" bsmsg "github.com/ipfs/boxo/bitswap/message" bsnet "github.com/ipfs/boxo/bitswap/network" - delay "github.com/ipfs/go-ipfs-delay" - tnet "github.com/libp2p/go-libp2p-testing/net" "github.com/libp2p/go-libp2p/core/connmgr" "github.com/libp2p/go-libp2p/core/peer" @@ -75,7 +74,7 @@ type message struct { // for type receiverQueue struct { receiver *networkClient - queue []*message + queue deque.Deque[*message] active bool lk sync.Mutex } @@ -346,7 +345,7 @@ func (nc *networkClient) DisconnectFrom(_ context.Context, p peer.ID) error { func (rq *receiverQueue) enqueue(m *message) { rq.lk.Lock() defer rq.lk.Unlock() - rq.queue = append(rq.queue, m) + rq.queue.PushBack(m) if !rq.active { rq.active = true go rq.process() @@ -354,29 +353,29 @@ func (rq *receiverQueue) enqueue(m *message) { } func (rq *receiverQueue) Swap(i, j int) { - rq.queue[i], rq.queue[j] = rq.queue[j], rq.queue[i] + rq.queue.Swap(i, j) } func (rq *receiverQueue) Len() int { - return len(rq.queue) + return rq.queue.Len() } func (rq *receiverQueue) Less(i, j int) bool { - return rq.queue[i].shouldSend.UnixNano() < rq.queue[j].shouldSend.UnixNano() + return rq.queue.At(i).shouldSend.UnixNano() < rq.queue.At(j).shouldSend.UnixNano() } func (rq *receiverQueue) process() { for { rq.lk.Lock() - sort.Sort(rq) - if len(rq.queue) == 0 { + if rq.queue.Len() == 0 { rq.active = false rq.lk.Unlock() return } - m := rq.queue[0] + sort.Sort(rq) + m := rq.queue.Front() if time.Until(m.shouldSend).Seconds() < 0.1 { - rq.queue = rq.queue[1:] + rq.queue.PopFront() rq.lk.Unlock() time.Sleep(time.Until(m.shouldSend)) atomic.AddUint64(&rq.receiver.stats.MessagesRecvd, 1) diff --git a/ipld/merkledag/merkledag.go b/ipld/merkledag/merkledag.go index a227780ff..1c638d139 100644 --- a/ipld/merkledag/merkledag.go +++ b/ipld/merkledag/merkledag.go @@ -6,6 +6,7 @@ import ( "errors" "sync" + "github.com/gammazero/deque" bserv "github.com/ipfs/boxo/blockservice" blocks "github.com/ipfs/go-block-format" cid "github.com/ipfs/go-cid" @@ -535,7 +536,7 @@ func parallelWalkDepth(ctx context.Context, getLinks GetLinks, root cid.Cid, vis defer close(feed) send := feed - var todoQueue []cidDepth + var todoQueue deque.Deque[cidDepth] var inProgress int next := cidDepth{ @@ -547,9 +548,8 @@ func parallelWalkDepth(ctx context.Context, getLinks GetLinks, root cid.Cid, vis select { case send <- next: inProgress++ - if len(todoQueue) > 0 { - next = todoQueue[0] - todoQueue = todoQueue[1:] + if todoQueue.Len() > 0 { + next = todoQueue.PopFront() } else { next = cidDepth{} send = nil @@ -570,7 +570,7 @@ func parallelWalkDepth(ctx context.Context, getLinks GetLinks, root cid.Cid, vis next = cd send = feed } else { - todoQueue = append(todoQueue, cd) + todoQueue.PushBack(cd) } } case err := <-errChan: diff --git a/ipld/merkledag/traverse/traverse.go b/ipld/merkledag/traverse/traverse.go index a3836e385..125e5d7db 100644 --- a/ipld/merkledag/traverse/traverse.go +++ b/ipld/merkledag/traverse/traverse.go @@ -5,6 +5,7 @@ import ( "context" "errors" + "github.com/gammazero/deque" ipld "github.com/ipfs/go-ipld-format" ) @@ -167,10 +168,10 @@ func bfsTraverse(root State, t *traversal) error { return err } - var q queue - q.enq(root) - for q.len() > 0 { - curr := q.deq() + var q deque.Deque[State] + q.PushBack(root) + for q.Len() > 0 { + curr := q.PopFront() if curr.Node == nil { return errors.New("failed to dequeue though queue not empty") } @@ -189,7 +190,7 @@ func bfsTraverse(root State, t *traversal) error { continue } - q.enq(State{ + q.PushBack(State{ Node: node, Depth: curr.Depth + 1, }) @@ -197,24 +198,3 @@ func bfsTraverse(root State, t *traversal) error { } return nil } - -type queue struct { - s []State -} - -func (q *queue) enq(n State) { - q.s = append(q.s, n) -} - -func (q *queue) deq() State { - if len(q.s) < 1 { - return State{} - } - n := q.s[0] - q.s = q.s[1:] - return n -} - -func (q *queue) len() int { - return len(q.s) -} diff --git a/ipld/unixfs/hamt/hamt.go b/ipld/unixfs/hamt/hamt.go index 455d070c6..41bf7dd14 100644 --- a/ipld/unixfs/hamt/hamt.go +++ b/ipld/unixfs/hamt/hamt.go @@ -29,10 +29,10 @@ import ( "os" "sync" + "github.com/gammazero/deque" + dag "github.com/ipfs/boxo/ipld/merkledag" format "github.com/ipfs/boxo/ipld/unixfs" "github.com/ipfs/boxo/ipld/unixfs/internal" - - dag "github.com/ipfs/boxo/ipld/merkledag" bitfield "github.com/ipfs/go-bitfield" cid "github.com/ipfs/go-cid" ipld "github.com/ipfs/go-ipld-format" @@ -563,7 +563,7 @@ func parallelShardWalk(ctx context.Context, root *Shard, dserv ipld.DAGService, } send := feed - var todoQueue []*listCidsAndShards + var todoQueue deque.Deque[*listCidsAndShards] var inProgress int next := &listCidsAndShards{ @@ -575,9 +575,8 @@ dispatcherLoop: select { case send <- next: inProgress++ - if len(todoQueue) > 0 { - next = todoQueue[0] - todoQueue = todoQueue[1:] + if todoQueue.Len() > 0 { + next = todoQueue.PopFront() } else { next = nil send = nil @@ -592,7 +591,7 @@ dispatcherLoop: next = nextNodes send = feed } else { - todoQueue = append(todoQueue, nextNodes) + todoQueue.PushBack(nextNodes) } case <-errGrpCtx.Done(): break dispatcherLoop From 266017f0be8d20a25dda17e702e0ce5709973a88 Mon Sep 17 00:00:00 2001 From: gammazero <11790789+gammazero@users.noreply.github.com> Date: Mon, 9 Dec 2024 09:18:58 -1000 Subject: [PATCH 35/36] tidy changelog --- CHANGELOG.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7f4d0c094..5c4d93690 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -14,6 +14,18 @@ The following emojis are used to highlight certain changes: ## [Unreleased] +### Added + +### Changed + +### Removed + +### Fixed + +### Security + +## [v0.25.0] + - `bitswap`, `routing`, `exchange` ([#641](https://github.com/ipfs/boxo/pull/641)): - ✨ Bitswap is no longer in charge of providing blocks to the newtork: providing functionality is now handled by a `exchange/providing.Exchange`, meant to be used with `provider.System` so that all provides follow the same rules (multiple parts of the code where handling provides) before. - 🛠 `bitswap/client/internal/providerquerymanager` has been moved to `routing/providerquerymanager` where it belongs. In order to keep compatibility, Bitswap now receives a `routing.ContentDiscovery` parameter which implements `FindProvidersAsync(...)` and uses it to create a `providerquerymanager` with the default settings as before. Custom settings can be used by using a custom `providerquerymanager` to manually wrap a `ContentDiscovery` object and pass that in as `ContentDiscovery` on initialization while setting `bitswap.WithDefaultProviderQueryManager(false)` (to avoid re-wrapping it again). From 87c39147f8c5988bd1aae402caba16d0b82e5f3d Mon Sep 17 00:00:00 2001 From: gammazero <11790789+gammazero@users.noreply.github.com> Date: Mon, 9 Dec 2024 09:39:08 -1000 Subject: [PATCH 36/36] update version --- version.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.json b/version.json index 336b08fb6..0bab82458 100644 --- a/version.json +++ b/version.json @@ -1,3 +1,3 @@ { - "version": "v0.24.3" + "version": "v0.25.0" }