From 5362e55792d87687832f5eb66ac3635842267da7 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Tue, 21 Feb 2023 11:08:38 +0100 Subject: [PATCH 01/17] added check to avoid adding unresponsive dht peers to the dht routing table --- dht.go | 105 ++++++++++++++++++++++++------------------ dht_bootstrap_test.go | 4 +- dht_net.go | 2 +- ext_test.go | 6 +-- handlers_test.go | 2 +- query.go | 2 +- subscriber_notifee.go | 2 +- 7 files changed, 68 insertions(+), 55 deletions(-) diff --git a/dht.go b/dht.go index 689de6240..2764c7857 100644 --- a/dht.go +++ b/dht.go @@ -43,6 +43,10 @@ var ( baseLogger = logger.Desugar() rtFreezeTimeout = 1 * time.Minute + + // max time given to connecting to, and querying a peers before + // adding it to the Routing Table + protocolCheckTimeout = 10 * time.Second ) const ( @@ -68,11 +72,6 @@ const ( protectedBuckets = 2 ) -type addPeerRTReq struct { - p peer.ID - queryPeer bool -} - // IpfsDHT is an implementation of Kademlia with S/Kademlia modifications. // It is used to implement the base Routing module. type IpfsDHT struct { @@ -125,6 +124,10 @@ type IpfsDHT struct { autoRefresh bool + // A function performing a lookup request to a remote peer.ID, verifying that it is able to + // answer it correctly + protocolCheck func(context.Context, peer.ID) error + // A function returning a set of bootstrap peers to fallback on if all other attempts to fix // the routing table fail (or, e.g., this is the first time this node is // connecting to the network). @@ -140,7 +143,7 @@ type IpfsDHT struct { disableFixLowPeers bool fixLowPeersChan chan struct{} - addPeerToRTChan chan addPeerRTReq + addPeerToRTChan chan peer.ID refreshFinishedCh chan struct{} rtFreezeTimeout time.Duration @@ -233,7 +236,7 @@ func New(ctx context.Context, h host.Host, options ...Option) (*IpfsDHT, error) // Fill routing table with currently connected peers that are DHT servers dht.plk.Lock() for _, p := range dht.host.Network().Peers() { - dht.peerFound(dht.ctx, p, false) + dht.peerFound(dht.ctx, p) } dht.plk.Unlock() @@ -294,7 +297,7 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err fixLowPeersChan: make(chan struct{}, 1), - addPeerToRTChan: make(chan addPeerRTReq), + addPeerToRTChan: make(chan peer.ID), refreshFinishedCh: make(chan struct{}), } @@ -321,6 +324,17 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err dht.routingTable = rt dht.bootstrapPeers = cfg.BootstrapPeers + dht.protocolCheck = func(ctx context.Context, p peer.ID) error { + // lookup request to p requesting for its own peer.ID + peerids, err := dht.protoMessenger.GetClosestPeers(ctx, p, p) + // p should return at least 2 peer.ID, itself plus at least one neighbor + // otherwise the response is considered as invalid + if err == nil && len(peerids) < 2 { + return fmt.Errorf("peer %s failed to return its closest peers", p) + } + return err + } + // rt refresh manager rtRefresh, err := makeRtRefreshManager(dht, cfg, maxLastSuccessfulOutboundThreshold) if err != nil { @@ -363,16 +377,11 @@ func makeRtRefreshManager(dht *IpfsDHT, cfg dhtcfg.Config, maxLastSuccessfulOutb return err } - pingFnc := func(ctx context.Context, p peer.ID) error { - _, err := dht.protoMessenger.GetClosestPeers(ctx, p, p) // don't use the PING message type as it's deprecated - return err - } - r, err := rtrefresh.NewRtRefreshManager( dht.host, dht.routingTable, cfg.RoutingTable.AutoRefresh, keyGenFnc, queryFnc, - pingFnc, + dht.protocolCheck, cfg.RoutingTable.RefreshQueryTimeout, cfg.RoutingTable.RefreshInterval, maxLastSuccessfulOutboundThreshold, @@ -480,7 +489,7 @@ func (dht *IpfsDHT) fixLowPeers(ctx context.Context) { // we try to add all peers we are connected to to the Routing Table // in case they aren't already there. for _, p := range dht.host.Network().Peers() { - dht.peerFound(ctx, p, false) + dht.peerFound(ctx, p) } // TODO Active Bootstrapping @@ -591,22 +600,23 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { select { case <-timerCh: dht.routingTable.MarkAllPeersIrreplaceable() - case addReq := <-dht.addPeerToRTChan: + case p := <-dht.addPeerToRTChan: prevSize := dht.routingTable.Size() if prevSize == 0 { isBootsrapping = true bootstrapCount = 0 timerCh = nil } - newlyAdded, err := dht.routingTable.TryAddPeer(addReq.p, addReq.queryPeer, isBootsrapping) + // queryPeer set to true as we only try to add queried peers to the RT + newlyAdded, err := dht.routingTable.TryAddPeer(p, true, isBootsrapping) if err != nil { // peer not added. continue } - if !newlyAdded && addReq.queryPeer { + if !newlyAdded { // the peer is already in our RT, but we just successfully queried it and so let's give it a // bump on the query time so we don't ping it too soon for a liveliness check. - dht.routingTable.UpdateLastSuccessfulOutboundQueryAt(addReq.p, time.Now()) + dht.routingTable.UpdateLastSuccessfulOutboundQueryAt(p, time.Now()) } case <-dht.refreshFinishedCh: bootstrapCount = bootstrapCount + 1 @@ -626,39 +636,42 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { } } -// peerFound signals the routingTable that we've found a peer that -// might support the DHT protocol. -// If we have a connection a peer but no exchange of a query RPC -> -// -// LastQueriedAt=time.Now (so we don't ping it for some time for a liveliness check) -// LastUsefulAt=0 -// -// If we connect to a peer and then exchange a query RPC -> -// -// LastQueriedAt=time.Now (same reason as above) -// LastUsefulAt=time.Now (so we give it some life in the RT without immediately evicting it) -// -// If we query a peer we already have in our Routing Table -> -// -// LastQueriedAt=time.Now() -// LastUsefulAt remains unchanged -// -// If we connect to a peer we already have in the RT but do not exchange a query (rare) -// -// Do Nothing. -func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID, queryPeer bool) { - if c := baseLogger.Check(zap.DebugLevel, "peer found"); c != nil { - c.Write(zap.String("peer", p.String())) - } +// peerFound verifies whether the found peer advertises DHT protocols +// and probe it to make sure it answers DHT queries as expected. If +// it fails to answer, it isn't added to the routingTable. +func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { b, err := dht.validRTPeer(p) if err != nil { logger.Errorw("failed to validate if peer is a DHT peer", "peer", p, "error", err) } else if b { - select { - case dht.addPeerToRTChan <- addPeerRTReq{p, queryPeer}: - case <-dht.ctx.Done(): + livelinessCtx, cancel := context.WithTimeout(ctx, protocolCheckTimeout) + defer cancel() + + if err := dht.host.Connect(livelinessCtx, peer.AddrInfo{ID: p}); err != nil { + logger.Debugw("failed connection to DHT peer", "peer", p, "error", err) return } + + if err := dht.protocolCheck(livelinessCtx, p); err != nil { + logger.Debugw("connected peer not answering DHT request as expected", "peer", p, "error", err) + return + } + + dht.validPeerFound(ctx, p) + } +} + +// validPeerFound signals the routingTable that we've found a peer that +// supports the DHT protocol, and just answered correctly to a DHT FindPeers +func (dht *IpfsDHT) validPeerFound(ctx context.Context, p peer.ID) { + if c := baseLogger.Check(zap.DebugLevel, "peer found"); c != nil { + c.Write(zap.String("peer", p.String())) + } + + select { + case dht.addPeerToRTChan <- p: + case <-dht.ctx.Done(): + return } } diff --git a/dht_bootstrap_test.go b/dht_bootstrap_test.go index d97e3b7e7..9dd496c0a 100644 --- a/dht_bootstrap_test.go +++ b/dht_bootstrap_test.go @@ -191,8 +191,8 @@ func TestBootstrappersReplacable(t *testing.T) { require.NoError(t, d.host.Network().ClosePeer(d5.self)) connectNoSync(t, ctx, d, d1) connectNoSync(t, ctx, d, d5) - d.peerFound(ctx, d5.self, true) - d.peerFound(ctx, d1.self, true) + d.peerFound(ctx, d5.self) + d.peerFound(ctx, d1.self) time.Sleep(1 * time.Second) require.Len(t, d.routingTable.ListPeers(), 2) diff --git a/dht_net.go b/dht_net.go index 2a31a0b7c..9c075fb3f 100644 --- a/dht_net.go +++ b/dht_net.go @@ -111,7 +111,7 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { } // a peer has queried us, let's add it to RT - dht.peerFound(dht.ctx, mPeer, true) + dht.peerFound(dht.ctx, mPeer) if c := baseLogger.Check(zap.DebugLevel, "handling message"); c != nil { c.Write(zap.String("from", mPeer.String()), diff --git a/ext_test.go b/ext_test.go index 0ade6ac93..c2b88dcd5 100644 --- a/ext_test.go +++ b/ext_test.go @@ -293,7 +293,7 @@ func TestNotFound(t *testing.T) { } for _, p := range hosts { - d.peerFound(ctx, p.ID(), true) + d.peerFound(ctx, p.ID()) } // long timeout to ensure timing is not at play. @@ -343,7 +343,7 @@ func TestLessThanKResponses(t *testing.T) { } for i := 1; i < 5; i++ { - d.peerFound(ctx, hosts[i].ID(), true) + d.peerFound(ctx, hosts[i].ID()) } // Reply with random peers to every message @@ -415,7 +415,7 @@ func TestMultipleQueries(t *testing.T) { t.Fatal(err) } - d.peerFound(ctx, hosts[1].ID(), true) + d.peerFound(ctx, hosts[1].ID()) for _, proto := range d.serverProtocols { // It would be nice to be able to just get a value and succeed but then diff --git a/handlers_test.go b/handlers_test.go index c4c1e4ca1..d829e38b1 100644 --- a/handlers_test.go +++ b/handlers_test.go @@ -111,7 +111,7 @@ func BenchmarkHandleFindPeer(b *testing.B) { panic(err) } - d.peerFound(ctx, id, true) + d.peerFound(ctx, id) peers = append(peers, id) a, err := ma.NewMultiaddr(fmt.Sprintf("/ip4/127.0.0.1/tcp/%d", 2000+i)) diff --git a/query.go b/query.go index 0761a1c4a..f3582d29b 100644 --- a/query.go +++ b/query.go @@ -417,7 +417,7 @@ func (q *query) queryPeer(ctx context.Context, ch chan<- *queryUpdate, p peer.ID queryDuration := time.Since(startQuery) // query successful, try to add to RT - q.dht.peerFound(q.dht.ctx, p, true) + q.dht.validPeerFound(q.dht.ctx, p) // process new peers saw := []peer.ID{} diff --git a/subscriber_notifee.go b/subscriber_notifee.go index ec9eca146..6897c58e5 100644 --- a/subscriber_notifee.go +++ b/subscriber_notifee.go @@ -108,7 +108,7 @@ func handlePeerChangeEvent(dht *IpfsDHT, p peer.ID) { logger.Errorf("could not check peerstore for protocol support: err: %s", err) return } else if valid { - dht.peerFound(dht.ctx, p, false) + dht.peerFound(dht.ctx, p) dht.fixRTIfNeeded() } else { dht.peerStoppedDHT(dht.ctx, p) From 9357ac95bb069175c5c4ab9f5f9b272686843777 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Wed, 22 Feb 2023 15:39:56 +0100 Subject: [PATCH 02/17] removed lock in adding peers to the rt --- dht.go | 7 +++---- dht_net.go | 2 +- ext_test.go | 3 +++ 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/dht.go b/dht.go index 2764c7857..6802658fc 100644 --- a/dht.go +++ b/dht.go @@ -327,10 +327,9 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err dht.protocolCheck = func(ctx context.Context, p peer.ID) error { // lookup request to p requesting for its own peer.ID peerids, err := dht.protoMessenger.GetClosestPeers(ctx, p, p) - // p should return at least 2 peer.ID, itself plus at least one neighbor - // otherwise the response is considered as invalid - if err == nil && len(peerids) < 2 { - return fmt.Errorf("peer %s failed to return its closest peers", p) + // p should return at least its own peerid + if err == nil && len(peerids) == 0 { + return fmt.Errorf("peer %s failed to return its closest peers, got %d", p, len(peerids)) } return err } diff --git a/dht_net.go b/dht_net.go index 9c075fb3f..d71ed3d91 100644 --- a/dht_net.go +++ b/dht_net.go @@ -111,7 +111,7 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { } // a peer has queried us, let's add it to RT - dht.peerFound(dht.ctx, mPeer) + go dht.peerFound(dht.ctx, mPeer) if c := baseLogger.Check(zap.DebugLevel, "handling message"); c != nil { c.Write(zap.String("from", mPeer.String()), diff --git a/ext_test.go b/ext_test.go index c2b88dcd5..d0afc7409 100644 --- a/ext_test.go +++ b/ext_test.go @@ -279,6 +279,7 @@ func TestNotFound(t *testing.T) { if err := pbw.WriteMsg(resp); err != nil { return } + case pb.Message_FIND_NODE: default: panic("Shouldnt recieve this.") } @@ -372,6 +373,7 @@ func TestLessThanKResponses(t *testing.T) { if err := pbw.WriteMsg(resp); err != nil { panic(err) } + case pb.Message_FIND_NODE: default: panic("Shouldnt recieve this.") } @@ -442,6 +444,7 @@ func TestMultipleQueries(t *testing.T) { if err := pbw.WriteMsg(resp); err != nil { panic(err) } + case pb.Message_FIND_NODE: default: panic("Shouldnt recieve this.") } From d0d6d73ffdd5314db57658ea7f46a5e926e28ddf Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Thu, 23 Feb 2023 10:49:57 +0100 Subject: [PATCH 03/17] made variable names more meaningful --- dht.go | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/dht.go b/dht.go index 6802658fc..ee23c3d88 100644 --- a/dht.go +++ b/dht.go @@ -43,10 +43,6 @@ var ( baseLogger = logger.Desugar() rtFreezeTimeout = 1 * time.Minute - - // max time given to connecting to, and querying a peers before - // adding it to the Routing Table - protocolCheckTimeout = 10 * time.Second ) const ( @@ -126,7 +122,8 @@ type IpfsDHT struct { // A function performing a lookup request to a remote peer.ID, verifying that it is able to // answer it correctly - protocolCheck func(context.Context, peer.ID) error + lookupCheck func(context.Context, peer.ID) error + lookupCheckTimeout time.Duration // A function returning a set of bootstrap peers to fallback on if all other attempts to fix // the routing table fail (or, e.g., this is the first time this node is @@ -324,7 +321,7 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err dht.routingTable = rt dht.bootstrapPeers = cfg.BootstrapPeers - dht.protocolCheck = func(ctx context.Context, p peer.ID) error { + dht.lookupCheck = func(ctx context.Context, p peer.ID) error { // lookup request to p requesting for its own peer.ID peerids, err := dht.protoMessenger.GetClosestPeers(ctx, p, p) // p should return at least its own peerid @@ -333,6 +330,7 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err } return err } + dht.lookupCheckTimeout = cfg.RoutingTable.RefreshQueryTimeout // rt refresh manager rtRefresh, err := makeRtRefreshManager(dht, cfg, maxLastSuccessfulOutboundThreshold) @@ -380,7 +378,7 @@ func makeRtRefreshManager(dht *IpfsDHT, cfg dhtcfg.Config, maxLastSuccessfulOutb dht.host, dht.routingTable, cfg.RoutingTable.AutoRefresh, keyGenFnc, queryFnc, - dht.protocolCheck, + dht.lookupCheck, cfg.RoutingTable.RefreshQueryTimeout, cfg.RoutingTable.RefreshInterval, maxLastSuccessfulOutboundThreshold, @@ -643,7 +641,7 @@ func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { if err != nil { logger.Errorw("failed to validate if peer is a DHT peer", "peer", p, "error", err) } else if b { - livelinessCtx, cancel := context.WithTimeout(ctx, protocolCheckTimeout) + livelinessCtx, cancel := context.WithTimeout(ctx, dht.lookupCheckTimeout) defer cancel() if err := dht.host.Connect(livelinessCtx, peer.AddrInfo{ID: p}); err != nil { @@ -651,7 +649,7 @@ func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { return } - if err := dht.protocolCheck(livelinessCtx, p); err != nil { + if err := dht.lookupCheck(livelinessCtx, p); err != nil { logger.Debugw("connected peer not answering DHT request as expected", "peer", p, "error", err) return } From 8be8d1e78d729e02395f537d9e28ca05f6645a1a Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Thu, 23 Feb 2023 17:20:12 +0100 Subject: [PATCH 04/17] fixed network loop and corrected tests --- dht.go | 58 ++++- dht_bootstrap_test.go | 4 + dht_test.go | 65 ++++++ ext_test.go | 431 +------------------------------------- internal/config/config.go | 31 +-- 5 files changed, 142 insertions(+), 447 deletions(-) diff --git a/dht.go b/dht.go index ee23c3d88..366206f95 100644 --- a/dht.go +++ b/dht.go @@ -122,8 +122,13 @@ type IpfsDHT struct { // A function performing a lookup request to a remote peer.ID, verifying that it is able to // answer it correctly - lookupCheck func(context.Context, peer.ID) error - lookupCheckTimeout time.Duration + lookupCheck func(context.Context, peer.ID) error + lookupCheckTimeout time.Duration + lookupCheckInterval time.Duration // time interval during which we don't try to query the same peer again + // recentlyCheckedPeers contains the peers recently queried with the time at which they were queried + recentlyCheckedPeers map[peer.ID]time.Time + recentlyCheckedPeersLk sync.Mutex + peerRecentlyQueried func(peer.ID) bool // A function returning a set of bootstrap peers to fallback on if all other attempts to fix // the routing table fail (or, e.g., this is the first time this node is @@ -183,6 +188,7 @@ func New(ctx context.Context, h host.Host, options ...Option) (*IpfsDHT, error) dht.autoRefresh = cfg.RoutingTable.AutoRefresh + dht.lookupCheckInterval = cfg.LookupCheckInterval dht.maxRecordAge = cfg.MaxRecordAge dht.enableProviders = cfg.EnableProviders dht.enableValues = cfg.EnableValues @@ -331,6 +337,26 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err return err } dht.lookupCheckTimeout = cfg.RoutingTable.RefreshQueryTimeout + dht.recentlyCheckedPeers = make(map[peer.ID]time.Time) + dht.peerRecentlyQueried = func(p peer.ID) bool { + dht.recentlyCheckedPeersLk.Lock() + + now := time.Now() + + // clean recentlyCheckedPeers + for peerid, t := range dht.recentlyCheckedPeers { + // remove peers that have been queried more than lookupCheckInterval ago + if t.Add(dht.lookupCheckInterval).Before(now) { + delete(dht.recentlyCheckedPeers, peerid) + } + } + + // if p still in recentlyCheckedPeers, it has been queried less than + // lookupCheckInterval ago + _, ok := dht.recentlyCheckedPeers[p] + dht.recentlyCheckedPeersLk.Unlock() + return ok + } // rt refresh manager rtRefresh, err := makeRtRefreshManager(dht, cfg, maxLastSuccessfulOutboundThreshold) @@ -598,8 +624,7 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { case <-timerCh: dht.routingTable.MarkAllPeersIrreplaceable() case p := <-dht.addPeerToRTChan: - prevSize := dht.routingTable.Size() - if prevSize == 0 { + if dht.routingTable.Size() == 0 { isBootsrapping = true bootstrapCount = 0 timerCh = nil @@ -637,23 +662,46 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { // and probe it to make sure it answers DHT queries as expected. If // it fails to answer, it isn't added to the routingTable. func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { + + // TODO: verify whether the appropriate bucket still has space + /* + cpl := kb.CommonPrefixLen(dht.selfKey, kb.ConvertPeerID(p)) + if dht.routingTable.NPeersForCpl(uint(cpl)) >= dht.bucketSize { + logger.Debugw("bucket already full, not querying", p) + return + } + */ + + // verify whether the remote peer advertises the right dht protocol b, err := dht.validRTPeer(p) if err != nil { logger.Errorw("failed to validate if peer is a DHT peer", "peer", p, "error", err) } else if b { + if dht.peerRecentlyQueried(p) { + // peer was already queried recently and didn't make it to the bucket + return + } + livelinessCtx, cancel := context.WithTimeout(ctx, dht.lookupCheckTimeout) defer cancel() + // connecting to the remote peer if err := dht.host.Connect(livelinessCtx, peer.AddrInfo{ID: p}); err != nil { logger.Debugw("failed connection to DHT peer", "peer", p, "error", err) return } + // add peer.ID to recently queried peers + dht.recentlyCheckedPeersLk.Lock() + dht.recentlyCheckedPeers[p] = time.Now() + dht.recentlyCheckedPeersLk.Unlock() + + // performing a FIND_NODE query if err := dht.lookupCheck(livelinessCtx, p); err != nil { logger.Debugw("connected peer not answering DHT request as expected", "peer", p, "error", err) return } - + // if the FIND_NODE succeeded, the peer is considered as valid dht.validPeerFound(ctx, p) } } diff --git a/dht_bootstrap_test.go b/dht_bootstrap_test.go index 9dd496c0a..ef2f5926e 100644 --- a/dht_bootstrap_test.go +++ b/dht_bootstrap_test.go @@ -164,6 +164,10 @@ func TestBootstrappersReplacable(t *testing.T) { } require.Len(t, d.routingTable.ListPeers(), 0) + d.recentlyCheckedPeersLk.Lock() + d.recentlyCheckedPeers = make(map[peer.ID]time.Time) + d.recentlyCheckedPeersLk.Unlock() + // adding d1 & d2 works now because there is space in the Routing Table require.NoError(t, d.host.Network().ClosePeer(d1.self)) require.NoError(t, d.host.Network().ClosePeer(d2.self)) diff --git a/dht_test.go b/dht_test.go index 094017c25..f21c2d0e2 100644 --- a/dht_test.go +++ b/dht_test.go @@ -767,6 +767,7 @@ func TestRefreshBelowMinRTThreshold(t *testing.T) { // and because of the above bootstrap, A also discovers E ! waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 4, 4, 20*time.Second) + time.Sleep(10 * time.Millisecond) assert.Equal(t, dhtE.self, dhtA.routingTable.Find(dhtE.self), "A's routing table should have peer E!") } @@ -1325,6 +1326,49 @@ func TestClientModeConnect(t *testing.T) { } } +func TestInvalidServer(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + a := setupDHT(ctx, t, false) + b := setupDHT(ctx, t, true) + + // make b advertise all dht server protocols + for _, proto := range a.serverProtocols { + // Hang on every request. + b.host.SetStreamHandler(proto, func(s network.Stream) { + defer s.Reset() // nolint + <-ctx.Done() + }) + } + + connectNoSync(t, ctx, a, b) + + c := testCaseCids[0] + p := peer.ID("TestPeer") + a.ProviderStore().AddProvider(ctx, c.Hash(), peer.AddrInfo{ID: p}) + time.Sleep(time.Millisecond * 5) // just in case... + + provs, err := b.FindProviders(ctx, c) + if err != nil { + t.Fatal(err) + } + + if len(provs) == 0 { + t.Fatal("Expected to get a provider back") + } + + if provs[0].ID != p { + t.Fatal("expected it to be our test peer") + } + if a.routingTable.Find(b.self) != "" { + t.Fatal("DHT clients should not be added to routing tables") + } + if b.routingTable.Find(a.self) == "" { + t.Fatal("DHT server should have been added to the dht client's routing table") + } +} + func TestClientModeFindPeer(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() @@ -1519,6 +1563,11 @@ func TestFixLowPeers(t *testing.T) { mainD.routingTable.RemovePeer(d.self) } + // remove blacklist of already contacted peers + mainD.recentlyCheckedPeersLk.Lock() + mainD.recentlyCheckedPeers = make(map[peer.ID]time.Time) + mainD.recentlyCheckedPeersLk.Unlock() + // but we will still get enough peers in the RT because of fix low Peers waitForWellFormedTables(t, []*IpfsDHT{mainD}, minRTRefreshThreshold, minRTRefreshThreshold, 5*time.Second) } @@ -1624,6 +1673,15 @@ func TestHandleRemotePeerProtocolChanges(t *testing.T) { connect(t, ctx, dhtA, dhtB) + // clear connection history + dhtA.recentlyCheckedPeersLk.Lock() + dhtA.recentlyCheckedPeers = make(map[peer.ID]time.Time) + dhtA.recentlyCheckedPeersLk.Unlock() + + dhtB.recentlyCheckedPeersLk.Lock() + dhtB.recentlyCheckedPeers = make(map[peer.ID]time.Time) + dhtB.recentlyCheckedPeersLk.Unlock() + // now assert both have each other in their RT waitForWellFormedTables(t, []*IpfsDHT{dhtA, dhtB}, 1, 1, 10*time.Second) @@ -2124,6 +2182,13 @@ func TestPreconnectedNodes(t *testing.T) { require.NoError(t, err) defer h2.Close() + // clear d2 recent checked peers + d2.recentlyCheckedPeersLk.Lock() + d2.recentlyCheckedPeers = make(map[peer.ID]time.Time) + d2.recentlyCheckedPeersLk.Unlock() + + connect(t, ctx, d1, d2) + // See if it works peers, err := d2.GetClosestPeers(ctx, "testkey") require.NoError(t, err) diff --git a/ext_test.go b/ext_test.go index d0afc7409..51340f334 100644 --- a/ext_test.go +++ b/ext_test.go @@ -2,36 +2,20 @@ package dht import ( "context" - "math/rand" "testing" "time" "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/routing" "github.com/stretchr/testify/require" - record "github.com/libp2p/go-libp2p-record" - bhost "github.com/libp2p/go-libp2p/p2p/host/basic" mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" - swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing" - - //lint:ignore SA1019 TODO migrate away from gogo pb - "github.com/libp2p/go-msgio/protoio" - - pb "github.com/libp2p/go-libp2p-kad-dht/pb" - - u "github.com/ipfs/go-ipfs-util" ) -// Test that one hung request to a peer doesn't prevent another request -// using that same peer from obeying its context. -func TestHungRequest(t *testing.T) { +func TestInvalidRemotePeers(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - mn, err := mocknet.FullMeshLinked(2) + mn, err := mocknet.FullMeshLinked(5) if err != nil { t.Fatal(err) } @@ -56,416 +40,7 @@ func TestHungRequest(t *testing.T) { t.Fatal("failed to connect peers", err) } - // Wait at a bit for a peer in our routing table. - for i := 0; i < 100 && d.routingTable.Size() == 0; i++ { - time.Sleep(10 * time.Millisecond) - } - if d.routingTable.Size() == 0 { - t.Fatal("failed to fill routing table") - } - - ctx1, cancel1 := context.WithTimeout(ctx, 1*time.Second) - defer cancel1() - - done := make(chan error, 1) - go func() { - _, err := d.GetClosestPeers(ctx1, testCaseCids[0].KeyString()) - done <- err - }() - time.Sleep(100 * time.Millisecond) - ctx2, cancel2 := context.WithTimeout(ctx, 100*time.Millisecond) - defer cancel2() - err = d.Provide(ctx2, testCaseCids[0], true) - if err != context.DeadlineExceeded { - t.Errorf("expected to fail with deadline exceeded, got: %s", ctx2.Err()) - } - select { - case err = <-done: - t.Error("GetClosestPeers should not have returned yet", err) - default: - err = <-done - if err != context.DeadlineExceeded { - t.Errorf("expected the deadline to be exceeded, got %s", err) - } - } - - if d.routingTable.Size() == 0 { - // make sure we didn't just disconnect - t.Fatal("expected peers in the routing table") - } -} - -func TestGetFailures(t *testing.T) { - if testing.Short() { - t.SkipNow() - } - - ctx := context.Background() - - host1, err := bhost.NewHost(swarmt.GenSwarm(t, swarmt.OptDisableReuseport), new(bhost.HostOpts)) - require.NoError(t, err) - host1.Start() - host2, err := bhost.NewHost(swarmt.GenSwarm(t, swarmt.OptDisableReuseport), new(bhost.HostOpts)) - require.NoError(t, err) - host2.Start() - - d, err := New(ctx, host1, testPrefix, DisableAutoRefresh(), Mode(ModeServer)) - require.NoError(t, err) - - // Reply with failures to every message - for _, proto := range d.serverProtocols { - host2.SetStreamHandler(proto, func(s network.Stream) { - time.Sleep(400 * time.Millisecond) - s.Close() - }) - } - - host1.Peerstore().AddAddrs(host2.ID(), host2.Addrs(), peerstore.ConnectedAddrTTL) - _, err = host1.Network().DialPeer(ctx, host2.ID()) - require.NoError(t, err) - time.Sleep(1 * time.Second) - - // This one should time out - ctx1, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond) - defer cancel() - if _, err := d.GetValue(ctx1, "test"); err != nil { - if merr, ok := err.(u.MultiErr); ok && len(merr) > 0 { - err = merr[0] - } - - if err != context.DeadlineExceeded { - t.Fatal("Got different error than we expected", err) - } - } else { - t.Fatal("Did not get expected error!") - } - - t.Log("Timeout test passed.") - - for _, proto := range d.serverProtocols { - // Reply with failures to every message - host2.SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - // user gave up - return - } - - resp := &pb.Message{ - Type: pmes.Type, - } - _ = pbw.WriteMsg(resp) - }) - } - - // This one should fail with NotFound. - // long context timeout to ensure we dont end too early. - // the dht should be exhausting its query and returning not found. - // (was 3 seconds before which should be _plenty_ of time, but maybe - // travis machines really have a hard time...) - ctx2, cancel := context.WithTimeout(context.Background(), 20*time.Second) - defer cancel() - _, err = d.GetValue(ctx2, "test") - if err != nil { - if merr, ok := err.(u.MultiErr); ok && len(merr) > 0 { - err = merr[0] - } - if err != routing.ErrNotFound { - t.Fatalf("Expected ErrNotFound, got: %s", err) - } - } else { - t.Fatal("expected error, got none.") - } - - t.Log("ErrNotFound check passed!") - - // Now we test this DHT's handleGetValue failure - { - typ := pb.Message_GET_VALUE - str := "hello" - - rec := record.MakePutRecord(str, []byte("blah")) - req := pb.Message{ - Type: typ, - Key: []byte(str), - Record: rec, - } - - s, err := host2.NewStream(context.Background(), host1.ID(), d.protocols...) - if err != nil { - t.Fatal(err) - } - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - if err := pbw.WriteMsg(&req); err != nil { - t.Fatal(err) - } - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - t.Fatal(err) - } - if pmes.GetRecord() != nil { - t.Fatal("shouldnt have value") - } - if pmes.GetProviderPeers() != nil { - t.Fatal("shouldnt have provider peers") - } - } - - if d.routingTable.Size() == 0 { - // make sure we didn't just disconnect - t.Fatal("expected peers in the routing table") - } -} - -func TestNotFound(t *testing.T) { - // t.Skip("skipping test to debug another") - if testing.Short() { - t.SkipNow() - } - - ctx := context.Background() - mn, err := mocknet.FullMeshConnected(16) - if err != nil { - t.Fatal(err) - } - defer mn.Close() - hosts := mn.Hosts() - - os := []Option{testPrefix, DisableAutoRefresh(), Mode(ModeServer)} - d, err := New(ctx, hosts[0], os...) - if err != nil { - t.Fatal(err) - } - - // Reply with random peers to every message - for _, host := range hosts { - host := host // shadow loop var - for _, proto := range d.serverProtocols { - host.SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - // this isn't an error, it just means the stream has died. - return - } - - switch pmes.GetType() { - case pb.Message_GET_VALUE: - resp := &pb.Message{Type: pmes.Type} - - ps := []peer.AddrInfo{} - for i := 0; i < 7; i++ { - p := hosts[rand.Intn(len(hosts))].ID() - pi := host.Peerstore().PeerInfo(p) - ps = append(ps, pi) - } - - resp.CloserPeers = pb.PeerInfosToPBPeers(d.host.Network(), ps) - if err := pbw.WriteMsg(resp); err != nil { - return - } - case pb.Message_FIND_NODE: - default: - panic("Shouldnt recieve this.") - } - }) - } - for _, peer := range hosts { - if host == peer { - continue - } - _ = peer.Peerstore().AddProtocols(host.ID(), d.serverProtocols...) - } - } - - for _, p := range hosts { - d.peerFound(ctx, p.ID()) - } - - // long timeout to ensure timing is not at play. - ctx, cancel := context.WithTimeout(ctx, time.Second*20) - defer cancel() - v, err := d.GetValue(ctx, "hello") - logger.Debugf("get value got %v", v) - if err != nil { - if merr, ok := err.(u.MultiErr); ok && len(merr) > 0 { - err = merr[0] - } - switch err { - case routing.ErrNotFound: - if d.routingTable.Size() == 0 { - // make sure we didn't just disconnect - t.Fatal("expected peers in the routing table") - } - // Success! - return - case u.ErrTimeout: - t.Fatal("Should not have gotten timeout!") - default: - t.Fatalf("Got unexpected error: %s", err) - } - } - t.Fatal("Expected to recieve an error.") -} - -// If less than K nodes are in the entire network, it should fail when we make -// a GET rpc and nobody has the value -func TestLessThanKResponses(t *testing.T) { - // t.Skip("skipping test to debug another") - // t.Skip("skipping test because it makes a lot of output") - - ctx := context.Background() - mn, err := mocknet.FullMeshConnected(6) - if err != nil { - t.Fatal(err) - } - defer mn.Close() - hosts := mn.Hosts() - - os := []Option{testPrefix, DisableAutoRefresh(), Mode(ModeServer)} - d, err := New(ctx, hosts[0], os...) - if err != nil { - t.Fatal(err) - } - - for i := 1; i < 5; i++ { - d.peerFound(ctx, hosts[i].ID()) - } - - // Reply with random peers to every message - for _, host := range hosts { - host := host // shadow loop var - for _, proto := range d.serverProtocols { - host.SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } - - switch pmes.GetType() { - case pb.Message_GET_VALUE: - pi := host.Peerstore().PeerInfo(hosts[1].ID()) - resp := &pb.Message{ - Type: pmes.Type, - CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), - } - - if err := pbw.WriteMsg(resp); err != nil { - panic(err) - } - case pb.Message_FIND_NODE: - default: - panic("Shouldnt recieve this.") - } - - }) - } - } - - ctx, cancel := context.WithTimeout(ctx, time.Second*30) - defer cancel() - if _, err := d.GetValue(ctx, "hello"); err != nil { - switch err { - case routing.ErrNotFound: - // Success! - return - case u.ErrTimeout: - t.Fatal("Should not have gotten timeout!") - default: - t.Fatalf("Got unexpected error: %s", err) - } - } - t.Fatal("Expected to recieve an error.") -} - -// Test multiple queries against a node that closes its stream after every query. -func TestMultipleQueries(t *testing.T) { - if testing.Short() { - t.SkipNow() - } - - ctx := context.Background() - mn, err := mocknet.FullMeshConnected(2) - if err != nil { - t.Fatal(err) - } - defer mn.Close() - hosts := mn.Hosts() - os := []Option{testPrefix, DisableAutoRefresh(), Mode(ModeServer)} - d, err := New(ctx, hosts[0], os...) - if err != nil { - t.Fatal(err) - } - - d.peerFound(ctx, hosts[1].ID()) - - for _, proto := range d.serverProtocols { - // It would be nice to be able to just get a value and succeed but then - // we'd need to deal with selectors and validators... - hosts[1].SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } - - switch pmes.GetType() { - case pb.Message_GET_VALUE: - pi := hosts[1].Peerstore().PeerInfo(hosts[0].ID()) - resp := &pb.Message{ - Type: pmes.Type, - CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), - } - - if err := pbw.WriteMsg(resp); err != nil { - panic(err) - } - case pb.Message_FIND_NODE: - default: - panic("Shouldnt recieve this.") - } - }) - } - - // long timeout to ensure timing is not at play. - ctx, cancel := context.WithTimeout(ctx, time.Second*20) - defer cancel() - for i := 0; i < 10; i++ { - if _, err := d.GetValue(ctx, "hello"); err != nil { - switch err { - case routing.ErrNotFound: - // Success! - continue - case u.ErrTimeout: - t.Fatal("Should not have gotten timeout!") - default: - t.Fatalf("Got unexpected error: %s", err) - } - } - t.Fatal("Expected to recieve an error.") - } + require.Equal(t, 0, d.routingTable.Size()) } diff --git a/internal/config/config.go b/internal/config/config.go index 7a1d14fca..c95514c7d 100644 --- a/internal/config/config.go +++ b/internal/config/config.go @@ -32,20 +32,21 @@ type RouteTableFilterFunc func(dht interface{}, p peer.ID) bool // Config is a structure containing all the options that can be used when constructing a DHT. type Config struct { - Datastore ds.Batching - Validator record.Validator - ValidatorChanged bool // if true implies that the validator has been changed and that Defaults should not be used - Mode ModeOpt - ProtocolPrefix protocol.ID - V1ProtocolOverride protocol.ID - BucketSize int - Concurrency int - Resiliency int - MaxRecordAge time.Duration - EnableProviders bool - EnableValues bool - ProviderStore providers.ProviderStore - QueryPeerFilter QueryFilterFunc + Datastore ds.Batching + Validator record.Validator + ValidatorChanged bool // if true implies that the validator has been changed and that Defaults should not be used + Mode ModeOpt + ProtocolPrefix protocol.ID + V1ProtocolOverride protocol.ID + BucketSize int + Concurrency int + Resiliency int + LookupCheckInterval time.Duration + MaxRecordAge time.Duration + EnableProviders bool + EnableValues bool + ProviderStore providers.ProviderStore + QueryPeerFilter QueryFilterFunc RoutingTable struct { RefreshQueryTimeout time.Duration @@ -114,6 +115,8 @@ var Defaults = func(o *Config) error { o.RoutingTable.RefreshInterval = 10 * time.Minute o.RoutingTable.AutoRefresh = true o.RoutingTable.PeerFilter = EmptyRTFilter + + o.LookupCheckInterval = 5 * time.Second o.MaxRecordAge = providers.ProvideValidity o.BucketSize = defaultBucketSize From 7d046a2f850d05cbd3fe16efb16e55594d046256 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Thu, 23 Feb 2023 17:32:36 +0100 Subject: [PATCH 05/17] added UsefulPeer() references from current PR --- dht.go | 12 ++++-------- go.mod | 26 +++++++++++++------------- go.sum | 26 ++++++++++++++++++++++++++ 3 files changed, 43 insertions(+), 21 deletions(-) diff --git a/dht.go b/dht.go index 366206f95..856de2a6f 100644 --- a/dht.go +++ b/dht.go @@ -663,14 +663,10 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { // it fails to answer, it isn't added to the routingTable. func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { - // TODO: verify whether the appropriate bucket still has space - /* - cpl := kb.CommonPrefixLen(dht.selfKey, kb.ConvertPeerID(p)) - if dht.routingTable.NPeersForCpl(uint(cpl)) >= dht.bucketSize { - logger.Debugw("bucket already full, not querying", p) - return - } - */ + // if the appropriate bucket is already full, don't try to add the new peer.ID + if !dht.routingTable.UsefulPeer(p) { + return + } // verify whether the remote peer advertises the right dht protocol b, err := dht.validRTPeer(p) diff --git a/go.mod b/go.mod index 0c0259a3b..fa25dc41e 100644 --- a/go.mod +++ b/go.mod @@ -15,8 +15,8 @@ require ( github.com/ipfs/go-ipns v0.2.0 github.com/ipfs/go-log v1.0.5 github.com/jbenet/goprocess v0.1.4 - github.com/libp2p/go-libp2p v0.25.1 - github.com/libp2p/go-libp2p-kbucket v0.5.0 + github.com/libp2p/go-libp2p v0.26.0 + github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f github.com/libp2p/go-libp2p-record v0.2.0 github.com/libp2p/go-libp2p-routing-helpers v0.4.0 github.com/libp2p/go-libp2p-testing v0.12.0 @@ -64,7 +64,7 @@ require ( github.com/jackpal/go-nat-pmp v1.0.2 // indirect github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect github.com/klauspost/compress v1.15.12 // indirect - github.com/klauspost/cpuid/v2 v2.2.1 // indirect + github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/koron/go-ssdp v0.0.3 // indirect github.com/libp2p/go-buffer-pool v0.1.0 // indirect github.com/libp2p/go-cidranger v1.1.0 // indirect @@ -74,7 +74,7 @@ require ( github.com/libp2p/go-reuseport v0.2.0 // indirect github.com/libp2p/go-yamux/v4 v4.0.0 // indirect github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd // indirect - github.com/mattn/go-isatty v0.0.16 // indirect + github.com/mattn/go-isatty v0.0.17 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/miekg/dns v1.1.50 // indirect github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b // indirect @@ -84,7 +84,7 @@ require ( github.com/multiformats/go-base36 v0.2.0 // indirect github.com/multiformats/go-multiaddr-dns v0.3.1 // indirect github.com/multiformats/go-multiaddr-fmt v0.1.0 // indirect - github.com/multiformats/go-multicodec v0.7.0 // indirect + github.com/multiformats/go-multicodec v0.8.1 // indirect github.com/multiformats/go-varint v0.0.7 // indirect github.com/onsi/ginkgo/v2 v2.5.1 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect @@ -99,23 +99,23 @@ require ( github.com/prometheus/procfs v0.8.0 // indirect github.com/quic-go/qpack v0.4.0 // indirect github.com/quic-go/qtls-go1-18 v0.2.0 // indirect - github.com/quic-go/qtls-go1-19 v0.2.0 // indirect - github.com/quic-go/qtls-go1-20 v0.1.0 // indirect - github.com/quic-go/quic-go v0.32.0 // indirect + github.com/quic-go/qtls-go1-19 v0.2.1 // indirect + github.com/quic-go/qtls-go1-20 v0.1.1 // indirect + github.com/quic-go/quic-go v0.33.0 // indirect github.com/quic-go/webtransport-go v0.5.1 // indirect github.com/raulk/go-watchdog v1.3.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/dig v1.15.0 // indirect go.uber.org/fx v1.18.2 // indirect - go.uber.org/multierr v1.8.0 // indirect - golang.org/x/crypto v0.4.0 // indirect + go.uber.org/multierr v1.9.0 // indirect + golang.org/x/crypto v0.6.0 // indirect golang.org/x/exp v0.0.0-20221205204356-47842c84f3db // indirect golang.org/x/mod v0.7.0 // indirect - golang.org/x/net v0.4.0 // indirect + golang.org/x/net v0.6.0 // indirect golang.org/x/sync v0.1.0 // indirect - golang.org/x/sys v0.3.0 // indirect - golang.org/x/text v0.5.0 // indirect + golang.org/x/sys v0.5.0 // indirect + golang.org/x/text v0.7.0 // indirect golang.org/x/tools v0.3.0 // indirect google.golang.org/protobuf v1.28.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 063a2620b..56b45d43d 100644 --- a/go.sum +++ b/go.sum @@ -336,6 +336,8 @@ github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa02 github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.1 h1:U33DW0aiEj633gHYw3LoDNfkDiYnE5Q8M/TKJn2f2jI= github.com/klauspost/cpuid/v2 v2.2.1/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= +github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= +github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/koron/go-ssdp v0.0.0-20191105050749-2e1c40ed0b5d/go.mod h1:5Ky9EC2xfoUKUor0Hjgi2BJhCSXJfMOFlmyYrVKGQMk= @@ -363,6 +365,8 @@ github.com/libp2p/go-flow-metrics v0.1.0 h1:0iPhMI8PskQwzh57jB9WxIuIOQ0r+15PChFG github.com/libp2p/go-flow-metrics v0.1.0/go.mod h1:4Xi8MX8wj5aWNDAZttg6UPmc0ZrnFNsMtpsYUClFtro= github.com/libp2p/go-libp2p v0.25.1 h1:YK+YDCHpYyTvitKWVxa5PfElgIpOONU01X5UcLEwJGA= github.com/libp2p/go-libp2p v0.25.1/go.mod h1:xnK9/1d9+jeQCVvi/f1g12KqtVi/jP/SijtKV1hML3g= +github.com/libp2p/go-libp2p v0.26.0 h1:0FE0bP9/G9YADjruqoFvf1snBBFvrdh1MmTuEeUkl2E= +github.com/libp2p/go-libp2p v0.26.0/go.mod h1:R8N+XhwPDPLNb4TKboKJKnDeg9vPw8+zlC6g793dTGw= github.com/libp2p/go-libp2p-asn-util v0.2.0 h1:rg3+Os8jbnO5DxkC7K/Utdi+DkY3q/d1/1q+8WeNAsw= github.com/libp2p/go-libp2p-asn-util v0.2.0/go.mod h1:WoaWxbHKBymSN41hWSq/lGKJEca7TNm58+gGJi2WsLI= github.com/libp2p/go-libp2p-core v0.2.4/go.mod h1:STh4fdfa5vDYr0/SzYYeqnt+E6KfEV5VxfIrm0bcI0g= @@ -370,6 +374,8 @@ github.com/libp2p/go-libp2p-core v0.3.0/go.mod h1:ACp3DmS3/N64c2jDzcV429ukDpicbL github.com/libp2p/go-libp2p-kbucket v0.3.1/go.mod h1:oyjT5O7tS9CQurok++ERgc46YLwEpuGoFq9ubvoUOio= github.com/libp2p/go-libp2p-kbucket v0.5.0 h1:g/7tVm8ACHDxH29BGrpsQlnNeu+6OF1A9bno/4/U1oA= github.com/libp2p/go-libp2p-kbucket v0.5.0/go.mod h1:zGzGCpQd78b5BNTDGHNDLaTt9aDK/A02xeZp9QeFC4U= +github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f h1:e1ibEklHRE4Oh5tpzzjRRCvYylG3f01ZbPrPulPFpPg= +github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f/go.mod h1:efnPrfoP+WT/ONcC5eB0iADCDIJFXauXhylgJYO+VWw= github.com/libp2p/go-libp2p-peerstore v0.1.4/go.mod h1:+4BDbDiiKf4PzpANZDAT+knVdLxvqh7hXOujessqdzs= github.com/libp2p/go-libp2p-record v0.2.0 h1:oiNUOCWno2BFuxt3my4i1frNrt7PerzB3queqa1NkQ0= github.com/libp2p/go-libp2p-record v0.2.0/go.mod h1:I+3zMkvvg5m2OcSdoL0KPljyJyvNDFGKX7QdlpYUcwk= @@ -408,6 +414,8 @@ github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Ky github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ= github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-isatty v0.0.17 h1:BTarxUcIeDqL27Mc+vyvdWYSL28zpIhv3RoTdsLMPng= +github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= @@ -462,6 +470,8 @@ github.com/multiformats/go-multibase v0.1.1 h1:3ASCDsuLX8+j4kx58qnJ4YFq/JWTJpCyD github.com/multiformats/go-multibase v0.1.1/go.mod h1:ZEjHE+IsUrgp5mhlEAYjMtZwK1k4haNkcaPg9aoe1a8= github.com/multiformats/go-multicodec v0.7.0 h1:rTUjGOwjlhGHbEMbPoSUJowG1spZTVsITRANCjKTUAQ= github.com/multiformats/go-multicodec v0.7.0/go.mod h1:GUC8upxSBE4oG+q3kWZRw/+6yC1BqO550bjhWsJbZlw= +github.com/multiformats/go-multicodec v0.8.1 h1:ycepHwavHafh3grIbR1jIXnKCsFm0fqsfEOsJ8NtKE8= +github.com/multiformats/go-multicodec v0.8.1/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.10/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= @@ -538,10 +548,16 @@ github.com/quic-go/qtls-go1-18 v0.2.0 h1:5ViXqBZ90wpUcZS0ge79rf029yx0dYB0McyPJwq github.com/quic-go/qtls-go1-18 v0.2.0/go.mod h1:moGulGHK7o6O8lSPSZNoOwcLvJKJ85vVNc7oJFD65bc= github.com/quic-go/qtls-go1-19 v0.2.0 h1:Cvn2WdhyViFUHoOqK52i51k4nDX8EwIh5VJiVM4nttk= github.com/quic-go/qtls-go1-19 v0.2.0/go.mod h1:ySOI96ew8lnoKPtSqx2BlI5wCpUVPT05RMAlajtnyOI= +github.com/quic-go/qtls-go1-19 v0.2.1 h1:aJcKNMkH5ASEJB9FXNeZCyTEIHU1J7MmHyz1Q1TSG1A= +github.com/quic-go/qtls-go1-19 v0.2.1/go.mod h1:ySOI96ew8lnoKPtSqx2BlI5wCpUVPT05RMAlajtnyOI= github.com/quic-go/qtls-go1-20 v0.1.0 h1:d1PK3ErFy9t7zxKsG3NXBJXZjp/kMLoIb3y/kV54oAI= github.com/quic-go/qtls-go1-20 v0.1.0/go.mod h1:JKtK6mjbAVcUTN/9jZpvLbGxvdWIKS8uT7EiStoU1SM= +github.com/quic-go/qtls-go1-20 v0.1.1 h1:KbChDlg82d3IHqaj2bn6GfKRj84Per2VGf5XV3wSwQk= +github.com/quic-go/qtls-go1-20 v0.1.1/go.mod h1:JKtK6mjbAVcUTN/9jZpvLbGxvdWIKS8uT7EiStoU1SM= github.com/quic-go/quic-go v0.32.0 h1:lY02md31s1JgPiiyfqJijpu/UX/Iun304FI3yUqX7tA= github.com/quic-go/quic-go v0.32.0/go.mod h1:/fCsKANhQIeD5l76c2JFU+07gVE3KaA0FP+0zMWwfwo= +github.com/quic-go/quic-go v0.33.0 h1:ItNoTDN/Fm/zBlq769lLJc8ECe9gYaW40veHCCco7y0= +github.com/quic-go/quic-go v0.33.0/go.mod h1:YMuhaAV9/jIu0XclDXwZPAsP/2Kgr5yMYhe9oxhhOFA= github.com/quic-go/webtransport-go v0.5.1 h1:1eVb7WDWCRoaeTtFHpFBJ6WDN1bSrPrRoW6tZgSw0Ow= github.com/quic-go/webtransport-go v0.5.1/go.mod h1:OhmmgJIzTTqXK5xvtuX0oBpLV2GkLWNDA+UeTGJXErU= github.com/raulk/go-watchdog v1.3.0 h1:oUmdlHxdkXRJlwfG0O9omj8ukerm8MEQavSiDTEtBsk= @@ -657,6 +673,8 @@ go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKY go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= +go.uber.org/multierr v1.9.0 h1:7fIwc/ZtS0q++VgcfqFDxSBZVv/Xo49/SYnDFupUwlI= +go.uber.org/multierr v1.9.0/go.mod h1:X2jQV1h+kxSjClGpnseKVIxpmcjrj7MNnI0bnlfKTVQ= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= @@ -681,6 +699,8 @@ golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWP golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.4.0 h1:UVQgzMY87xqpKNgb+kDsll2Igd33HszWHFLmpaRMq/8= golang.org/x/crypto v0.4.0/go.mod h1:3quD/ATkf6oY+rnes5c3ExXTbLc8mueNue5/DoinL80= +golang.org/x/crypto v0.6.0 h1:qfktjS5LUO+fFKeJXZ+ikTRijMmljikvG68fpMMruSc= +golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -762,6 +782,8 @@ golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.4.0 h1:Q5QPcMlvfxFTAPV0+07Xz/MpK9NTXu2VDUuy0FeMfaU= golang.org/x/net v0.4.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= +golang.org/x/net v0.6.0 h1:L4ZwwTvKW9gr0ZMS1yrHD9GZhIuVjOBBnaKH+SPQK0Q= +golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -847,6 +869,8 @@ golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.3.0 h1:w8ZOecv6NaNa/zC8944JTU3vz4u6Lagfk4RPQxv92NQ= golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -859,6 +883,8 @@ golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.5.0 h1:OLmvp0KP+FVG99Ct/qFiL/Fhk4zp4QQnZ7b2U+5piUM= golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= From cbbba915e915f8c2ed4678291c1ea98d43f892a4 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Fri, 24 Feb 2023 09:07:01 +0100 Subject: [PATCH 06/17] go mod tidy --- go.mod | 1 - go.sum | 28 ---------------------------- 2 files changed, 29 deletions(-) diff --git a/go.mod b/go.mod index fa25dc41e..d9a5a395f 100644 --- a/go.mod +++ b/go.mod @@ -98,7 +98,6 @@ require ( github.com/prometheus/common v0.37.0 // indirect github.com/prometheus/procfs v0.8.0 // indirect github.com/quic-go/qpack v0.4.0 // indirect - github.com/quic-go/qtls-go1-18 v0.2.0 // indirect github.com/quic-go/qtls-go1-19 v0.2.1 // indirect github.com/quic-go/qtls-go1-20 v0.1.1 // indirect github.com/quic-go/quic-go v0.33.0 // indirect diff --git a/go.sum b/go.sum index 56b45d43d..67ba20b8b 100644 --- a/go.sum +++ b/go.sum @@ -334,8 +334,6 @@ github.com/klauspost/compress v1.15.12 h1:YClS/PImqYbn+UILDnqxQCZ3RehC9N318SU3kE github.com/klauspost/compress v1.15.12/go.mod h1:QPwzmACJjUTFsnSHH934V6woptycfrDDJnH7hvFVbGM= github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.2.1 h1:U33DW0aiEj633gHYw3LoDNfkDiYnE5Q8M/TKJn2f2jI= -github.com/klauspost/cpuid/v2 v2.2.1/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -363,8 +361,6 @@ github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZ github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-flow-metrics v0.1.0 h1:0iPhMI8PskQwzh57jB9WxIuIOQ0r+15PChFGkx3Q3WM= github.com/libp2p/go-flow-metrics v0.1.0/go.mod h1:4Xi8MX8wj5aWNDAZttg6UPmc0ZrnFNsMtpsYUClFtro= -github.com/libp2p/go-libp2p v0.25.1 h1:YK+YDCHpYyTvitKWVxa5PfElgIpOONU01X5UcLEwJGA= -github.com/libp2p/go-libp2p v0.25.1/go.mod h1:xnK9/1d9+jeQCVvi/f1g12KqtVi/jP/SijtKV1hML3g= github.com/libp2p/go-libp2p v0.26.0 h1:0FE0bP9/G9YADjruqoFvf1snBBFvrdh1MmTuEeUkl2E= github.com/libp2p/go-libp2p v0.26.0/go.mod h1:R8N+XhwPDPLNb4TKboKJKnDeg9vPw8+zlC6g793dTGw= github.com/libp2p/go-libp2p-asn-util v0.2.0 h1:rg3+Os8jbnO5DxkC7K/Utdi+DkY3q/d1/1q+8WeNAsw= @@ -372,8 +368,6 @@ github.com/libp2p/go-libp2p-asn-util v0.2.0/go.mod h1:WoaWxbHKBymSN41hWSq/lGKJEc github.com/libp2p/go-libp2p-core v0.2.4/go.mod h1:STh4fdfa5vDYr0/SzYYeqnt+E6KfEV5VxfIrm0bcI0g= github.com/libp2p/go-libp2p-core v0.3.0/go.mod h1:ACp3DmS3/N64c2jDzcV429ukDpicbL6+TrrxANBjPGw= github.com/libp2p/go-libp2p-kbucket v0.3.1/go.mod h1:oyjT5O7tS9CQurok++ERgc46YLwEpuGoFq9ubvoUOio= -github.com/libp2p/go-libp2p-kbucket v0.5.0 h1:g/7tVm8ACHDxH29BGrpsQlnNeu+6OF1A9bno/4/U1oA= -github.com/libp2p/go-libp2p-kbucket v0.5.0/go.mod h1:zGzGCpQd78b5BNTDGHNDLaTt9aDK/A02xeZp9QeFC4U= github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f h1:e1ibEklHRE4Oh5tpzzjRRCvYylG3f01ZbPrPulPFpPg= github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f/go.mod h1:efnPrfoP+WT/ONcC5eB0iADCDIJFXauXhylgJYO+VWw= github.com/libp2p/go-libp2p-peerstore v0.1.4/go.mod h1:+4BDbDiiKf4PzpANZDAT+knVdLxvqh7hXOujessqdzs= @@ -412,8 +406,6 @@ github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNx github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= -github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ= -github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-isatty v0.0.17 h1:BTarxUcIeDqL27Mc+vyvdWYSL28zpIhv3RoTdsLMPng= github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -468,8 +460,6 @@ github.com/multiformats/go-multiaddr-net v0.1.1/go.mod h1:5JNbcfBOP4dnhoZOv10JJV github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= github.com/multiformats/go-multibase v0.1.1 h1:3ASCDsuLX8+j4kx58qnJ4YFq/JWTJpCyDW27ztsVTOI= github.com/multiformats/go-multibase v0.1.1/go.mod h1:ZEjHE+IsUrgp5mhlEAYjMtZwK1k4haNkcaPg9aoe1a8= -github.com/multiformats/go-multicodec v0.7.0 h1:rTUjGOwjlhGHbEMbPoSUJowG1spZTVsITRANCjKTUAQ= -github.com/multiformats/go-multicodec v0.7.0/go.mod h1:GUC8upxSBE4oG+q3kWZRw/+6yC1BqO550bjhWsJbZlw= github.com/multiformats/go-multicodec v0.8.1 h1:ycepHwavHafh3grIbR1jIXnKCsFm0fqsfEOsJ8NtKE8= github.com/multiformats/go-multicodec v0.8.1/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= @@ -544,18 +534,10 @@ github.com/prometheus/procfs v0.8.0 h1:ODq8ZFEaYeCaZOJlZZdJA2AbQR98dSHSM1KW/You5 github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4= github.com/quic-go/qpack v0.4.0 h1:Cr9BXA1sQS2SmDUWjSofMPNKmvF6IiIfDRmgU0w1ZCo= github.com/quic-go/qpack v0.4.0/go.mod h1:UZVnYIfi5GRk+zI9UMaCPsmZ2xKJP7XBUvVyT1Knj9A= -github.com/quic-go/qtls-go1-18 v0.2.0 h1:5ViXqBZ90wpUcZS0ge79rf029yx0dYB0McyPJwqqj7U= -github.com/quic-go/qtls-go1-18 v0.2.0/go.mod h1:moGulGHK7o6O8lSPSZNoOwcLvJKJ85vVNc7oJFD65bc= -github.com/quic-go/qtls-go1-19 v0.2.0 h1:Cvn2WdhyViFUHoOqK52i51k4nDX8EwIh5VJiVM4nttk= -github.com/quic-go/qtls-go1-19 v0.2.0/go.mod h1:ySOI96ew8lnoKPtSqx2BlI5wCpUVPT05RMAlajtnyOI= github.com/quic-go/qtls-go1-19 v0.2.1 h1:aJcKNMkH5ASEJB9FXNeZCyTEIHU1J7MmHyz1Q1TSG1A= github.com/quic-go/qtls-go1-19 v0.2.1/go.mod h1:ySOI96ew8lnoKPtSqx2BlI5wCpUVPT05RMAlajtnyOI= -github.com/quic-go/qtls-go1-20 v0.1.0 h1:d1PK3ErFy9t7zxKsG3NXBJXZjp/kMLoIb3y/kV54oAI= -github.com/quic-go/qtls-go1-20 v0.1.0/go.mod h1:JKtK6mjbAVcUTN/9jZpvLbGxvdWIKS8uT7EiStoU1SM= github.com/quic-go/qtls-go1-20 v0.1.1 h1:KbChDlg82d3IHqaj2bn6GfKRj84Per2VGf5XV3wSwQk= github.com/quic-go/qtls-go1-20 v0.1.1/go.mod h1:JKtK6mjbAVcUTN/9jZpvLbGxvdWIKS8uT7EiStoU1SM= -github.com/quic-go/quic-go v0.32.0 h1:lY02md31s1JgPiiyfqJijpu/UX/Iun304FI3yUqX7tA= -github.com/quic-go/quic-go v0.32.0/go.mod h1:/fCsKANhQIeD5l76c2JFU+07gVE3KaA0FP+0zMWwfwo= github.com/quic-go/quic-go v0.33.0 h1:ItNoTDN/Fm/zBlq769lLJc8ECe9gYaW40veHCCco7y0= github.com/quic-go/quic-go v0.33.0/go.mod h1:YMuhaAV9/jIu0XclDXwZPAsP/2Kgr5yMYhe9oxhhOFA= github.com/quic-go/webtransport-go v0.5.1 h1:1eVb7WDWCRoaeTtFHpFBJ6WDN1bSrPrRoW6tZgSw0Ow= @@ -671,8 +653,6 @@ go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpK go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= -go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/multierr v1.9.0 h1:7fIwc/ZtS0q++VgcfqFDxSBZVv/Xo49/SYnDFupUwlI= go.uber.org/multierr v1.9.0/go.mod h1:X2jQV1h+kxSjClGpnseKVIxpmcjrj7MNnI0bnlfKTVQ= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= @@ -697,8 +677,6 @@ golang.org/x/crypto v0.0.0-20200602180216-279210d13fed/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= -golang.org/x/crypto v0.4.0 h1:UVQgzMY87xqpKNgb+kDsll2Igd33HszWHFLmpaRMq/8= -golang.org/x/crypto v0.4.0/go.mod h1:3quD/ATkf6oY+rnes5c3ExXTbLc8mueNue5/DoinL80= golang.org/x/crypto v0.6.0 h1:qfktjS5LUO+fFKeJXZ+ikTRijMmljikvG68fpMMruSc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -780,8 +758,6 @@ golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.4.0 h1:Q5QPcMlvfxFTAPV0+07Xz/MpK9NTXu2VDUuy0FeMfaU= -golang.org/x/net v0.4.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= golang.org/x/net v0.6.0 h1:L4ZwwTvKW9gr0ZMS1yrHD9GZhIuVjOBBnaKH+SPQK0Q= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -867,8 +843,6 @@ golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.3.0 h1:w8ZOecv6NaNa/zC8944JTU3vz4u6Lagfk4RPQxv92NQ= -golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= @@ -881,8 +855,6 @@ golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.5.0 h1:OLmvp0KP+FVG99Ct/qFiL/Fhk4zp4QQnZ7b2U+5piUM= -golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= From be2500854be9793a53e00140477dc963a310cbe6 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Fri, 24 Feb 2023 10:18:46 +0100 Subject: [PATCH 07/17] added delay in TestRefreshBelowMinRTThreshold --- dht_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dht_test.go b/dht_test.go index f21c2d0e2..d29d3a3d7 100644 --- a/dht_test.go +++ b/dht_test.go @@ -767,7 +767,7 @@ func TestRefreshBelowMinRTThreshold(t *testing.T) { // and because of the above bootstrap, A also discovers E ! waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 4, 4, 20*time.Second) - time.Sleep(10 * time.Millisecond) + time.Sleep(100 * time.Millisecond) assert.Equal(t, dhtE.self, dhtA.routingTable.Find(dhtE.self), "A's routing table should have peer E!") } From 5e2ef88403e21e2001becad9a1a030d2e5b04693 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Tue, 7 Mar 2023 11:51:50 +0100 Subject: [PATCH 08/17] addressed review --- dht.go | 83 ++++++++++++++++++++++++++--------------------------- dht_test.go | 9 +++--- 2 files changed, 45 insertions(+), 47 deletions(-) diff --git a/dht.go b/dht.go index 856de2a6f..cd72fb2d5 100644 --- a/dht.go +++ b/dht.go @@ -120,15 +120,13 @@ type IpfsDHT struct { autoRefresh bool - // A function performing a lookup request to a remote peer.ID, verifying that it is able to - // answer it correctly - lookupCheck func(context.Context, peer.ID) error - lookupCheckTimeout time.Duration - lookupCheckInterval time.Duration // time interval during which we don't try to query the same peer again + // timeout for the lookupCheck operation + lookupCheckTimeout time.Duration + // time interval during which we don't try to query the same peer again + lookupCheckInterval time.Duration // recentlyCheckedPeers contains the peers recently queried with the time at which they were queried recentlyCheckedPeers map[peer.ID]time.Time recentlyCheckedPeersLk sync.Mutex - peerRecentlyQueried func(peer.ID) bool // A function returning a set of bootstrap peers to fallback on if all other attempts to fix // the routing table fail (or, e.g., this is the first time this node is @@ -327,36 +325,8 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err dht.routingTable = rt dht.bootstrapPeers = cfg.BootstrapPeers - dht.lookupCheck = func(ctx context.Context, p peer.ID) error { - // lookup request to p requesting for its own peer.ID - peerids, err := dht.protoMessenger.GetClosestPeers(ctx, p, p) - // p should return at least its own peerid - if err == nil && len(peerids) == 0 { - return fmt.Errorf("peer %s failed to return its closest peers, got %d", p, len(peerids)) - } - return err - } dht.lookupCheckTimeout = cfg.RoutingTable.RefreshQueryTimeout dht.recentlyCheckedPeers = make(map[peer.ID]time.Time) - dht.peerRecentlyQueried = func(p peer.ID) bool { - dht.recentlyCheckedPeersLk.Lock() - - now := time.Now() - - // clean recentlyCheckedPeers - for peerid, t := range dht.recentlyCheckedPeers { - // remove peers that have been queried more than lookupCheckInterval ago - if t.Add(dht.lookupCheckInterval).Before(now) { - delete(dht.recentlyCheckedPeers, peerid) - } - } - - // if p still in recentlyCheckedPeers, it has been queried less than - // lookupCheckInterval ago - _, ok := dht.recentlyCheckedPeers[p] - dht.recentlyCheckedPeersLk.Unlock() - return ok - } // rt refresh manager rtRefresh, err := makeRtRefreshManager(dht, cfg, maxLastSuccessfulOutboundThreshold) @@ -389,6 +359,39 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err return dht, nil } +// lookupCheck performs a lookup request to a remote peer.ID, verifying that it is able to +// answer it correctly +func (dht *IpfsDHT) lookupCheck(ctx context.Context, p peer.ID) error { + // lookup request to p requesting for its own peer.ID + peerids, err := dht.protoMessenger.GetClosestPeers(ctx, p, p) + // p should return at least its own peerid + if err == nil && len(peerids) == 0 { + return fmt.Errorf("peer %s failed to return its closest peers, got %d", p, len(peerids)) + } + return err +} + +// peerRecentlyQueried returns true if p has been queried less than dht.lookupCheckInterval ago +func (dht *IpfsDHT) peerRecentlyQueried(p peer.ID) bool { + dht.recentlyCheckedPeersLk.Lock() + defer dht.recentlyCheckedPeersLk.Unlock() + + now := time.Now() + + // clean recentlyCheckedPeers + for peerid, t := range dht.recentlyCheckedPeers { + // remove peers that have been queried more than lookupCheckInterval ago + if t.Add(dht.lookupCheckInterval).Before(now) { + delete(dht.recentlyCheckedPeers, peerid) + } + } + + // if p still in recentlyCheckedPeers, it has been queried less than + // lookupCheckInterval ago + _, ok := dht.recentlyCheckedPeers[p] + return ok +} + func makeRtRefreshManager(dht *IpfsDHT, cfg dhtcfg.Config, maxLastSuccessfulOutboundThreshold time.Duration) (*rtrefresh.RtRefreshManager, error) { keyGenFnc := func(cpl uint) (string, error) { p, err := dht.routingTable.GenRandPeerID(cpl) @@ -681,22 +684,16 @@ func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { livelinessCtx, cancel := context.WithTimeout(ctx, dht.lookupCheckTimeout) defer cancel() - // connecting to the remote peer - if err := dht.host.Connect(livelinessCtx, peer.AddrInfo{ID: p}); err != nil { - logger.Debugw("failed connection to DHT peer", "peer", p, "error", err) + // performing a FIND_NODE query + if err := dht.lookupCheck(livelinessCtx, p); err != nil { + logger.Debugw("connected peer not answering DHT request as expected", "peer", p, "error", err) return } - // add peer.ID to recently queried peers dht.recentlyCheckedPeersLk.Lock() dht.recentlyCheckedPeers[p] = time.Now() dht.recentlyCheckedPeersLk.Unlock() - // performing a FIND_NODE query - if err := dht.lookupCheck(livelinessCtx, p); err != nil { - logger.Debugw("connected peer not answering DHT request as expected", "peer", p, "error", err) - return - } // if the FIND_NODE succeeded, the peer is considered as valid dht.validPeerFound(ctx, p) } diff --git a/dht_test.go b/dht_test.go index d29d3a3d7..f8590c4f4 100644 --- a/dht_test.go +++ b/dht_test.go @@ -23,6 +23,7 @@ import ( ma "github.com/multiformats/go-multiaddr" "github.com/multiformats/go-multihash" "github.com/multiformats/go-multistream" + "golang.org/x/exp/maps" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -1565,7 +1566,7 @@ func TestFixLowPeers(t *testing.T) { // remove blacklist of already contacted peers mainD.recentlyCheckedPeersLk.Lock() - mainD.recentlyCheckedPeers = make(map[peer.ID]time.Time) + maps.Clear(mainD.recentlyCheckedPeers) mainD.recentlyCheckedPeersLk.Unlock() // but we will still get enough peers in the RT because of fix low Peers @@ -1675,11 +1676,11 @@ func TestHandleRemotePeerProtocolChanges(t *testing.T) { // clear connection history dhtA.recentlyCheckedPeersLk.Lock() - dhtA.recentlyCheckedPeers = make(map[peer.ID]time.Time) + maps.Clear(dhtA.recentlyCheckedPeers) dhtA.recentlyCheckedPeersLk.Unlock() dhtB.recentlyCheckedPeersLk.Lock() - dhtB.recentlyCheckedPeers = make(map[peer.ID]time.Time) + maps.Clear(dhtB.recentlyCheckedPeers) dhtB.recentlyCheckedPeersLk.Unlock() // now assert both have each other in their RT @@ -2184,7 +2185,7 @@ func TestPreconnectedNodes(t *testing.T) { // clear d2 recent checked peers d2.recentlyCheckedPeersLk.Lock() - d2.recentlyCheckedPeers = make(map[peer.ID]time.Time) + maps.Clear(d2.recentlyCheckedPeers) d2.recentlyCheckedPeersLk.Unlock() connect(t, ctx, d1, d2) From f083cb399ca1ab1e770e1da281cd74a547942419 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Tue, 7 Mar 2023 14:09:22 +0100 Subject: [PATCH 09/17] go mod tidy --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index d9a5a395f..e23633e62 100644 --- a/go.mod +++ b/go.mod @@ -34,6 +34,7 @@ require ( go.opentelemetry.io/otel v1.7.0 go.opentelemetry.io/otel/trace v1.7.0 go.uber.org/zap v1.24.0 + golang.org/x/exp v0.0.0-20221205204356-47842c84f3db ) require ( @@ -109,7 +110,6 @@ require ( go.uber.org/fx v1.18.2 // indirect go.uber.org/multierr v1.9.0 // indirect golang.org/x/crypto v0.6.0 // indirect - golang.org/x/exp v0.0.0-20221205204356-47842c84f3db // indirect golang.org/x/mod v0.7.0 // indirect golang.org/x/net v0.6.0 // indirect golang.org/x/sync v0.1.0 // indirect From 48081c3e1a642807856b7d4596bc41ac3f460aa4 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Wed, 7 Jun 2023 12:39:22 +0200 Subject: [PATCH 10/17] addressed Jorropo review --- dht.go | 18 +++++++++--------- dht_test.go | 2 +- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dht.go b/dht.go index cd72fb2d5..c87215b56 100644 --- a/dht.go +++ b/dht.go @@ -371,8 +371,9 @@ func (dht *IpfsDHT) lookupCheck(ctx context.Context, p peer.ID) error { return err } -// peerRecentlyQueried returns true if p has been queried less than dht.lookupCheckInterval ago -func (dht *IpfsDHT) peerRecentlyQueried(p peer.ID) bool { +// verifyAndLogPeerQuery returns true if p has been queried less than dht.lookupCheckInterval ago. +// If the peer has not been queried recently, log that it is being queried. +func (dht *IpfsDHT) verifyAndLogPeerQuery(p peer.ID) bool { dht.recentlyCheckedPeersLk.Lock() defer dht.recentlyCheckedPeersLk.Unlock() @@ -389,6 +390,10 @@ func (dht *IpfsDHT) peerRecentlyQueried(p peer.ID) bool { // if p still in recentlyCheckedPeers, it has been queried less than // lookupCheckInterval ago _, ok := dht.recentlyCheckedPeers[p] + if !ok { + // log that the peer is being queried + dht.recentlyCheckedPeers[p] = now + } return ok } @@ -665,7 +670,6 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { // and probe it to make sure it answers DHT queries as expected. If // it fails to answer, it isn't added to the routingTable. func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { - // if the appropriate bucket is already full, don't try to add the new peer.ID if !dht.routingTable.UsefulPeer(p) { return @@ -676,8 +680,8 @@ func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { if err != nil { logger.Errorw("failed to validate if peer is a DHT peer", "peer", p, "error", err) } else if b { - if dht.peerRecentlyQueried(p) { - // peer was already queried recently and didn't make it to the bucket + if dht.verifyAndLogPeerQuery(p) { + // peer was already queried recently, don't query it again return } @@ -689,10 +693,6 @@ func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { logger.Debugw("connected peer not answering DHT request as expected", "peer", p, "error", err) return } - // add peer.ID to recently queried peers - dht.recentlyCheckedPeersLk.Lock() - dht.recentlyCheckedPeers[p] = time.Now() - dht.recentlyCheckedPeersLk.Unlock() // if the FIND_NODE succeeded, the peer is considered as valid dht.validPeerFound(ctx, p) diff --git a/dht_test.go b/dht_test.go index f8590c4f4..d0410185e 100644 --- a/dht_test.go +++ b/dht_test.go @@ -767,7 +767,7 @@ func TestRefreshBelowMinRTThreshold(t *testing.T) { connect(t, ctx, dhtA, dhtD) // and because of the above bootstrap, A also discovers E ! - waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 4, 4, 20*time.Second) + waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 4, 4, 10*time.Second) time.Sleep(100 * time.Millisecond) assert.Equal(t, dhtE.self, dhtA.routingTable.Find(dhtE.self), "A's routing table should have peer E!") } From 5cf86808d61e44f7b477515d9957e49419c8484e Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Mon, 12 Jun 2023 09:42:21 +0200 Subject: [PATCH 11/17] added comments --- dht_bootstrap_test.go | 1 + dht_net.go | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dht_bootstrap_test.go b/dht_bootstrap_test.go index ef2f5926e..7fc1646ff 100644 --- a/dht_bootstrap_test.go +++ b/dht_bootstrap_test.go @@ -164,6 +164,7 @@ func TestBootstrappersReplacable(t *testing.T) { } require.Len(t, d.routingTable.ListPeers(), 0) + // Empty recentlyCheckedPeers, to make calls to peerFound predictable d.recentlyCheckedPeersLk.Lock() d.recentlyCheckedPeers = make(map[peer.ID]time.Time) d.recentlyCheckedPeersLk.Unlock() diff --git a/dht_net.go b/dht_net.go index d71ed3d91..f1eedd5d9 100644 --- a/dht_net.go +++ b/dht_net.go @@ -110,7 +110,9 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { return false } - // a peer has queried us, let's add it to RT + // a peer has queried us, let's add it to RT. A new go routine is required + // because we can't block the stream handler until the remote peer answers + // our query. go dht.peerFound(dht.ctx, mPeer) if c := baseLogger.Check(zap.DebugLevel, "handling message"); c != nil { From 7bdc62299072dddda5c5448368f8a52ebe2f7e7d Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Mon, 12 Jun 2023 11:02:24 +0200 Subject: [PATCH 12/17] removed state of peers probed recently --- dht.go | 48 ++++++++------------------------------- dht_bootstrap_test.go | 5 ---- dht_test.go | 20 ---------------- go.mod | 2 +- internal/config/config.go | 30 ++++++++++++------------ query_test.go | 4 ++-- 6 files changed, 26 insertions(+), 83 deletions(-) diff --git a/dht.go b/dht.go index c87215b56..ff9a5e3f5 100644 --- a/dht.go +++ b/dht.go @@ -122,11 +122,6 @@ type IpfsDHT struct { // timeout for the lookupCheck operation lookupCheckTimeout time.Duration - // time interval during which we don't try to query the same peer again - lookupCheckInterval time.Duration - // recentlyCheckedPeers contains the peers recently queried with the time at which they were queried - recentlyCheckedPeers map[peer.ID]time.Time - recentlyCheckedPeersLk sync.Mutex // A function returning a set of bootstrap peers to fallback on if all other attempts to fix // the routing table fail (or, e.g., this is the first time this node is @@ -186,7 +181,6 @@ func New(ctx context.Context, h host.Host, options ...Option) (*IpfsDHT, error) dht.autoRefresh = cfg.RoutingTable.AutoRefresh - dht.lookupCheckInterval = cfg.LookupCheckInterval dht.maxRecordAge = cfg.MaxRecordAge dht.enableProviders = cfg.EnableProviders dht.enableValues = cfg.EnableValues @@ -326,7 +320,6 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err dht.bootstrapPeers = cfg.BootstrapPeers dht.lookupCheckTimeout = cfg.RoutingTable.RefreshQueryTimeout - dht.recentlyCheckedPeers = make(map[peer.ID]time.Time) // rt refresh manager rtRefresh, err := makeRtRefreshManager(dht, cfg, maxLastSuccessfulOutboundThreshold) @@ -371,32 +364,6 @@ func (dht *IpfsDHT) lookupCheck(ctx context.Context, p peer.ID) error { return err } -// verifyAndLogPeerQuery returns true if p has been queried less than dht.lookupCheckInterval ago. -// If the peer has not been queried recently, log that it is being queried. -func (dht *IpfsDHT) verifyAndLogPeerQuery(p peer.ID) bool { - dht.recentlyCheckedPeersLk.Lock() - defer dht.recentlyCheckedPeersLk.Unlock() - - now := time.Now() - - // clean recentlyCheckedPeers - for peerid, t := range dht.recentlyCheckedPeers { - // remove peers that have been queried more than lookupCheckInterval ago - if t.Add(dht.lookupCheckInterval).Before(now) { - delete(dht.recentlyCheckedPeers, peerid) - } - } - - // if p still in recentlyCheckedPeers, it has been queried less than - // lookupCheckInterval ago - _, ok := dht.recentlyCheckedPeers[p] - if !ok { - // log that the peer is being queried - dht.recentlyCheckedPeers[p] = now - } - return ok -} - func makeRtRefreshManager(dht *IpfsDHT, cfg dhtcfg.Config, maxLastSuccessfulOutboundThreshold time.Duration) (*rtrefresh.RtRefreshManager, error) { keyGenFnc := func(cpl uint) (string, error) { p, err := dht.routingTable.GenRandPeerID(cpl) @@ -670,8 +637,9 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { // and probe it to make sure it answers DHT queries as expected. If // it fails to answer, it isn't added to the routingTable. func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { - // if the appropriate bucket is already full, don't try to add the new peer.ID - if !dht.routingTable.UsefulPeer(p) { + // if the peer is already in the routing table or the appropriate bucket is + // already full, don't try to add the new peer.ID + if dht.routingTable.Find(p) != "" || !dht.routingTable.UsefulPeer(p) { return } @@ -680,10 +648,12 @@ func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { if err != nil { logger.Errorw("failed to validate if peer is a DHT peer", "peer", p, "error", err) } else if b { - if dht.verifyAndLogPeerQuery(p) { - // peer was already queried recently, don't query it again - return - } + /* + if dht.verifyAndLogPeerQuery(p) { + // peer was already queried recently, don't query it again + return + } + */ livelinessCtx, cancel := context.WithTimeout(ctx, dht.lookupCheckTimeout) defer cancel() diff --git a/dht_bootstrap_test.go b/dht_bootstrap_test.go index 7fc1646ff..9dd496c0a 100644 --- a/dht_bootstrap_test.go +++ b/dht_bootstrap_test.go @@ -164,11 +164,6 @@ func TestBootstrappersReplacable(t *testing.T) { } require.Len(t, d.routingTable.ListPeers(), 0) - // Empty recentlyCheckedPeers, to make calls to peerFound predictable - d.recentlyCheckedPeersLk.Lock() - d.recentlyCheckedPeers = make(map[peer.ID]time.Time) - d.recentlyCheckedPeersLk.Unlock() - // adding d1 & d2 works now because there is space in the Routing Table require.NoError(t, d.host.Network().ClosePeer(d1.self)) require.NoError(t, d.host.Network().ClosePeer(d2.self)) diff --git a/dht_test.go b/dht_test.go index d0410185e..dab653408 100644 --- a/dht_test.go +++ b/dht_test.go @@ -23,7 +23,6 @@ import ( ma "github.com/multiformats/go-multiaddr" "github.com/multiformats/go-multihash" "github.com/multiformats/go-multistream" - "golang.org/x/exp/maps" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -1564,11 +1563,6 @@ func TestFixLowPeers(t *testing.T) { mainD.routingTable.RemovePeer(d.self) } - // remove blacklist of already contacted peers - mainD.recentlyCheckedPeersLk.Lock() - maps.Clear(mainD.recentlyCheckedPeers) - mainD.recentlyCheckedPeersLk.Unlock() - // but we will still get enough peers in the RT because of fix low Peers waitForWellFormedTables(t, []*IpfsDHT{mainD}, minRTRefreshThreshold, minRTRefreshThreshold, 5*time.Second) } @@ -1674,15 +1668,6 @@ func TestHandleRemotePeerProtocolChanges(t *testing.T) { connect(t, ctx, dhtA, dhtB) - // clear connection history - dhtA.recentlyCheckedPeersLk.Lock() - maps.Clear(dhtA.recentlyCheckedPeers) - dhtA.recentlyCheckedPeersLk.Unlock() - - dhtB.recentlyCheckedPeersLk.Lock() - maps.Clear(dhtB.recentlyCheckedPeers) - dhtB.recentlyCheckedPeersLk.Unlock() - // now assert both have each other in their RT waitForWellFormedTables(t, []*IpfsDHT{dhtA, dhtB}, 1, 1, 10*time.Second) @@ -2183,11 +2168,6 @@ func TestPreconnectedNodes(t *testing.T) { require.NoError(t, err) defer h2.Close() - // clear d2 recent checked peers - d2.recentlyCheckedPeersLk.Lock() - maps.Clear(d2.recentlyCheckedPeers) - d2.recentlyCheckedPeersLk.Unlock() - connect(t, ctx, d1, d2) // See if it works diff --git a/go.mod b/go.mod index e23633e62..d9a5a395f 100644 --- a/go.mod +++ b/go.mod @@ -34,7 +34,6 @@ require ( go.opentelemetry.io/otel v1.7.0 go.opentelemetry.io/otel/trace v1.7.0 go.uber.org/zap v1.24.0 - golang.org/x/exp v0.0.0-20221205204356-47842c84f3db ) require ( @@ -110,6 +109,7 @@ require ( go.uber.org/fx v1.18.2 // indirect go.uber.org/multierr v1.9.0 // indirect golang.org/x/crypto v0.6.0 // indirect + golang.org/x/exp v0.0.0-20221205204356-47842c84f3db // indirect golang.org/x/mod v0.7.0 // indirect golang.org/x/net v0.6.0 // indirect golang.org/x/sync v0.1.0 // indirect diff --git a/internal/config/config.go b/internal/config/config.go index c95514c7d..8b7962ed4 100644 --- a/internal/config/config.go +++ b/internal/config/config.go @@ -32,21 +32,20 @@ type RouteTableFilterFunc func(dht interface{}, p peer.ID) bool // Config is a structure containing all the options that can be used when constructing a DHT. type Config struct { - Datastore ds.Batching - Validator record.Validator - ValidatorChanged bool // if true implies that the validator has been changed and that Defaults should not be used - Mode ModeOpt - ProtocolPrefix protocol.ID - V1ProtocolOverride protocol.ID - BucketSize int - Concurrency int - Resiliency int - LookupCheckInterval time.Duration - MaxRecordAge time.Duration - EnableProviders bool - EnableValues bool - ProviderStore providers.ProviderStore - QueryPeerFilter QueryFilterFunc + Datastore ds.Batching + Validator record.Validator + ValidatorChanged bool // if true implies that the validator has been changed and that Defaults should not be used + Mode ModeOpt + ProtocolPrefix protocol.ID + V1ProtocolOverride protocol.ID + BucketSize int + Concurrency int + Resiliency int + MaxRecordAge time.Duration + EnableProviders bool + EnableValues bool + ProviderStore providers.ProviderStore + QueryPeerFilter QueryFilterFunc RoutingTable struct { RefreshQueryTimeout time.Duration @@ -116,7 +115,6 @@ var Defaults = func(o *Config) error { o.RoutingTable.AutoRefresh = true o.RoutingTable.PeerFilter = EmptyRTFilter - o.LookupCheckInterval = 5 * time.Second o.MaxRecordAge = providers.ProvideValidity o.BucketSize = defaultBucketSize diff --git a/query_test.go b/query_test.go index e79af514f..91669641a 100644 --- a/query_test.go +++ b/query_test.go @@ -34,7 +34,7 @@ func TestRTEvictionOnFailedQuery(t *testing.T) { // peers should be in the RT because of fixLowPeers require.NoError(t, tu.WaitFor(ctx, func() error { if !checkRoutingTable(d1, d2) { - return fmt.Errorf("should have routes") + return fmt.Errorf("should have routes") } return nil })) @@ -45,7 +45,7 @@ func TestRTEvictionOnFailedQuery(t *testing.T) { // peers will still be in the RT because we have decoupled membership from connectivity require.NoError(t, tu.WaitFor(ctx, func() error { if !checkRoutingTable(d1, d2) { - return fmt.Errorf("should have routes") + return fmt.Errorf("should have routes") } return nil })) From e85f001bc867e143d823b73411564c62c36b4ef3 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Mon, 12 Jun 2023 11:43:44 +0200 Subject: [PATCH 13/17] fix conflicts merge --- dht.go | 8 +- dht_bootstrap_test.go | 4 +- dht_net.go | 2 +- ext_test.go | 456 ------------------------------------------ handlers_test.go | 2 +- query.go | 2 +- subscriber_notifee.go | 2 +- 7 files changed, 10 insertions(+), 466 deletions(-) diff --git a/dht.go b/dht.go index 959543392..0d4b0a8b4 100644 --- a/dht.go +++ b/dht.go @@ -245,7 +245,7 @@ func New(ctx context.Context, h host.Host, options ...Option) (*IpfsDHT, error) // Fill routing table with currently connected peers that are DHT servers dht.plk.Lock() for _, p := range dht.host.Network().Peers() { - dht.peerFound(p) + dht.peerFound(dht.ctx, p) } dht.plk.Unlock() @@ -339,7 +339,7 @@ func makeDHT(ctx context.Context, h host.Host, cfg dhtcfg.Config) (*IpfsDHT, err dht.lookupCheckTimeout = cfg.RoutingTable.RefreshQueryTimeout - // init network size estimator + // init network size estimator dht.nsEstimator = netsize.NewEstimator(h.ID(), rt, cfg.BucketSize) if dht.enableOptProv { @@ -512,7 +512,7 @@ func (dht *IpfsDHT) fixLowPeers(ctx context.Context) { // we try to add all peers we are connected to to the Routing Table // in case they aren't already there. for _, p := range dht.host.Network().Peers() { - dht.peerFound(p) + dht.peerFound(ctx, p) } // TODO Active Bootstrapping @@ -661,7 +661,7 @@ func (dht *IpfsDHT) rtPeerLoop(proc goprocess.Process) { // peerFound verifies whether the found peer advertises DHT protocols // and probe it to make sure it answers DHT queries as expected. If // it fails to answer, it isn't added to the routingTable. -func (dht *IpfsDHT) peerFound(p peer.ID) { +func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { // if the peer is already in the routing table or the appropriate bucket is // already full, don't try to add the new peer.ID if dht.routingTable.Find(p) != "" || !dht.routingTable.UsefulPeer(p) { diff --git a/dht_bootstrap_test.go b/dht_bootstrap_test.go index e2236f5a1..9dd496c0a 100644 --- a/dht_bootstrap_test.go +++ b/dht_bootstrap_test.go @@ -191,8 +191,8 @@ func TestBootstrappersReplacable(t *testing.T) { require.NoError(t, d.host.Network().ClosePeer(d5.self)) connectNoSync(t, ctx, d, d1) connectNoSync(t, ctx, d, d5) - d.peerFound(d5.self) - d.peerFound(d1.self) + d.peerFound(ctx, d5.self) + d.peerFound(ctx, d1.self) time.Sleep(1 * time.Second) require.Len(t, d.routingTable.ListPeers(), 2) diff --git a/dht_net.go b/dht_net.go index 81f233f3a..f1eedd5d9 100644 --- a/dht_net.go +++ b/dht_net.go @@ -113,7 +113,7 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { // a peer has queried us, let's add it to RT. A new go routine is required // because we can't block the stream handler until the remote peer answers // our query. - go dht.peerFound(mPeer) + go dht.peerFound(dht.ctx, mPeer) if c := baseLogger.Check(zap.DebugLevel, "handling message"); c != nil { c.Write(zap.String("from", mPeer.String()), diff --git a/ext_test.go b/ext_test.go index 1b2554687..51340f334 100644 --- a/ext_test.go +++ b/ext_test.go @@ -2,471 +2,15 @@ package dht import ( "context" - "math/rand" "testing" "time" "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/routing" "github.com/stretchr/testify/require" - record "github.com/libp2p/go-libp2p-record" - bhost "github.com/libp2p/go-libp2p/p2p/host/basic" mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" - swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing" - - //lint:ignore SA1019 TODO migrate away from gogo pb - "github.com/libp2p/go-msgio/protoio" - - pb "github.com/libp2p/go-libp2p-kad-dht/pb" - - u "github.com/ipfs/boxo/util" ) -// Test that one hung request to a peer doesn't prevent another request -// using that same peer from obeying its context. -func TestHungRequest(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - mn, err := mocknet.FullMeshLinked(2) - if err != nil { - t.Fatal(err) - } - defer mn.Close() - hosts := mn.Hosts() - - os := []Option{testPrefix, DisableAutoRefresh(), Mode(ModeServer)} - d, err := New(ctx, hosts[0], os...) - if err != nil { - t.Fatal(err) - } - for _, proto := range d.serverProtocols { - // Hang on every request. - hosts[1].SetStreamHandler(proto, func(s network.Stream) { - defer s.Reset() // nolint - <-ctx.Done() - }) - } - - err = mn.ConnectAllButSelf() - if err != nil { - t.Fatal("failed to connect peers", err) - } - - // Wait at a bit for a peer in our routing table. - for i := 0; i < 100 && d.routingTable.Size() == 0; i++ { - time.Sleep(10 * time.Millisecond) - } - if d.routingTable.Size() == 0 { - t.Fatal("failed to fill routing table") - } - - ctx1, cancel1 := context.WithTimeout(ctx, 1*time.Second) - defer cancel1() - - done := make(chan error, 1) - go func() { - _, err := d.GetClosestPeers(ctx1, testCaseCids[0].KeyString()) - done <- err - }() - - time.Sleep(100 * time.Millisecond) - ctx2, cancel2 := context.WithTimeout(ctx, 100*time.Millisecond) - defer cancel2() - err = d.Provide(ctx2, testCaseCids[0], true) - if err != context.DeadlineExceeded { - t.Errorf("expected to fail with deadline exceeded, got: %s", ctx2.Err()) - } - select { - case err = <-done: - t.Error("GetClosestPeers should not have returned yet", err) - default: - err = <-done - if err != context.DeadlineExceeded { - t.Errorf("expected the deadline to be exceeded, got %s", err) - } - } - - if d.routingTable.Size() == 0 { - // make sure we didn't just disconnect - t.Fatal("expected peers in the routing table") - } -} - -func TestGetFailures(t *testing.T) { - if testing.Short() { - t.SkipNow() - } - - ctx := context.Background() - - host1, err := bhost.NewHost(swarmt.GenSwarm(t, swarmt.OptDisableReuseport), new(bhost.HostOpts)) - require.NoError(t, err) - host1.Start() - host2, err := bhost.NewHost(swarmt.GenSwarm(t, swarmt.OptDisableReuseport), new(bhost.HostOpts)) - require.NoError(t, err) - host2.Start() - - d, err := New(ctx, host1, testPrefix, DisableAutoRefresh(), Mode(ModeServer)) - require.NoError(t, err) - - // Reply with failures to every message - for _, proto := range d.serverProtocols { - host2.SetStreamHandler(proto, func(s network.Stream) { - time.Sleep(400 * time.Millisecond) - s.Close() - }) - } - - host1.Peerstore().AddAddrs(host2.ID(), host2.Addrs(), peerstore.ConnectedAddrTTL) - _, err = host1.Network().DialPeer(ctx, host2.ID()) - require.NoError(t, err) - time.Sleep(1 * time.Second) - - // This one should time out - ctx1, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond) - defer cancel() - if _, err := d.GetValue(ctx1, "test"); err != nil { - if merr, ok := err.(u.MultiErr); ok && len(merr) > 0 { - err = merr[0] - } - - if err != context.DeadlineExceeded { - t.Fatal("Got different error than we expected", err) - } - } else { - t.Fatal("Did not get expected error!") - } - - t.Log("Timeout test passed.") - - for _, proto := range d.serverProtocols { - // Reply with failures to every message - host2.SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - // user gave up - return - } - - resp := &pb.Message{ - Type: pmes.Type, - } - _ = pbw.WriteMsg(resp) - }) - } - - // This one should fail with NotFound. - // long context timeout to ensure we dont end too early. - // the dht should be exhausting its query and returning not found. - // (was 3 seconds before which should be _plenty_ of time, but maybe - // travis machines really have a hard time...) - ctx2, cancel := context.WithTimeout(context.Background(), 20*time.Second) - defer cancel() - _, err = d.GetValue(ctx2, "test") - if err != nil { - if merr, ok := err.(u.MultiErr); ok && len(merr) > 0 { - err = merr[0] - } - if err != routing.ErrNotFound { - t.Fatalf("Expected ErrNotFound, got: %s", err) - } - } else { - t.Fatal("expected error, got none.") - } - - t.Log("ErrNotFound check passed!") - - // Now we test this DHT's handleGetValue failure - { - typ := pb.Message_GET_VALUE - str := "hello" - - rec := record.MakePutRecord(str, []byte("blah")) - req := pb.Message{ - Type: typ, - Key: []byte(str), - Record: rec, - } - - s, err := host2.NewStream(context.Background(), host1.ID(), d.protocols...) - if err != nil { - t.Fatal(err) - } - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - if err := pbw.WriteMsg(&req); err != nil { - t.Fatal(err) - } - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - t.Fatal(err) - } - if pmes.GetRecord() != nil { - t.Fatal("shouldnt have value") - } - if pmes.GetProviderPeers() != nil { - t.Fatal("shouldnt have provider peers") - } - } - - if d.routingTable.Size() == 0 { - // make sure we didn't just disconnect - t.Fatal("expected peers in the routing table") - } -} - -func TestNotFound(t *testing.T) { - // t.Skip("skipping test to debug another") - if testing.Short() { - t.SkipNow() - } - - ctx := context.Background() - mn, err := mocknet.FullMeshConnected(16) - if err != nil { - t.Fatal(err) - } - defer mn.Close() - hosts := mn.Hosts() - - os := []Option{testPrefix, DisableAutoRefresh(), Mode(ModeServer)} - d, err := New(ctx, hosts[0], os...) - if err != nil { - t.Fatal(err) - } - - // Reply with random peers to every message - for _, host := range hosts { - host := host // shadow loop var - for _, proto := range d.serverProtocols { - host.SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - // this isn't an error, it just means the stream has died. - return - } - - switch pmes.GetType() { - case pb.Message_GET_VALUE: - resp := &pb.Message{Type: pmes.Type} - - ps := []peer.AddrInfo{} - for i := 0; i < 7; i++ { - p := hosts[rand.Intn(len(hosts))].ID() - pi := host.Peerstore().PeerInfo(p) - ps = append(ps, pi) - } - - resp.CloserPeers = pb.PeerInfosToPBPeers(d.host.Network(), ps) - if err := pbw.WriteMsg(resp); err != nil { - return - } - default: - panic("Shouldnt recieve this.") - } - }) - } - for _, peer := range hosts { - if host == peer { - continue - } - _ = peer.Peerstore().AddProtocols(host.ID(), d.serverProtocols...) - } - } - - for _, p := range hosts { - d.peerFound(p.ID()) - } - - // long timeout to ensure timing is not at play. - ctx, cancel := context.WithTimeout(ctx, time.Second*20) - defer cancel() - v, err := d.GetValue(ctx, "hello") - logger.Debugf("get value got %v", v) - if err != nil { - if merr, ok := err.(u.MultiErr); ok && len(merr) > 0 { - err = merr[0] - } - switch err { - case routing.ErrNotFound: - if d.routingTable.Size() == 0 { - // make sure we didn't just disconnect - t.Fatal("expected peers in the routing table") - } - // Success! - return - case u.ErrTimeout: - t.Fatal("Should not have gotten timeout!") - default: - t.Fatalf("Got unexpected error: %s", err) - } - } - t.Fatal("Expected to recieve an error.") -} - -// If less than K nodes are in the entire network, it should fail when we make -// a GET rpc and nobody has the value -func TestLessThanKResponses(t *testing.T) { - // t.Skip("skipping test to debug another") - // t.Skip("skipping test because it makes a lot of output") - - ctx := context.Background() - mn, err := mocknet.FullMeshConnected(6) - if err != nil { - t.Fatal(err) - } - defer mn.Close() - hosts := mn.Hosts() - - os := []Option{testPrefix, DisableAutoRefresh(), Mode(ModeServer)} - d, err := New(ctx, hosts[0], os...) - if err != nil { - t.Fatal(err) - } - - for i := 1; i < 5; i++ { - d.peerFound(hosts[i].ID()) - } - - // Reply with random peers to every message - for _, host := range hosts { - host := host // shadow loop var - for _, proto := range d.serverProtocols { - host.SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } - - switch pmes.GetType() { - case pb.Message_GET_VALUE: - pi := host.Peerstore().PeerInfo(hosts[1].ID()) - resp := &pb.Message{ - Type: pmes.Type, - CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), - } - - if err := pbw.WriteMsg(resp); err != nil { - panic(err) - } - default: - panic("Shouldnt recieve this.") - } - - }) - } - } - - ctx, cancel := context.WithTimeout(ctx, time.Second*30) - defer cancel() - if _, err := d.GetValue(ctx, "hello"); err != nil { - switch err { - case routing.ErrNotFound: - // Success! - return - case u.ErrTimeout: - t.Fatal("Should not have gotten timeout!") - default: - t.Fatalf("Got unexpected error: %s", err) - } - } - t.Fatal("Expected to recieve an error.") -} - -// Test multiple queries against a node that closes its stream after every query. -func TestMultipleQueries(t *testing.T) { - if testing.Short() { - t.SkipNow() - } - - ctx := context.Background() - mn, err := mocknet.FullMeshConnected(2) - if err != nil { - t.Fatal(err) - } - defer mn.Close() - hosts := mn.Hosts() - os := []Option{testPrefix, DisableAutoRefresh(), Mode(ModeServer)} - d, err := New(ctx, hosts[0], os...) - if err != nil { - t.Fatal(err) - } - - d.peerFound(hosts[1].ID()) - - for _, proto := range d.serverProtocols { - // It would be nice to be able to just get a value and succeed but then - // we'd need to deal with selectors and validators... - hosts[1].SetStreamHandler(proto, func(s network.Stream) { - defer s.Close() - - pbr := protoio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := protoio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } - - switch pmes.GetType() { - case pb.Message_GET_VALUE: - pi := hosts[1].Peerstore().PeerInfo(hosts[0].ID()) - resp := &pb.Message{ - Type: pmes.Type, - CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), - } - - if err := pbw.WriteMsg(resp); err != nil { - panic(err) - } - default: - panic("Shouldnt recieve this.") - } - }) - } - - // long timeout to ensure timing is not at play. - ctx, cancel := context.WithTimeout(ctx, time.Second*20) - defer cancel() - for i := 0; i < 10; i++ { - if _, err := d.GetValue(ctx, "hello"); err != nil { - switch err { - case routing.ErrNotFound: - // Success! - continue - case u.ErrTimeout: - t.Fatal("Should not have gotten timeout!") - default: - t.Fatalf("Got unexpected error: %s", err) - } - } - t.Fatal("Expected to recieve an error.") - } -} - func TestInvalidRemotePeers(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/handlers_test.go b/handlers_test.go index 35959df62..d829e38b1 100644 --- a/handlers_test.go +++ b/handlers_test.go @@ -111,7 +111,7 @@ func BenchmarkHandleFindPeer(b *testing.B) { panic(err) } - d.peerFound(id) + d.peerFound(ctx, id) peers = append(peers, id) a, err := ma.NewMultiaddr(fmt.Sprintf("/ip4/127.0.0.1/tcp/%d", 2000+i)) diff --git a/query.go b/query.go index 524269aec..c8b07d650 100644 --- a/query.go +++ b/query.go @@ -446,7 +446,7 @@ func (q *query) queryPeer(ctx context.Context, ch chan<- *queryUpdate, p peer.ID queryDuration := time.Since(startQuery) // query successful, try to add to RT - q.dht.validPeerFound(p) + q.dht.validPeerFound(q.dht.ctx, p) // process new peers saw := []peer.ID{} diff --git a/subscriber_notifee.go b/subscriber_notifee.go index 55ec69d15..23c21ffb9 100644 --- a/subscriber_notifee.go +++ b/subscriber_notifee.go @@ -108,7 +108,7 @@ func handlePeerChangeEvent(dht *IpfsDHT, p peer.ID) { logger.Errorf("could not check peerstore for protocol support: err: %s", err) return } else if valid { - dht.peerFound(p) + dht.peerFound(dht.ctx, p) dht.fixRTIfNeeded() } else { dht.peerStoppedDHT(p) From 89474e587635e5e03731549f2be7a312eb187828 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Mon, 12 Jun 2023 12:22:14 +0200 Subject: [PATCH 14/17] updated deps --- dht_filters_test.go | 11 +- go.mod | 92 ++++---- go.sum | 496 +++++++++----------------------------------- 3 files changed, 151 insertions(+), 448 deletions(-) diff --git a/dht_filters_test.go b/dht_filters_test.go index 3273011d8..7714b8d9e 100644 --- a/dht_filters_test.go +++ b/dht_filters_test.go @@ -3,6 +3,7 @@ package dht import ( "context" "net" + "sync/atomic" "testing" ic "github.com/libp2p/go-libp2p/core/crypto" @@ -31,12 +32,17 @@ func TestIsRelay(t *testing.T) { type mockConn struct { local peer.AddrInfo remote peer.AddrInfo + + isClosed atomic.Bool } var _ network.Conn = (*mockConn)(nil) -func (m *mockConn) ID() string { return "0" } -func (m *mockConn) Close() error { return nil } +func (m *mockConn) ID() string { return "0" } +func (m *mockConn) Close() error { + m.isClosed.Store(true) + return nil +} func (m *mockConn) NewStream(context.Context) (network.Stream, error) { return nil, nil } func (m *mockConn) GetStreams() []network.Stream { return []network.Stream{} } func (m *mockConn) Stat() network.ConnStats { @@ -50,6 +56,7 @@ func (m *mockConn) LocalPrivateKey() ic.PrivKey { return nil } func (m *mockConn) RemotePeer() peer.ID { return m.remote.ID } func (m *mockConn) RemotePublicKey() ic.PubKey { return nil } func (m *mockConn) ConnState() network.ConnectionState { return network.ConnectionState{} } +func (m *mockConn) IsClosed() bool { return m.isClosed.Load() } func TestFilterCaching(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) diff --git a/go.mod b/go.mod index a08f89beb..0467adade 100644 --- a/go.mod +++ b/go.mod @@ -8,86 +8,85 @@ require ( github.com/google/uuid v1.3.0 github.com/hashicorp/go-multierror v1.1.1 github.com/hashicorp/golang-lru v0.5.4 - github.com/ipfs/boxo v0.8.0 - github.com/ipfs/go-cid v0.4.0 + github.com/ipfs/boxo v0.10.0 + github.com/ipfs/go-cid v0.4.1 github.com/ipfs/go-datastore v0.6.0 github.com/ipfs/go-detect-race v0.0.1 github.com/ipfs/go-log v1.0.5 github.com/jbenet/goprocess v0.1.4 - github.com/libp2p/go-libp2p v0.26.3 - github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f + github.com/libp2p/go-libp2p v0.27.5 + github.com/libp2p/go-libp2p-kbucket v0.6.1 github.com/libp2p/go-libp2p-record v0.2.0 - github.com/libp2p/go-libp2p-routing-helpers v0.4.0 + github.com/libp2p/go-libp2p-routing-helpers v0.7.0 github.com/libp2p/go-libp2p-testing v0.12.0 github.com/libp2p/go-libp2p-xor v0.1.0 github.com/libp2p/go-msgio v0.3.0 github.com/libp2p/go-netroute v0.2.1 github.com/multiformats/go-base32 v0.1.0 - github.com/multiformats/go-multiaddr v0.8.0 - github.com/multiformats/go-multibase v0.1.1 - github.com/multiformats/go-multihash v0.2.1 + github.com/multiformats/go-multiaddr v0.9.0 + github.com/multiformats/go-multibase v0.2.0 + github.com/multiformats/go-multihash v0.2.3 github.com/multiformats/go-multistream v0.4.1 - github.com/stretchr/testify v1.8.2 + github.com/stretchr/testify v1.8.4 github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 go.opencensus.io v0.24.0 - go.opentelemetry.io/otel v1.14.0 - go.opentelemetry.io/otel/trace v1.14.0 + go.opentelemetry.io/otel v1.16.0 + go.opentelemetry.io/otel/trace v1.16.0 go.uber.org/zap v1.24.0 - gonum.org/v1/gonum v0.11.0 + gonum.org/v1/gonum v0.13.0 ) require ( github.com/benbjohnson/clock v1.3.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/containerd/cgroups v1.0.4 // indirect + github.com/containerd/cgroups v1.1.0 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect - github.com/decred/dcrd/dcrec/secp256k1/v4 v4.1.0 // indirect + github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/elastic/gosigar v0.14.2 // indirect github.com/flynn/noise v1.0.0 // indirect github.com/francoispqt/gojay v1.2.13 // indirect - github.com/go-logr/logr v1.2.3 // indirect + github.com/go-logr/logr v1.2.4 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 // indirect + github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect github.com/godbus/dbus/v5 v5.1.0 // indirect github.com/golang/mock v1.6.0 // indirect - github.com/golang/protobuf v1.5.2 // indirect - github.com/google/pprof v0.0.0-20221203041831-ce31453925ec // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/pprof v0.0.0-20230405160723-4a4c7d95572b // indirect github.com/gorilla/websocket v1.5.0 // indirect github.com/hashicorp/errwrap v1.1.0 // indirect - github.com/huin/goupnp v1.0.3 // indirect - github.com/ipfs/go-ipfs-util v0.0.2 // indirect + github.com/huin/goupnp v1.1.0 // indirect github.com/ipfs/go-log/v2 v2.5.1 // indirect github.com/ipld/go-ipld-prime v0.20.0 // indirect github.com/jackpal/go-nat-pmp v1.0.2 // indirect github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect - github.com/klauspost/compress v1.15.12 // indirect - github.com/klauspost/cpuid/v2 v2.2.4 // indirect - github.com/koron/go-ssdp v0.0.3 // indirect + github.com/klauspost/compress v1.16.4 // indirect + github.com/klauspost/cpuid/v2 v2.2.5 // indirect + github.com/koron/go-ssdp v0.0.4 // indirect github.com/libp2p/go-buffer-pool v0.1.0 // indirect github.com/libp2p/go-cidranger v1.1.0 // indirect github.com/libp2p/go-flow-metrics v0.1.0 // indirect - github.com/libp2p/go-libp2p-asn-util v0.2.0 // indirect + github.com/libp2p/go-libp2p-asn-util v0.3.0 // indirect github.com/libp2p/go-nat v0.1.0 // indirect github.com/libp2p/go-reuseport v0.2.0 // indirect github.com/libp2p/go-yamux/v4 v4.0.0 // indirect github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd // indirect - github.com/mattn/go-isatty v0.0.17 // indirect + github.com/mattn/go-isatty v0.0.19 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect - github.com/miekg/dns v1.1.50 // indirect + github.com/miekg/dns v1.1.53 // indirect github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b // indirect github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc // indirect - github.com/minio/sha256-simd v1.0.0 // indirect + github.com/minio/sha256-simd v1.0.1 // indirect github.com/mr-tron/base58 v1.2.0 // indirect github.com/multiformats/go-base36 v0.2.0 // indirect github.com/multiformats/go-multiaddr-dns v0.3.1 // indirect github.com/multiformats/go-multiaddr-fmt v0.1.0 // indirect - github.com/multiformats/go-multicodec v0.8.1 // indirect + github.com/multiformats/go-multicodec v0.9.0 // indirect github.com/multiformats/go-varint v0.0.7 // indirect - github.com/onsi/ginkgo/v2 v2.5.1 // indirect + github.com/onsi/ginkgo/v2 v2.9.2 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 // indirect @@ -96,29 +95,30 @@ require ( github.com/polydawn/refmt v0.89.0 // indirect github.com/prometheus/client_golang v1.14.0 // indirect github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.37.0 // indirect - github.com/prometheus/procfs v0.8.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect github.com/quic-go/qpack v0.4.0 // indirect - github.com/quic-go/qtls-go1-19 v0.2.1 // indirect - github.com/quic-go/qtls-go1-20 v0.1.1 // indirect + github.com/quic-go/qtls-go1-19 v0.3.2 // indirect + github.com/quic-go/qtls-go1-20 v0.2.2 // indirect github.com/quic-go/quic-go v0.33.0 // indirect github.com/quic-go/webtransport-go v0.5.2 // indirect github.com/raulk/go-watchdog v1.3.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect - go.uber.org/atomic v1.10.0 // indirect - go.uber.org/dig v1.15.0 // indirect - go.uber.org/fx v1.18.2 // indirect - go.uber.org/multierr v1.9.0 // indirect - golang.org/x/crypto v0.6.0 // indirect - golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb // indirect - golang.org/x/mod v0.7.0 // indirect - golang.org/x/net v0.7.0 // indirect + go.opentelemetry.io/otel/metric v1.16.0 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/dig v1.16.1 // indirect + go.uber.org/fx v1.19.2 // indirect + go.uber.org/multierr v1.11.0 // indirect + golang.org/x/crypto v0.9.0 // indirect + golang.org/x/exp v0.0.0-20230321023759-10a507213a29 // indirect + golang.org/x/mod v0.10.0 // indirect + golang.org/x/net v0.10.0 // indirect golang.org/x/sync v0.1.0 // indirect - golang.org/x/sys v0.6.0 // indirect - golang.org/x/text v0.7.0 // indirect - golang.org/x/tools v0.3.0 // indirect - google.golang.org/protobuf v1.28.1 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + golang.org/x/tools v0.7.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect - lukechampine.com/blake3 v1.1.7 // indirect + lukechampine.com/blake3 v1.2.1 // indirect nhooyr.io/websocket v1.8.7 // indirect ) diff --git a/go.sum b/go.sum index ecad0c3e7..158ec8621 100644 --- a/go.sum +++ b/go.sum @@ -2,58 +2,20 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.31.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.37.0/go.mod h1:TS1dMSSfndXH133OKGwekG838Om/cQT0BUHV3HcBgoo= -cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= -cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= -cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= -cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= -cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= -cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= -cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= -cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= -cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= -cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= -cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= -cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= -cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= -cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= -cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= -cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= -cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= -cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= -cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= -cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= -cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= -cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= -cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= -cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= -cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= -cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3/go.mod h1:Yl+fi1br7+Rr3LqpNJf1/uxUdtRUV+Tnj0o93V2B9MU= -dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0/go.mod h1:JLBrvjyP0v+ecvNYvCpyZgu5/xkfAUhi6wJj28eUfSU= dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412/go.mod h1:a1inKt/atXimZ4Mv927x+r7UpyzRUf4emIoiiSC2TN4= dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D6DFvNNtx+9ybjezNCa8XF0xaYcETyp6rHWU= git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= -github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= @@ -68,19 +30,14 @@ github.com/btcsuite/websocket v0.0.0-20150119174127-31079b680792/go.mod h1:ghJtE github.com/btcsuite/winsvc v1.0.0/go.mod h1:jsenWakMcC0zFBFurPLEAyrnc/teJEM1O46fmI40EZs= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= -github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= -github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/cilium/ebpf v0.2.0/go.mod h1:To2CFviqOWL/M0gIMsvSMlqe7em/l1ALkX1PyjrX2Qs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/containerd/cgroups v0.0.0-20201119153540-4cbc285b3327/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE= -github.com/containerd/cgroups v1.0.4 h1:jN/mbWBEaz+T1pi5OFtnkQ+8qnmEbAr1Oo1FRm5B0dA= -github.com/containerd/cgroups v1.0.4/go.mod h1:nLNQtsF7Sl2HxNebu77i1R0oDlhiTG+kO4JTrUzo6IA= +github.com/containerd/cgroups v1.1.0 h1:v8rEWFl6EoqHB+swVNjVoCJE8o3jX7e8nqBGPLaDFBM= +github.com/containerd/cgroups v1.1.0/go.mod h1:6ppBcbh/NOOUU+dMKrykgaBnK9lCIBxHqJDGwsa1mIw= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= @@ -98,9 +55,9 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c h1:pFUpOrbxDR6AkioZ1ySsx5yxlDQZ8stG2b88gTPxgJU= github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c/go.mod h1:6UhI8N9EjYm1c2odKpFpAYeR8dsBeM7PtzQhRgxRr9U= -github.com/decred/dcrd/crypto/blake256 v1.0.0 h1:/8DMNYp9SGi5f0w7uCm6d6M4OU2rGFK09Y2A4Xv7EE0= -github.com/decred/dcrd/dcrec/secp256k1/v4 v4.1.0 h1:HbphB4TFFXpv7MNrT52FGrrgVXF1owhMVTHFZIlnvd4= -github.com/decred/dcrd/dcrec/secp256k1/v4 v4.1.0/go.mod h1:DZGJHZMqrU4JJqFAWUS2UO1+lbSKsdiOoYi9Zzey7Fc= +github.com/decred/dcrd/crypto/blake256 v1.0.1 h1:7PltbUIQB7u/FfZ39+DGa/ShuMyJ5ilcvdfma9wOH6Y= +github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 h1:8UrgZ3GkP4i/CLijOJx79Yu+etlyjdBU4sfcs2WYQMs= +github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0/go.mod h1:v57UDF4pDQJcEfFUCRop3lJL149eHGSe9Jvczhzjo/0= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= @@ -129,20 +86,9 @@ github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwv github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= -github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-kit/log v0.2.0/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= -github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= -github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ= +github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-playground/assert/v2 v2.0.1/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= @@ -152,9 +98,8 @@ github.com/go-playground/universal-translator v0.17.0 h1:icxd5fm+REJzpZx7ZfpaD87 github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= github.com/go-playground/validator/v10 v10.2.0 h1:KgJ0snyC2R9VXYN2rneOtQcw5aHQB1Vv0sFl1UcHBOY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 h1:p104kn46Q8WdvHunIJ9dAyjPVtrBPhSr3KT2yUst43I= -github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= +github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 h1:tfuBGBXKqDEevZMzYi5KSi8KkcZtzBcTgAUUtapy0OI= +github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572/go.mod h1:9Pwr4B2jHnOSGXyyzV8ROjYa2ojvAY6HCGYYfMoC3Ls= github.com/go-yaml/yaml v2.1.0+incompatible/go.mod h1:w2MrLa16VYP0jy6N7M5kHaCkaLENm+P+Tv+MfurjSw0= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee h1:s+21KNqlpePfkah2I+gwHF8xmJWRjooY+5248k6m4A0= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= @@ -174,24 +119,17 @@ 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-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= 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/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= -github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= -github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= @@ -199,24 +137,19 @@ github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrU github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= @@ -226,16 +159,9 @@ github.com/google/gopacket v1.1.17/go.mod h1:UdDNZ1OO62aGYVnPhxT1U6aI7ukYtA/kB8v github.com/google/gopacket v1.1.19 h1:ves8RnFZPGiFnTS0uPQStjwru6uO6h+nlr9j6fL7kF8= github.com/google/gopacket v1.1.19/go.mod h1:iJ8V8n6KS+z2U1A8pUwu8bW5SyEMkXJB8Yo/Vo+TKTo= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= -github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20221203041831-ce31453925ec h1:fR20TYVVwhK4O7r7y+McjRYyaTH6/vjwJOajE+XhlzM= -github.com/google/pprof v0.0.0-20221203041831-ce31453925ec/go.mod h1:dDKJzRmX4S37WGHujM7tX//fmj1uioxKzKxz3lo4HJo= +github.com/google/pprof v0.0.0-20230405160723-4a4c7d95572b h1:Qcx5LM0fSiks9uCyFZwDBUasd3lxd1RM0GYpL+Li5o4= +github.com/google/pprof v0.0.0-20230405160723-4a4c7d95572b/go.mod h1:79YE0hCXdHag9sBkw2o+N/YnZtTkXi0UT9Nnixa5eYk= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -243,8 +169,6 @@ github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= -github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= -github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gopherjs/gopherjs v0.0.0-20190430165422-3e4dfb77656c h1:7lF+Vz0LqiRidnzC1Oq86fpX1q/iEv2KJdrCtttYjT4= github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= @@ -259,23 +183,21 @@ github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= -github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/huin/goupnp v1.0.0/go.mod h1:n9v9KO1tAxYH82qOn+UTIFQDmx5n1Zxd/ClZDMX7Bnc= -github.com/huin/goupnp v1.0.3 h1:N8No57ls+MnjlB+JPiCVSOyy/ot7MJTqlo7rn+NYSqQ= -github.com/huin/goupnp v1.0.3/go.mod h1:ZxNlw5WqJj6wSsRK5+YfflQGXYfccj5VgQsMNixHM7Y= +github.com/huin/goupnp v1.1.0 h1:gEe0Dp/lZmPZiDFzJJaOfUpOvv2MKUkoBX8lDrn9vKU= +github.com/huin/goupnp v1.1.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= -github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/ipfs/boxo v0.8.0 h1:UdjAJmHzQHo/j3g3b1bAcAXCj/GM6iTwvSlBDvPBNBs= -github.com/ipfs/boxo v0.8.0/go.mod h1:RIsi4CnTyQ7AUsNn5gXljJYZlQrHBMnJp94p73liFiA= +github.com/ipfs/boxo v0.10.0 h1:tdDAxq8jrsbRkYoF+5Rcqyeb91hgWe2hp7iLu7ORZLY= +github.com/ipfs/boxo v0.10.0/go.mod h1:Fg+BnfxZ0RPzR0nOodzdIq3A7KgoWAOWsEIImrIQdBM= github.com/ipfs/go-cid v0.0.3/go.mod h1:GHWU/WuQdMPmIosc4Yn1bcCT7dSeX4lBafM7iqUPQvM= -github.com/ipfs/go-cid v0.4.0 h1:a4pdZq0sx6ZSxbCizebnKiMCx/xI/aBBFlB73IgH4rA= -github.com/ipfs/go-cid v0.4.0/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk= +github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s= +github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk= github.com/ipfs/go-datastore v0.1.0/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= github.com/ipfs/go-datastore v0.1.1/go.mod h1:w38XXW9kVFNp57Zj5knbKWM2T+KOZCGDRVNdgPHtbHw= github.com/ipfs/go-datastore v0.6.0 h1:JKyz+Gvz1QEZw0LsX1IBn+JFCJQH4SJVFtM4uWU0Myk= @@ -287,7 +209,6 @@ github.com/ipfs/go-ds-leveldb v0.1.0/go.mod h1:hqAW8y4bwX5LWcCtku2rFNX3vjDZCy5LZ github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-util v0.0.1/go.mod h1:spsl5z8KUnrve+73pOhSVZND1SIxPW5RyBCNzQxlJBc= github.com/ipfs/go-ipfs-util v0.0.2 h1:59Sswnk1MFaiq+VcaknX7aYEyGyGDAA73ilhEK2POp8= -github.com/ipfs/go-ipfs-util v0.0.2/go.mod h1:CbPtkWJzjLdEcezDns2XYaehFVNXG9zrdrtMecczcsQ= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-log v1.0.5 h1:2dOuUCB1Z7uoczMWgAyDck5JLb72zHzrMnGnCNNbvY8= github.com/ipfs/go-log v1.0.5/go.mod h1:j0b8ZoR+7+R99LD9jZ6+AJsrzkPbSXbZfGakb5JPtIo= @@ -308,20 +229,13 @@ github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZl github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= -github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= -github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d/go.mod h1:P2viExyCEfeWGU259JnaQ34Inuec4R38JCyBx2edgD0= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= @@ -329,18 +243,14 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= github.com/klauspost/compress v1.10.3/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/compress v1.15.12 h1:YClS/PImqYbn+UILDnqxQCZ3RehC9N318SU3kElDUEM= -github.com/klauspost/compress v1.15.12/go.mod h1:QPwzmACJjUTFsnSHH934V6woptycfrDDJnH7hvFVbGM= -github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= -github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= +github.com/klauspost/compress v1.16.4 h1:91KN02FnsOYhuunwU4ssRe8lc2JosWmizWa91B5v1PU= +github.com/klauspost/compress v1.16.4/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= +github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/koron/go-ssdp v0.0.0-20191105050749-2e1c40ed0b5d/go.mod h1:5Ky9EC2xfoUKUor0Hjgi2BJhCSXJfMOFlmyYrVKGQMk= -github.com/koron/go-ssdp v0.0.3 h1:JivLMY45N76b4p/vsWGOKewBQu6uf39y8l+AQ7sDKx8= -github.com/koron/go-ssdp v0.0.3/go.mod h1:b2MxI6yh02pKrsyNoQUsk4+YNikaGhe4894J+Q5lDvA= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/koron/go-ssdp v0.0.4 h1:1IDwrghSKYM7yLf7XCzbByg2sJ/JcNOZRXS2jczTwz0= +github.com/koron/go-ssdp v0.0.4/go.mod h1:oDXq+E5IL5q0U8uSBcoAXzTzInwy5lEgC91HoKtbmZk= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= @@ -360,20 +270,20 @@ github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZ github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-flow-metrics v0.1.0 h1:0iPhMI8PskQwzh57jB9WxIuIOQ0r+15PChFGkx3Q3WM= github.com/libp2p/go-flow-metrics v0.1.0/go.mod h1:4Xi8MX8wj5aWNDAZttg6UPmc0ZrnFNsMtpsYUClFtro= -github.com/libp2p/go-libp2p v0.26.3 h1:6g/psubqwdaBqNNoidbRKSTBEYgaOuKBhHl8Q5tO+PM= -github.com/libp2p/go-libp2p v0.26.3/go.mod h1:x75BN32YbwuY0Awm2Uix4d4KOz+/4piInkp4Wr3yOo8= -github.com/libp2p/go-libp2p-asn-util v0.2.0 h1:rg3+Os8jbnO5DxkC7K/Utdi+DkY3q/d1/1q+8WeNAsw= -github.com/libp2p/go-libp2p-asn-util v0.2.0/go.mod h1:WoaWxbHKBymSN41hWSq/lGKJEca7TNm58+gGJi2WsLI= +github.com/libp2p/go-libp2p v0.27.5 h1:KwA7pXKXpz8hG6Cr1fMA7UkgleogcwQj0sxl5qquWRg= +github.com/libp2p/go-libp2p v0.27.5/go.mod h1:oMfQGTb9CHnrOuSM6yMmyK2lXz3qIhnkn2+oK3B1Y2g= +github.com/libp2p/go-libp2p-asn-util v0.3.0 h1:gMDcMyYiZKkocGXDQ5nsUQyquC9+H+iLEQHwOCZ7s8s= +github.com/libp2p/go-libp2p-asn-util v0.3.0/go.mod h1:B1mcOrKUE35Xq/ASTmQ4tN3LNzVVaMNmq2NACuqyB9w= github.com/libp2p/go-libp2p-core v0.2.4/go.mod h1:STh4fdfa5vDYr0/SzYYeqnt+E6KfEV5VxfIrm0bcI0g= github.com/libp2p/go-libp2p-core v0.3.0/go.mod h1:ACp3DmS3/N64c2jDzcV429ukDpicbL6+TrrxANBjPGw= github.com/libp2p/go-libp2p-kbucket v0.3.1/go.mod h1:oyjT5O7tS9CQurok++ERgc46YLwEpuGoFq9ubvoUOio= -github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f h1:e1ibEklHRE4Oh5tpzzjRRCvYylG3f01ZbPrPulPFpPg= -github.com/libp2p/go-libp2p-kbucket v0.5.1-0.20230223111028-43db4412c39f/go.mod h1:efnPrfoP+WT/ONcC5eB0iADCDIJFXauXhylgJYO+VWw= +github.com/libp2p/go-libp2p-kbucket v0.6.1 h1:Y/NIvALuY5/fJlOpaJor9Azg4eor15JskGs9Lb2EhH0= +github.com/libp2p/go-libp2p-kbucket v0.6.1/go.mod h1:dvWO707Oq/vhMVuUhyfLkw0QsOrJFETepbNfpVHSELI= github.com/libp2p/go-libp2p-peerstore v0.1.4/go.mod h1:+4BDbDiiKf4PzpANZDAT+knVdLxvqh7hXOujessqdzs= github.com/libp2p/go-libp2p-record v0.2.0 h1:oiNUOCWno2BFuxt3my4i1frNrt7PerzB3queqa1NkQ0= github.com/libp2p/go-libp2p-record v0.2.0/go.mod h1:I+3zMkvvg5m2OcSdoL0KPljyJyvNDFGKX7QdlpYUcwk= -github.com/libp2p/go-libp2p-routing-helpers v0.4.0 h1:b7y4aixQ7AwbqYfcOQ6wTw8DQvuRZeTAA0Od3YYN5yc= -github.com/libp2p/go-libp2p-routing-helpers v0.4.0/go.mod h1:dYEAgkVhqho3/YKxfOEGdFMIcWfAFNlZX8iAIihYA2E= +github.com/libp2p/go-libp2p-routing-helpers v0.7.0 h1:sirOYVD0wGWjkDwHZvinunIpaqPLBXkcnXApVHwZFGA= +github.com/libp2p/go-libp2p-routing-helpers v0.7.0/go.mod h1:R289GUxUMzRXIbWGSuUUTPrlVJZ3Y/pPz495+qgXJX8= github.com/libp2p/go-libp2p-testing v0.12.0 h1:EPvBb4kKMWO29qP4mZGyhVzUyR25dvfUIK5WDu6iPUA= github.com/libp2p/go-libp2p-testing v0.12.0/go.mod h1:KcGDRXyN7sQCllucn1cOOS+Dmm7ujhfEyXQL5lvkcPg= github.com/libp2p/go-libp2p-xor v0.1.0 h1:hhQwT4uGrBcuAkUGXADuPltalOdpf9aag9kaYNT2tLA= @@ -405,16 +315,16 @@ github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNx github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= -github.com/mattn/go-isatty v0.0.17 h1:BTarxUcIeDqL27Mc+vyvdWYSL28zpIhv3RoTdsLMPng= -github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-isatty v0.0.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= +github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b/go.mod h1:01TrycV0kFyexm33Z7vhZRXopbI8J3TDReVlkTgMUxE= github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= -github.com/miekg/dns v1.1.50 h1:DQUfb9uc6smULcREF09Uc+/Gd46YWqJd5DbpPE9xkcA= -github.com/miekg/dns v1.1.50/go.mod h1:e3IlAVfNqAllflbibAZEWOXOQ+Ynzk/dDozDxY7XnME= +github.com/miekg/dns v1.1.53 h1:ZBkuHr5dxHtB1caEOlZTLPo7D3L3TWckgUUs/RHfDxw= +github.com/miekg/dns v1.1.53/go.mod h1:uInx36IzPl7FYnDcMeVWxj9byh7DutNykX4G9Sj60FY= github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c h1:bzE/A84HN25pxAuk9Eej1Kz9OUelF97nAc82bDquQI8= github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c/go.mod h1:0SQS9kMwD2VsyFEB++InYyBJroV/FRmBgcydeSUcJms= github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b h1:z78hV3sbSMAUoyUMM0I83AUIT6Hu17AWfgjzIbtrYFc= @@ -425,8 +335,8 @@ github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8Rv github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= github.com/minio/sha256-simd v0.1.1/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= -github.com/minio/sha256-simd v1.0.0 h1:v1ta+49hkWZyvaKwrQB8elexRqm6Y0aMLjCNsrYxo6g= -github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM= +github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= +github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -435,7 +345,6 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= -github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/mr-tron/base58 v1.1.0/go.mod h1:xcD2VGqlgYjBdcBLw+TuYLr8afG+Hj8g2eTVqeSzSU8= github.com/mr-tron/base58 v1.1.2/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= github.com/mr-tron/base58 v1.1.3/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= @@ -449,40 +358,38 @@ github.com/multiformats/go-base36 v0.2.0/go.mod h1:qvnKE++v+2MWCfePClUEjE78Z7P2a github.com/multiformats/go-multiaddr v0.1.0/go.mod h1:xKVEak1K9cS1VdmPZW3LSIb6lgmoS58qz/pzqmAxV44= github.com/multiformats/go-multiaddr v0.1.1/go.mod h1:aMKBKNEYmzmDmxfX88/vz+J5IU55txyt0p4aiWVohjo= github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y95ncPj0vFwVF6k6wJ4= -github.com/multiformats/go-multiaddr v0.8.0 h1:aqjksEcqK+iD/Foe1RRFsGZh8+XFiGo7FgUCZlpv3LU= -github.com/multiformats/go-multiaddr v0.8.0/go.mod h1:Fs50eBDWvZu+l3/9S6xAE7ZYj6yhxlvaVZjakWN7xRs= +github.com/multiformats/go-multiaddr v0.9.0 h1:3h4V1LHIk5w4hJHekMKWALPXErDfz/sggzwC/NcqbDQ= +github.com/multiformats/go-multiaddr v0.9.0/go.mod h1:mI67Lb1EeTOYb8GQfL/7wpIZwc46ElrvzhYnoJOmTT0= github.com/multiformats/go-multiaddr-dns v0.3.1 h1:QgQgR+LQVt3NPTjbrLLpsaT2ufAA2y0Mkk+QRVJbW3A= github.com/multiformats/go-multiaddr-dns v0.3.1/go.mod h1:G/245BRQ6FJGmryJCrOuTdB37AMA5AMOVuO6NY3JwTk= 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-multiaddr-net v0.1.1/go.mod h1:5JNbcfBOP4dnhoZOv10JJVkJO0pCCEf8mTnipAo2UZQ= github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= -github.com/multiformats/go-multibase v0.1.1 h1:3ASCDsuLX8+j4kx58qnJ4YFq/JWTJpCyDW27ztsVTOI= -github.com/multiformats/go-multibase v0.1.1/go.mod h1:ZEjHE+IsUrgp5mhlEAYjMtZwK1k4haNkcaPg9aoe1a8= -github.com/multiformats/go-multicodec v0.8.1 h1:ycepHwavHafh3grIbR1jIXnKCsFm0fqsfEOsJ8NtKE8= -github.com/multiformats/go-multicodec v0.8.1/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= +github.com/multiformats/go-multibase v0.2.0 h1:isdYCVLvksgWlMW9OZRYJEa9pZETFivncJHmHnnd87g= +github.com/multiformats/go-multibase v0.2.0/go.mod h1:bFBZX4lKCA/2lyOFSAoKH5SS6oPyjtnzK/XTFDPkNuk= +github.com/multiformats/go-multicodec v0.9.0 h1:pb/dlPnzee/Sxv/j4PmkDRxCOi3hXTz3IbPKOXWJkmg= +github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.10/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.13/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= -github.com/multiformats/go-multihash v0.2.1 h1:aem8ZT0VA2nCHHk7bPJ1BjUbHNciqZC/d16Vve9l108= -github.com/multiformats/go-multihash v0.2.1/go.mod h1:WxoMcYG85AZVQUyRyo9s4wULvW5qrI9vb2Lt6evduFc= +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.4.1 h1:rFy0Iiyn3YT0asivDUIR05leAdwZq3de4741sbiSdfo= github.com/multiformats/go-multistream v0.4.1/go.mod h1:Mz5eykRVAjJWckE2U78c6xqdtyNUEhKSM0Lwar2p77Q= github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.5/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= -github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo/v2 v2.5.1 h1:auzK7OI497k6x4OvWq+TKAcpcSAlod0doAH72oIN0Jw= -github.com/onsi/ginkgo/v2 v2.5.1/go.mod h1:63DOGlLAH8+REH8jUGdL3YpCpu7JODesutUjdENfUAc= +github.com/onsi/ginkgo/v2 v2.9.2 h1:BA2GMJOtfGAfagzYtrAlufIP0lq6QERkFmHLMLPwFSU= +github.com/onsi/ginkgo/v2 v2.9.2/go.mod h1:WHcJJG2dIlcCqVfBAwUCrJxSPFb6v4azBwgxeMeDuts= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= -github.com/onsi/gomega v1.24.0 h1:+0glovB9Jd6z3VR+ScSwQqXVTIfJcGA9UBM8yzQxhqg= +github.com/onsi/gomega v1.27.4 h1:Z2AnStgsdSayCMDiCU42qIz+HLqEPcgiOCXjAU/w+8E= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -492,7 +399,6 @@ github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTm github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 h1:onHthvaw9LFnH4t2DcNVpwGmV9E1BkGknEliJkfwQj0= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58/go.mod h1:DXv8WO4yhMYhSNPKjeNKa5WY9YCIEBRbNzFFPJbWO6Y= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -501,40 +407,24 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/polydawn/refmt v0.89.0 h1:ADJTApkvkeBZsN0tBTx8QjpD9JkmxbKp0cxfr9qszm4= github.com/polydawn/refmt v0.89.0/go.mod h1:/zvteZs/GwLtCgZ4BL6CBsk9IKIlexP43ObX9AxTqTw= github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= -github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= -github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= github.com/prometheus/client_golang v1.14.0 h1:nJdhIvne2eSX/XRAFV9PcvFFRbrjbcTUj0VP62TMhnw= github.com/prometheus/client_golang v1.14.0/go.mod h1:8vpkKitgIVNcqrRBWh1C4TIUQgYNtG/XQE4E/Zae36Y= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.3.0 h1:UBgGFHqYdG/TPFD1B1ogZywDqEkwp3fBMvqdiQ7Xew4= github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= -github.com/prometheus/common v0.37.0 h1:ccBbHCgIiT9uSoFY0vX8H3zsNR5eLt17/RQLUvn8pXE= -github.com/prometheus/common v0.37.0/go.mod h1:phzohg0JFMnBEFGxTDbfu3QyL5GI8gTQJFhYO5B3mfA= +github.com/prometheus/common v0.42.0 h1:EKsfXEYo4JpWMHH5cg+KOUWeuJSov1Id8zGR8eeI1YM= +github.com/prometheus/common v0.42.0/go.mod h1:xBwqVerjNdUDjgODMpudtOMwlOwf2SaTr1yjz4b7Zbc= github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.8.0 h1:ODq8ZFEaYeCaZOJlZZdJA2AbQR98dSHSM1KW/You5mo= -github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4= +github.com/prometheus/procfs v0.9.0 h1:wzCHvIvM5SxWqYvwgVL7yJY8Lz3PKn49KQtpgMYJfhI= +github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY= github.com/quic-go/qpack v0.4.0 h1:Cr9BXA1sQS2SmDUWjSofMPNKmvF6IiIfDRmgU0w1ZCo= github.com/quic-go/qpack v0.4.0/go.mod h1:UZVnYIfi5GRk+zI9UMaCPsmZ2xKJP7XBUvVyT1Knj9A= -github.com/quic-go/qtls-go1-19 v0.2.1 h1:aJcKNMkH5ASEJB9FXNeZCyTEIHU1J7MmHyz1Q1TSG1A= -github.com/quic-go/qtls-go1-19 v0.2.1/go.mod h1:ySOI96ew8lnoKPtSqx2BlI5wCpUVPT05RMAlajtnyOI= -github.com/quic-go/qtls-go1-20 v0.1.1 h1:KbChDlg82d3IHqaj2bn6GfKRj84Per2VGf5XV3wSwQk= -github.com/quic-go/qtls-go1-20 v0.1.1/go.mod h1:JKtK6mjbAVcUTN/9jZpvLbGxvdWIKS8uT7EiStoU1SM= +github.com/quic-go/qtls-go1-19 v0.3.2 h1:tFxjCFcTQzK+oMxG6Zcvp4Dq8dx4yD3dDiIiyc86Z5U= +github.com/quic-go/qtls-go1-19 v0.3.2/go.mod h1:ySOI96ew8lnoKPtSqx2BlI5wCpUVPT05RMAlajtnyOI= +github.com/quic-go/qtls-go1-20 v0.2.2 h1:WLOPx6OY/hxtTxKV1Zrq20FtXtDEkeY00CGQm8GEa3E= +github.com/quic-go/qtls-go1-20 v0.2.2/go.mod h1:JKtK6mjbAVcUTN/9jZpvLbGxvdWIKS8uT7EiStoU1SM= github.com/quic-go/quic-go v0.33.0 h1:ItNoTDN/Fm/zBlq769lLJc8ECe9gYaW40veHCCco7y0= github.com/quic-go/quic-go v0.33.0/go.mod h1:YMuhaAV9/jIu0XclDXwZPAsP/2Kgr5yMYhe9oxhhOFA= github.com/quic-go/webtransport-go v0.5.2 h1:GA6Bl6oZY+g/flt00Pnu0XtivSD8vukOu3lYhJjnGEk= @@ -570,8 +460,6 @@ 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.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v1.2.0 h1:42S6lae5dvLc7BrLu/0ugRtcFVjoJNMC/N3yZFZkDFs= github.com/smartystreets/assertions v1.2.0/go.mod h1:tcbTF8ujkAEcZ8TElKY+i30BzYlVhC/LOxJk7iOWnoo= @@ -597,13 +485,13 @@ github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpE github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= -github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/syndtr/goleveldb v1.0.0/go.mod h1:ZVVdQEZoIme9iO1Ch2Jdy24qqXrMMOU6lpPAyBWyWuQ= github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA= github.com/ugorji/go v1.1.7 h1:/68gy2h+1mWMrwZFeD1kQialdSzAb432dtpeJ42ovdo= @@ -623,38 +511,34 @@ github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc/go.mod h1:bopw91TMyo8J3tvftk8xmU2kPmlrt4nScJQZU2hE5EM= github.com/x-cray/logrus-prefixed-formatter v0.5.2/go.mod h1:2duySbKsL6M18s5GU7VPsoEPHyzalCE06qoARUCeBBE= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= -github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.opencensus.io v0.18.0/go.mod h1:vKdFvxhtzZ9onBp9VKHK8z/sRpBMnKAsufL7wlDrCOA= -go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= -go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.1/go.mod h1:Ap50jQcDJrx6rB6VgeeFPtuPIf3wMRvRfrfYDO6+BmA= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/otel v1.14.0 h1:/79Huy8wbf5DnIPhemGB+zEPVwnN6fuQybr/SRXa6hM= -go.opentelemetry.io/otel v1.14.0/go.mod h1:o4buv+dJzx8rohcUeRmWUZhqupFvzWis188WlggnNeU= -go.opentelemetry.io/otel/trace v1.14.0 h1:wp2Mmvj41tDsyAJXiWDWpfNsOiIyd38fy85pyKcFq/M= -go.opentelemetry.io/otel/trace v1.14.0/go.mod h1:8avnQLK+CG77yNLUae4ea2JDQ6iT+gozhnZjy/rw9G8= +go.opentelemetry.io/otel v1.16.0 h1:Z7GVAX/UkAXPKsy94IU+i6thsQS4nb7LviLpnaNeW8s= +go.opentelemetry.io/otel v1.16.0/go.mod h1:vl0h9NUa1D5s1nv3A5vZOYWn8av4K8Ml6JDeHrT/bx4= +go.opentelemetry.io/otel/metric v1.16.0 h1:RbrpwVG1Hfv85LgnZ7+txXioPDoh6EdbZHo26Q3hqOo= +go.opentelemetry.io/otel/metric v1.16.0/go.mod h1:QE47cpOmkwipPiefDwo2wDzwJrlfxxNYodqc4xnGCo4= +go.opentelemetry.io/otel/trace v1.16.0 h1:8JRpaObFoW0pxuVPapkgH8UhHQj+bJW8jJsCZEu5MQs= +go.opentelemetry.io/otel/trace v1.16.0/go.mod h1:Yt9vYq1SdNz3xdjZZK7wcXv1qv2pwLkqr2QVwea0ef0= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= -go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= -go.uber.org/dig v1.15.0 h1:vq3YWr8zRj1eFGC7Gvf907hE0eRjPTZ1d3xHadD6liE= -go.uber.org/dig v1.15.0/go.mod h1:pKHs0wMynzL6brANhB2hLMro+zalv1osARTviTcqHLM= -go.uber.org/fx v1.18.2 h1:bUNI6oShr+OVFQeU8cDNbnN7VFsu+SsjHzUF51V/GAU= -go.uber.org/fx v1.18.2/go.mod h1:g0V1KMQ66zIRk8bLu3Ea5Jt2w/cHlOIp4wdRsgh0JaY= +go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= +go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= +go.uber.org/dig v1.16.1 h1:+alNIBsl0qfY0j6epRubp/9obgtrObRAc5aD+6jbWY8= +go.uber.org/dig v1.16.1/go.mod h1:557JTAUZT5bUK0SvCwikmLPPtdQhfvLYtO5tJgQSbnk= +go.uber.org/fx v1.19.2 h1:SyFgYQFr1Wl0AYstE8vyYIzP4bFz2URrScjwC4cwUvY= +go.uber.org/fx v1.19.2/go.mod h1:43G1VcqSzbIv77y00p1DRAsyZS8WdzuYdhZXmEUkMyQ= go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.9.0 h1:7fIwc/ZtS0q++VgcfqFDxSBZVv/Xo49/SYnDFupUwlI= -go.uber.org/multierr v1.9.0/go.mod h1:X2jQV1h+kxSjClGpnseKVIxpmcjrj7MNnI0bnlfKTVQ= +go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= +go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= @@ -670,104 +554,54 @@ golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnf golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200602180216-279210d13fed/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= -golang.org/x/crypto v0.6.0 h1:qfktjS5LUO+fFKeJXZ+ikTRijMmljikvG68fpMMruSc= -golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= +golang.org/x/crypto v0.9.0 h1:LF6fAI+IutBocDJ2OT0Q1g8plpYljMZ4+lty+dsqw3g= +golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= -golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= -golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= -golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= -golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb h1:PaBZQdo+iSDyHT053FjUCgZQ/9uqVwPOcl7KSWhKn6w= -golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= -golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= -golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/exp v0.0.0-20230321023759-10a507213a29 h1:ooxPy7fPvB4kwsA2h+iBNHkAbp/4JxTSwCmvdjEYmug= +golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= -golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= -golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.7.0 h1:LapD9S96VoQRhi/GrNTqeBJFrUjs5UHCAtTlgwA5oZA= -golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.10.0 h1:lFO9qtOdlre5W1jxS3r/4szv2/6iXxScdzjoBMXNhYk= +golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181011144130-49bb7cea24b1/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181029044818-c44066c5c816/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190227160552-c95aed5357e7/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190313220215-9f648a60d977/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= -golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= -golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.10.0 h1:X2//UzNDwYmtCLn7To6G58Wr6f5ahEAQgKNzv9Y951M= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852/go.mod h1:JLpeXjPJfIyPr5TlbXLkXWLhP8nz10XfvxElABhCtcw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -775,10 +609,7 @@ golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -787,76 +618,39 @@ golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181029174526-d69651ed3497/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190219092855-153ac476189d/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190228124157-a34e9553db1e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190316082340-a2f829d7f35f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190405154228-4b34438f7a67/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0 h1:EBmGv8NaZBZTWvrbjNoL6HVt+IVy3QDQpJs7VRIw3tU= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= -golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.9.0 h1:2sjJmO8cDvYveuX97RDLsxlyUxLl+GHoLxBiRdHllBE= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -867,133 +661,48 @@ golang.org/x/tools v0.0.0-20181130052023-1c3d964395ce/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= -golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.6-0.20210726203631-07bc1bf47fb2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.3.0 h1:SrNbZl6ECOS1qFzgTdQfWXZM9XBkiA6tkFrH9YSTPHM= -golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= +golang.org/x/tools v0.7.0 h1:W4OVu8VVOaIO0yzWMNdepAulS7YfoS3Zabrm8DOXXU4= +golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -gonum.org/v1/gonum v0.11.0 h1:f1IJhK4Km5tBJmaiJXtk/PkL4cdVX6J+tGiM187uT5E= -gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= +gonum.org/v1/gonum v0.13.0 h1:a0T3bh+7fhRyqeNbiC3qVHYmkiQgit3wnNan/2c0HMM= +gonum.org/v1/gonum v0.13.0/go.mod h1:/WPYRckkfWrhWefxyYTfrTtQR0KH4iyHNuzxqXAKyAU= google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.1.0/go.mod h1:UGEZY7KEX120AnNLIHFMKIo4obdJhkp2tPbaPlQx13Y= -google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= -google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= -google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= -google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= -google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181029155118-b69ba1387ce2/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181202183823-bd91e49a0898/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg= google.golang.org/genproto v0.0.0-20190306203927-b5d61aea6440/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= -google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= -google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= -google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= -google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1003,16 +712,13 @@ google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzi google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= -google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= +google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= +google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= @@ -1023,12 +729,8 @@ gopkg.in/src-d/go-log.v1 v1.0.1/go.mod h1:GN34hKP0g305ysm2/hctJ0Y8nWP3zxXXJ8GFab gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= @@ -1037,17 +739,11 @@ grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJd honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -lukechampine.com/blake3 v1.1.7 h1:GgRMhmdsuK8+ii6UZFDL8Nb+VyMwadAgcJyfYHxG6n0= -lukechampine.com/blake3 v1.1.7/go.mod h1:tkKEOtDkNtklkXtLNEOGNq5tcV90tJiA1vAA12R78LA= +lukechampine.com/blake3 v1.2.1 h1:YuqqRuaqsGV71BV/nm9xlI0MKUv4QC54jQnBChWbGnI= +lukechampine.com/blake3 v1.2.1/go.mod h1:0OFRp7fBtAylGVCO40o87sbupkyIGgbpv1+M1k1LM6k= nhooyr.io/websocket v1.8.7 h1:usjR2uOr/zjjkVMy0lW+PPohFok7PCow5sDjLgX4P4g= nhooyr.io/websocket v1.8.7/go.mod h1:B70DZP8IakI65RVQ51MsWP/8jndNma26DVA/nFSCgW0= -rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= -rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= -rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= sourcegraph.com/sourcegraph/go-diff v0.5.0/go.mod h1:kuch7UrkMzY0X+p9CRK03kfuPQ2zzQcaEFbx8wA8rck= sourcegraph.com/sqs/pbtypes v0.0.0-20180604144634-d3ebe8f20ae4/go.mod h1:ketZ/q3QxT9HOBeFhu6RdvsftgpsbFHBF5Cas6cDKZ0= From 810d7dd723d4e0f32bce787507a6298278ff6c08 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Mon, 12 Jun 2023 12:37:45 +0200 Subject: [PATCH 15/17] added optimizations documentation --- optimizations.md | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 optimizations.md diff --git a/optimizations.md b/optimizations.md new file mode 100644 index 000000000..82e8953cf --- /dev/null +++ b/optimizations.md @@ -0,0 +1,9 @@ +# Client-side optimizations + +This document reflects client-side optimizations that are implemented in this repository. Client-side optimizations are not part of the [Kademlia spec](https://github.com/libp2p/specs/tree/master/kad-dht), and are not required to be implemented on all clients. + +## Checking before Adding + +Owner: [Guillaume Michel](https://github.com/guillaumemichel) + +A Kademlia server should try to add remote peers querying it to its routing table. However, the Kademlia server has no guarantee that remote peers issuing requests are able to answer Kademlia requests correctly, even though they advertise speaking the Kademlia server protocol. It is important that only server nodes able to answer Kademlia requests end up in other peers' routing tables. Hence, before adding a remote peer to the Kademlia server's routing table, the Kademlia server will send a trivial `FIND_NODE` request to the remote peer, and add it to its routing table only if it is able to provide a valid response. \ No newline at end of file From 2450e4599b710d2231a991d670702a1b1cb4ca70 Mon Sep 17 00:00:00 2001 From: Jorropo Date: Mon, 12 Jun 2023 14:01:28 +0200 Subject: [PATCH 16/17] Update dht.go --- dht.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/dht.go b/dht.go index 0d4b0a8b4..7127a4018 100644 --- a/dht.go +++ b/dht.go @@ -673,12 +673,6 @@ func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID) { if err != nil { logger.Errorw("failed to validate if peer is a DHT peer", "peer", p, "error", err) } else if b { - /* - if dht.verifyAndLogPeerQuery(p) { - // peer was already queried recently, don't query it again - return - } - */ livelinessCtx, cancel := context.WithTimeout(ctx, dht.lookupCheckTimeout) defer cancel() From 38793b91668fd1633ada80d1f2e6e52fc81fe517 Mon Sep 17 00:00:00 2001 From: guillaumemichel Date: Mon, 12 Jun 2023 14:07:42 +0200 Subject: [PATCH 17/17] updated md files --- README.md | 5 +++++ optimizations.md | 2 -- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 517bc9f3a..b9b0a69b6 100644 --- a/README.md +++ b/README.md @@ -11,6 +11,7 @@ - [Install](#install) - [Usage](#usage) +- [Optimizations](#optimizations) - [Contribute](#contribute) - [Maintainers](#maintainers) - [License](#license) @@ -21,6 +22,10 @@ go get github.com/libp2p/go-libp2p-kad-dht ``` +## Optimizations + +Client-side optimizations are described in [optimizations.md](./optimizations.md) + ## Usage Go to https://godoc.org/github.com/libp2p/go-libp2p-kad-dht. diff --git a/optimizations.md b/optimizations.md index 82e8953cf..214c1796f 100644 --- a/optimizations.md +++ b/optimizations.md @@ -4,6 +4,4 @@ This document reflects client-side optimizations that are implemented in this re ## Checking before Adding -Owner: [Guillaume Michel](https://github.com/guillaumemichel) - A Kademlia server should try to add remote peers querying it to its routing table. However, the Kademlia server has no guarantee that remote peers issuing requests are able to answer Kademlia requests correctly, even though they advertise speaking the Kademlia server protocol. It is important that only server nodes able to answer Kademlia requests end up in other peers' routing tables. Hence, before adding a remote peer to the Kademlia server's routing table, the Kademlia server will send a trivial `FIND_NODE` request to the remote peer, and add it to its routing table only if it is able to provide a valid response. \ No newline at end of file