feat:add contexts to all peerstore methods (#2312)

This commit is contained in:
Prem Chaitanya Prathi
2023-06-03 14:54:50 +05:30
committed by GitHub
parent e89814c520
commit 8864d1c33e
61 changed files with 757 additions and 740 deletions

View File

@@ -1,6 +1,7 @@
package config
import (
"context"
"crypto/rand"
"errors"
"fmt"
@@ -150,10 +151,10 @@ func (cfg *Config) makeSwarm(eventBus event.Bus, enableMetrics bool) (*swarm.Swa
return nil, err
}
if err := cfg.Peerstore.AddPrivKey(pid, cfg.PeerKey); err != nil {
if err := cfg.Peerstore.AddPrivKey(context.Background(), pid, cfg.PeerKey); err != nil {
return nil, err
}
if err := cfg.Peerstore.AddPubKey(pid, cfg.PeerKey.GetPublic()); err != nil {
if err := cfg.Peerstore.AddPubKey(context.Background(), pid, cfg.PeerKey.GetPublic()); err != nil {
return nil, err
}
@@ -194,7 +195,7 @@ func (cfg *Config) addTransports(h host.Host) error {
fx.Supply(cfg.Muxers),
fx.Supply(h.ID()),
fx.Provide(func() host.Host { return h }),
fx.Provide(func() crypto.PrivKey { return h.Peerstore().PrivKey(h.ID()) }),
fx.Provide(func() crypto.PrivKey { return h.Peerstore().PrivKey(context.Background(), h.ID()) }),
fx.Provide(func() connmgr.ConnectionGater { return cfg.ConnectionGater }),
fx.Provide(func() pnet.PSK { return cfg.PSK }),
fx.Provide(func() network.ResourceManager { return cfg.ResourceManager }),

View File

@@ -1,14 +1,16 @@
package peerstore
import (
"context"
"github.com/libp2p/go-libp2p/core/peer"
)
// AddrInfos returns an AddrInfo for each specified peer ID, in-order.
func AddrInfos(ps Peerstore, peers []peer.ID) []peer.AddrInfo {
func AddrInfos(ctx context.Context, ps Peerstore, peers []peer.ID) []peer.AddrInfo {
pi := make([]peer.AddrInfo, len(peers))
for i, p := range peers {
pi[i] = ps.PeerInfo(p)
pi[i] = ps.PeerInfo(ctx, p)
}
return pi
}

View File

@@ -60,10 +60,10 @@ type Peerstore interface {
// PeerInfo returns a peer.PeerInfo struct for given peer.ID.
// This is a small slice of the information Peerstore has on
// that peer, useful to other services.
PeerInfo(peer.ID) peer.AddrInfo
PeerInfo(context.Context, peer.ID) peer.AddrInfo
// Peers returns all of the peer IDs stored across all inner stores.
Peers() peer.IDSlice
Peers(context.Context) peer.IDSlice
}
// PeerMetadata can handle values of any type. Serializing values is
@@ -77,36 +77,36 @@ type PeerMetadata interface {
// Get / Put is a simple registry for other peer-related key/value pairs.
// If we find something we use often, it should become its own set of
// methods. This is a last resort.
Get(p peer.ID, key string) (interface{}, error)
Put(p peer.ID, key string, val interface{}) error
Get(ctx context.Context, p peer.ID, key string) (interface{}, error)
Put(ctx context.Context, p peer.ID, key string, val interface{}) error
// RemovePeer removes all values stored for a peer.
RemovePeer(peer.ID)
RemovePeer(context.Context, peer.ID)
}
// AddrBook holds the multiaddrs of peers.
type AddrBook interface {
// AddAddr calls AddAddrs(p, []ma.Multiaddr{addr}, ttl)
AddAddr(p peer.ID, addr ma.Multiaddr, ttl time.Duration)
AddAddr(context.Context, peer.ID, ma.Multiaddr, time.Duration)
// AddAddrs gives this AddrBook addresses to use, with a given ttl
// (time-to-live), after which the address is no longer valid.
// If the manager has a longer TTL, the operation is a no-op for that address
AddAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duration)
AddAddrs(context.Context, peer.ID, []ma.Multiaddr, time.Duration)
// SetAddr calls mgr.SetAddrs(p, addr, ttl)
SetAddr(p peer.ID, addr ma.Multiaddr, ttl time.Duration)
SetAddr(context.Context, peer.ID, ma.Multiaddr, time.Duration)
// SetAddrs sets the ttl on addresses. This clears any TTL there previously.
// This is used when we receive the best estimate of the validity of an address.
SetAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duration)
SetAddrs(context.Context, peer.ID, []ma.Multiaddr, time.Duration)
// UpdateAddrs updates the addresses associated with the given peer that have
// the given oldTTL to have the given newTTL.
UpdateAddrs(p peer.ID, oldTTL time.Duration, newTTL time.Duration)
UpdateAddrs(context.Context, peer.ID, time.Duration, time.Duration)
// Addrs returns all known (and valid) addresses for a given peer.
Addrs(p peer.ID) []ma.Multiaddr
Addrs(context.Context, peer.ID) []ma.Multiaddr
// AddrStream returns a channel that gets all addresses for a given
// peer sent on it. If new addresses are added after the call is made
@@ -114,10 +114,10 @@ type AddrBook interface {
AddrStream(context.Context, peer.ID) <-chan ma.Multiaddr
// ClearAddresses removes all previously stored addresses.
ClearAddrs(p peer.ID)
ClearAddrs(context.Context, peer.ID)
// PeersWithAddrs returns all of the peer IDs stored in the AddrBook.
PeersWithAddrs() peer.IDSlice
PeersWithAddrs(context.Context) peer.IDSlice
}
// CertifiedAddrBook manages "self-certified" addresses for remote peers.
@@ -172,12 +172,12 @@ type CertifiedAddrBook interface {
// AddrBook.SetAddrs will be ignored. AddrBook.SetAddrs may still be used
// to update the TTL of certified addresses that have previously been
// added via ConsumePeerRecord.
ConsumePeerRecord(s *record.Envelope, ttl time.Duration) (accepted bool, err error)
ConsumePeerRecord(context.Context, *record.Envelope, time.Duration) (accepted bool, err error)
// GetPeerRecord returns a Envelope containing a PeerRecord for the
// given peer id, if one exists.
// Returns nil if no signed PeerRecord exists for the peer.
GetPeerRecord(p peer.ID) *record.Envelope
GetPeerRecord(context.Context, peer.ID) *record.Envelope
}
// GetCertifiedAddrBook is a helper to "upcast" an AddrBook to a
@@ -196,24 +196,24 @@ func GetCertifiedAddrBook(ab AddrBook) (cab CertifiedAddrBook, ok bool) {
// KeyBook tracks the keys of Peers.
type KeyBook interface {
// PubKey stores the public key of a peer.
PubKey(peer.ID) ic.PubKey
PubKey(context.Context, peer.ID) ic.PubKey
// AddPubKey stores the public key of a peer.
AddPubKey(peer.ID, ic.PubKey) error
AddPubKey(context.Context, peer.ID, ic.PubKey) error
// PrivKey returns the private key of a peer, if known. Generally this might only be our own
// private key, see
// https://discuss.libp2p.io/t/what-is-the-purpose-of-having-map-peer-id-privatekey-in-peerstore/74.
PrivKey(peer.ID) ic.PrivKey
PrivKey(context.Context, peer.ID) ic.PrivKey
// AddPrivKey stores the private key of a peer.
AddPrivKey(peer.ID, ic.PrivKey) error
AddPrivKey(context.Context, peer.ID, ic.PrivKey) error
// PeersWithKeys returns all the peer IDs stored in the KeyBook.
PeersWithKeys() peer.IDSlice
PeersWithKeys(context.Context) peer.IDSlice
// RemovePeer removes all keys associated with a peer.
RemovePeer(peer.ID)
RemovePeer(context.Context, peer.ID)
}
// Metrics tracks metrics across a set of peers.
@@ -226,25 +226,25 @@ type Metrics interface {
LatencyEWMA(peer.ID) time.Duration
// RemovePeer removes all metrics stored for a peer.
RemovePeer(peer.ID)
RemovePeer(context.Context, peer.ID)
}
// ProtoBook tracks the protocols supported by peers.
type ProtoBook interface {
GetProtocols(peer.ID) ([]protocol.ID, error)
AddProtocols(peer.ID, ...protocol.ID) error
SetProtocols(peer.ID, ...protocol.ID) error
RemoveProtocols(peer.ID, ...protocol.ID) error
GetProtocols(context.Context, peer.ID) ([]protocol.ID, error)
AddProtocols(context.Context, peer.ID, ...protocol.ID) error
SetProtocols(context.Context, peer.ID, ...protocol.ID) error
RemoveProtocols(context.Context, peer.ID, ...protocol.ID) error
// SupportsProtocols returns the set of protocols the peer supports from among the given protocols.
// If the returned error is not nil, the result is indeterminate.
SupportsProtocols(peer.ID, ...protocol.ID) ([]protocol.ID, error)
SupportsProtocols(context.Context, peer.ID, ...protocol.ID) ([]protocol.ID, error)
// FirstSupportedProtocol returns the first protocol that the peer supports among the given protocols.
// If the peer does not support any of the given protocols, this function will return an empty protocol.ID and a nil error.
// If the returned error is not nil, the result is indeterminate.
FirstSupportedProtocol(peer.ID, ...protocol.ID) (protocol.ID, error)
FirstSupportedProtocol(context.Context, peer.ID, ...protocol.ID) (protocol.ID, error)
// RemovePeer removes all protocols associated with a peer.
RemovePeer(peer.ID)
RemovePeer(context.Context, peer.ID)
}

View File

@@ -55,7 +55,7 @@ func getNetHosts(t *testing.T, n int) []host.Host {
func loadCh(peers []host.Host) <-chan peer.AddrInfo {
ch := make(chan peer.AddrInfo, len(peers))
for _, p := range peers {
ch <- p.Peerstore().PeerInfo(p.ID())
ch <- p.Peerstore().PeerInfo(context.Background(), p.ID())
}
close(ch)
return ch

View File

@@ -188,7 +188,7 @@ func (as *AmbientAutoNAT) background() {
as.confidence--
}
case event.EvtPeerIdentificationCompleted:
if s, err := as.host.Peerstore().SupportsProtocols(e.Peer, AutoNATProto); err == nil && len(s) > 0 {
if s, err := as.host.Peerstore().SupportsProtocols(context.Background(), e.Peer, AutoNATProto); err == nil && len(s) > 0 {
currentStatus := *as.status.Load()
if currentStatus == network.ReachabilityUnknown {
as.tryProbe(e.Peer)
@@ -367,7 +367,7 @@ func (as *AmbientAutoNAT) tryProbe(p peer.ID) bool {
}
as.cleanupRecentProbes()
info := as.host.Peerstore().PeerInfo(p)
info := as.host.Peerstore().PeerInfo(context.Background(), p)
if !as.config.dialPolicy.skipPeer(info.Addrs) {
as.recentProbes[p] = time.Now()
@@ -402,9 +402,9 @@ func (as *AmbientAutoNAT) getPeerToProbe() peer.ID {
candidates := make([]peer.ID, 0, len(peers))
for _, p := range peers {
info := as.host.Peerstore().PeerInfo(p)
info := as.host.Peerstore().PeerInfo(context.Background(), p)
// Exclude peers which don't support the autonat protocol.
if proto, err := as.host.Peerstore().SupportsProtocols(p, AutoNATProto); len(proto) == 0 || err != nil {
if proto, err := as.host.Peerstore().SupportsProtocols(context.Background(), p, AutoNATProto); len(proto) == 0 || err != nil {
continue
}

View File

@@ -94,8 +94,8 @@ func makeAutoNATServicePublic(t *testing.T) host.Host {
func makeAutoNAT(t *testing.T, ash host.Host) (host.Host, AutoNAT) {
h := bhost.NewBlankHost(swarmt.GenSwarm(t))
h.Peerstore().AddAddrs(ash.ID(), ash.Addrs(), time.Minute)
h.Peerstore().AddProtocols(ash.ID(), AutoNATProto)
h.Peerstore().AddAddrs(context.Background(), ash.ID(), ash.Addrs(), time.Minute)
h.Peerstore().AddProtocols(context.Background(), ash.ID(), AutoNATProto)
a, _ := New(h, WithSchedule(100*time.Millisecond, time.Second), WithoutStartupDelay())
a.(*AmbientAutoNAT).config.dialPolicy.allowSelfDials = true
a.(*AmbientAutoNAT).config.throttlePeerPeriod = 100 * time.Millisecond
@@ -103,8 +103,8 @@ func makeAutoNAT(t *testing.T, ash host.Host) (host.Host, AutoNAT) {
}
func identifyAsServer(server, recip host.Host) {
recip.Peerstore().AddAddrs(server.ID(), server.Addrs(), time.Minute)
recip.Peerstore().AddProtocols(server.ID(), AutoNATProto)
recip.Peerstore().AddAddrs(context.Background(), server.ID(), server.Addrs(), time.Minute)
recip.Peerstore().AddProtocols(context.Background(), server.ID(), AutoNATProto)
}

View File

@@ -223,13 +223,13 @@ func (as *autoNATService) doDial(pi peer.AddrInfo) *pb.Message_DialResponse {
ctx, cancel := context.WithTimeout(context.Background(), as.config.dialTimeout)
defer cancel()
as.config.dialer.Peerstore().ClearAddrs(pi.ID)
as.config.dialer.Peerstore().ClearAddrs(ctx, pi.ID)
as.config.dialer.Peerstore().AddAddrs(pi.ID, pi.Addrs, peerstore.TempAddrTTL)
as.config.dialer.Peerstore().AddAddrs(ctx, pi.ID, pi.Addrs, peerstore.TempAddrTTL)
defer func() {
as.config.dialer.Peerstore().ClearAddrs(pi.ID)
as.config.dialer.Peerstore().RemovePeer(pi.ID)
as.config.dialer.Peerstore().ClearAddrs(ctx, pi.ID)
as.config.dialer.Peerstore().RemovePeer(ctx, pi.ID)
}()
conn, err := as.config.dialer.DialPeer(ctx, pi.ID)

View File

@@ -29,8 +29,8 @@ func TestAutonatRoundtrip(t *testing.T) {
t.Fatal(err)
}
client.Peerstore().AddAddrs(service.ID(), service.Addrs(), time.Hour)
require.NoError(t, client.Connect(context.Background(), service.Peerstore().PeerInfo(service.ID())))
client.Peerstore().AddAddrs(context.Background(), service.ID(), service.Addrs(), time.Hour)
require.NoError(t, client.Connect(context.Background(), service.Peerstore().PeerInfo(context.Background(), service.ID())))
cSub, err := client.EventBus().Subscribe(new(event.EvtLocalReachabilityChanged))
require.NoError(t, err)

View File

@@ -493,7 +493,7 @@ func (rf *relayFinder) tryNode(ctx context.Context, pi peer.AddrInfo) (supportsR
return false, ctx.Err()
}
protos, err := rf.host.Peerstore().SupportsProtocols(pi.ID, protoIDv2)
protos, err := rf.host.Peerstore().SupportsProtocols(ctx, pi.ID, protoIDv2)
if err != nil {
return false, fmt.Errorf("error checking relay protocol support for peer %s: %w", pi.ID, err)
}
@@ -734,7 +734,7 @@ func (rf *relayFinder) relayAddrs(addrs []ma.Multiaddr) []ma.Multiaddr {
// add relay specific addrs to the list
relayAddrCnt := 0
for p := range rf.relays {
addrs := cleanupAddressSet(rf.host.Peerstore().Addrs(p))
addrs := cleanupAddressSet(rf.host.Peerstore().Addrs(context.Background(), p))
relayAddrCnt += len(addrs)
circuit := ma.StringCast(fmt.Sprintf("/p2p/%s/p2p-circuit", p.Pretty()))
for _, addr := range addrs {

View File

@@ -207,7 +207,7 @@ func NewHost(n network.Network, opts *HostOpts) (*BasicHost, error) {
}
h.caBook = cab
h.signKey = h.Peerstore().PrivKey(h.ID())
h.signKey = h.Peerstore().PrivKey(context.Background(), h.ID())
if h.signKey == nil {
return nil, errors.New("unable to access host key")
}
@@ -221,7 +221,7 @@ func NewHost(n network.Network, opts *HostOpts) (*BasicHost, error) {
if err != nil {
return nil, fmt.Errorf("failed to create signed record for self: %w", err)
}
if _, err := cab.ConsumePeerRecord(ev, peerstore.PermanentAddrTTL); err != nil {
if _, err := cab.ConsumePeerRecord(context.Background(), ev, peerstore.PermanentAddrTTL); err != nil {
return nil, fmt.Errorf("failed to persist signed record to peerstore: %w", err)
}
}
@@ -515,7 +515,7 @@ func (h *BasicHost) background() {
changeEvt.SignedPeerRecord = sr
// persist the signed record to the peerstore
if _, err := h.caBook.ConsumePeerRecord(sr, peerstore.PermanentAddrTTL); err != nil {
if _, err := h.caBook.ConsumePeerRecord(context.Background(), sr, peerstore.PermanentAddrTTL); err != nil {
log.Errorf("failed to persist signed peer record in peer store, err=%s", err)
return
}
@@ -656,7 +656,7 @@ func (h *BasicHost) NewStream(ctx context.Context, p peer.ID, pids ...protocol.I
return nil, ctx.Err()
}
pref, err := h.preferredProtocol(p, pids)
pref, err := h.preferredProtocol(ctx, p, pids)
if err != nil {
_ = s.Reset()
return nil, err
@@ -692,12 +692,12 @@ func (h *BasicHost) NewStream(ctx context.Context, p peer.ID, pids ...protocol.I
}
s.SetProtocol(selected)
h.Peerstore().AddProtocols(p, selected)
h.Peerstore().AddProtocols(ctx, p, selected)
return s, nil
}
func (h *BasicHost) preferredProtocol(p peer.ID, pids []protocol.ID) (protocol.ID, error) {
supported, err := h.Peerstore().SupportsProtocols(p, pids...)
func (h *BasicHost) preferredProtocol(ctx context.Context, p peer.ID, pids []protocol.ID) (protocol.ID, error) {
supported, err := h.Peerstore().SupportsProtocols(ctx, p, pids...)
if err != nil {
return "", err
}
@@ -716,7 +716,7 @@ func (h *BasicHost) preferredProtocol(p peer.ID, pids []protocol.ID) (protocol.I
// It will also resolve any /dns4, /dns6, and /dnsaddr addresses.
func (h *BasicHost) Connect(ctx context.Context, pi peer.AddrInfo) error {
// absorb addresses into peerstore
h.Peerstore().AddAddrs(pi.ID, pi.Addrs, peerstore.TempAddrTTL)
h.Peerstore().AddAddrs(ctx, pi.ID, pi.Addrs, peerstore.TempAddrTTL)
forceDirect, _ := network.GetForceDirectDial(ctx)
if !forceDirect {

View File

@@ -47,7 +47,7 @@ func TestHostSimple(t *testing.T) {
defer h2.Close()
h2.Start()
h2pi := h2.Peerstore().PeerInfo(h2.ID())
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2.ID())
require.NoError(t, h1.Connect(ctx, h2pi))
piper, pipew := io.Pipe()
@@ -105,7 +105,7 @@ func TestSignedPeerRecordWithNoListenAddrs(t *testing.T) {
// the signed record with the new addr is added async
var env *record.Envelope
require.Eventually(t, func() bool {
env = cab.GetPeerRecord(h.ID())
env = cab.GetPeerRecord(context.Background(), h.ID())
return env != nil
}, 500*time.Millisecond, 10*time.Millisecond)
rec, err := env.Record()
@@ -255,7 +255,7 @@ func getHostPair(t *testing.T) (host.Host, host.Host) {
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
defer cancel()
h2pi := h2.Peerstore().PeerInfo(h2.ID())
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2.ID())
require.NoError(t, h1.Connect(ctx, h2pi))
return h1, h2
}
@@ -369,7 +369,7 @@ func TestHostProtoPreknowledge(t *testing.T) {
// Prevent pushing identify information so this test actually _uses_ the super protocol.
h1.RemoveStreamHandler(identify.IDPush)
h2pi := h2.Peerstore().PeerInfo(h2.ID())
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2.ID())
// Filter to only 1 address so that we don't have to think about parallel
// connections in this test
h2pi.Addrs = h2pi.Addrs[:1]
@@ -395,7 +395,7 @@ func TestHostProtoPreknowledge(t *testing.T) {
h2.SetStreamHandler("/foo", handler)
require.Never(t, func() bool {
protos, err := h1.Peerstore().GetProtocols(h2.ID())
protos, err := h1.Peerstore().GetProtocols(context.Background(), h2.ID())
require.NoError(t, err)
for _, p := range protos {
if p == "/foo" {
@@ -458,7 +458,7 @@ func TestNewStreamResolve(t *testing.T) {
defer cancel()
// Get the tcp port that h2 is listening on.
h2pi := h2.Peerstore().PeerInfo(h2.ID())
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2.ID())
var dialAddr string
const tcpPrefix = "/ip4/127.0.0.1/tcp/"
for _, addr := range h2pi.Addrs {
@@ -474,7 +474,7 @@ func TestNewStreamResolve(t *testing.T) {
// Add the DNS multiaddr to h1's peerstore.
maddr, err := ma.NewMultiaddr(dialAddr)
require.NoError(t, err)
h1.Peerstore().AddAddr(h2.ID(), maddr, time.Second)
h1.Peerstore().AddAddr(context.Background(), h2.ID(), maddr, time.Second)
connectedOn := make(chan protocol.ID)
h2.SetStreamHandler("/testing", func(s network.Stream) {
@@ -535,7 +535,7 @@ func TestProtoDowngrade(t *testing.T) {
// This is _almost_ instantaneous, but this test fails once every ~1k runs without this.
time.Sleep(time.Millisecond)
h2pi := h2.Peerstore().PeerInfo(h2.ID())
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2.ID())
require.NoError(t, h1.Connect(ctx, h2pi))
s2, err := h1.NewStream(ctx, h2.ID(), "/testing/1.0.0", "/testing")
@@ -587,7 +587,7 @@ func TestAddrChangeImmediatelyIfAddressNonEmpty(t *testing.T) {
require.Equal(t, taddrs, rc.Addrs)
// assert it's in the peerstore
ev := h.Peerstore().(peerstore.CertifiedAddrBook).GetPeerRecord(h.ID())
ev := h.Peerstore().(peerstore.CertifiedAddrBook).GetPeerRecord(context.Background(), h.ID())
require.NotNil(t, ev)
rc = peerRecordFromEnvelope(t, ev)
require.Equal(t, taddrs, rc.Addrs)
@@ -696,7 +696,7 @@ func TestHostAddrChangeDetection(t *testing.T) {
require.Equal(t, addrSets[i], rc.Addrs)
// assert it's in the peerstore
ev := h.Peerstore().(peerstore.CertifiedAddrBook).GetPeerRecord(h.ID())
ev := h.Peerstore().(peerstore.CertifiedAddrBook).GetPeerRecord(context.Background(), h.ID())
require.NotNil(t, ev)
rc = peerRecordFromEnvelope(t, ev)
require.Equal(t, addrSets[i], rc.Addrs)

View File

@@ -97,12 +97,12 @@ func (bh *BlankHost) initSignedRecord() error {
return errors.New("peerstore does not support signed records")
}
rec := peer.PeerRecordFromAddrInfo(peer.AddrInfo{ID: bh.ID(), Addrs: bh.Addrs()})
ev, err := record.Seal(rec, bh.Peerstore().PrivKey(bh.ID()))
ev, err := record.Seal(rec, bh.Peerstore().PrivKey(context.Background(), bh.ID()))
if err != nil {
log.Errorf("failed to create signed record for self, err=%s", err)
return fmt.Errorf("failed to create signed record for self, err=%s", err)
}
_, err = cab.ConsumePeerRecord(ev, peerstore.PermanentAddrTTL)
_, err = cab.ConsumePeerRecord(context.Background(), ev, peerstore.PermanentAddrTTL)
if err != nil {
log.Errorf("failed to persist signed record to peerstore,err=%s", err)
return fmt.Errorf("failed to persist signed record for self, err=%s", err)
@@ -128,7 +128,7 @@ func (bh *BlankHost) Close() error {
func (bh *BlankHost) Connect(ctx context.Context, ai peer.AddrInfo) error {
// absorb addresses into peerstore
bh.Peerstore().AddAddrs(ai.ID, ai.Addrs, peerstore.TempAddrTTL)
bh.Peerstore().AddAddrs(ctx, ai.ID, ai.Addrs, peerstore.TempAddrTTL)
cs := bh.n.ConnsToPeer(ai.ID)
if len(cs) > 0 {
@@ -160,7 +160,7 @@ func (bh *BlankHost) NewStream(ctx context.Context, p peer.ID, protos ...protoco
}
s.SetProtocol(selected)
bh.Peerstore().AddProtocols(p, selected)
bh.Peerstore().AddProtocols(ctx, p, selected)
return s, nil
}

View File

@@ -1,6 +1,7 @@
package peerstore
import (
"context"
"sync"
"time"
@@ -51,7 +52,7 @@ func (m *metrics) LatencyEWMA(p peer.ID) time.Duration {
return m.latmap[p]
}
func (m *metrics) RemovePeer(p peer.ID) {
func (m *metrics) RemovePeer(ctx context.Context, p peer.ID) {
m.mutex.Lock()
delete(m.latmap, p)
m.mutex.Unlock()

View File

@@ -1,14 +1,16 @@
package peerstore
import (
"context"
"github.com/libp2p/go-libp2p/core/peer"
pstore "github.com/libp2p/go-libp2p/core/peerstore"
)
func PeerInfos(ps pstore.Peerstore, peers peer.IDSlice) []peer.AddrInfo {
func PeerInfos(ctx context.Context, ps pstore.Peerstore, peers peer.IDSlice) []peer.AddrInfo {
pi := make([]peer.AddrInfo, len(peers))
for i, p := range peers {
pi[i] = ps.PeerInfo(p)
pi[i] = ps.PeerInfo(ctx, p)
}
return pi
}

View File

@@ -47,11 +47,11 @@ type addrsRecord struct {
// flush writes the record to the datastore by calling ds.Put, unless the record is
// marked for deletion, in which case we call ds.Delete. To be called within a lock.
func (r *addrsRecord) flush(write ds.Write) (err error) {
func (r *addrsRecord) flush(ctx context.Context, write ds.Write) (err error) {
key := addrBookBase.ChildString(b32.RawStdEncoding.EncodeToString(r.Id))
if len(r.Addrs) == 0 {
if err = write.Delete(context.TODO(), key); err == nil {
if err = write.Delete(ctx, key); err == nil {
r.dirty = false
}
return err
@@ -61,7 +61,7 @@ func (r *addrsRecord) flush(write ds.Write) (err error) {
if err != nil {
return err
}
if err = write.Put(context.TODO(), key, data); err != nil {
if err = write.Put(ctx, key, data); err != nil {
return err
}
// write succeeded; record is no longer dirty.
@@ -229,13 +229,13 @@ func (ab *dsAddrBook) Close() error {
// as a result and the update argument is true, the resulting state is saved in the datastore.
//
// If the cache argument is true, the record is inserted in the cache when loaded from the datastore.
func (ab *dsAddrBook) loadRecord(id peer.ID, cache bool, update bool) (pr *addrsRecord, err error) {
func (ab *dsAddrBook) loadRecord(ctx context.Context, id peer.ID, cache bool, update bool) (pr *addrsRecord, err error) {
if pr, ok := ab.cache.Get(id); ok {
pr.Lock()
defer pr.Unlock()
if pr.clean(ab.clock.Now()) && update {
err = pr.flush(ab.ds)
err = pr.flush(ctx, ab.ds)
}
return pr, err
}
@@ -254,7 +254,7 @@ func (ab *dsAddrBook) loadRecord(id peer.ID, cache bool, update bool) (pr *addrs
}
// this record is new and local for now (not in cache), so we don't need to lock.
if pr.clean(ab.clock.Now()) && update {
err = pr.flush(ab.ds)
err = pr.flush(ctx, ab.ds)
}
default:
return nil, err
@@ -267,23 +267,23 @@ func (ab *dsAddrBook) loadRecord(id peer.ID, cache bool, update bool) (pr *addrs
}
// AddAddr will add a new address if it's not already in the AddrBook.
func (ab *dsAddrBook) AddAddr(p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
ab.AddAddrs(p, []ma.Multiaddr{addr}, ttl)
func (ab *dsAddrBook) AddAddr(ctx context.Context, p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
ab.AddAddrs(ctx, p, []ma.Multiaddr{addr}, ttl)
}
// AddAddrs will add many new addresses if they're not already in the AddrBook.
func (ab *dsAddrBook) AddAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
func (ab *dsAddrBook) AddAddrs(ctx context.Context, p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
if ttl <= 0 {
return
}
addrs = cleanAddrs(addrs, p)
ab.setAddrs(p, addrs, ttl, ttlExtend, false)
ab.setAddrs(ctx, p, addrs, ttl, ttlExtend, false)
}
// ConsumePeerRecord adds addresses from a signed peer.PeerRecord (contained in
// a record.Envelope), which will expire after the given TTL.
// See https://godoc.org/github.com/libp2p/go-libp2p/core/peerstore#CertifiedAddrBook for more details.
func (ab *dsAddrBook) ConsumePeerRecord(recordEnvelope *record.Envelope, ttl time.Duration) (bool, error) {
func (ab *dsAddrBook) ConsumePeerRecord(ctx context.Context, recordEnvelope *record.Envelope, ttl time.Duration) (bool, error) {
r, err := recordEnvelope.Record()
if err != nil {
return false, err
@@ -298,25 +298,25 @@ func (ab *dsAddrBook) ConsumePeerRecord(recordEnvelope *record.Envelope, ttl tim
// ensure that the seq number from envelope is >= any previously received seq no
// update when equal to extend the ttls
if ab.latestPeerRecordSeq(rec.PeerID) > rec.Seq {
if ab.latestPeerRecordSeq(ctx, rec.PeerID) > rec.Seq {
return false, nil
}
addrs := cleanAddrs(rec.Addrs, rec.PeerID)
err = ab.setAddrs(rec.PeerID, addrs, ttl, ttlExtend, true)
err = ab.setAddrs(ctx, rec.PeerID, addrs, ttl, ttlExtend, true)
if err != nil {
return false, err
}
err = ab.storeSignedPeerRecord(rec.PeerID, recordEnvelope, rec)
err = ab.storeSignedPeerRecord(ctx, rec.PeerID, recordEnvelope, rec)
if err != nil {
return false, err
}
return true, nil
}
func (ab *dsAddrBook) latestPeerRecordSeq(p peer.ID) uint64 {
pr, err := ab.loadRecord(p, true, false)
func (ab *dsAddrBook) latestPeerRecordSeq(ctx context.Context, p peer.ID) uint64 {
pr, err := ab.loadRecord(ctx, p, true, false)
if err != nil {
// We ignore the error because we don't want to fail storing a new record in this
// case.
@@ -332,7 +332,7 @@ func (ab *dsAddrBook) latestPeerRecordSeq(p peer.ID) uint64 {
return pr.CertifiedRecord.Seq
}
func (ab *dsAddrBook) storeSignedPeerRecord(p peer.ID, envelope *record.Envelope, rec *peer.PeerRecord) error {
func (ab *dsAddrBook) storeSignedPeerRecord(ctx context.Context, p peer.ID, envelope *record.Envelope, rec *peer.PeerRecord) error {
envelopeBytes, err := envelope.Marshal()
if err != nil {
return err
@@ -341,7 +341,7 @@ func (ab *dsAddrBook) storeSignedPeerRecord(p peer.ID, envelope *record.Envelope
// this has to be done after we add the addresses, since if
// we try to flush a datastore record with no addresses,
// it will just get deleted
pr, err := ab.loadRecord(p, true, false)
pr, err := ab.loadRecord(ctx, p, true, false)
if err != nil {
return err
}
@@ -352,15 +352,15 @@ func (ab *dsAddrBook) storeSignedPeerRecord(p peer.ID, envelope *record.Envelope
Raw: envelopeBytes,
}
pr.dirty = true
err = pr.flush(ab.ds)
err = pr.flush(ctx, ab.ds)
return err
}
// GetPeerRecord returns a record.Envelope containing a peer.PeerRecord for the
// given peer id, if one exists.
// Returns nil if no signed PeerRecord exists for the peer.
func (ab *dsAddrBook) GetPeerRecord(p peer.ID) *record.Envelope {
pr, err := ab.loadRecord(p, true, false)
func (ab *dsAddrBook) GetPeerRecord(ctx context.Context, p peer.ID) *record.Envelope {
pr, err := ab.loadRecord(ctx, p, true, false)
if err != nil {
log.Errorf("unable to load record for peer %s: %v", p.Pretty(), err)
return nil
@@ -379,24 +379,24 @@ func (ab *dsAddrBook) GetPeerRecord(p peer.ID) *record.Envelope {
}
// SetAddr will add or update the TTL of an address in the AddrBook.
func (ab *dsAddrBook) SetAddr(p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
ab.SetAddrs(p, []ma.Multiaddr{addr}, ttl)
func (ab *dsAddrBook) SetAddr(ctx context.Context, p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
ab.SetAddrs(ctx, p, []ma.Multiaddr{addr}, ttl)
}
// SetAddrs will add or update the TTLs of addresses in the AddrBook.
func (ab *dsAddrBook) SetAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
func (ab *dsAddrBook) SetAddrs(ctx context.Context, p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
addrs = cleanAddrs(addrs, p)
if ttl <= 0 {
ab.deleteAddrs(p, addrs)
ab.deleteAddrs(ctx, p, addrs)
return
}
ab.setAddrs(p, addrs, ttl, ttlOverride, false)
ab.setAddrs(ctx, p, addrs, ttl, ttlOverride, false)
}
// UpdateAddrs will update any addresses for a given peer and TTL combination to
// have a new TTL.
func (ab *dsAddrBook) UpdateAddrs(p peer.ID, oldTTL time.Duration, newTTL time.Duration) {
pr, err := ab.loadRecord(p, true, false)
func (ab *dsAddrBook) UpdateAddrs(ctx context.Context, p peer.ID, oldTTL time.Duration, newTTL time.Duration) {
pr, err := ab.loadRecord(ctx, p, true, false)
if err != nil {
log.Errorf("failed to update ttls for peer %s: %s\n", p.Pretty(), err)
return
@@ -415,13 +415,13 @@ func (ab *dsAddrBook) UpdateAddrs(p peer.ID, oldTTL time.Duration, newTTL time.D
}
if pr.clean(ab.clock.Now()) {
pr.flush(ab.ds)
pr.flush(ctx, ab.ds)
}
}
// Addrs returns all of the non-expired addresses for a given peer.
func (ab *dsAddrBook) Addrs(p peer.ID) []ma.Multiaddr {
pr, err := ab.loadRecord(p, true, true)
func (ab *dsAddrBook) Addrs(ctx context.Context, p peer.ID) []ma.Multiaddr {
pr, err := ab.loadRecord(ctx, p, true, true)
if err != nil {
log.Warn("failed to load peerstore entry for peer %v while querying addrs, err: %v", p, err)
return nil
@@ -443,8 +443,8 @@ func (ab *dsAddrBook) Addrs(p peer.ID) []ma.Multiaddr {
}
// Peers returns all of the peer IDs for which the AddrBook has addresses.
func (ab *dsAddrBook) PeersWithAddrs() peer.IDSlice {
ids, err := uniquePeerIds(ab.ds, addrBookBase, func(result query.Result) string {
func (ab *dsAddrBook) PeersWithAddrs(ctx context.Context) peer.IDSlice {
ids, err := uniquePeerIds(ctx, ab.ds, addrBookBase, func(result query.Result) string {
return ds.RawKey(result.Key).Name()
})
if err != nil {
@@ -456,26 +456,26 @@ func (ab *dsAddrBook) PeersWithAddrs() peer.IDSlice {
// AddrStream returns a channel on which all new addresses discovered for a
// given peer ID will be published.
func (ab *dsAddrBook) AddrStream(ctx context.Context, p peer.ID) <-chan ma.Multiaddr {
initial := ab.Addrs(p)
initial := ab.Addrs(ctx, p)
return ab.subsManager.AddrStream(ctx, p, initial)
}
// ClearAddrs will delete all known addresses for a peer ID.
func (ab *dsAddrBook) ClearAddrs(p peer.ID) {
func (ab *dsAddrBook) ClearAddrs(ctx context.Context, p peer.ID) {
ab.cache.Remove(p)
key := addrBookBase.ChildString(b32.RawStdEncoding.EncodeToString([]byte(p)))
if err := ab.ds.Delete(context.TODO(), key); err != nil {
if err := ab.ds.Delete(ctx, key); err != nil {
log.Errorf("failed to clear addresses for peer %s: %v", p.Pretty(), err)
}
}
func (ab *dsAddrBook) setAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duration, mode ttlWriteMode, signed bool) (err error) {
func (ab *dsAddrBook) setAddrs(ctx context.Context, p peer.ID, addrs []ma.Multiaddr, ttl time.Duration, mode ttlWriteMode, signed bool) (err error) {
if len(addrs) == 0 {
return nil
}
pr, err := ab.loadRecord(p, true, false)
pr, err := ab.loadRecord(ctx, p, true, false)
if err != nil {
return fmt.Errorf("failed to load peerstore entry for peer %v while setting addrs, err: %v", p, err)
}
@@ -549,7 +549,7 @@ func (ab *dsAddrBook) setAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duratio
pr.dirty = true
pr.clean(ab.clock.Now())
return pr.flush(ab.ds)
return pr.flush(ctx, ab.ds)
}
// deletes addresses in place, avoiding copies until we encounter the first deletion.
@@ -578,8 +578,8 @@ Outer:
return s[:survived]
}
func (ab *dsAddrBook) deleteAddrs(p peer.ID, addrs []ma.Multiaddr) (err error) {
pr, err := ab.loadRecord(p, false, false)
func (ab *dsAddrBook) deleteAddrs(ctx context.Context, p peer.ID, addrs []ma.Multiaddr) (err error) {
pr, err := ab.loadRecord(ctx, p, false, false)
if err != nil {
return fmt.Errorf("failed to load peerstore entry for peer %v while deleting addrs, err: %v", p, err)
}
@@ -595,7 +595,7 @@ func (ab *dsAddrBook) deleteAddrs(p peer.ID, addrs []ma.Multiaddr) (err error) {
pr.dirty = true
pr.clean(ab.clock.Now())
return pr.flush(ab.ds)
return pr.flush(ctx, ab.ds)
}
func cleanAddrs(addrs []ma.Multiaddr, pid peer.ID) []ma.Multiaddr {

View File

@@ -48,7 +48,7 @@ type dsAddrBookGc struct {
ab *dsAddrBook
running chan struct{}
lookaheadEnabled bool
purgeFunc func()
purgeFunc func(ctx context.Context)
currWindowEnd int64
}
@@ -108,18 +108,18 @@ func (gc *dsAddrBookGc) background() {
if gc.lookaheadEnabled {
lookaheadTimer := time.NewTicker(gc.ab.opts.GCLookaheadInterval)
lookaheadCh = lookaheadTimer.C
gc.populateLookahead() // do a lookahead now
gc.populateLookahead(gc.ctx) // do a lookahead now
defer lookaheadTimer.Stop()
}
for {
select {
case <-purgeTimer.C:
gc.purgeFunc()
gc.purgeFunc(gc.ctx)
case <-lookaheadCh:
// will never trigger if lookahead is disabled (nil Duration).
gc.populateLookahead()
gc.populateLookahead(gc.ctx)
case <-gc.ctx.Done():
return
@@ -129,7 +129,7 @@ func (gc *dsAddrBookGc) background() {
// purgeCycle runs a single GC purge cycle. It operates within the lookahead window if lookahead is enabled; else it
// visits all entries in the datastore, deleting the addresses that have expired.
func (gc *dsAddrBookGc) purgeLookahead() {
func (gc *dsAddrBookGc) purgeLookahead(ctx context.Context) {
select {
case gc.running <- struct{}{}:
defer func() { <-gc.running }()
@@ -152,7 +152,7 @@ func (gc *dsAddrBookGc) purgeLookahead() {
if err != nil {
log.Warnf("failed while %s record with GC key: %v, err: %v; deleting", msg, key, err)
}
if err = batch.Delete(context.TODO(), key); err != nil {
if err = batch.Delete(ctx, key); err != nil {
log.Warnf("failed to delete corrupt GC lookahead entry: %v, err: %v", key, err)
}
}
@@ -160,20 +160,20 @@ func (gc *dsAddrBookGc) purgeLookahead() {
// This function drops a GC key if the entry is cleaned correctly. It may reschedule another visit
// if the next earliest expiry falls within the current window again.
dropOrReschedule := func(key ds.Key, ar *addrsRecord) {
if err := batch.Delete(context.TODO(), key); err != nil {
if err := batch.Delete(ctx, key); err != nil {
log.Warnf("failed to delete lookahead entry: %v, err: %v", key, err)
}
// re-add the record if it needs to be visited again in this window.
if len(ar.Addrs) != 0 && ar.Addrs[0].Expiry <= gc.currWindowEnd {
gcKey := gcLookaheadBase.ChildString(fmt.Sprintf("%d/%s", ar.Addrs[0].Expiry, key.Name()))
if err := batch.Put(context.TODO(), gcKey, []byte{}); err != nil {
if err := batch.Put(ctx, gcKey, []byte{}); err != nil {
log.Warnf("failed to add new GC key: %v, err: %v", gcKey, err)
}
}
}
results, err := gc.ab.ds.Query(context.TODO(), purgeLookaheadQuery)
results, err := gc.ab.ds.Query(ctx, purgeLookaheadQuery)
if err != nil {
log.Warnf("failed while fetching entries to purge: %v", err)
return
@@ -214,7 +214,7 @@ func (gc *dsAddrBookGc) purgeLookahead() {
if cached, ok := gc.ab.cache.Peek(id); ok {
cached.Lock()
if cached.clean(gc.ab.clock.Now()) {
if err = cached.flush(batch); err != nil {
if err = cached.flush(ctx, batch); err != nil {
log.Warnf("failed to flush entry modified by GC for peer: &v, err: %v", id.Pretty(), err)
}
}
@@ -227,7 +227,7 @@ func (gc *dsAddrBookGc) purgeLookahead() {
// otherwise, fetch it from the store, clean it and flush it.
entryKey := addrBookBase.ChildString(gcKey.Name())
val, err := gc.ab.ds.Get(context.TODO(), entryKey)
val, err := gc.ab.ds.Get(ctx, entryKey)
if err != nil {
// captures all errors, including ErrNotFound.
dropInError(gcKey, err, "fetching entry")
@@ -239,7 +239,7 @@ func (gc *dsAddrBookGc) purgeLookahead() {
continue
}
if record.clean(gc.ab.clock.Now()) {
err = record.flush(batch)
err = record.flush(ctx, batch)
if err != nil {
log.Warnf("failed to flush entry modified by GC for peer: &v, err: %v", id.Pretty(), err)
}
@@ -247,12 +247,12 @@ func (gc *dsAddrBookGc) purgeLookahead() {
dropOrReschedule(gcKey, record)
}
if err = batch.Commit(context.TODO()); err != nil {
if err = batch.Commit(ctx); err != nil {
log.Warnf("failed to commit GC purge batch: %v", err)
}
}
func (gc *dsAddrBookGc) purgeStore() {
func (gc *dsAddrBookGc) purgeStore(ctx context.Context) {
select {
case gc.running <- struct{}{}:
defer func() { <-gc.running }()
@@ -267,7 +267,7 @@ func (gc *dsAddrBookGc) purgeStore() {
log.Warnf("failed while creating batch to purge GC entries: %v", err)
}
results, err := gc.ab.ds.Query(context.TODO(), purgeStoreQuery)
results, err := gc.ab.ds.Query(ctx, purgeStoreQuery)
if err != nil {
log.Warnf("failed while opening iterator: %v", err)
return
@@ -287,13 +287,13 @@ func (gc *dsAddrBookGc) purgeStore() {
continue
}
if err := record.flush(batch); err != nil {
if err := record.flush(ctx, batch); err != nil {
log.Warnf("failed to flush entry modified by GC for peer: &v, err: %v", id, err)
}
gc.ab.cache.Remove(peer.ID(id))
}
if err = batch.Commit(context.TODO()); err != nil {
if err = batch.Commit(ctx); err != nil {
log.Warnf("failed to commit GC purge batch: %v", err)
}
}
@@ -303,7 +303,7 @@ func (gc *dsAddrBookGc) purgeStore() {
//
// Those entries are stored in the lookahead region in the store, indexed by the timestamp when they need to be
// visited, to facilitate temporal range scans.
func (gc *dsAddrBookGc) populateLookahead() {
func (gc *dsAddrBookGc) populateLookahead(ctx context.Context) {
if gc.ab.opts.GCLookaheadInterval == 0 {
return
}
@@ -320,7 +320,7 @@ func (gc *dsAddrBookGc) populateLookahead() {
var id peer.ID
record := &addrsRecord{AddrBookRecord: &pb.AddrBookRecord{}}
results, err := gc.ab.ds.Query(context.TODO(), populateLookaheadQuery)
results, err := gc.ab.ds.Query(ctx, populateLookaheadQuery)
if err != nil {
log.Warnf("failed while querying to populate lookahead GC window: %v", err)
return
@@ -352,7 +352,7 @@ func (gc *dsAddrBookGc) populateLookahead() {
continue
}
gcKey := gcLookaheadBase.ChildString(fmt.Sprintf("%d/%s", cached.Addrs[0].Expiry, idb32))
if err = batch.Put(context.TODO(), gcKey, []byte{}); err != nil {
if err = batch.Put(ctx, gcKey, []byte{}); err != nil {
log.Warnf("failed while inserting GC entry for peer: %v, err: %v", id.Pretty(), err)
}
cached.RUnlock()
@@ -361,7 +361,7 @@ func (gc *dsAddrBookGc) populateLookahead() {
record.Reset()
val, err := gc.ab.ds.Get(context.TODO(), ds.RawKey(result.Key))
val, err := gc.ab.ds.Get(ctx, ds.RawKey(result.Key))
if err != nil {
log.Warnf("failed which getting record from store for peer: %v, err: %v", id.Pretty(), err)
continue
@@ -372,13 +372,13 @@ func (gc *dsAddrBookGc) populateLookahead() {
}
if len(record.Addrs) > 0 && record.Addrs[0].Expiry <= until {
gcKey := gcLookaheadBase.ChildString(fmt.Sprintf("%d/%s", record.Addrs[0].Expiry, idb32))
if err = batch.Put(context.TODO(), gcKey, []byte{}); err != nil {
if err = batch.Put(ctx, gcKey, []byte{}); err != nil {
log.Warnf("failed while inserting GC entry for peer: %v, err: %v", id.Pretty(), err)
}
}
}
if err = batch.Commit(context.TODO()); err != nil {
if err = batch.Commit(ctx); err != nil {
log.Warnf("failed to commit GC lookahead batch: %v", err)
}

View File

@@ -59,29 +59,29 @@ func TestGCLookahead(t *testing.T) {
addrs := test.GenerateAddrs(100)
// lookahead is 10 seconds, so these entries will be outside the lookahead window.
ab.AddAddrs(ids[0], addrs[:10], time.Hour)
ab.AddAddrs(ids[1], addrs[10:20], time.Hour)
ab.AddAddrs(ids[2], addrs[20:30], time.Hour)
ab.AddAddrs(context.Background(), ids[0], addrs[:10], time.Hour)
ab.AddAddrs(context.Background(), ids[1], addrs[10:20], time.Hour)
ab.AddAddrs(context.Background(), ids[2], addrs[20:30], time.Hour)
gc.populateLookahead()
gc.populateLookahead(context.Background())
if i := tp.countLookaheadEntries(); i != 0 {
t.Errorf("expected no GC lookahead entries, got: %v", i)
}
// change addresses of a peer to have TTL 1 second, placing them in the lookahead window.
ab.UpdateAddrs(ids[1], time.Hour, time.Second)
ab.UpdateAddrs(context.Background(), ids[1], time.Hour, time.Second)
// Purge the cache, to exercise a different path in the lookahead cycle.
tp.clearCache()
gc.populateLookahead()
gc.populateLookahead(context.Background())
if i := tp.countLookaheadEntries(); i != 1 {
t.Errorf("expected 1 GC lookahead entry, got: %v", i)
}
// change addresses of another to have TTL 5 second, placing them in the lookahead window.
ab.UpdateAddrs(ids[2], time.Hour, 5*time.Second)
gc.populateLookahead()
ab.UpdateAddrs(context.Background(), ids[2], time.Hour, 5*time.Second)
gc.populateLookahead(context.Background())
if i := tp.countLookaheadEntries(); i != 2 {
t.Errorf("expected 2 GC lookahead entries, got: %v", i)
}
@@ -108,26 +108,26 @@ func TestGCPurging(t *testing.T) {
addrs := test.GenerateAddrs(100)
// stagger addresses within the lookahead window, but stagger them.
ab.AddAddrs(ids[0], addrs[:10], 1*time.Second)
ab.AddAddrs(ids[1], addrs[30:40], 1*time.Second)
ab.AddAddrs(ids[2], addrs[60:70], 1*time.Second)
ab.AddAddrs(context.Background(), ids[0], addrs[:10], 1*time.Second)
ab.AddAddrs(context.Background(), ids[1], addrs[30:40], 1*time.Second)
ab.AddAddrs(context.Background(), ids[2], addrs[60:70], 1*time.Second)
ab.AddAddrs(ids[0], addrs[10:20], 4*time.Second)
ab.AddAddrs(ids[1], addrs[40:50], 4*time.Second)
ab.AddAddrs(context.Background(), ids[0], addrs[10:20], 4*time.Second)
ab.AddAddrs(context.Background(), ids[1], addrs[40:50], 4*time.Second)
ab.AddAddrs(ids[0], addrs[20:30], 10*time.Second)
ab.AddAddrs(ids[1], addrs[50:60], 10*time.Second)
ab.AddAddrs(context.Background(), ids[0], addrs[20:30], 10*time.Second)
ab.AddAddrs(context.Background(), ids[1], addrs[50:60], 10*time.Second)
// this is inside the window, but it will survive the purges we do in the test.
ab.AddAddrs(ids[3], addrs[70:80], 15*time.Second)
ab.AddAddrs(context.Background(), ids[3], addrs[70:80], 15*time.Second)
gc.populateLookahead()
gc.populateLookahead(context.Background())
if i := tp.countLookaheadEntries(); i != 4 {
t.Errorf("expected 4 GC lookahead entries, got: %v", i)
}
clk.Add(2 * time.Second)
gc.purgeLookahead()
gc.purgeLookahead(context.Background())
if i := tp.countLookaheadEntries(); i != 3 {
t.Errorf("expected 3 GC lookahead entries, got: %v", i)
}
@@ -136,20 +136,20 @@ func TestGCPurging(t *testing.T) {
tp.clearCache()
clk.Add(5 * time.Second)
gc.purgeLookahead()
gc.purgeLookahead(context.Background())
if i := tp.countLookaheadEntries(); i != 3 {
t.Errorf("expected 3 GC lookahead entries, got: %v", i)
}
clk.Add(5 * time.Second)
gc.purgeLookahead()
gc.purgeLookahead(context.Background())
if i := tp.countLookaheadEntries(); i != 1 {
t.Errorf("expected 1 GC lookahead entries, got: %v", i)
}
if i := len(ab.PeersWithAddrs()); i != 1 {
if i := len(ab.PeersWithAddrs(context.Background())); i != 1 {
t.Errorf("expected 1 entries in database, got: %v", i)
}
if p := ab.PeersWithAddrs()[0]; p != ids[3] {
if p := ab.PeersWithAddrs(context.Background())[0]; p != ids[3] {
t.Errorf("expected remaining peer to be #3, got: %v, expected: %v", p, ids[3])
}
}
@@ -173,7 +173,7 @@ func TestGCDelay(t *testing.T) {
tp := &testProbe{t, ab}
ab.AddAddrs(ids[0], addrs, 1*time.Second)
ab.AddAddrs(context.Background(), ids[0], addrs, 1*time.Second)
// immediately after we should be having no lookahead entries.
if i := tp.countLookaheadEntries(); i != 0 {
@@ -209,11 +209,11 @@ func TestGCLookaheadDisabled(t *testing.T) {
// ids[1] has 20 addresses; 50% expire in 500ms and 50% in 10 hours.
// ids[2] has 10 addresses; all expire in 10 hours.
// ids[3] has 60 addresses; all expire in 10 hours.
ab.AddAddrs(ids[0], addrs[:10], 500*time.Millisecond)
ab.AddAddrs(ids[1], addrs[10:20], 500*time.Millisecond)
ab.AddAddrs(ids[1], addrs[20:30], 10*time.Hour)
ab.AddAddrs(ids[2], addrs[30:40], 10*time.Hour)
ab.AddAddrs(ids[3], addrs[40:], 10*time.Hour)
ab.AddAddrs(context.Background(), ids[0], addrs[:10], 500*time.Millisecond)
ab.AddAddrs(context.Background(), ids[1], addrs[10:20], 500*time.Millisecond)
ab.AddAddrs(context.Background(), ids[1], addrs[20:30], 10*time.Hour)
ab.AddAddrs(context.Background(), ids[2], addrs[30:40], 10*time.Hour)
ab.AddAddrs(context.Background(), ids[3], addrs[40:], 10*time.Hour)
clk.Add(100 * time.Millisecond)
@@ -223,13 +223,13 @@ func TestGCLookaheadDisabled(t *testing.T) {
clk.Add(500 * time.Millisecond)
gc := ab.(*dsAddrBook).gc
gc.purgeFunc()
gc.purgeFunc(context.Background())
var empty []ma.Multiaddr
test.AssertAddressesEqual(t, empty, ab.Addrs(ids[0]))
test.AssertAddressesEqual(t, addrs[20:30], ab.Addrs(ids[1]))
test.AssertAddressesEqual(t, addrs[30:40], ab.Addrs(ids[2]))
test.AssertAddressesEqual(t, addrs[40:], ab.Addrs(ids[3]))
test.AssertAddressesEqual(t, empty, ab.Addrs(context.Background(), ids[0]))
test.AssertAddressesEqual(t, addrs[20:30], ab.Addrs(context.Background(), ids[1]))
test.AssertAddressesEqual(t, addrs[30:40], ab.Addrs(context.Background(), ids[2]))
test.AssertAddressesEqual(t, addrs[40:], ab.Addrs(context.Background(), ids[3]))
}
func BenchmarkLookaheadCycle(b *testing.B) {
@@ -254,11 +254,11 @@ func BenchmarkLookaheadCycle(b *testing.B) {
} else {
ttl = outside
}
ab.AddAddrs(id, addrs, ttl)
ab.AddAddrs(context.Background(), id, addrs, ttl)
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
ab.(*dsAddrBook).gc.populateLookahead()
ab.(*dsAddrBook).gc.populateLookahead(context.Background())
}
}

View File

@@ -31,11 +31,11 @@ func NewKeyBook(_ context.Context, store ds.Datastore, _ Options) (*dsKeyBook, e
return &dsKeyBook{store}, nil
}
func (kb *dsKeyBook) PubKey(p peer.ID) ic.PubKey {
func (kb *dsKeyBook) PubKey(ctx context.Context, p peer.ID) ic.PubKey {
key := peerToKey(p, pubSuffix)
var pk ic.PubKey
if value, err := kb.ds.Get(context.TODO(), key); err == nil {
if value, err := kb.ds.Get(ctx, key); err == nil {
pk, err = ic.UnmarshalPublicKey(value)
if err != nil {
log.Errorf("error when unmarshalling pubkey from datastore for peer %s: %s\n", p.Pretty(), err)
@@ -55,7 +55,7 @@ func (kb *dsKeyBook) PubKey(p peer.ID) ic.PubKey {
log.Errorf("error when turning extracted pubkey into bytes for peer %s: %s\n", p.Pretty(), err)
return nil
}
if err := kb.ds.Put(context.TODO(), key, pkb); err != nil {
if err := kb.ds.Put(ctx, key, pkb); err != nil {
log.Errorf("error when adding extracted pubkey to peerstore for peer %s: %s\n", p.Pretty(), err)
return nil
}
@@ -66,7 +66,7 @@ func (kb *dsKeyBook) PubKey(p peer.ID) ic.PubKey {
return pk
}
func (kb *dsKeyBook) AddPubKey(p peer.ID, pk ic.PubKey) error {
func (kb *dsKeyBook) AddPubKey(ctx context.Context, p peer.ID, pk ic.PubKey) error {
// check it's correct.
if !p.MatchesPublicKey(pk) {
return errors.New("peer ID does not match public key")
@@ -77,15 +77,15 @@ func (kb *dsKeyBook) AddPubKey(p peer.ID, pk ic.PubKey) error {
log.Errorf("error while converting pubkey byte string for peer %s: %s\n", p.Pretty(), err)
return err
}
if err := kb.ds.Put(context.TODO(), peerToKey(p, pubSuffix), val); err != nil {
if err := kb.ds.Put(ctx, peerToKey(p, pubSuffix), val); err != nil {
log.Errorf("error while updating pubkey in datastore for peer %s: %s\n", p.Pretty(), err)
return err
}
return nil
}
func (kb *dsKeyBook) PrivKey(p peer.ID) ic.PrivKey {
value, err := kb.ds.Get(context.TODO(), peerToKey(p, privSuffix))
func (kb *dsKeyBook) PrivKey(ctx context.Context, p peer.ID) ic.PrivKey {
value, err := kb.ds.Get(ctx, peerToKey(p, privSuffix))
if err != nil {
return nil
}
@@ -96,7 +96,7 @@ func (kb *dsKeyBook) PrivKey(p peer.ID) ic.PrivKey {
return sk
}
func (kb *dsKeyBook) AddPrivKey(p peer.ID, sk ic.PrivKey) error {
func (kb *dsKeyBook) AddPrivKey(ctx context.Context, p peer.ID, sk ic.PrivKey) error {
if sk == nil {
return errors.New("private key is nil")
}
@@ -110,14 +110,14 @@ func (kb *dsKeyBook) AddPrivKey(p peer.ID, sk ic.PrivKey) error {
log.Errorf("error while converting privkey byte string for peer %s: %s\n", p.Pretty(), err)
return err
}
if err := kb.ds.Put(context.TODO(), peerToKey(p, privSuffix), val); err != nil {
if err := kb.ds.Put(ctx, peerToKey(p, privSuffix), val); err != nil {
log.Errorf("error while updating privkey in datastore for peer %s: %s\n", p.Pretty(), err)
}
return err
}
func (kb *dsKeyBook) PeersWithKeys() peer.IDSlice {
ids, err := uniquePeerIds(kb.ds, kbBase, func(result query.Result) string {
func (kb *dsKeyBook) PeersWithKeys(ctx context.Context) peer.IDSlice {
ids, err := uniquePeerIds(ctx, kb.ds, kbBase, func(result query.Result) string {
return ds.RawKey(result.Key).Parent().Name()
})
if err != nil {
@@ -126,9 +126,9 @@ func (kb *dsKeyBook) PeersWithKeys() peer.IDSlice {
return ids
}
func (kb *dsKeyBook) RemovePeer(p peer.ID) {
kb.ds.Delete(context.TODO(), peerToKey(p, privSuffix))
kb.ds.Delete(context.TODO(), peerToKey(p, pubSuffix))
func (kb *dsKeyBook) RemovePeer(ctx context.Context, p peer.ID) {
kb.ds.Delete(ctx, peerToKey(p, privSuffix))
kb.ds.Delete(ctx, peerToKey(p, pubSuffix))
}
func peerToKey(p peer.ID, suffix ds.Key) ds.Key {

View File

@@ -41,9 +41,9 @@ func NewPeerMetadata(_ context.Context, store ds.Datastore, _ Options) (*dsPeerM
return &dsPeerMetadata{store}, nil
}
func (pm *dsPeerMetadata) Get(p peer.ID, key string) (interface{}, error) {
func (pm *dsPeerMetadata) Get(ctx context.Context, p peer.ID, key string) (interface{}, error) {
k := pmBase.ChildString(base32.RawStdEncoding.EncodeToString([]byte(p))).ChildString(key)
value, err := pm.ds.Get(context.TODO(), k)
value, err := pm.ds.Get(ctx, k)
if err != nil {
if err == ds.ErrNotFound {
err = pstore.ErrNotFound
@@ -58,17 +58,17 @@ func (pm *dsPeerMetadata) Get(p peer.ID, key string) (interface{}, error) {
return res, nil
}
func (pm *dsPeerMetadata) Put(p peer.ID, key string, val interface{}) error {
func (pm *dsPeerMetadata) Put(ctx context.Context, p peer.ID, key string, val interface{}) error {
k := pmBase.ChildString(base32.RawStdEncoding.EncodeToString([]byte(p))).ChildString(key)
var buf pool.Buffer
if err := gob.NewEncoder(&buf).Encode(&val); err != nil {
return err
}
return pm.ds.Put(context.TODO(), k, buf.Bytes())
return pm.ds.Put(ctx, k, buf.Bytes())
}
func (pm *dsPeerMetadata) RemovePeer(p peer.ID) {
result, err := pm.ds.Query(context.TODO(), query.Query{
func (pm *dsPeerMetadata) RemovePeer(ctx context.Context, p peer.ID) {
result, err := pm.ds.Query(ctx, query.Query{
Prefix: pmBase.ChildString(base32.RawStdEncoding.EncodeToString([]byte(p))).String(),
KeysOnly: true,
})
@@ -77,6 +77,6 @@ func (pm *dsPeerMetadata) RemovePeer(p peer.ID) {
return
}
for entry := range result.Next() {
pm.ds.Delete(context.TODO(), ds.NewKey(entry.Key))
pm.ds.Delete(ctx, ds.NewKey(entry.Key))
}
}

View File

@@ -103,14 +103,14 @@ func NewPeerstore(ctx context.Context, store ds.Batching, opts Options) (*pstore
}
// uniquePeerIds extracts and returns unique peer IDs from database keys.
func uniquePeerIds(ds ds.Datastore, prefix ds.Key, extractor func(result query.Result) string) (peer.IDSlice, error) {
func uniquePeerIds(ctx context.Context, ds ds.Datastore, prefix ds.Key, extractor func(result query.Result) string) (peer.IDSlice, error) {
var (
q = query.Query{Prefix: prefix.String(), KeysOnly: true}
results query.Results
err error
)
if results, err = ds.Query(context.TODO(), q); err != nil {
if results, err = ds.Query(ctx, q); err != nil {
log.Error(err)
return nil, err
}
@@ -156,12 +156,12 @@ func (ps *pstoreds) Close() (err error) {
return nil
}
func (ps *pstoreds) Peers() peer.IDSlice {
func (ps *pstoreds) Peers(ctx context.Context) peer.IDSlice {
set := map[peer.ID]struct{}{}
for _, p := range ps.PeersWithKeys() {
for _, p := range ps.PeersWithKeys(ctx) {
set[p] = struct{}{}
}
for _, p := range ps.PeersWithAddrs() {
for _, p := range ps.PeersWithAddrs(ctx) {
set[p] = struct{}{}
}
@@ -172,10 +172,10 @@ func (ps *pstoreds) Peers() peer.IDSlice {
return pps
}
func (ps *pstoreds) PeerInfo(p peer.ID) peer.AddrInfo {
func (ps *pstoreds) PeerInfo(ctx context.Context, p peer.ID) peer.AddrInfo {
return peer.AddrInfo{
ID: p,
Addrs: ps.dsAddrBook.Addrs(p),
Addrs: ps.dsAddrBook.Addrs(ctx, p),
}
}
@@ -185,9 +185,9 @@ func (ps *pstoreds) PeerInfo(p peer.ID) peer.AddrInfo {
// * the PeerMetadata
// * the Metrics
// It DOES NOT remove the peer from the AddrBook.
func (ps *pstoreds) RemovePeer(p peer.ID) {
ps.dsKeyBook.RemovePeer(p)
ps.dsProtoBook.RemovePeer(p)
ps.dsPeerMetadata.RemovePeer(p)
ps.Metrics.RemovePeer(p)
func (ps *pstoreds) RemovePeer(ctx context.Context, p peer.ID) {
ps.dsKeyBook.RemovePeer(ctx, p)
ps.dsProtoBook.RemovePeer(ctx, p)
ps.dsPeerMetadata.RemovePeer(ctx, p)
ps.Metrics.RemovePeer(ctx, p)
}

View File

@@ -1,6 +1,7 @@
package pstoreds
import (
"context"
"errors"
"fmt"
"sync"
@@ -59,7 +60,7 @@ func NewProtoBook(meta pstore.PeerMetadata, opts ...ProtoBookOption) (*dsProtoBo
return pb, nil
}
func (pb *dsProtoBook) SetProtocols(p peer.ID, protos ...protocol.ID) error {
func (pb *dsProtoBook) SetProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) error {
if len(protos) > pb.maxProtos {
return errTooManyProtocols
}
@@ -73,15 +74,15 @@ func (pb *dsProtoBook) SetProtocols(p peer.ID, protos ...protocol.ID) error {
s.Lock()
defer s.Unlock()
return pb.meta.Put(p, "protocols", protomap)
return pb.meta.Put(ctx, p, "protocols", protomap)
}
func (pb *dsProtoBook) AddProtocols(p peer.ID, protos ...protocol.ID) error {
func (pb *dsProtoBook) AddProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) error {
s := pb.segments.get(p)
s.Lock()
defer s.Unlock()
pmap, err := pb.getProtocolMap(p)
pmap, err := pb.getProtocolMap(ctx, p)
if err != nil {
return err
}
@@ -93,15 +94,15 @@ func (pb *dsProtoBook) AddProtocols(p peer.ID, protos ...protocol.ID) error {
pmap[proto] = struct{}{}
}
return pb.meta.Put(p, "protocols", pmap)
return pb.meta.Put(ctx, p, "protocols", pmap)
}
func (pb *dsProtoBook) GetProtocols(p peer.ID) ([]protocol.ID, error) {
func (pb *dsProtoBook) GetProtocols(ctx context.Context, p peer.ID) ([]protocol.ID, error) {
s := pb.segments.get(p)
s.RLock()
defer s.RUnlock()
pmap, err := pb.getProtocolMap(p)
pmap, err := pb.getProtocolMap(ctx, p)
if err != nil {
return nil, err
}
@@ -114,12 +115,12 @@ func (pb *dsProtoBook) GetProtocols(p peer.ID) ([]protocol.ID, error) {
return res, nil
}
func (pb *dsProtoBook) SupportsProtocols(p peer.ID, protos ...protocol.ID) ([]protocol.ID, error) {
func (pb *dsProtoBook) SupportsProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) ([]protocol.ID, error) {
s := pb.segments.get(p)
s.RLock()
defer s.RUnlock()
pmap, err := pb.getProtocolMap(p)
pmap, err := pb.getProtocolMap(ctx, p)
if err != nil {
return nil, err
}
@@ -134,12 +135,12 @@ func (pb *dsProtoBook) SupportsProtocols(p peer.ID, protos ...protocol.ID) ([]pr
return res, nil
}
func (pb *dsProtoBook) FirstSupportedProtocol(p peer.ID, protos ...protocol.ID) (protocol.ID, error) {
func (pb *dsProtoBook) FirstSupportedProtocol(ctx context.Context, p peer.ID, protos ...protocol.ID) (protocol.ID, error) {
s := pb.segments.get(p)
s.RLock()
defer s.RUnlock()
pmap, err := pb.getProtocolMap(p)
pmap, err := pb.getProtocolMap(ctx, p)
if err != nil {
return "", err
}
@@ -152,12 +153,12 @@ func (pb *dsProtoBook) FirstSupportedProtocol(p peer.ID, protos ...protocol.ID)
return "", nil
}
func (pb *dsProtoBook) RemoveProtocols(p peer.ID, protos ...protocol.ID) error {
func (pb *dsProtoBook) RemoveProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) error {
s := pb.segments.get(p)
s.Lock()
defer s.Unlock()
pmap, err := pb.getProtocolMap(p)
pmap, err := pb.getProtocolMap(ctx, p)
if err != nil {
return err
}
@@ -171,11 +172,11 @@ func (pb *dsProtoBook) RemoveProtocols(p peer.ID, protos ...protocol.ID) error {
delete(pmap, proto)
}
return pb.meta.Put(p, "protocols", pmap)
return pb.meta.Put(ctx, p, "protocols", pmap)
}
func (pb *dsProtoBook) getProtocolMap(p peer.ID) (map[protocol.ID]struct{}, error) {
iprotomap, err := pb.meta.Get(p, "protocols")
func (pb *dsProtoBook) getProtocolMap(ctx context.Context, p peer.ID) (map[protocol.ID]struct{}, error) {
iprotomap, err := pb.meta.Get(ctx, p, "protocols")
switch err {
default:
return nil, err
@@ -191,6 +192,6 @@ func (pb *dsProtoBook) getProtocolMap(p peer.ID) (map[protocol.ID]struct{}, erro
}
}
func (pb *dsProtoBook) RemovePeer(p peer.ID) {
pb.meta.RemovePeer(p)
func (pb *dsProtoBook) RemovePeer(ctx context.Context, p peer.ID) {
pb.meta.RemovePeer(ctx, p)
}

View File

@@ -148,7 +148,7 @@ func (mab *memoryAddrBook) gc() {
}
}
func (mab *memoryAddrBook) PeersWithAddrs() peer.IDSlice {
func (mab *memoryAddrBook) PeersWithAddrs(ctx context.Context) peer.IDSlice {
// deduplicate, since the same peer could have both signed & unsigned addrs
set := make(map[peer.ID]struct{})
for _, s := range mab.segments {
@@ -168,14 +168,14 @@ func (mab *memoryAddrBook) PeersWithAddrs() peer.IDSlice {
}
// AddAddr calls AddAddrs(p, []ma.Multiaddr{addr}, ttl)
func (mab *memoryAddrBook) AddAddr(p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
mab.AddAddrs(p, []ma.Multiaddr{addr}, ttl)
func (mab *memoryAddrBook) AddAddr(ctx context.Context, p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
mab.AddAddrs(ctx, p, []ma.Multiaddr{addr}, ttl)
}
// AddAddrs gives memoryAddrBook addresses to use, with a given ttl
// (time-to-live), after which the address is no longer valid.
// This function never reduces the TTL or expiration of an address.
func (mab *memoryAddrBook) AddAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
func (mab *memoryAddrBook) AddAddrs(ctx context.Context, p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
// if we have a valid peer record, ignore unsigned addrs
// peerRec := mab.GetPeerRecord(p)
// if peerRec != nil {
@@ -187,7 +187,7 @@ func (mab *memoryAddrBook) AddAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Du
// ConsumePeerRecord adds addresses from a signed peer.PeerRecord (contained in
// a record.Envelope), which will expire after the given TTL.
// See https://godoc.org/github.com/libp2p/go-libp2p/core/peerstore#CertifiedAddrBook for more details.
func (mab *memoryAddrBook) ConsumePeerRecord(recordEnvelope *record.Envelope, ttl time.Duration) (bool, error) {
func (mab *memoryAddrBook) ConsumePeerRecord(ctx context.Context, recordEnvelope *record.Envelope, ttl time.Duration) (bool, error) {
r, err := recordEnvelope.Record()
if err != nil {
return false, err
@@ -269,13 +269,13 @@ func (mab *memoryAddrBook) addAddrsUnlocked(s *addrSegment, p peer.ID, addrs []m
}
// SetAddr calls mgr.SetAddrs(p, addr, ttl)
func (mab *memoryAddrBook) SetAddr(p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
mab.SetAddrs(p, []ma.Multiaddr{addr}, ttl)
func (mab *memoryAddrBook) SetAddr(ctx context.Context, p peer.ID, addr ma.Multiaddr, ttl time.Duration) {
mab.SetAddrs(ctx, p, []ma.Multiaddr{addr}, ttl)
}
// SetAddrs sets the ttl on addresses. This clears any TTL there previously.
// This is used when we receive the best estimate of the validity of an address.
func (mab *memoryAddrBook) SetAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
func (mab *memoryAddrBook) SetAddrs(ctx context.Context, p peer.ID, addrs []ma.Multiaddr, ttl time.Duration) {
s := mab.segments.get(p)
s.Lock()
defer s.Unlock()
@@ -312,7 +312,7 @@ func (mab *memoryAddrBook) SetAddrs(p peer.ID, addrs []ma.Multiaddr, ttl time.Du
// UpdateAddrs updates the addresses associated with the given peer that have
// the given oldTTL to have the given newTTL.
func (mab *memoryAddrBook) UpdateAddrs(p peer.ID, oldTTL time.Duration, newTTL time.Duration) {
func (mab *memoryAddrBook) UpdateAddrs(ctx context.Context, p peer.ID, oldTTL time.Duration, newTTL time.Duration) {
s := mab.segments.get(p)
s.Lock()
defer s.Unlock()
@@ -336,7 +336,7 @@ func (mab *memoryAddrBook) UpdateAddrs(p peer.ID, oldTTL time.Duration, newTTL t
}
// Addrs returns all known (and valid) addresses for a given peer
func (mab *memoryAddrBook) Addrs(p peer.ID) []ma.Multiaddr {
func (mab *memoryAddrBook) Addrs(ctx context.Context, p peer.ID) []ma.Multiaddr {
s := mab.segments.get(p)
s.RLock()
defer s.RUnlock()
@@ -361,7 +361,7 @@ func validAddrs(now time.Time, amap map[string]*expiringAddr) []ma.Multiaddr {
// GetPeerRecord returns a Envelope containing a PeerRecord for the
// given peer id, if one exists.
// Returns nil if no signed PeerRecord exists for the peer.
func (mab *memoryAddrBook) GetPeerRecord(p peer.ID) *record.Envelope {
func (mab *memoryAddrBook) GetPeerRecord(ctx context.Context, p peer.ID) *record.Envelope {
s := mab.segments.get(p)
s.RLock()
defer s.RUnlock()
@@ -381,7 +381,7 @@ func (mab *memoryAddrBook) GetPeerRecord(p peer.ID) *record.Envelope {
}
// ClearAddrs removes all previously stored addresses
func (mab *memoryAddrBook) ClearAddrs(p peer.ID) {
func (mab *memoryAddrBook) ClearAddrs(ctx context.Context, p peer.ID) {
s := mab.segments.get(p)
s.Lock()
defer s.Unlock()

View File

@@ -1,6 +1,7 @@
package pstoremem
import (
"context"
"errors"
"sync"
@@ -24,7 +25,7 @@ func NewKeyBook() *memoryKeyBook {
}
}
func (mkb *memoryKeyBook) PeersWithKeys() peer.IDSlice {
func (mkb *memoryKeyBook) PeersWithKeys(ctx context.Context) peer.IDSlice {
mkb.RLock()
ps := make(peer.IDSlice, 0, len(mkb.pks)+len(mkb.sks))
for p := range mkb.pks {
@@ -39,7 +40,7 @@ func (mkb *memoryKeyBook) PeersWithKeys() peer.IDSlice {
return ps
}
func (mkb *memoryKeyBook) PubKey(p peer.ID) ic.PubKey {
func (mkb *memoryKeyBook) PubKey(ctx context.Context, p peer.ID) ic.PubKey {
mkb.RLock()
pk := mkb.pks[p]
mkb.RUnlock()
@@ -55,7 +56,7 @@ func (mkb *memoryKeyBook) PubKey(p peer.ID) ic.PubKey {
return pk
}
func (mkb *memoryKeyBook) AddPubKey(p peer.ID, pk ic.PubKey) error {
func (mkb *memoryKeyBook) AddPubKey(ctx context.Context, p peer.ID, pk ic.PubKey) error {
// check it's correct first
if !p.MatchesPublicKey(pk) {
return errors.New("ID does not match PublicKey")
@@ -67,13 +68,13 @@ func (mkb *memoryKeyBook) AddPubKey(p peer.ID, pk ic.PubKey) error {
return nil
}
func (mkb *memoryKeyBook) PrivKey(p peer.ID) ic.PrivKey {
func (mkb *memoryKeyBook) PrivKey(ctx context.Context, p peer.ID) ic.PrivKey {
mkb.RLock()
defer mkb.RUnlock()
return mkb.sks[p]
}
func (mkb *memoryKeyBook) AddPrivKey(p peer.ID, sk ic.PrivKey) error {
func (mkb *memoryKeyBook) AddPrivKey(ctx context.Context, p peer.ID, sk ic.PrivKey) error {
if sk == nil {
return errors.New("sk is nil (PrivKey)")
}
@@ -89,7 +90,7 @@ func (mkb *memoryKeyBook) AddPrivKey(p peer.ID, sk ic.PrivKey) error {
return nil
}
func (mkb *memoryKeyBook) RemovePeer(p peer.ID) {
func (mkb *memoryKeyBook) RemovePeer(ctx context.Context, p peer.ID) {
mkb.Lock()
delete(mkb.sks, p)
delete(mkb.pks, p)

View File

@@ -1,6 +1,7 @@
package pstoremem
import (
"context"
"sync"
"github.com/libp2p/go-libp2p/core/peer"
@@ -21,7 +22,7 @@ func NewPeerMetadata() *memoryPeerMetadata {
}
}
func (ps *memoryPeerMetadata) Put(p peer.ID, key string, val interface{}) error {
func (ps *memoryPeerMetadata) Put(ctx context.Context, p peer.ID, key string, val interface{}) error {
ps.dslock.Lock()
defer ps.dslock.Unlock()
m, ok := ps.ds[p]
@@ -33,7 +34,7 @@ func (ps *memoryPeerMetadata) Put(p peer.ID, key string, val interface{}) error
return nil
}
func (ps *memoryPeerMetadata) Get(p peer.ID, key string) (interface{}, error) {
func (ps *memoryPeerMetadata) Get(ctx context.Context, p peer.ID, key string) (interface{}, error) {
ps.dslock.RLock()
defer ps.dslock.RUnlock()
m, ok := ps.ds[p]
@@ -47,7 +48,7 @@ func (ps *memoryPeerMetadata) Get(p peer.ID, key string) (interface{}, error) {
return val, nil
}
func (ps *memoryPeerMetadata) RemovePeer(p peer.ID) {
func (ps *memoryPeerMetadata) RemovePeer(ctx context.Context, p peer.ID) {
ps.dslock.Lock()
delete(ps.ds, p)
ps.dslock.Unlock()

View File

@@ -1,6 +1,7 @@
package pstoremem
import (
"context"
"fmt"
"io"
@@ -77,12 +78,12 @@ func (ps *pstoremem) Close() (err error) {
return nil
}
func (ps *pstoremem) Peers() peer.IDSlice {
func (ps *pstoremem) Peers(ctx context.Context) peer.IDSlice {
set := map[peer.ID]struct{}{}
for _, p := range ps.PeersWithKeys() {
for _, p := range ps.PeersWithKeys(ctx) {
set[p] = struct{}{}
}
for _, p := range ps.PeersWithAddrs() {
for _, p := range ps.PeersWithAddrs(ctx) {
set[p] = struct{}{}
}
@@ -93,10 +94,10 @@ func (ps *pstoremem) Peers() peer.IDSlice {
return pps
}
func (ps *pstoremem) PeerInfo(p peer.ID) peer.AddrInfo {
func (ps *pstoremem) PeerInfo(ctx context.Context, p peer.ID) peer.AddrInfo {
return peer.AddrInfo{
ID: p,
Addrs: ps.memoryAddrBook.Addrs(p),
Addrs: ps.memoryAddrBook.Addrs(ctx, p),
}
}
@@ -106,9 +107,9 @@ func (ps *pstoremem) PeerInfo(p peer.ID) peer.AddrInfo {
// * the PeerMetadata
// * the Metrics
// It DOES NOT remove the peer from the AddrBook.
func (ps *pstoremem) RemovePeer(p peer.ID) {
ps.memoryKeyBook.RemovePeer(p)
ps.memoryProtoBook.RemovePeer(p)
ps.memoryPeerMetadata.RemovePeer(p)
ps.Metrics.RemovePeer(p)
func (ps *pstoremem) RemovePeer(ctx context.Context, p peer.ID) {
ps.memoryKeyBook.RemovePeer(ctx, p)
ps.memoryProtoBook.RemovePeer(ctx, p)
ps.memoryPeerMetadata.RemovePeer(ctx, p)
ps.Metrics.RemovePeer(ctx, p)
}

View File

@@ -1,6 +1,7 @@
package pstoremem
import (
"context"
"errors"
"sync"
@@ -88,7 +89,7 @@ func (pb *memoryProtoBook) internProtocol(proto protocol.ID) protocol.ID {
return proto
}
func (pb *memoryProtoBook) SetProtocols(p peer.ID, protos ...protocol.ID) error {
func (pb *memoryProtoBook) SetProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) error {
if len(protos) > pb.maxProtos {
return errTooManyProtocols
}
@@ -106,7 +107,7 @@ func (pb *memoryProtoBook) SetProtocols(p peer.ID, protos ...protocol.ID) error
return nil
}
func (pb *memoryProtoBook) AddProtocols(p peer.ID, protos ...protocol.ID) error {
func (pb *memoryProtoBook) AddProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) error {
s := pb.segments.get(p)
s.Lock()
defer s.Unlock()
@@ -126,7 +127,7 @@ func (pb *memoryProtoBook) AddProtocols(p peer.ID, protos ...protocol.ID) error
return nil
}
func (pb *memoryProtoBook) GetProtocols(p peer.ID) ([]protocol.ID, error) {
func (pb *memoryProtoBook) GetProtocols(ctx context.Context, p peer.ID) ([]protocol.ID, error) {
s := pb.segments.get(p)
s.RLock()
defer s.RUnlock()
@@ -139,7 +140,7 @@ func (pb *memoryProtoBook) GetProtocols(p peer.ID) ([]protocol.ID, error) {
return out, nil
}
func (pb *memoryProtoBook) RemoveProtocols(p peer.ID, protos ...protocol.ID) error {
func (pb *memoryProtoBook) RemoveProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) error {
s := pb.segments.get(p)
s.Lock()
defer s.Unlock()
@@ -156,7 +157,7 @@ func (pb *memoryProtoBook) RemoveProtocols(p peer.ID, protos ...protocol.ID) err
return nil
}
func (pb *memoryProtoBook) SupportsProtocols(p peer.ID, protos ...protocol.ID) ([]protocol.ID, error) {
func (pb *memoryProtoBook) SupportsProtocols(ctx context.Context, p peer.ID, protos ...protocol.ID) ([]protocol.ID, error) {
s := pb.segments.get(p)
s.RLock()
defer s.RUnlock()
@@ -171,7 +172,7 @@ func (pb *memoryProtoBook) SupportsProtocols(p peer.ID, protos ...protocol.ID) (
return out, nil
}
func (pb *memoryProtoBook) FirstSupportedProtocol(p peer.ID, protos ...protocol.ID) (protocol.ID, error) {
func (pb *memoryProtoBook) FirstSupportedProtocol(ctx context.Context, p peer.ID, protos ...protocol.ID) (protocol.ID, error) {
s := pb.segments.get(p)
s.RLock()
defer s.RUnlock()
@@ -184,7 +185,7 @@ func (pb *memoryProtoBook) FirstSupportedProtocol(p peer.ID, protos ...protocol.
return "", nil
}
func (pb *memoryProtoBook) RemovePeer(p peer.ID) {
func (pb *memoryProtoBook) RemovePeer(ctx context.Context, p peer.ID) {
s := pb.segments.get(p)
s.Lock()
delete(s.protocols, p)

View File

@@ -1,6 +1,7 @@
package test
import (
"context"
"testing"
"time"
@@ -49,105 +50,105 @@ func testAddAddress(ab pstore.AddrBook, clk *mockClock.Mock) func(*testing.T) {
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(1)
ab.AddAddr(id, addrs[0], time.Hour)
ab.AddAddr(context.Background(), id, addrs[0], time.Hour)
AssertAddressesEqual(t, addrs, ab.Addrs(id))
AssertAddressesEqual(t, addrs, ab.Addrs(context.Background(), id))
})
t.Run("idempotent add single address", func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(1)
ab.AddAddr(id, addrs[0], time.Hour)
ab.AddAddr(id, addrs[0], time.Hour)
ab.AddAddr(context.Background(), id, addrs[0], time.Hour)
ab.AddAddr(context.Background(), id, addrs[0], time.Hour)
AssertAddressesEqual(t, addrs, ab.Addrs(id))
AssertAddressesEqual(t, addrs, ab.Addrs(context.Background(), id))
})
t.Run("add multiple addresses", func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(3)
ab.AddAddrs(id, addrs, time.Hour)
AssertAddressesEqual(t, addrs, ab.Addrs(id))
ab.AddAddrs(context.Background(), id, addrs, time.Hour)
AssertAddressesEqual(t, addrs, ab.Addrs(context.Background(), id))
})
t.Run("idempotent add multiple addresses", func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(3)
ab.AddAddrs(id, addrs, time.Hour)
ab.AddAddrs(id, addrs, time.Hour)
ab.AddAddrs(context.Background(), id, addrs, time.Hour)
ab.AddAddrs(context.Background(), id, addrs, time.Hour)
AssertAddressesEqual(t, addrs, ab.Addrs(id))
AssertAddressesEqual(t, addrs, ab.Addrs(context.Background(), id))
})
t.Run("adding an existing address with a later expiration extends its ttl", func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(3)
ab.AddAddrs(id, addrs, time.Second)
ab.AddAddrs(context.Background(), id, addrs, time.Second)
// same address as before but with a higher TTL
ab.AddAddrs(id, addrs[2:], time.Hour)
ab.AddAddrs(context.Background(), id, addrs[2:], time.Hour)
// after the initial TTL has expired, check that only the third address is present.
clk.Add(1200 * time.Millisecond)
AssertAddressesEqual(t, addrs[2:], ab.Addrs(id))
AssertAddressesEqual(t, addrs[2:], ab.Addrs(context.Background(), id))
// make sure we actually set the TTL
ab.UpdateAddrs(id, time.Hour, 0)
AssertAddressesEqual(t, nil, ab.Addrs(id))
ab.UpdateAddrs(context.Background(), id, time.Hour, 0)
AssertAddressesEqual(t, nil, ab.Addrs(context.Background(), id))
})
t.Run("adding an existing address with an earlier expiration never reduces the expiration", func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(3)
ab.AddAddrs(id, addrs, time.Hour)
ab.AddAddrs(context.Background(), id, addrs, time.Hour)
// same address as before but with a lower TTL
ab.AddAddrs(id, addrs[2:], time.Second)
ab.AddAddrs(context.Background(), id, addrs[2:], time.Second)
// after the initial TTL has expired, check that all three addresses are still present (i.e. the TTL on
// the modified one was not shortened).
clk.Add(2100 * time.Millisecond)
AssertAddressesEqual(t, addrs, ab.Addrs(id))
AssertAddressesEqual(t, addrs, ab.Addrs(context.Background(), id))
})
t.Run("adding an existing address with an earlier expiration never reduces the TTL", func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(1)
ab.AddAddrs(id, addrs, 4*time.Second)
ab.AddAddrs(context.Background(), id, addrs, 4*time.Second)
// 4 seconds left
clk.Add(2 * time.Second)
// 2 second left
ab.AddAddrs(id, addrs, 3*time.Second)
ab.AddAddrs(context.Background(), id, addrs, 3*time.Second)
// 3 seconds left
clk.Add(1 * time.Second)
// 2 seconds left.
// We still have the address.
AssertAddressesEqual(t, addrs, ab.Addrs(id))
AssertAddressesEqual(t, addrs, ab.Addrs(context.Background(), id))
// The TTL wasn't reduced
ab.UpdateAddrs(id, 4*time.Second, 0)
AssertAddressesEqual(t, nil, ab.Addrs(id))
ab.UpdateAddrs(context.Background(), id, 4*time.Second, 0)
AssertAddressesEqual(t, nil, ab.Addrs(context.Background(), id))
})
t.Run("accessing an empty peer ID", func(t *testing.T) {
addrs := GenerateAddrs(5)
ab.AddAddrs("", addrs, time.Hour)
AssertAddressesEqual(t, addrs, ab.Addrs(""))
ab.AddAddrs(context.Background(), "", addrs, time.Hour)
AssertAddressesEqual(t, addrs, ab.Addrs(context.Background(), ""))
})
t.Run("add a /p2p address with valid peerid", func(t *testing.T) {
peerId := GeneratePeerIDs(1)[0]
addr := GenerateAddrs(1)
p2pAddr := addr[0].Encapsulate(Multiaddr("/p2p/" + peerId.String()))
ab.AddAddr(peerId, p2pAddr, time.Hour)
AssertAddressesEqual(t, addr, ab.Addrs(peerId))
ab.AddAddr(context.Background(), peerId, p2pAddr, time.Hour)
AssertAddressesEqual(t, addr, ab.Addrs(context.Background(), peerId))
})
t.Run("add a /p2p address with invalid peerid", func(t *testing.T) {
@@ -156,8 +157,8 @@ func testAddAddress(ab pstore.AddrBook, clk *mockClock.Mock) func(*testing.T) {
pid2 := pids[1]
addr := GenerateAddrs(1)
p2pAddr := addr[0].Encapsulate(Multiaddr("/p2p/" + pid1.String()))
ab.AddAddr(pid2, p2pAddr, time.Hour)
AssertAddressesEqual(t, nil, ab.Addrs(pid2))
ab.AddAddr(context.Background(), pid2, p2pAddr, time.Hour)
AssertAddressesEqual(t, nil, ab.Addrs(context.Background(), pid2))
})
}
}
@@ -167,19 +168,19 @@ func testClearWorks(ab pstore.AddrBook, clk *mockClock.Mock) func(t *testing.T)
ids := GeneratePeerIDs(2)
addrs := GenerateAddrs(5)
ab.AddAddrs(ids[0], addrs[0:3], time.Hour)
ab.AddAddrs(ids[1], addrs[3:], time.Hour)
ab.AddAddrs(context.Background(), ids[0], addrs[0:3], time.Hour)
ab.AddAddrs(context.Background(), ids[1], addrs[3:], time.Hour)
AssertAddressesEqual(t, addrs[0:3], ab.Addrs(ids[0]))
AssertAddressesEqual(t, addrs[3:], ab.Addrs(ids[1]))
AssertAddressesEqual(t, addrs[0:3], ab.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs[3:], ab.Addrs(context.Background(), ids[1]))
ab.ClearAddrs(ids[0])
AssertAddressesEqual(t, nil, ab.Addrs(ids[0]))
AssertAddressesEqual(t, addrs[3:], ab.Addrs(ids[1]))
ab.ClearAddrs(context.Background(), ids[0])
AssertAddressesEqual(t, nil, ab.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs[3:], ab.Addrs(context.Background(), ids[1]))
ab.ClearAddrs(ids[1])
AssertAddressesEqual(t, nil, ab.Addrs(ids[0]))
AssertAddressesEqual(t, nil, ab.Addrs(ids[1]))
ab.ClearAddrs(context.Background(), ids[1])
AssertAddressesEqual(t, nil, ab.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, nil, ab.Addrs(context.Background(), ids[1]))
}
}
@@ -188,41 +189,41 @@ func testSetNegativeTTLClears(m pstore.AddrBook, clk *mockClock.Mock) func(t *te
id := GeneratePeerIDs(1)[0]
addrs := GenerateAddrs(100)
m.SetAddrs(id, addrs, time.Hour)
AssertAddressesEqual(t, addrs, m.Addrs(id))
m.SetAddrs(context.Background(), id, addrs, time.Hour)
AssertAddressesEqual(t, addrs, m.Addrs(context.Background(), id))
// remove two addresses.
m.SetAddr(id, addrs[50], -1)
m.SetAddr(id, addrs[75], -1)
m.SetAddr(context.Background(), id, addrs[50], -1)
m.SetAddr(context.Background(), id, addrs[75], -1)
// calculate the survivors
survivors := append(addrs[0:50], addrs[51:]...)
survivors = append(survivors[0:74], survivors[75:]...)
AssertAddressesEqual(t, survivors, m.Addrs(id))
AssertAddressesEqual(t, survivors, m.Addrs(context.Background(), id))
// remove _all_ the addresses
m.SetAddrs(id, survivors, -1)
if len(m.Addrs(id)) != 0 {
m.SetAddrs(context.Background(), id, survivors, -1)
if len(m.Addrs(context.Background(), id)) != 0 {
t.Error("expected empty address list after clearing all addresses")
}
// add half, but try to remove more than we added
m.SetAddrs(id, addrs[:50], time.Hour)
m.SetAddrs(id, addrs, -1)
if len(m.Addrs(id)) != 0 {
m.SetAddrs(context.Background(), id, addrs[:50], time.Hour)
m.SetAddrs(context.Background(), id, addrs, -1)
if len(m.Addrs(context.Background(), id)) != 0 {
t.Error("expected empty address list after clearing all addresses")
}
// try to remove the same addr multiple times
m.SetAddrs(id, addrs[:5], time.Hour)
m.SetAddrs(context.Background(), id, addrs[:5], time.Hour)
repeated := make([]multiaddr.Multiaddr, 10)
for i := 0; i < len(repeated); i++ {
repeated[i] = addrs[0]
}
m.SetAddrs(id, repeated, -1)
if len(m.Addrs(id)) != 4 {
t.Errorf("expected 4 addrs after removing one, got %d", len(m.Addrs(id)))
m.SetAddrs(context.Background(), id, repeated, -1)
if len(m.Addrs(context.Background(), id)) != 4 {
t.Errorf("expected 4 addrs after removing one, got %d", len(m.Addrs(context.Background(), id)))
}
}
}
@@ -233,7 +234,7 @@ func testUpdateTTLs(m pstore.AddrBook, clk *mockClock.Mock) func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
// Shouldn't panic.
m.UpdateAddrs(id, time.Hour, time.Minute)
m.UpdateAddrs(context.Background(), id, time.Hour, time.Minute)
})
t.Run("update to 0 clears addrs", func(t *testing.T) {
@@ -241,9 +242,9 @@ func testUpdateTTLs(m pstore.AddrBook, clk *mockClock.Mock) func(t *testing.T) {
addrs := GenerateAddrs(1)
// Shouldn't panic.
m.SetAddrs(id, addrs, time.Hour)
m.UpdateAddrs(id, time.Hour, 0)
if len(m.Addrs(id)) != 0 {
m.SetAddrs(context.Background(), id, addrs, time.Hour)
m.UpdateAddrs(context.Background(), id, time.Hour, 0)
if len(m.Addrs(context.Background(), id)) != 0 {
t.Error("expected no addresses")
}
})
@@ -253,41 +254,41 @@ func testUpdateTTLs(m pstore.AddrBook, clk *mockClock.Mock) func(t *testing.T) {
addrs1, addrs2 := GenerateAddrs(2), GenerateAddrs(2)
// set two keys with different ttls for each peer.
m.SetAddr(ids[0], addrs1[0], time.Hour)
m.SetAddr(ids[0], addrs1[1], time.Minute)
m.SetAddr(ids[1], addrs2[0], time.Hour)
m.SetAddr(ids[1], addrs2[1], time.Minute)
m.SetAddr(context.Background(), ids[0], addrs1[0], time.Hour)
m.SetAddr(context.Background(), ids[0], addrs1[1], time.Minute)
m.SetAddr(context.Background(), ids[1], addrs2[0], time.Hour)
m.SetAddr(context.Background(), ids[1], addrs2[1], time.Minute)
// Sanity check.
AssertAddressesEqual(t, addrs1, m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1, m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
// Will only affect addrs1[0].
// Badger does not support subsecond TTLs.
// https://github.com/dgraph-io/badger/issues/339
m.UpdateAddrs(ids[0], time.Hour, 1*time.Second)
m.UpdateAddrs(context.Background(), ids[0], time.Hour, 1*time.Second)
// No immediate effect.
AssertAddressesEqual(t, addrs1, m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1, m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
// After a wait, addrs[0] is gone.
clk.Add(2 * time.Second)
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
// Will only affect addrs2[0].
m.UpdateAddrs(ids[1], time.Hour, 1*time.Second)
m.UpdateAddrs(context.Background(), ids[1], time.Hour, 1*time.Second)
// No immediate effect.
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
clk.Add(2 * time.Second)
// First addrs is gone in both.
AssertAddressesEqual(t, addrs1[1:], m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2[1:], m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1[1:], m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2[1:], m.Addrs(context.Background(), ids[1]))
})
}
@@ -297,8 +298,8 @@ func testNilAddrsDontBreak(m pstore.AddrBook, clk *mockClock.Mock) func(t *testi
return func(t *testing.T) {
id := GeneratePeerIDs(1)[0]
m.SetAddr(id, nil, time.Hour)
m.AddAddr(id, nil, time.Hour)
m.SetAddr(context.Background(), id, nil, time.Hour)
m.AddAddr(context.Background(), id, nil, time.Hour)
}
}
@@ -308,42 +309,42 @@ func testAddressesExpire(m pstore.AddrBook, clk *mockClock.Mock) func(t *testing
addrs1 := GenerateAddrs(3)
addrs2 := GenerateAddrs(2)
m.AddAddrs(ids[0], addrs1, time.Hour)
m.AddAddrs(ids[1], addrs2, time.Hour)
m.AddAddrs(context.Background(), ids[0], addrs1, time.Hour)
m.AddAddrs(context.Background(), ids[1], addrs2, time.Hour)
AssertAddressesEqual(t, addrs1, m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1, m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
m.AddAddrs(ids[0], addrs1, 2*time.Hour)
m.AddAddrs(ids[1], addrs2, 2*time.Hour)
m.AddAddrs(context.Background(), ids[0], addrs1, 2*time.Hour)
m.AddAddrs(context.Background(), ids[1], addrs2, 2*time.Hour)
AssertAddressesEqual(t, addrs1, m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1, m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
m.SetAddr(ids[0], addrs1[0], 100*time.Microsecond)
m.SetAddr(context.Background(), ids[0], addrs1[0], 100*time.Microsecond)
clk.Add(100 * time.Millisecond)
AssertAddressesEqual(t, addrs1[1:3], m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1[1:3], m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
m.SetAddr(ids[0], addrs1[2], 100*time.Microsecond)
m.SetAddr(context.Background(), ids[0], addrs1[2], 100*time.Microsecond)
clk.Add(100 * time.Millisecond)
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2, m.Addrs(context.Background(), ids[1]))
m.SetAddr(ids[1], addrs2[0], 100*time.Microsecond)
m.SetAddr(context.Background(), ids[1], addrs2[0], 100*time.Microsecond)
clk.Add(100 * time.Millisecond)
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(ids[0]))
AssertAddressesEqual(t, addrs2[1:], m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, addrs2[1:], m.Addrs(context.Background(), ids[1]))
m.SetAddr(ids[1], addrs2[1], 100*time.Microsecond)
m.SetAddr(context.Background(), ids[1], addrs2[1], 100*time.Microsecond)
clk.Add(100 * time.Millisecond)
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(ids[0]))
AssertAddressesEqual(t, nil, m.Addrs(ids[1]))
AssertAddressesEqual(t, addrs1[1:2], m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, nil, m.Addrs(context.Background(), ids[1]))
m.SetAddr(ids[0], addrs1[1], 100*time.Microsecond)
m.SetAddr(context.Background(), ids[0], addrs1[1], 100*time.Microsecond)
clk.Add(100 * time.Millisecond)
AssertAddressesEqual(t, nil, m.Addrs(ids[0]))
AssertAddressesEqual(t, nil, m.Addrs(ids[1]))
AssertAddressesEqual(t, nil, m.Addrs(context.Background(), ids[0]))
AssertAddressesEqual(t, nil, m.Addrs(context.Background(), ids[1]))
}
}
@@ -353,22 +354,22 @@ func testClearWithIterator(m pstore.AddrBook, clk *mockClock.Mock) func(t *testi
addrs := GenerateAddrs(100)
// Add the peers with 50 addresses each.
m.AddAddrs(ids[0], addrs[:50], pstore.PermanentAddrTTL)
m.AddAddrs(ids[1], addrs[50:], pstore.PermanentAddrTTL)
m.AddAddrs(context.Background(), ids[0], addrs[:50], pstore.PermanentAddrTTL)
m.AddAddrs(context.Background(), ids[1], addrs[50:], pstore.PermanentAddrTTL)
if all := append(m.Addrs(ids[0]), m.Addrs(ids[1])...); len(all) != 100 {
if all := append(m.Addrs(context.Background(), ids[0]), m.Addrs(context.Background(), ids[1])...); len(all) != 100 {
t.Fatal("expected pstore to contain both peers with all their maddrs")
}
// Since we don't fetch these peers, they won't be present in cache.
m.ClearAddrs(ids[0])
if all := append(m.Addrs(ids[0]), m.Addrs(ids[1])...); len(all) != 50 {
m.ClearAddrs(context.Background(), ids[0])
if all := append(m.Addrs(context.Background(), ids[0]), m.Addrs(context.Background(), ids[1])...); len(all) != 50 {
t.Fatal("expected pstore to contain only addrs of peer 2")
}
m.ClearAddrs(ids[1])
if all := append(m.Addrs(ids[0]), m.Addrs(ids[1])...); len(all) != 0 {
m.ClearAddrs(context.Background(), ids[1])
if all := append(m.Addrs(context.Background(), ids[0]), m.Addrs(context.Background(), ids[1])...); len(all) != 0 {
t.Fatal("expected pstore to contain no addresses")
}
}
@@ -381,7 +382,7 @@ func testPeersWithAddrs(m pstore.AddrBook, clk *mockClock.Mock) func(t *testing.
// see https://blog.golang.org/subtests
t.Run("empty addrbook", func(t *testing.T) {
if peers := m.PeersWithAddrs(); len(peers) != 0 {
if peers := m.PeersWithAddrs(context.Background()); len(peers) != 0 {
t.Fatal("expected to find no peers")
}
})
@@ -390,10 +391,10 @@ func testPeersWithAddrs(m pstore.AddrBook, clk *mockClock.Mock) func(t *testing.
ids := GeneratePeerIDs(2)
addrs := GenerateAddrs(10)
m.AddAddrs(ids[0], addrs[:5], pstore.PermanentAddrTTL)
m.AddAddrs(ids[1], addrs[5:], pstore.PermanentAddrTTL)
m.AddAddrs(context.Background(), ids[0], addrs[:5], pstore.PermanentAddrTTL)
m.AddAddrs(context.Background(), ids[1], addrs[5:], pstore.PermanentAddrTTL)
if peers := m.PeersWithAddrs(); len(peers) != 2 {
if peers := m.PeersWithAddrs(context.Background()); len(peers) != 2 {
t.Fatal("expected to find 2 peers")
}
})
@@ -430,13 +431,13 @@ func testCertifiedAddresses(m pstore.AddrBook, clk *mockClock.Mock) func(*testin
}
// add a few non-certified addrs
m.AddAddrs(id, uncertifiedAddrs, time.Hour)
m.AddAddrs(context.Background(), id, uncertifiedAddrs, time.Hour)
// make sure they're present
AssertAddressesEqual(t, uncertifiedAddrs, m.Addrs(id))
AssertAddressesEqual(t, uncertifiedAddrs, m.Addrs(context.Background(), id))
// add the signed record to addr book
accepted, err := cab.ConsumePeerRecord(signedRec2, time.Hour)
accepted, err := cab.ConsumePeerRecord(context.Background(), signedRec2, time.Hour)
if err != nil {
t.Errorf("error adding signed routing record to addrbook: %v", err)
}
@@ -445,16 +446,16 @@ func testCertifiedAddresses(m pstore.AddrBook, clk *mockClock.Mock) func(*testin
}
// the non-certified addrs should be gone & we should get only certified addrs back from Addrs
// AssertAddressesEqual(t, certifiedAddrs, m.Addrs(id))
AssertAddressesEqual(t, allAddrs, m.Addrs(id))
// AssertAddressesEqual(t, certifiedAddrs, m.Addrs(context.Background(), id))
AssertAddressesEqual(t, allAddrs, m.Addrs(context.Background(), id))
// PeersWithAddrs should return a single peer
if len(m.PeersWithAddrs()) != 1 {
t.Errorf("expected PeersWithAddrs to return 1, got %d", len(m.PeersWithAddrs()))
if len(m.PeersWithAddrs(context.Background())) != 1 {
t.Errorf("expected PeersWithAddrs to return 1, got %d", len(m.PeersWithAddrs(context.Background())))
}
// Adding an old record should fail
accepted, err = cab.ConsumePeerRecord(signedRec1, time.Hour)
accepted, err = cab.ConsumePeerRecord(context.Background(), signedRec1, time.Hour)
if accepted {
t.Error("We should have failed to accept a record with an old sequence number")
}
@@ -463,14 +464,14 @@ func testCertifiedAddresses(m pstore.AddrBook, clk *mockClock.Mock) func(*testin
}
// once certified addrs exist, trying to add non-certified addrs should have no effect
// m.AddAddrs(id, uncertifiedAddrs, time.Hour)
// AssertAddressesEqual(t, certifiedAddrs, m.Addrs(id))
// m.AddAddrs(context.Background(), id, uncertifiedAddrs, time.Hour)
// AssertAddressesEqual(t, certifiedAddrs, m.Addrs(context.Background(), id))
// XXX: Disabled until signed records are required
m.AddAddrs(id, uncertifiedAddrs, time.Hour)
AssertAddressesEqual(t, allAddrs, m.Addrs(id))
m.AddAddrs(context.Background(), id, uncertifiedAddrs, time.Hour)
AssertAddressesEqual(t, allAddrs, m.Addrs(context.Background(), id))
// we should be able to retrieve the signed peer record
rec3 := cab.GetPeerRecord(id)
rec3 := cab.GetPeerRecord(context.Background(), id)
if rec3 == nil || !signedRec2.Equal(rec3) {
t.Error("unable to retrieve signed routing record from addrbook")
}
@@ -483,36 +484,36 @@ func testCertifiedAddresses(m pstore.AddrBook, clk *mockClock.Mock) func(*testin
rec4.Addrs = certifiedAddrs
signedRec4, err := record.Seal(rec4, priv)
test.AssertNilError(t, err)
accepted, err = cab.ConsumePeerRecord(signedRec4, time.Hour)
accepted, err = cab.ConsumePeerRecord(context.Background(), signedRec4, time.Hour)
test.AssertNilError(t, err)
if !accepted {
t.Error("expected peer record to be accepted")
}
// AssertAddressesEqual(t, certifiedAddrs, m.Addrs(id))
AssertAddressesEqual(t, allAddrs, m.Addrs(id))
// AssertAddressesEqual(t, certifiedAddrs, m.Addrs(context.Background(), id))
AssertAddressesEqual(t, allAddrs, m.Addrs(context.Background(), id))
// update TTL on signed addrs to -1 to remove them.
// the signed routing record should be deleted
// m.SetAddrs(id, certifiedAddrs, -1)
// m.SetAddrs(context.Background(), id, certifiedAddrs, -1)
// XXX: Disabled until signed records are required
m.SetAddrs(id, allAddrs, -1)
if len(m.Addrs(id)) != 0 {
m.SetAddrs(context.Background(), id, allAddrs, -1)
if len(m.Addrs(context.Background(), id)) != 0 {
t.Error("expected zero certified addrs after setting TTL to -1")
}
if cab.GetPeerRecord(id) != nil {
if cab.GetPeerRecord(context.Background(), id) != nil {
t.Error("expected signed peer record to be removed when addresses expire")
}
// Test that natural TTL expiration clears signed peer records
accepted, err = cab.ConsumePeerRecord(signedRec4, time.Second)
accepted, err = cab.ConsumePeerRecord(context.Background(), signedRec4, time.Second)
if !accepted {
t.Error("expected peer record to be accepted")
}
test.AssertNilError(t, err)
AssertAddressesEqual(t, certifiedAddrs, m.Addrs(id))
AssertAddressesEqual(t, certifiedAddrs, m.Addrs(context.Background(), id))
clk.Add(2 * time.Second)
if cab.GetPeerRecord(id) != nil {
if cab.GetPeerRecord(context.Background(), id) != nil {
t.Error("expected signed peer record to be removed when addresses expire")
}
@@ -522,7 +523,7 @@ func testCertifiedAddresses(m pstore.AddrBook, clk *mockClock.Mock) func(*testin
env, err := record.Seal(rec4, priv2)
test.AssertNilError(t, err)
accepted, err = cab.ConsumePeerRecord(env, time.Second)
accepted, err = cab.ConsumePeerRecord(context.Background(), env, time.Second)
if accepted || err == nil {
t.Error("expected adding a PeerRecord that's signed with the wrong key to fail")
}

View File

@@ -68,7 +68,7 @@ func benchmarkAddAddrs(ps pstore.Peerstore, addrs chan *peerpair) func(*testing.
b.ResetTimer()
for i := 0; i < b.N; i++ {
pp := <-addrs
ps.AddAddrs(pp.ID, pp.Addr, pstore.PermanentAddrTTL)
ps.AddAddrs(context.Background(), pp.ID, pp.Addr, pstore.PermanentAddrTTL)
}
}
}
@@ -78,7 +78,7 @@ func benchmarkSetAddrs(ps pstore.Peerstore, addrs chan *peerpair) func(*testing.
b.ResetTimer()
for i := 0; i < b.N; i++ {
pp := <-addrs
ps.SetAddrs(pp.ID, pp.Addr, pstore.PermanentAddrTTL)
ps.SetAddrs(context.Background(), pp.ID, pp.Addr, pstore.PermanentAddrTTL)
}
}
}
@@ -86,11 +86,11 @@ func benchmarkSetAddrs(ps pstore.Peerstore, addrs chan *peerpair) func(*testing.
func benchmarkGetAddrs(ps pstore.Peerstore, addrs chan *peerpair) func(*testing.B) {
return func(b *testing.B) {
pp := <-addrs
ps.SetAddrs(pp.ID, pp.Addr, pstore.PermanentAddrTTL)
ps.SetAddrs(context.Background(), pp.ID, pp.Addr, pstore.PermanentAddrTTL)
b.ResetTimer()
for i := 0; i < b.N; i++ {
_ = ps.Addrs(pp.ID)
_ = ps.Addrs(context.Background(), pp.ID)
}
}
}
@@ -100,9 +100,9 @@ func benchmarkAddGetAndClearAddrs(ps pstore.Peerstore, addrs chan *peerpair) fun
b.ResetTimer()
for i := 0; i < b.N; i++ {
pp := <-addrs
ps.AddAddrs(pp.ID, pp.Addr, pstore.PermanentAddrTTL)
ps.Addrs(pp.ID)
ps.ClearAddrs(pp.ID)
ps.AddAddrs(context.Background(), pp.ID, pp.Addr, pstore.PermanentAddrTTL)
ps.Addrs(context.Background(), pp.ID)
ps.ClearAddrs(context.Background(), pp.ID)
}
}
}
@@ -112,13 +112,13 @@ func benchmarkGet1000PeersWithAddrs(ps pstore.Peerstore, addrs chan *peerpair) f
var peers = make([]*peerpair, 1000)
for i := range peers {
pp := <-addrs
ps.AddAddrs(pp.ID, pp.Addr, pstore.PermanentAddrTTL)
ps.AddAddrs(context.Background(), pp.ID, pp.Addr, pstore.PermanentAddrTTL)
peers[i] = pp
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
_ = ps.PeersWithAddrs()
_ = ps.PeersWithAddrs(context.Background())
}
}
}

View File

@@ -1,6 +1,7 @@
package test
import (
"context"
"sort"
"testing"
@@ -39,7 +40,7 @@ func TestKeyBook(t *testing.T, factory KeyBookFactory) {
func testKeybookPrivKey(kb pstore.KeyBook) func(t *testing.T) {
return func(t *testing.T) {
if peers := kb.PeersWithKeys(); len(peers) > 0 {
if peers := kb.PeersWithKeys(context.Background()); len(peers) > 0 {
t.Error("expected peers to be empty on init")
}
@@ -53,20 +54,20 @@ func testKeybookPrivKey(kb pstore.KeyBook) func(t *testing.T) {
t.Error(err)
}
if res := kb.PrivKey(id); res != nil {
if res := kb.PrivKey(context.Background(), id); res != nil {
t.Error("retrieving private key should have failed")
}
err = kb.AddPrivKey(id, priv)
err = kb.AddPrivKey(context.Background(), id, priv)
if err != nil {
t.Error(err)
}
if res := kb.PrivKey(id); !priv.Equals(res) {
if res := kb.PrivKey(context.Background(), id); !priv.Equals(res) {
t.Error("retrieved private key did not match stored private key")
}
if peers := kb.PeersWithKeys(); len(peers) != 1 || peers[0] != id {
if peers := kb.PeersWithKeys(context.Background()); len(peers) != 1 || peers[0] != id {
t.Error("list of peers did not include test peer")
}
}
@@ -74,7 +75,7 @@ func testKeybookPrivKey(kb pstore.KeyBook) func(t *testing.T) {
func testKeyBookPubKey(kb pstore.KeyBook) func(t *testing.T) {
return func(t *testing.T) {
if peers := kb.PeersWithKeys(); len(peers) > 0 {
if peers := kb.PeersWithKeys(context.Background()); len(peers) > 0 {
t.Error("expected peers to be empty on init")
}
@@ -88,20 +89,20 @@ func testKeyBookPubKey(kb pstore.KeyBook) func(t *testing.T) {
t.Fatal(err)
}
if res := kb.PubKey(id); res != nil {
if res := kb.PubKey(context.Background(), id); res != nil {
t.Error("retrieving public key should have failed")
}
err = kb.AddPubKey(id, pub)
err = kb.AddPubKey(context.Background(), id, pub)
if err != nil {
t.Error(err)
}
if res := kb.PubKey(id); !pub.Equals(res) {
if res := kb.PubKey(context.Background(), id); !pub.Equals(res) {
t.Error("retrieved public key did not match stored public key")
}
if peers := kb.PeersWithKeys(); len(peers) != 1 || peers[0] != id {
if peers := kb.PeersWithKeys(context.Background()); len(peers) != 1 || peers[0] != id {
t.Error("list of peers did not include test peer")
}
}
@@ -109,7 +110,7 @@ func testKeyBookPubKey(kb pstore.KeyBook) func(t *testing.T) {
func testKeyBookPeers(kb pstore.KeyBook) func(t *testing.T) {
return func(t *testing.T) {
if peers := kb.PeersWithKeys(); len(peers) > 0 {
if peers := kb.PeersWithKeys(context.Background()); len(peers) > 0 {
t.Error("expected peers to be empty on init")
}
@@ -124,7 +125,7 @@ func testKeyBookPeers(kb pstore.KeyBook) func(t *testing.T) {
if err != nil {
t.Fatal(err)
}
kb.AddPubKey(p1, pub)
kb.AddPubKey(context.Background(), p1, pub)
// Add a private key.
priv, _, err := pt.RandTestKeyPair(ic.RSA, 2048)
@@ -135,12 +136,12 @@ func testKeyBookPeers(kb pstore.KeyBook) func(t *testing.T) {
if err != nil {
t.Fatal(err)
}
kb.AddPrivKey(p2, priv)
kb.AddPrivKey(context.Background(), p2, priv)
peers = append(peers, []peer.ID{p1, p2}...)
}
kbPeers := kb.PeersWithKeys()
kbPeers := kb.PeersWithKeys(context.Background())
sort.Sort(kbPeers)
sort.Sort(peers)
@@ -156,7 +157,7 @@ func testInlinedPubKeyAddedOnRetrieve(kb pstore.KeyBook) func(t *testing.T) {
return func(t *testing.T) {
t.Skip("key inlining disabled for now: see libp2p/specs#111")
if peers := kb.PeersWithKeys(); len(peers) > 0 {
if peers := kb.PeersWithKeys(context.Background()); len(peers) > 0 {
t.Error("expected peers to be empty on init")
}
@@ -171,7 +172,7 @@ func testInlinedPubKeyAddedOnRetrieve(kb pstore.KeyBook) func(t *testing.T) {
t.Error(err)
}
pubKey := kb.PubKey(id)
pubKey := kb.PubKey(context.Background(), id)
if !pubKey.Equals(pub) {
t.Error("mismatch between original public key and keybook-calculated one")
}
@@ -185,13 +186,13 @@ func testKeyBookDelete(kb pstore.KeyBook) func(t *testing.T) {
require.NoError(t, err)
p, err := peer.IDFromPublicKey(pub)
require.NoError(t, err)
require.NoError(t, kb.AddPubKey(p, pub))
require.NoError(t, kb.AddPrivKey(p, priv))
require.NotNil(t, kb.PrivKey(p))
require.NotNil(t, kb.PubKey(p))
kb.RemovePeer(p)
require.Nil(t, kb.PrivKey(p))
require.Nil(t, kb.PubKey(p))
require.NoError(t, kb.AddPubKey(context.Background(), p, pub))
require.NoError(t, kb.AddPrivKey(context.Background(), p, priv))
require.NotNil(t, kb.PrivKey(context.Background(), p))
require.NotNil(t, kb.PubKey(context.Background(), p))
kb.RemovePeer(context.Background(), p)
require.Nil(t, kb.PrivKey(context.Background(), p))
require.Nil(t, kb.PubKey(context.Background(), p))
}
}
@@ -233,14 +234,14 @@ func benchmarkPubKey(kb pstore.KeyBook) func(*testing.B) {
b.Fatal(err)
}
err = kb.AddPubKey(id, pub)
err = kb.AddPubKey(context.Background(), id, pub)
if err != nil {
b.Fatal(err)
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
kb.PubKey(id)
kb.PubKey(context.Background(), id)
}
}
}
@@ -259,7 +260,7 @@ func benchmarkAddPubKey(kb pstore.KeyBook) func(*testing.B) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
kb.AddPubKey(id, pub)
kb.AddPubKey(context.Background(), id, pub)
}
}
}
@@ -276,14 +277,14 @@ func benchmarkPrivKey(kb pstore.KeyBook) func(*testing.B) {
b.Fatal(err)
}
err = kb.AddPrivKey(id, priv)
err = kb.AddPrivKey(context.Background(), id, priv)
if err != nil {
b.Fatal(err)
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
kb.PrivKey(id)
kb.PrivKey(context.Background(), id)
}
}
}
@@ -302,7 +303,7 @@ func benchmarkAddPrivKey(kb pstore.KeyBook) func(*testing.B) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
kb.AddPrivKey(id, priv)
kb.AddPrivKey(context.Background(), id, priv)
}
}
}
@@ -320,18 +321,18 @@ func benchmarkPeersWithKeys(kb pstore.KeyBook) func(*testing.B) {
b.Fatal(err)
}
err = kb.AddPubKey(id, pub)
err = kb.AddPubKey(context.Background(), id, pub)
if err != nil {
b.Fatal(err)
}
err = kb.AddPrivKey(id, priv)
err = kb.AddPrivKey(context.Background(), id, priv)
if err != nil {
b.Fatal(err)
}
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
kb.PeersWithKeys()
kb.PeersWithKeys(context.Background())
}
}
}

View File

@@ -52,7 +52,7 @@ func sortProtos(protos []protocol.ID) {
func testAddrStream(ps pstore.Peerstore) func(t *testing.T) {
return func(t *testing.T) {
addrs, pid := getAddrs(t, 100), peer.ID("testpeer")
ps.AddAddrs(pid, addrs[:10], time.Hour)
ps.AddAddrs(context.Background(), pid, addrs[:10], time.Hour)
ctx, cancel := context.WithCancel(context.Background())
addrch := ps.AddrStream(ctx, pid)
@@ -60,7 +60,7 @@ func testAddrStream(ps pstore.Peerstore) func(t *testing.T) {
// while that subscription is active, publish ten more addrs
// this tests that it doesnt hang
for i := 10; i < 20; i++ {
ps.AddAddr(pid, addrs[i], time.Hour)
ps.AddAddr(context.Background(), pid, addrs[i], time.Hour)
}
// now receive them (without hanging)
@@ -82,7 +82,7 @@ func testAddrStream(ps pstore.Peerstore) func(t *testing.T) {
defer close(done)
// now send the rest of the addresses
for _, a := range addrs[20:80] {
ps.AddAddr(pid, a, time.Hour)
ps.AddAddr(context.Background(), pid, a, time.Hour)
}
}()
@@ -118,7 +118,7 @@ func testAddrStream(ps pstore.Peerstore) func(t *testing.T) {
// and add a few more addresses it doesnt hang afterwards
for _, a := range addrs[80:] {
ps.AddAddr(pid, a, time.Hour)
ps.AddAddr(context.Background(), pid, a, time.Hour)
}
}
}
@@ -132,7 +132,7 @@ func testGetStreamBeforePeerAdded(ps pstore.Peerstore) func(t *testing.T) {
ach := ps.AddrStream(ctx, pid)
for i := 0; i < 10; i++ {
ps.AddAddr(pid, addrs[i], time.Hour)
ps.AddAddr(context.Background(), pid, addrs[i], time.Hour)
}
received := make(map[string]bool)
@@ -182,8 +182,8 @@ func testAddrStreamDuplicates(ps pstore.Peerstore) func(t *testing.T) {
ach := ps.AddrStream(ctx, pid)
go func() {
for i := 0; i < 10; i++ {
ps.AddAddr(pid, addrs[i], time.Hour)
ps.AddAddr(pid, addrs[rand.Intn(10)], time.Hour)
ps.AddAddr(context.Background(), pid, addrs[i], time.Hour)
ps.AddAddr(context.Background(), pid, addrs[rand.Intn(10)], time.Hour)
}
// make sure that all addresses get processed before context is cancelled
@@ -216,8 +216,8 @@ func testPeerstoreProtoStore(ps pstore.Peerstore) func(t *testing.T) {
p1 := peer.ID("TESTPEER")
protos := []protocol.ID{"a", "b", "c", "d"}
require.NoError(t, ps.AddProtocols(p1, protos...))
out, err := ps.GetProtocols(p1)
require.NoError(t, ps.AddProtocols(context.Background(), p1, protos...))
out, err := ps.GetProtocols(context.Background(), p1)
require.NoError(t, err)
require.Len(t, out, len(protos), "got wrong number of protocols back")
@@ -228,7 +228,7 @@ func testPeerstoreProtoStore(ps pstore.Peerstore) func(t *testing.T) {
}
}
supported, err := ps.SupportsProtocols(p1, "q", "w", "a", "y", "b")
supported, err := ps.SupportsProtocols(context.Background(), p1, "q", "w", "a", "y", "b")
require.NoError(t, err)
require.Len(t, supported, 2, "only expected 2 supported")
@@ -236,26 +236,26 @@ func testPeerstoreProtoStore(ps pstore.Peerstore) func(t *testing.T) {
t.Fatal("got wrong supported array: ", supported)
}
b, err := ps.FirstSupportedProtocol(p1, "q", "w", "a", "y", "b")
b, err := ps.FirstSupportedProtocol(context.Background(), p1, "q", "w", "a", "y", "b")
require.NoError(t, err)
require.Equal(t, protocol.ID("a"), b)
b, err = ps.FirstSupportedProtocol(p1, "q", "x", "z")
b, err = ps.FirstSupportedProtocol(context.Background(), p1, "q", "x", "z")
require.NoError(t, err)
require.Empty(t, b)
b, err = ps.FirstSupportedProtocol(p1, "a")
b, err = ps.FirstSupportedProtocol(context.Background(), p1, "a")
require.NoError(t, err)
require.Equal(t, protocol.ID("a"), b)
protos = []protocol.ID{"other", "yet another", "one more"}
require.NoError(t, ps.SetProtocols(p1, protos...))
require.NoError(t, ps.SetProtocols(context.Background(), p1, protos...))
supported, err = ps.SupportsProtocols(p1, "q", "w", "a", "y", "b")
supported, err = ps.SupportsProtocols(context.Background(), p1, "q", "w", "a", "y", "b")
require.NoError(t, err)
require.Empty(t, supported, "none of those protocols should have been supported")
supported, err = ps.GetProtocols(p1)
supported, err = ps.GetProtocols(context.Background(), p1)
require.NoError(t, err)
sortProtos(supported)
@@ -264,9 +264,9 @@ func testPeerstoreProtoStore(ps pstore.Peerstore) func(t *testing.T) {
t.Fatalf("expected previously set protos; expected: %v, have: %v", protos, supported)
}
require.NoError(t, ps.RemoveProtocols(p1, protos[:2]...))
require.NoError(t, ps.RemoveProtocols(context.Background(), p1, protos[:2]...))
supported, err = ps.GetProtocols(p1)
supported, err = ps.GetProtocols(context.Background(), p1)
require.NoError(t, err)
if !reflect.DeepEqual(supported, protos[2:]) {
t.Fatal("expected only one protocol to remain")
@@ -277,12 +277,12 @@ func testPeerstoreProtoStore(ps pstore.Peerstore) func(t *testing.T) {
p := peer.ID("foobar")
protos := []protocol.ID{"a", "b"}
require.NoError(t, ps.SetProtocols(p, protos...))
out, err := ps.GetProtocols(p)
require.NoError(t, ps.SetProtocols(context.Background(), p, protos...))
out, err := ps.GetProtocols(context.Background(), p)
require.NoError(t, err)
require.Len(t, out, 2)
ps.RemovePeer(p)
out, err = ps.GetProtocols(p)
ps.RemovePeer(context.Background(), p)
out, err = ps.GetProtocols(context.Background(), p)
require.NoError(t, err)
require.Empty(t, out)
})
@@ -304,15 +304,15 @@ func testBasicPeerstore(ps pstore.Peerstore) func(t *testing.T) {
t.Fatal(err)
}
pids = append(pids, p)
ps.AddAddr(p, a, pstore.PermanentAddrTTL)
ps.AddAddr(context.Background(), p, a, pstore.PermanentAddrTTL)
}
peers := ps.Peers()
peers := ps.Peers(context.Background())
if len(peers) != 10 {
t.Fatal("expected ten peers, got", len(peers))
}
pinfo := ps.PeerInfo(pids[0])
pinfo := ps.PeerInfo(context.Background(), pids[0])
if !pinfo.Addrs[0].Equal(addrs[0]) {
t.Fatal("stored wrong address")
}
@@ -331,15 +331,15 @@ func testMetadata(ps pstore.Peerstore) func(t *testing.T) {
pids[i] = p
}
for _, p := range pids {
require.NoError(t, ps.Put(p, "AgentVersion", "string"), "failed to put AgentVersion")
require.NoError(t, ps.Put(p, "bar", 1), "failed to put bar")
require.NoError(t, ps.Put(context.Background(), p, "AgentVersion", "string"), "failed to put AgentVersion")
require.NoError(t, ps.Put(context.Background(), p, "bar", 1), "failed to put bar")
}
for _, p := range pids {
v, err := ps.Get(p, "AgentVersion")
v, err := ps.Get(context.Background(), p, "AgentVersion")
require.NoError(t, err)
require.Equal(t, v, "string")
v, err = ps.Get(p, "bar")
v, err = ps.Get(context.Background(), p, "bar")
require.NoError(t, err)
require.Equal(t, v, 1)
}
@@ -348,16 +348,16 @@ func testMetadata(ps pstore.Peerstore) func(t *testing.T) {
t.Run("removing a peer", func(t *testing.T) {
p := peer.ID("foo")
otherP := peer.ID("foobar")
require.NoError(t, ps.Put(otherP, "AgentVersion", "v1"))
require.NoError(t, ps.Put(p, "AgentVersion", "v1"))
require.NoError(t, ps.Put(p, "bar", 1))
ps.RemovePeer(p)
_, err := ps.Get(p, "AgentVersion")
require.NoError(t, ps.Put(context.Background(), otherP, "AgentVersion", "v1"))
require.NoError(t, ps.Put(context.Background(), p, "AgentVersion", "v1"))
require.NoError(t, ps.Put(context.Background(), p, "bar", 1))
ps.RemovePeer(context.Background(), p)
_, err := ps.Get(context.Background(), p, "AgentVersion")
require.ErrorIs(t, err, pstore.ErrNotFound)
_, err = ps.Get(p, "bar")
_, err = ps.Get(context.Background(), p, "bar")
require.ErrorIs(t, err, pstore.ErrNotFound)
// make sure that entries for otherP were not deleted
val, err := ps.Get(otherP, "AgentVersion")
val, err := ps.Get(context.Background(), otherP, "AgentVersion")
require.NoError(t, err)
require.Equal(t, val, "v1")
})
@@ -394,14 +394,14 @@ func TestPeerstoreProtoStoreLimits(t *testing.T, ps pstore.Peerstore, limit int)
}
t.Run("setting protocols", func(t *testing.T) {
require.NoError(t, ps.SetProtocols(p, protocols...))
require.EqualError(t, ps.SetProtocols(p, append(protocols, "proto")...), "too many protocols")
require.NoError(t, ps.SetProtocols(context.Background(), p, protocols...))
require.EqualError(t, ps.SetProtocols(context.Background(), p, append(protocols, "proto")...), "too many protocols")
})
t.Run("adding protocols", func(t *testing.T) {
p1 := protocols[:limit/2]
p2 := protocols[limit/2:]
require.NoError(t, ps.SetProtocols(p, p1...))
require.NoError(t, ps.AddProtocols(p, p2...))
require.EqualError(t, ps.AddProtocols(p, "proto"), "too many protocols")
require.NoError(t, ps.SetProtocols(context.Background(), p, p1...))
require.NoError(t, ps.AddProtocols(context.Background(), p, p2...))
require.EqualError(t, ps.AddProtocols(context.Background(), p, "proto"), "too many protocols")
})
}

View File

@@ -40,48 +40,48 @@ func (m *MockPeerstore) EXPECT() *MockPeerstoreMockRecorder {
}
// AddAddr mocks base method.
func (m *MockPeerstore) AddAddr(arg0 peer.ID, arg1 multiaddr.Multiaddr, arg2 time.Duration) {
func (m *MockPeerstore) AddAddr(arg0 context.Context, arg1 peer.ID, arg2 multiaddr.Multiaddr, arg3 time.Duration) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "AddAddr", arg0, arg1, arg2)
m.ctrl.Call(m, "AddAddr", arg0, arg1, arg2, arg3)
}
// AddAddr indicates an expected call of AddAddr.
func (mr *MockPeerstoreMockRecorder) AddAddr(arg0, arg1, arg2 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) AddAddr(arg0, arg1, arg2, arg3 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddAddr", reflect.TypeOf((*MockPeerstore)(nil).AddAddr), arg0, arg1, arg2)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddAddr", reflect.TypeOf((*MockPeerstore)(nil).AddAddr), arg0, arg1, arg2, arg3)
}
// AddAddrs mocks base method.
func (m *MockPeerstore) AddAddrs(arg0 peer.ID, arg1 []multiaddr.Multiaddr, arg2 time.Duration) {
func (m *MockPeerstore) AddAddrs(arg0 context.Context, arg1 peer.ID, arg2 []multiaddr.Multiaddr, arg3 time.Duration) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "AddAddrs", arg0, arg1, arg2)
m.ctrl.Call(m, "AddAddrs", arg0, arg1, arg2, arg3)
}
// AddAddrs indicates an expected call of AddAddrs.
func (mr *MockPeerstoreMockRecorder) AddAddrs(arg0, arg1, arg2 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) AddAddrs(arg0, arg1, arg2, arg3 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddAddrs", reflect.TypeOf((*MockPeerstore)(nil).AddAddrs), arg0, arg1, arg2)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddAddrs", reflect.TypeOf((*MockPeerstore)(nil).AddAddrs), arg0, arg1, arg2, arg3)
}
// AddPrivKey mocks base method.
func (m *MockPeerstore) AddPrivKey(arg0 peer.ID, arg1 crypto.PrivKey) error {
func (m *MockPeerstore) AddPrivKey(arg0 context.Context, arg1 peer.ID, arg2 crypto.PrivKey) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "AddPrivKey", arg0, arg1)
ret := m.ctrl.Call(m, "AddPrivKey", arg0, arg1, arg2)
ret0, _ := ret[0].(error)
return ret0
}
// AddPrivKey indicates an expected call of AddPrivKey.
func (mr *MockPeerstoreMockRecorder) AddPrivKey(arg0, arg1 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) AddPrivKey(arg0, arg1, arg2 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddPrivKey", reflect.TypeOf((*MockPeerstore)(nil).AddPrivKey), arg0, arg1)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddPrivKey", reflect.TypeOf((*MockPeerstore)(nil).AddPrivKey), arg0, arg1, arg2)
}
// AddProtocols mocks base method.
func (m *MockPeerstore) AddProtocols(arg0 peer.ID, arg1 ...protocol.ID) error {
func (m *MockPeerstore) AddProtocols(arg0 context.Context, arg1 peer.ID, arg2 ...protocol.ID) error {
m.ctrl.T.Helper()
varargs := []interface{}{arg0}
for _, a := range arg1 {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "AddProtocols", varargs...)
@@ -90,24 +90,24 @@ func (m *MockPeerstore) AddProtocols(arg0 peer.ID, arg1 ...protocol.ID) error {
}
// AddProtocols indicates an expected call of AddProtocols.
func (mr *MockPeerstoreMockRecorder) AddProtocols(arg0 interface{}, arg1 ...interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) AddProtocols(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]interface{}{arg0}, arg1...)
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddProtocols", reflect.TypeOf((*MockPeerstore)(nil).AddProtocols), varargs...)
}
// AddPubKey mocks base method.
func (m *MockPeerstore) AddPubKey(arg0 peer.ID, arg1 crypto.PubKey) error {
func (m *MockPeerstore) AddPubKey(arg0 context.Context, arg1 peer.ID, arg2 crypto.PubKey) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "AddPubKey", arg0, arg1)
ret := m.ctrl.Call(m, "AddPubKey", arg0, arg1, arg2)
ret0, _ := ret[0].(error)
return ret0
}
// AddPubKey indicates an expected call of AddPubKey.
func (mr *MockPeerstoreMockRecorder) AddPubKey(arg0, arg1 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) AddPubKey(arg0, arg1, arg2 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddPubKey", reflect.TypeOf((*MockPeerstore)(nil).AddPubKey), arg0, arg1)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddPubKey", reflect.TypeOf((*MockPeerstore)(nil).AddPubKey), arg0, arg1, arg2)
}
// AddrStream mocks base method.
@@ -125,29 +125,29 @@ func (mr *MockPeerstoreMockRecorder) AddrStream(arg0, arg1 interface{}) *gomock.
}
// Addrs mocks base method.
func (m *MockPeerstore) Addrs(arg0 peer.ID) []multiaddr.Multiaddr {
func (m *MockPeerstore) Addrs(arg0 context.Context, arg1 peer.ID) []multiaddr.Multiaddr {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Addrs", arg0)
ret := m.ctrl.Call(m, "Addrs", arg0, arg1)
ret0, _ := ret[0].([]multiaddr.Multiaddr)
return ret0
}
// Addrs indicates an expected call of Addrs.
func (mr *MockPeerstoreMockRecorder) Addrs(arg0 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) Addrs(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Addrs", reflect.TypeOf((*MockPeerstore)(nil).Addrs), arg0)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Addrs", reflect.TypeOf((*MockPeerstore)(nil).Addrs), arg0, arg1)
}
// ClearAddrs mocks base method.
func (m *MockPeerstore) ClearAddrs(arg0 peer.ID) {
func (m *MockPeerstore) ClearAddrs(arg0 context.Context, arg1 peer.ID) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "ClearAddrs", arg0)
m.ctrl.Call(m, "ClearAddrs", arg0, arg1)
}
// ClearAddrs indicates an expected call of ClearAddrs.
func (mr *MockPeerstoreMockRecorder) ClearAddrs(arg0 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) ClearAddrs(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ClearAddrs", reflect.TypeOf((*MockPeerstore)(nil).ClearAddrs), arg0)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ClearAddrs", reflect.TypeOf((*MockPeerstore)(nil).ClearAddrs), arg0, arg1)
}
// Close mocks base method.
@@ -165,10 +165,10 @@ func (mr *MockPeerstoreMockRecorder) Close() *gomock.Call {
}
// FirstSupportedProtocol mocks base method.
func (m *MockPeerstore) FirstSupportedProtocol(arg0 peer.ID, arg1 ...protocol.ID) (protocol.ID, error) {
func (m *MockPeerstore) FirstSupportedProtocol(arg0 context.Context, arg1 peer.ID, arg2 ...protocol.ID) (protocol.ID, error) {
m.ctrl.T.Helper()
varargs := []interface{}{arg0}
for _, a := range arg1 {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "FirstSupportedProtocol", varargs...)
@@ -178,40 +178,40 @@ func (m *MockPeerstore) FirstSupportedProtocol(arg0 peer.ID, arg1 ...protocol.ID
}
// FirstSupportedProtocol indicates an expected call of FirstSupportedProtocol.
func (mr *MockPeerstoreMockRecorder) FirstSupportedProtocol(arg0 interface{}, arg1 ...interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) FirstSupportedProtocol(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]interface{}{arg0}, arg1...)
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FirstSupportedProtocol", reflect.TypeOf((*MockPeerstore)(nil).FirstSupportedProtocol), varargs...)
}
// Get mocks base method.
func (m *MockPeerstore) Get(arg0 peer.ID, arg1 string) (interface{}, error) {
func (m *MockPeerstore) Get(arg0 context.Context, arg1 peer.ID, arg2 string) (interface{}, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Get", arg0, arg1)
ret := m.ctrl.Call(m, "Get", arg0, arg1, arg2)
ret0, _ := ret[0].(interface{})
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Get indicates an expected call of Get.
func (mr *MockPeerstoreMockRecorder) Get(arg0, arg1 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) Get(arg0, arg1, arg2 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockPeerstore)(nil).Get), arg0, arg1)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockPeerstore)(nil).Get), arg0, arg1, arg2)
}
// GetProtocols mocks base method.
func (m *MockPeerstore) GetProtocols(arg0 peer.ID) ([]protocol.ID, error) {
func (m *MockPeerstore) GetProtocols(arg0 context.Context, arg1 peer.ID) ([]protocol.ID, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "GetProtocols", arg0)
ret := m.ctrl.Call(m, "GetProtocols", arg0, arg1)
ret0, _ := ret[0].([]protocol.ID)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// GetProtocols indicates an expected call of GetProtocols.
func (mr *MockPeerstoreMockRecorder) GetProtocols(arg0 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) GetProtocols(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetProtocols", reflect.TypeOf((*MockPeerstore)(nil).GetProtocols), arg0)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetProtocols", reflect.TypeOf((*MockPeerstore)(nil).GetProtocols), arg0, arg1)
}
// LatencyEWMA mocks base method.
@@ -229,101 +229,101 @@ func (mr *MockPeerstoreMockRecorder) LatencyEWMA(arg0 interface{}) *gomock.Call
}
// PeerInfo mocks base method.
func (m *MockPeerstore) PeerInfo(arg0 peer.ID) peer.AddrInfo {
func (m *MockPeerstore) PeerInfo(arg0 context.Context, arg1 peer.ID) peer.AddrInfo {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "PeerInfo", arg0)
ret := m.ctrl.Call(m, "PeerInfo", arg0, arg1)
ret0, _ := ret[0].(peer.AddrInfo)
return ret0
}
// PeerInfo indicates an expected call of PeerInfo.
func (mr *MockPeerstoreMockRecorder) PeerInfo(arg0 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) PeerInfo(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PeerInfo", reflect.TypeOf((*MockPeerstore)(nil).PeerInfo), arg0)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PeerInfo", reflect.TypeOf((*MockPeerstore)(nil).PeerInfo), arg0, arg1)
}
// Peers mocks base method.
func (m *MockPeerstore) Peers() peer.IDSlice {
func (m *MockPeerstore) Peers(arg0 context.Context) peer.IDSlice {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Peers")
ret := m.ctrl.Call(m, "Peers", arg0)
ret0, _ := ret[0].(peer.IDSlice)
return ret0
}
// Peers indicates an expected call of Peers.
func (mr *MockPeerstoreMockRecorder) Peers() *gomock.Call {
func (mr *MockPeerstoreMockRecorder) Peers(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Peers", reflect.TypeOf((*MockPeerstore)(nil).Peers))
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Peers", reflect.TypeOf((*MockPeerstore)(nil).Peers), arg0)
}
// PeersWithAddrs mocks base method.
func (m *MockPeerstore) PeersWithAddrs() peer.IDSlice {
func (m *MockPeerstore) PeersWithAddrs(arg0 context.Context) peer.IDSlice {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "PeersWithAddrs")
ret := m.ctrl.Call(m, "PeersWithAddrs", arg0)
ret0, _ := ret[0].(peer.IDSlice)
return ret0
}
// PeersWithAddrs indicates an expected call of PeersWithAddrs.
func (mr *MockPeerstoreMockRecorder) PeersWithAddrs() *gomock.Call {
func (mr *MockPeerstoreMockRecorder) PeersWithAddrs(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PeersWithAddrs", reflect.TypeOf((*MockPeerstore)(nil).PeersWithAddrs))
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PeersWithAddrs", reflect.TypeOf((*MockPeerstore)(nil).PeersWithAddrs), arg0)
}
// PeersWithKeys mocks base method.
func (m *MockPeerstore) PeersWithKeys() peer.IDSlice {
func (m *MockPeerstore) PeersWithKeys(arg0 context.Context) peer.IDSlice {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "PeersWithKeys")
ret := m.ctrl.Call(m, "PeersWithKeys", arg0)
ret0, _ := ret[0].(peer.IDSlice)
return ret0
}
// PeersWithKeys indicates an expected call of PeersWithKeys.
func (mr *MockPeerstoreMockRecorder) PeersWithKeys() *gomock.Call {
func (mr *MockPeerstoreMockRecorder) PeersWithKeys(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PeersWithKeys", reflect.TypeOf((*MockPeerstore)(nil).PeersWithKeys))
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PeersWithKeys", reflect.TypeOf((*MockPeerstore)(nil).PeersWithKeys), arg0)
}
// PrivKey mocks base method.
func (m *MockPeerstore) PrivKey(arg0 peer.ID) crypto.PrivKey {
func (m *MockPeerstore) PrivKey(arg0 context.Context, arg1 peer.ID) crypto.PrivKey {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "PrivKey", arg0)
ret := m.ctrl.Call(m, "PrivKey", arg0, arg1)
ret0, _ := ret[0].(crypto.PrivKey)
return ret0
}
// PrivKey indicates an expected call of PrivKey.
func (mr *MockPeerstoreMockRecorder) PrivKey(arg0 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) PrivKey(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PrivKey", reflect.TypeOf((*MockPeerstore)(nil).PrivKey), arg0)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PrivKey", reflect.TypeOf((*MockPeerstore)(nil).PrivKey), arg0, arg1)
}
// PubKey mocks base method.
func (m *MockPeerstore) PubKey(arg0 peer.ID) crypto.PubKey {
func (m *MockPeerstore) PubKey(arg0 context.Context, arg1 peer.ID) crypto.PubKey {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "PubKey", arg0)
ret := m.ctrl.Call(m, "PubKey", arg0, arg1)
ret0, _ := ret[0].(crypto.PubKey)
return ret0
}
// PubKey indicates an expected call of PubKey.
func (mr *MockPeerstoreMockRecorder) PubKey(arg0 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) PubKey(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PubKey", reflect.TypeOf((*MockPeerstore)(nil).PubKey), arg0)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PubKey", reflect.TypeOf((*MockPeerstore)(nil).PubKey), arg0, arg1)
}
// Put mocks base method.
func (m *MockPeerstore) Put(arg0 peer.ID, arg1 string, arg2 interface{}) error {
func (m *MockPeerstore) Put(arg0 context.Context, arg1 peer.ID, arg2 string, arg3 interface{}) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Put", arg0, arg1, arg2)
ret := m.ctrl.Call(m, "Put", arg0, arg1, arg2, arg3)
ret0, _ := ret[0].(error)
return ret0
}
// Put indicates an expected call of Put.
func (mr *MockPeerstoreMockRecorder) Put(arg0, arg1, arg2 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) Put(arg0, arg1, arg2, arg3 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Put", reflect.TypeOf((*MockPeerstore)(nil).Put), arg0, arg1, arg2)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Put", reflect.TypeOf((*MockPeerstore)(nil).Put), arg0, arg1, arg2, arg3)
}
// RecordLatency mocks base method.
@@ -339,22 +339,22 @@ func (mr *MockPeerstoreMockRecorder) RecordLatency(arg0, arg1 interface{}) *gomo
}
// RemovePeer mocks base method.
func (m *MockPeerstore) RemovePeer(arg0 peer.ID) {
func (m *MockPeerstore) RemovePeer(arg0 context.Context, arg1 peer.ID) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "RemovePeer", arg0)
m.ctrl.Call(m, "RemovePeer", arg0, arg1)
}
// RemovePeer indicates an expected call of RemovePeer.
func (mr *MockPeerstoreMockRecorder) RemovePeer(arg0 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) RemovePeer(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemovePeer", reflect.TypeOf((*MockPeerstore)(nil).RemovePeer), arg0)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemovePeer", reflect.TypeOf((*MockPeerstore)(nil).RemovePeer), arg0, arg1)
}
// RemoveProtocols mocks base method.
func (m *MockPeerstore) RemoveProtocols(arg0 peer.ID, arg1 ...protocol.ID) error {
func (m *MockPeerstore) RemoveProtocols(arg0 context.Context, arg1 peer.ID, arg2 ...protocol.ID) error {
m.ctrl.T.Helper()
varargs := []interface{}{arg0}
for _, a := range arg1 {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "RemoveProtocols", varargs...)
@@ -363,41 +363,41 @@ func (m *MockPeerstore) RemoveProtocols(arg0 peer.ID, arg1 ...protocol.ID) error
}
// RemoveProtocols indicates an expected call of RemoveProtocols.
func (mr *MockPeerstoreMockRecorder) RemoveProtocols(arg0 interface{}, arg1 ...interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) RemoveProtocols(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]interface{}{arg0}, arg1...)
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemoveProtocols", reflect.TypeOf((*MockPeerstore)(nil).RemoveProtocols), varargs...)
}
// SetAddr mocks base method.
func (m *MockPeerstore) SetAddr(arg0 peer.ID, arg1 multiaddr.Multiaddr, arg2 time.Duration) {
func (m *MockPeerstore) SetAddr(arg0 context.Context, arg1 peer.ID, arg2 multiaddr.Multiaddr, arg3 time.Duration) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "SetAddr", arg0, arg1, arg2)
m.ctrl.Call(m, "SetAddr", arg0, arg1, arg2, arg3)
}
// SetAddr indicates an expected call of SetAddr.
func (mr *MockPeerstoreMockRecorder) SetAddr(arg0, arg1, arg2 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) SetAddr(arg0, arg1, arg2, arg3 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetAddr", reflect.TypeOf((*MockPeerstore)(nil).SetAddr), arg0, arg1, arg2)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetAddr", reflect.TypeOf((*MockPeerstore)(nil).SetAddr), arg0, arg1, arg2, arg3)
}
// SetAddrs mocks base method.
func (m *MockPeerstore) SetAddrs(arg0 peer.ID, arg1 []multiaddr.Multiaddr, arg2 time.Duration) {
func (m *MockPeerstore) SetAddrs(arg0 context.Context, arg1 peer.ID, arg2 []multiaddr.Multiaddr, arg3 time.Duration) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "SetAddrs", arg0, arg1, arg2)
m.ctrl.Call(m, "SetAddrs", arg0, arg1, arg2, arg3)
}
// SetAddrs indicates an expected call of SetAddrs.
func (mr *MockPeerstoreMockRecorder) SetAddrs(arg0, arg1, arg2 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) SetAddrs(arg0, arg1, arg2, arg3 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetAddrs", reflect.TypeOf((*MockPeerstore)(nil).SetAddrs), arg0, arg1, arg2)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetAddrs", reflect.TypeOf((*MockPeerstore)(nil).SetAddrs), arg0, arg1, arg2, arg3)
}
// SetProtocols mocks base method.
func (m *MockPeerstore) SetProtocols(arg0 peer.ID, arg1 ...protocol.ID) error {
func (m *MockPeerstore) SetProtocols(arg0 context.Context, arg1 peer.ID, arg2 ...protocol.ID) error {
m.ctrl.T.Helper()
varargs := []interface{}{arg0}
for _, a := range arg1 {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SetProtocols", varargs...)
@@ -406,17 +406,17 @@ func (m *MockPeerstore) SetProtocols(arg0 peer.ID, arg1 ...protocol.ID) error {
}
// SetProtocols indicates an expected call of SetProtocols.
func (mr *MockPeerstoreMockRecorder) SetProtocols(arg0 interface{}, arg1 ...interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) SetProtocols(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]interface{}{arg0}, arg1...)
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetProtocols", reflect.TypeOf((*MockPeerstore)(nil).SetProtocols), varargs...)
}
// SupportsProtocols mocks base method.
func (m *MockPeerstore) SupportsProtocols(arg0 peer.ID, arg1 ...protocol.ID) ([]protocol.ID, error) {
func (m *MockPeerstore) SupportsProtocols(arg0 context.Context, arg1 peer.ID, arg2 ...protocol.ID) ([]protocol.ID, error) {
m.ctrl.T.Helper()
varargs := []interface{}{arg0}
for _, a := range arg1 {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SupportsProtocols", varargs...)
@@ -426,20 +426,20 @@ func (m *MockPeerstore) SupportsProtocols(arg0 peer.ID, arg1 ...protocol.ID) ([]
}
// SupportsProtocols indicates an expected call of SupportsProtocols.
func (mr *MockPeerstoreMockRecorder) SupportsProtocols(arg0 interface{}, arg1 ...interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) SupportsProtocols(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]interface{}{arg0}, arg1...)
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SupportsProtocols", reflect.TypeOf((*MockPeerstore)(nil).SupportsProtocols), varargs...)
}
// UpdateAddrs mocks base method.
func (m *MockPeerstore) UpdateAddrs(arg0 peer.ID, arg1, arg2 time.Duration) {
func (m *MockPeerstore) UpdateAddrs(arg0 context.Context, arg1 peer.ID, arg2, arg3 time.Duration) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "UpdateAddrs", arg0, arg1, arg2)
m.ctrl.Call(m, "UpdateAddrs", arg0, arg1, arg2, arg3)
}
// UpdateAddrs indicates an expected call of UpdateAddrs.
func (mr *MockPeerstoreMockRecorder) UpdateAddrs(arg0, arg1, arg2 interface{}) *gomock.Call {
func (mr *MockPeerstoreMockRecorder) UpdateAddrs(arg0, arg1, arg2, arg3 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateAddrs", reflect.TypeOf((*MockPeerstore)(nil).UpdateAddrs), arg0, arg1, arg2)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateAddrs", reflect.TypeOf((*MockPeerstore)(nil).UpdateAddrs), arg0, arg1, arg2, arg3)
}

View File

@@ -88,7 +88,7 @@ func (m *PeerstoreManager) background(ctx context.Context, sub event.Subscriptio
defer func() {
for p := range disconnected {
m.pstore.RemovePeer(p)
m.pstore.RemovePeer(context.Background(), p)
}
}()
@@ -113,7 +113,7 @@ func (m *PeerstoreManager) background(ctx context.Context, sub event.Subscriptio
now := time.Now()
for p, disconnectTime := range disconnected {
if disconnectTime.Add(m.gracePeriod).Before(now) {
m.pstore.RemovePeer(p)
m.pstore.RemovePeer(context.Background(), p)
delete(disconnected, p)
}
}

View File

@@ -1,6 +1,7 @@
package pstoremanager_test
import (
context "context"
"testing"
"time"
@@ -32,7 +33,7 @@ func TestGracePeriod(t *testing.T) {
require.NoError(t, err)
start := time.Now()
removed := make(chan struct{})
pstore.EXPECT().RemovePeer(peer.ID("foobar")).DoAndReturn(func(p peer.ID) {
pstore.EXPECT().RemovePeer(context.Background(), peer.ID("foobar")).DoAndReturn(func(ctx1 context.Context, p peer.ID) {
defer close(removed)
// make sure the call happened after the grace period
require.GreaterOrEqual(t, time.Since(start), gracePeriod)
@@ -101,7 +102,8 @@ func TestClose(t *testing.T) {
}
done := make(chan struct{})
pstore.EXPECT().RemovePeer(peer.ID("foobar")).Do(func(peer.ID) { close(done) })
pstore.EXPECT().RemovePeer(context.Background(), peer.ID("foobar")).Do(func(ctx context.Context, p peer.ID) { close(done) })
require.NoError(t, man.Close())
select {
case <-done:

View File

@@ -56,11 +56,11 @@ func (rh *RoutedHost) Connect(ctx context.Context, pi peer.AddrInfo) error {
// if we were given some addresses, keep + use them.
if len(pi.Addrs) > 0 {
rh.Peerstore().AddAddrs(pi.ID, pi.Addrs, peerstore.TempAddrTTL)
rh.Peerstore().AddAddrs(ctx, pi.ID, pi.Addrs, peerstore.TempAddrTTL)
}
// Check if we have some addresses in our recent memory.
addrs := rh.Peerstore().Addrs(pi.ID)
addrs := rh.Peerstore().Addrs(ctx, pi.ID)
if len(addrs) < 1 {
// no addrs? find some with the routing system.
var err error
@@ -91,7 +91,7 @@ func (rh *RoutedHost) Connect(ctx context.Context, pi peer.AddrInfo) error {
continue
}
if len(rh.Peerstore().Addrs(relayID)) > 0 {
if len(rh.Peerstore().Addrs(ctx, relayID)) > 0 {
// we already have addrs for this relay
continue
}
@@ -102,7 +102,7 @@ func (rh *RoutedHost) Connect(ctx context.Context, pi peer.AddrInfo) error {
continue
}
rh.Peerstore().AddAddrs(relayID, relayAddrs, peerstore.TempAddrTTL)
rh.Peerstore().AddAddrs(ctx, relayID, relayAddrs, peerstore.TempAddrTTL)
}
// if we're here, we got some addrs. let's use our wrapped host to connect.

View File

@@ -53,19 +53,19 @@ func newConn(ln, rn *peernet, l *link, dir network.Direction) *conn {
c.stat.Direction = dir
c.id = connCounter.Add(1)
c.localAddr = ln.ps.Addrs(ln.peer)[0]
for _, a := range rn.ps.Addrs(rn.peer) {
c.localAddr = ln.ps.Addrs(context.Background(), ln.peer)[0]
for _, a := range rn.ps.Addrs(context.Background(), rn.peer) {
if !manet.IsIPUnspecified(a) {
c.remoteAddr = a
break
}
}
if c.remoteAddr == nil {
c.remoteAddr = rn.ps.Addrs(rn.peer)[0]
c.remoteAddr = rn.ps.Addrs(context.Background(), rn.peer)[0]
}
c.localPrivKey = ln.ps.PrivKey(ln.peer)
c.remotePubKey = rn.ps.PubKey(rn.peer)
c.localPrivKey = ln.ps.PrivKey(context.Background(), ln.peer)
c.remotePubKey = rn.ps.PubKey(context.Background(), rn.peer)
return c
}

View File

@@ -101,9 +101,9 @@ func (mn *mocknet) AddPeer(k ic.PrivKey, a ma.Multiaddr) (host.Host, error) {
if err != nil {
return nil, err
}
ps.AddAddr(p, a, peerstore.PermanentAddrTTL)
ps.AddPrivKey(p, k)
ps.AddPubKey(p, k.GetPublic())
ps.AddAddr(context.Background(), p, a, peerstore.PermanentAddrTTL)
ps.AddPrivKey(context.Background(), p, k)
ps.AddPubKey(context.Background(), p, k.GetPublic())
return mn.AddPeerWithPeerstore(p, ps)
}

View File

@@ -318,13 +318,13 @@ func (pn *peernet) BandwidthTotals() (in uint64, out uint64) {
// Listen tells the network to start listening on given multiaddrs.
func (pn *peernet) Listen(addrs ...ma.Multiaddr) error {
pn.Peerstore().AddAddrs(pn.LocalPeer(), addrs, peerstore.PermanentAddrTTL)
pn.Peerstore().AddAddrs(context.Background(), pn.LocalPeer(), addrs, peerstore.PermanentAddrTTL)
return nil
}
// ListenAddresses returns a list of addresses at which this network listens.
func (pn *peernet) ListenAddresses() []ma.Multiaddr {
return pn.Peerstore().Addrs(pn.LocalPeer())
return pn.Peerstore().Addrs(context.Background(), pn.LocalPeer())
}
// InterfaceListenAddresses returns a list of addresses at which this network

View File

@@ -34,7 +34,7 @@ func TestBasicDialPeer(t *testing.T) {
s1 := swarms[0]
s2 := swarms[1]
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
c, err := s1.DialPeer(context.Background(), s2.LocalPeer())
require.NoError(t, err)
@@ -70,7 +70,7 @@ func TestBasicDialPeerWithResolver(t *testing.T) {
}
}
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2Addrs, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2Addrs, peerstore.PermanentAddrTTL)
c, err := s1.DialPeer(context.Background(), s2.LocalPeer())
require.NoError(t, err)
@@ -86,7 +86,7 @@ func TestDialWithNoListeners(t *testing.T) {
defer closeSwarms(swarms)
s2 := swarms[0]
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
c, err := s1.DialPeer(context.Background(), s2.LocalPeer())
require.NoError(t, err)
@@ -124,7 +124,7 @@ func TestSimultDials(t *testing.T) {
connect := func(s *swarm.Swarm, dst peer.ID, addr ma.Multiaddr) {
// copy for other peer
log.Debugf("TestSimultOpen: connecting: %s --> %s (%s)", s.LocalPeer(), dst, addr)
s.Peerstore().AddAddr(dst, addr, peerstore.TempAddrTTL)
s.Peerstore().AddAddr(context.Background(), dst, addr, peerstore.TempAddrTTL)
if _, err := s.DialPeer(ctx, dst); err != nil {
errs <- err
}
@@ -196,7 +196,7 @@ func TestDialWait(t *testing.T) {
s2p, s2addr, s2l := newSilentPeer(t)
go acceptAndHang(s2l)
defer s2l.Close()
s1.Peerstore().AddAddr(s2p, s2addr, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddr(context.Background(), s2p, s2addr, peerstore.PermanentAddrTTL)
before := time.Now()
if c, err := s1.DialPeer(context.Background(), s2p); err == nil {
@@ -233,13 +233,13 @@ func TestDialBackoff(t *testing.T) {
s2addrs, err := s2.InterfaceListenAddresses()
require.NoError(t, err)
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2addrs, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2addrs, peerstore.PermanentAddrTTL)
// dial to a non-existent peer.
s3p, s3addr, s3l := newSilentPeer(t)
go acceptAndHang(s3l)
defer s3l.Close()
s1.Peerstore().AddAddr(s3p, s3addr, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddr(context.Background(), s3p, s3addr, peerstore.PermanentAddrTTL)
// in this test we will:
// 1) dial 10x to each node.
@@ -438,7 +438,7 @@ func TestDialBackoffClears(t *testing.T) {
defer s2l.Close()
// phase 1 -- dial to non-operational addresses
s1.Peerstore().AddAddr(s2.LocalPeer(), s2bad, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddr(context.Background(), s2.LocalPeer(), s2bad, peerstore.PermanentAddrTTL)
before := time.Now()
_, err := s1.DialPeer(context.Background(), s2.LocalPeer())
@@ -456,7 +456,7 @@ func TestDialBackoffClears(t *testing.T) {
// phase 2 -- add the working address. dial should succeed.
ifaceAddrs1, err := s2.InterfaceListenAddresses()
require.NoError(t, err)
s1.Peerstore().AddAddrs(s2.LocalPeer(), ifaceAddrs1, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), ifaceAddrs1, peerstore.PermanentAddrTTL)
// backoffs are per address, not peer
c, err := s1.DialPeer(context.Background(), s2.LocalPeer())
@@ -476,7 +476,7 @@ func TestDialPeerFailed(t *testing.T) {
go acceptAndHang(silentPeerListener)
defer silentPeerListener.Close()
testedSwarm.Peerstore().AddAddr(targetSwarm.LocalPeer(), silentPeerAddress, peerstore.PermanentAddrTTL)
testedSwarm.Peerstore().AddAddr(context.Background(), targetSwarm.LocalPeer(), silentPeerAddress, peerstore.PermanentAddrTTL)
}
_, err := testedSwarm.DialPeer(context.Background(), targetSwarm.LocalPeer())
@@ -506,7 +506,7 @@ func TestDialExistingConnection(t *testing.T) {
// Only use one of the addresses here.
// Otherwise, we might dial TCP and QUIC simultaneously here, and end up with two connections,
// if the handshake latencies line up exactly.
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2.ListenAddresses()[:1], peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2.ListenAddresses()[:1], peerstore.PermanentAddrTTL)
c1, err := s1.DialPeer(context.Background(), s2.LocalPeer())
require.NoError(t, err)
@@ -553,7 +553,7 @@ func TestDialSimultaneousJoin(t *testing.T) {
// start a dial to s2 through the silent addr
go func() {
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2silentAddrs, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2silentAddrs, peerstore.PermanentAddrTTL)
c, err := s1.DialPeer(context.Background(), s2.LocalPeer())
if err != nil {
@@ -578,7 +578,7 @@ func TestDialSimultaneousJoin(t *testing.T) {
errs <- err
return
}
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2addrs[:1], peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2addrs[:1], peerstore.PermanentAddrTTL)
c, err := s1.DialPeer(context.Background(), s2.LocalPeer())
if err != nil {

View File

@@ -41,8 +41,8 @@ func makeSwarm(t *testing.T) *Swarm {
ps, err := pstoremem.NewPeerstore()
require.NoError(t, err)
ps.AddPubKey(id, priv.GetPublic())
ps.AddPrivKey(id, priv)
ps.AddPubKey(context.Background(), id, priv.GetPublic())
ps.AddPrivKey(context.Background(), id, priv)
t.Cleanup(func() { ps.Close() })
s, err := NewSwarm(id, ps, eventbus.NewBus(), WithDialTimeout(time.Second))
@@ -81,7 +81,7 @@ func makeSwarm(t *testing.T) *Swarm {
func makeUpgrader(t *testing.T, n *Swarm) transport.Upgrader {
id := n.LocalPeer()
pk := n.Peerstore().PrivKey(id)
pk := n.Peerstore().PrivKey(context.Background(), id)
st := insecure.NewWithIdentity(insecure.ID, id, pk)
u, err := tptu.New([]sec.SecureTransport{st}, []tptu.StreamMuxer{{ID: yamux.ID, Muxer: yamux.DefaultTransport}}, nil, nil, nil)
@@ -96,7 +96,7 @@ func TestDialWorkerLoopBasic(t *testing.T) {
defer s2.Close()
// Only pass in a single address here, otherwise we might end up with a TCP and QUIC connection dialed.
s1.Peerstore().AddAddrs(s2.LocalPeer(), []ma.Multiaddr{s2.ListenAddresses()[0]}, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), []ma.Multiaddr{s2.ListenAddresses()[0]}, peerstore.PermanentAddrTTL)
reqch := make(chan dialRequest)
resch := make(chan dialResponse)
@@ -142,7 +142,7 @@ func TestDialWorkerLoopConcurrent(t *testing.T) {
defer s1.Close()
defer s2.Close()
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
reqch := make(chan dialRequest)
worker := newDialWorker(s1, s2.LocalPeer(), reqch)
@@ -184,7 +184,7 @@ func TestDialWorkerLoopFailure(t *testing.T) {
_, p2 := newPeer(t)
s1.Peerstore().AddAddrs(p2, []ma.Multiaddr{ma.StringCast("/ip4/11.0.0.1/tcp/1234"), ma.StringCast("/ip4/11.0.0.1/udp/1234/quic")}, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), p2, []ma.Multiaddr{ma.StringCast("/ip4/11.0.0.1/tcp/1234"), ma.StringCast("/ip4/11.0.0.1/udp/1234/quic")}, peerstore.PermanentAddrTTL)
reqch := make(chan dialRequest)
resch := make(chan dialResponse)
@@ -209,7 +209,7 @@ func TestDialWorkerLoopConcurrentFailure(t *testing.T) {
_, p2 := newPeer(t)
s1.Peerstore().AddAddrs(p2, []ma.Multiaddr{ma.StringCast("/ip4/11.0.0.1/tcp/1234"), ma.StringCast("/ip4/11.0.0.1/udp/1234/quic")}, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), p2, []ma.Multiaddr{ma.StringCast("/ip4/11.0.0.1/tcp/1234"), ma.StringCast("/ip4/11.0.0.1/udp/1234/quic")}, peerstore.PermanentAddrTTL)
reqch := make(chan dialRequest)
worker := newDialWorker(s1, p2, reqch)
@@ -256,8 +256,8 @@ func TestDialWorkerLoopConcurrentMix(t *testing.T) {
defer s1.Close()
defer s2.Close()
s1.Peerstore().AddAddrs(s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(s2.LocalPeer(), []ma.Multiaddr{ma.StringCast("/ip4/11.0.0.1/tcp/1234"), ma.StringCast("/ip4/11.0.0.1/udp/1234/quic")}, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), s2.ListenAddresses(), peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), []ma.Multiaddr{ma.StringCast("/ip4/11.0.0.1/tcp/1234"), ma.StringCast("/ip4/11.0.0.1/udp/1234/quic")}, peerstore.PermanentAddrTTL)
reqch := make(chan dialRequest)
worker := newDialWorker(s1, s2.LocalPeer(), reqch)
@@ -303,7 +303,7 @@ func TestDialWorkerLoopConcurrentFailureStress(t *testing.T) {
for i := 0; i < 16; i++ {
addrs = append(addrs, ma.StringCast(fmt.Sprintf("/ip4/11.0.0.%d/tcp/%d", i%256, 1234+i)))
}
s1.Peerstore().AddAddrs(p2, addrs, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), p2, addrs, peerstore.PermanentAddrTTL)
reqch := make(chan dialRequest)
worker := newDialWorker(s1, p2, reqch)
@@ -379,7 +379,7 @@ func TestDialWorkerLoopAddrDedup(t *testing.T) {
defer close(closeCh)
<-ch // the routine has started listening on addr
s1.Peerstore().AddAddrs(s2.LocalPeer(), []ma.Multiaddr{t1}, peerstore.PermanentAddrTTL)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), []ma.Multiaddr{t1}, peerstore.PermanentAddrTTL)
reqch := make(chan dialRequest)
resch := make(chan dialResponse, 2)
@@ -395,8 +395,8 @@ func TestDialWorkerLoopAddrDedup(t *testing.T) {
// Need to clear backoff otherwise the dial attempt would not be made
s1.Backoff().Clear(s2.LocalPeer())
s1.Peerstore().ClearAddrs(s2.LocalPeer())
s1.Peerstore().AddAddrs(s2.LocalPeer(), []ma.Multiaddr{t2}, peerstore.PermanentAddrTTL)
s1.Peerstore().ClearAddrs(context.Background(), s2.LocalPeer())
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), []ma.Multiaddr{t2}, peerstore.PermanentAddrTTL)
reqch <- dialRequest{ctx: context.Background(), resch: resch}
select {

View File

@@ -21,7 +21,7 @@ func TestPeers(t *testing.T) {
connect := func(s *Swarm, dst peer.ID, addr ma.Multiaddr) {
// TODO: make a DialAddr func.
s.Peerstore().AddAddr(dst, addr, peerstore.PermanentAddrTTL)
s.Peerstore().AddAddr(context.Background(), dst, addr, peerstore.PermanentAddrTTL)
// t.Logf("connections from %s", s.LocalPeer())
// for _, c := range s.ConnsToPeer(dst) {
// t.Logf("connection from %s to %s: %v", s.LocalPeer(), dst, c)

View File

@@ -27,7 +27,7 @@ func TestSimultOpen(t *testing.T) {
defer wg.Done()
// copy for other peer
log.Debugf("TestSimultOpen: connecting: %s --> %s (%s)", s.LocalPeer(), dst, addr)
s.Peerstore().AddAddr(dst, addr, peerstore.PermanentAddrTTL)
s.Peerstore().AddAddr(context.Background(), dst, addr, peerstore.PermanentAddrTTL)
if _, err := s.DialPeer(context.Background(), dst); err != nil {
t.Error("error swarm dialing to peer", err)
}

View File

@@ -313,7 +313,7 @@ func (s *Swarm) addConn(tc transport.CapableConn, dir network.Direction) (*Conn,
// Add the public key.
if pk := tc.RemotePublicKey(); pk != nil {
s.peers.AddPubKey(p, pk)
s.peers.AddPubKey(context.Background(), p, pk)
}
// Clear any backoffs

View File

@@ -38,7 +38,7 @@ func TestDialBadAddrs(t *testing.T) {
test := func(a ma.Multiaddr) {
p := test.RandPeerIDFatal(t)
s.Peerstore().AddAddr(p, a, peerstore.PermanentAddrTTL)
s.Peerstore().AddAddr(context.Background(), p, a, peerstore.PermanentAddrTTL)
if _, err := s.DialPeer(context.Background(), p); err == nil {
t.Errorf("swarm should not dial: %s", p)
}

View File

@@ -300,7 +300,7 @@ func (s *Swarm) dialWorkerLoop(p peer.ID, reqch <-chan dialRequest) {
}
func (s *Swarm) addrsForDial(ctx context.Context, p peer.ID) ([]ma.Multiaddr, error) {
peerAddrs := s.peers.Addrs(p)
peerAddrs := s.peers.Addrs(ctx, p)
if len(peerAddrs) == 0 {
return nil, ErrNoAddresses
}
@@ -340,7 +340,7 @@ func (s *Swarm) addrsForDial(ctx context.Context, p peer.ID) ([]ma.Multiaddr, er
return nil, ErrNoGoodAddresses
}
s.peers.AddAddrs(p, goodAddrs, peerstore.TempAddrTTL)
s.peers.AddAddrs(ctx, p, goodAddrs, peerstore.TempAddrTTL)
return goodAddrs, nil
}

View File

@@ -42,8 +42,8 @@ func TestAddrsForDial(t *testing.T) {
ps, err := pstoremem.NewPeerstore()
require.NoError(t, err)
ps.AddPubKey(id, priv.GetPublic())
ps.AddPrivKey(id, priv)
ps.AddPubKey(context.Background(), id, priv.GetPublic())
ps.AddPrivKey(context.Background(), id, priv)
t.Cleanup(func() { ps.Close() })
tpt, err := websocket.New(nil, &network.NullResourceManager{})
@@ -56,7 +56,7 @@ func TestAddrsForDial(t *testing.T) {
otherPeer := test.RandPeerIDFatal(t)
ps.AddAddr(otherPeer, ma.StringCast("/dns4/example.com/tcp/1234/wss"), time.Hour)
ps.AddAddr(context.Background(), otherPeer, ma.StringCast("/dns4/example.com/tcp/1234/wss"), time.Hour)
ctx := context.Background()
mas, err := s.addrsForDial(ctx, otherPeer)
@@ -85,8 +85,8 @@ func TestDedupAddrsForDial(t *testing.T) {
ps, err := pstoremem.NewPeerstore()
require.NoError(t, err)
ps.AddPubKey(id, priv.GetPublic())
ps.AddPrivKey(id, priv)
ps.AddPubKey(context.Background(), id, priv.GetPublic())
ps.AddPrivKey(context.Background(), id, priv)
t.Cleanup(func() { ps.Close() })
s, err := NewSwarm(id, ps, eventbus.NewBus(), WithMultiaddrResolver(resolver))
@@ -100,8 +100,8 @@ func TestDedupAddrsForDial(t *testing.T) {
otherPeer := test.RandPeerIDFatal(t)
ps.AddAddr(otherPeer, ma.StringCast("/dns4/example.com/tcp/1234"), time.Hour)
ps.AddAddr(otherPeer, ma.StringCast("/ip4/1.2.3.4/tcp/1234"), time.Hour)
ps.AddAddr(context.Background(), otherPeer, ma.StringCast("/dns4/example.com/tcp/1234"), time.Hour)
ps.AddAddr(context.Background(), otherPeer, ma.StringCast("/ip4/1.2.3.4/tcp/1234"), time.Hour)
ctx := context.Background()
mas, err := s.addrsForDial(ctx, otherPeer)
@@ -117,8 +117,8 @@ func newTestSwarmWithResolver(t *testing.T, resolver *madns.Resolver) *Swarm {
require.NoError(t, err)
ps, err := pstoremem.NewPeerstore()
require.NoError(t, err)
ps.AddPubKey(id, priv.GetPublic())
ps.AddPrivKey(id, priv)
ps.AddPubKey(context.Background(), id, priv.GetPublic())
ps.AddPrivKey(context.Background(), id, priv)
t.Cleanup(func() { ps.Close() })
s, err := NewSwarm(id, ps, eventbus.NewBus(), WithMultiaddrResolver(resolver))
require.NoError(t, err)
@@ -156,7 +156,7 @@ func TestAddrResolution(t *testing.T) {
s := newTestSwarmWithResolver(t, resolver)
s.peers.AddAddr(p1, addr1, time.Hour)
s.peers.AddAddr(context.Background(), p1, addr1, time.Hour)
tctx, cancel := context.WithTimeout(ctx, time.Millisecond*100)
defer cancel()
@@ -166,7 +166,7 @@ func TestAddrResolution(t *testing.T) {
require.Len(t, mas, 1)
require.Contains(t, mas, addr2)
addrs := s.peers.Addrs(p1)
addrs := s.peers.Addrs(context.Background(), p1)
require.Len(t, addrs, 2)
require.Contains(t, addrs, addr1)
require.Contains(t, addrs, addr2)
@@ -217,11 +217,11 @@ func TestAddrResolutionRecursive(t *testing.T) {
tctx, cancel := context.WithTimeout(ctx, time.Millisecond*100)
defer cancel()
s.Peerstore().AddAddrs(pi1.ID, pi1.Addrs, peerstore.TempAddrTTL)
s.Peerstore().AddAddrs(context.Background(), pi1.ID, pi1.Addrs, peerstore.TempAddrTTL)
_, err = s.addrsForDial(tctx, p1)
require.NoError(t, err)
addrs1 := s.Peerstore().Addrs(pi1.ID)
addrs1 := s.Peerstore().Addrs(context.Background(), pi1.ID)
require.Len(t, addrs1, 2)
require.Contains(t, addrs1, addr1)
require.Contains(t, addrs1, addr2)
@@ -229,12 +229,12 @@ func TestAddrResolutionRecursive(t *testing.T) {
pi2, err := peer.AddrInfoFromP2pAddr(p2paddr2)
require.NoError(t, err)
s.Peerstore().AddAddrs(pi2.ID, pi2.Addrs, peerstore.TempAddrTTL)
s.Peerstore().AddAddrs(context.Background(), pi2.ID, pi2.Addrs, peerstore.TempAddrTTL)
_, err = s.addrsForDial(tctx, p2)
// This never resolves to a good address
require.Equal(t, ErrNoGoodAddresses, err)
addrs2 := s.Peerstore().Addrs(pi2.ID)
addrs2 := s.Peerstore().Addrs(context.Background(), pi2.ID)
require.Len(t, addrs2, 1)
require.Contains(t, addrs2, addr1)
}

View File

@@ -51,7 +51,7 @@ func TestConnectednessEventsSingleConn(t *testing.T) {
s1, sub1 := newSwarmWithSubscription(t)
s2, sub2 := newSwarmWithSubscription(t)
s1.Peerstore().AddAddrs(s2.LocalPeer(), []ma.Multiaddr{s2.ListenAddresses()[0]}, time.Hour)
s1.Peerstore().AddAddrs(context.Background(), s2.LocalPeer(), []ma.Multiaddr{s2.ListenAddresses()[0]}, time.Hour)
_, err := s1.DialPeer(context.Background(), s2.LocalPeer())
require.NoError(t, err)

View File

@@ -81,7 +81,7 @@ func connectSwarms(t *testing.T, ctx context.Context, swarms []*swarm.Swarm) {
var wg sync.WaitGroup
connect := func(s *swarm.Swarm, dst peer.ID, addr ma.Multiaddr) {
// TODO: make a DialAddr func.
s.Peerstore().AddAddr(dst, addr, peerstore.PermanentAddrTTL)
s.Peerstore().AddAddr(context.Background(), dst, addr, peerstore.PermanentAddrTTL)
if _, err := s.DialPeer(ctx, dst); err != nil {
t.Fatal("error swarm dialing to peer", err)
}
@@ -341,7 +341,7 @@ func TestConnectionGating(t *testing.T) {
p1 := sw1.LocalPeer()
p2 := sw2.LocalPeer()
sw1.Peerstore().AddAddr(p2, sw2.ListenAddresses()[0], peerstore.PermanentAddrTTL)
sw1.Peerstore().AddAddr(context.Background(), p2, sw2.ListenAddresses()[0], peerstore.PermanentAddrTTL)
// 1 -> 2
_, err := sw1.DialPeer(ctx, p2)
@@ -409,7 +409,7 @@ func TestPreventDialListenAddr(t *testing.T) {
}
}
remote := test.RandPeerIDFatal(t)
s.Peerstore().AddAddr(remote, addr, time.Hour)
s.Peerstore().AddAddr(context.Background(), remote, addr, time.Hour)
_, err = s.DialPeer(context.Background(), remote)
if !errors.Is(err, swarm.ErrNoGoodAddresses) {
t.Fatal("expected dial to fail: %w", err)

View File

@@ -1,6 +1,7 @@
package testing
import (
"context"
"crypto/rand"
"testing"
"time"
@@ -110,7 +111,7 @@ func EventBus(b event.Bus) Option {
// GenUpgrader creates a new connection upgrader for use with this swarm.
func GenUpgrader(t *testing.T, n *swarm.Swarm, connGater connmgr.ConnectionGater, opts ...tptu.Option) transport.Upgrader {
id := n.LocalPeer()
pk := n.Peerstore().PrivKey(id)
pk := n.Peerstore().PrivKey(context.Background(), id)
st := insecure.NewWithIdentity(insecure.ID, id, pk)
u, err := tptu.New([]sec.SecureTransport{st}, []tptu.StreamMuxer{{ID: yamux.ID, Muxer: yamux.DefaultTransport}}, nil, nil, connGater, opts...)
@@ -139,8 +140,8 @@ func GenSwarm(t *testing.T, opts ...Option) *swarm.Swarm {
ps, err := pstoremem.NewPeerstore(pstoremem.WithClock(cfg.clock))
require.NoError(t, err)
ps.AddPubKey(id, priv.GetPublic())
ps.AddPrivKey(id, priv)
ps.AddPubKey(context.Background(), id, priv.GetPublic())
ps.AddPrivKey(context.Background(), id, priv)
t.Cleanup(func() { ps.Close() })
swarmOpts := cfg.swarmOpts
@@ -193,7 +194,7 @@ func GenSwarm(t *testing.T, opts ...Option) *swarm.Swarm {
}
}
if !cfg.dialOnly {
s.Peerstore().AddAddrs(id, s.ListenAddresses(), peerstore.PermanentAddrTTL)
s.Peerstore().AddAddrs(context.Background(), id, s.ListenAddresses(), peerstore.PermanentAddrTTL)
}
return s
}
@@ -201,8 +202,8 @@ func GenSwarm(t *testing.T, opts ...Option) *swarm.Swarm {
// DivulgeAddresses adds swarm a's addresses to swarm b's peerstore.
func DivulgeAddresses(a, b network.Network) {
id := a.LocalPeer()
addrs := a.Peerstore().Addrs(id)
b.Peerstore().AddAddrs(id, addrs, peerstore.PermanentAddrTTL)
addrs := a.Peerstore().Addrs(context.Background(), id)
b.Peerstore().AddAddrs(context.Background(), id, addrs, peerstore.PermanentAddrTTL)
}
// MockConnectionGater is a mock connection gater to be used by the tests.

View File

@@ -118,7 +118,7 @@ func (c *Client) dialPeer(ctx context.Context, relay, dest peer.AddrInfo) (*Conn
log.Debugf("dialing peer %s through relay %s", dest.ID, relay.ID)
if len(relay.Addrs) > 0 {
c.host.Peerstore().AddAddrs(relay.ID, relay.Addrs, peerstore.TempAddrTTL)
c.host.Peerstore().AddAddrs(ctx, relay.ID, relay.Addrs, peerstore.TempAddrTTL)
}
dialCtx, cancel := context.WithTimeout(ctx, DialRelayTimeout)

View File

@@ -62,7 +62,7 @@ func (re ReservationError) Unwrap() error {
// Clients must reserve slots in order for the relay to relay connections to them.
func Reserve(ctx context.Context, h host.Host, ai peer.AddrInfo) (*Reservation, error) {
if len(ai.Addrs) > 0 {
h.Peerstore().AddAddrs(ai.ID, ai.Addrs, peerstore.TempAddrTTL)
h.Peerstore().AddAddrs(ctx, ai.ID, ai.Addrs, peerstore.TempAddrTTL)
}
s, err := h.NewStream(ctx, ai.ID, proto.ProtoIDv2Hop)

View File

@@ -592,7 +592,7 @@ func (r *Relay) makeReservationMsg(p peer.ID, expire time.Time) *pbv2.Reservatio
Expiration: expire,
}
envelope, err := record.Seal(voucher, r.host.Peerstore().PrivKey(r.host.ID()))
envelope, err := record.Seal(voucher, r.host.Peerstore().PrivKey(r.ctx, r.host.ID()))
if err != nil {
log.Errorf("error sealing voucher for %s: %s", p, err)
return rsvp

View File

@@ -43,7 +43,7 @@ func getNetHosts(t *testing.T, ctx context.Context, n int) (hosts []host.Host, u
if err != nil {
t.Fatal(err)
}
err = ps.AddPrivKey(p, privk)
err = ps.AddPrivKey(context.Background(), p, privk)
if err != nil {
t.Fatal(err)
}
@@ -130,7 +130,7 @@ func TestBasicRelay(t *testing.T) {
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
rinfo := hosts[1].Peerstore().PeerInfo(hosts[1].ID())
rinfo := hosts[1].Peerstore().PeerInfo(context.Background(), hosts[1].ID())
rsvp, err := client.Reserve(ctx, hosts[0], rinfo)
if err != nil {
t.Fatal(err)
@@ -209,7 +209,7 @@ func TestRelayLimitTime(t *testing.T) {
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
rinfo := hosts[1].Peerstore().PeerInfo(hosts[1].ID())
rinfo := hosts[1].Peerstore().PeerInfo(context.Background(), hosts[1].ID())
_, err = client.Reserve(ctx, hosts[0], rinfo)
if err != nil {
t.Fatal(err)
@@ -295,7 +295,7 @@ func TestRelayLimitData(t *testing.T) {
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
rinfo := hosts[1].Peerstore().PeerInfo(hosts[1].ID())
rinfo := hosts[1].Peerstore().PeerInfo(context.Background(), hosts[1].ID())
_, err = client.Reserve(ctx, hosts[0], rinfo)
if err != nil {
t.Fatal(err)

View File

@@ -113,7 +113,7 @@ func TestDirectDialWorks(t *testing.T) {
h2, _ := mkHostWithHolePunchSvc(t)
defer h2.Close()
h2.RemoveStreamHandler(holepunch.Protocol)
h1.Peerstore().AddAddrs(h2.ID(), h2.Addrs(), peerstore.ConnectedAddrTTL)
h1.Peerstore().AddAddrs(context.Background(), h2.ID(), h2.Addrs(), peerstore.ConnectedAddrTTL)
// try to hole punch without any connection and streams, if it works -> it's a direct connection
require.Len(t, h1.Network().ConnsToPeer(h2.ID()), 0)
@@ -227,7 +227,7 @@ func TestFailuresOnInitiator(t *testing.T) {
hps := addHolePunchService(t, h2, opts...)
// wait until the hole punching protocol has actually started
require.Eventually(t, func() bool {
protos, _ := h2.Peerstore().SupportsProtocols(h1.ID(), holepunch.Protocol)
protos, _ := h2.Peerstore().SupportsProtocols(context.Background(), h1.ID(), holepunch.Protocol)
return len(protos) > 0
}, 200*time.Millisecond, 10*time.Millisecond)
@@ -462,7 +462,7 @@ func makeRelayedHosts(t *testing.T, h1opt, h2opt []holepunch.Option, addHolePunc
defer h.Close()
require.NoError(t, h.Connect(context.Background(), peer.AddrInfo{ID: relay.ID(), Addrs: relay.Addrs()}))
require.Eventually(t, func() bool {
supported, err := h.Peerstore().SupportsProtocols(relay.ID(), proto.ProtoIDv2Hop)
supported, err := h.Peerstore().SupportsProtocols(context.Background(), relay.ID(), proto.ProtoIDv2Hop)
return err == nil && len(supported) > 0
}, 3*time.Second, 100*time.Millisecond)

View File

@@ -109,7 +109,7 @@ func (hp *holePuncher) directConnect(rp peer.ID) error {
// short-circuit hole punching if a direct dial works.
// attempt a direct connection ONLY if we have a public address for the remote peer
for _, a := range hp.host.Peerstore().Addrs(rp) {
for _, a := range hp.host.Peerstore().Addrs(hp.ctx, rp) {
if manet.IsPublicAddr(a) && !isRelayAddress(a) {
forceDirectConnCtx := network.WithForceDirectDial(hp.ctx, "hole-punching")
dialCtx, cancel := context.WithTimeout(forceDirectConnCtx, dialTimeout)

View File

@@ -522,7 +522,7 @@ func (ids *idService) handleIdentifyResponse(s network.Stream, isPush bool) erro
if !ok { // might already have disconnected
return nil
}
sup, err := ids.Host.Peerstore().SupportsProtocols(c.RemotePeer(), IDPush)
sup, err := ids.Host.Peerstore().SupportsProtocols(context.TODO(), c.RemotePeer(), IDPush)
if supportsIdentifyPush := err == nil && len(sup) > 0; supportsIdentifyPush {
e.PushSupport = identifyPushSupported
} else {
@@ -565,7 +565,7 @@ func (ids *idService) updateSnapshot() (updated bool) {
if !ids.disableSignedPeerRecord {
if cab, ok := peerstore.GetCertifiedAddrBook(ids.Host.Peerstore()); ok {
snapshot.record = cab.GetPeerRecord(ids.Host.ID())
snapshot.record = cab.GetPeerRecord(context.TODO(), ids.Host.ID())
}
}
@@ -624,13 +624,13 @@ func (ids *idService) createBaseIdentifyResponse(conn network.Conn, snapshot *id
mes.ListenAddrs = append(mes.ListenAddrs, addr.Bytes())
}
// set our public key
ownKey := ids.Host.Peerstore().PubKey(ids.Host.ID())
ownKey := ids.Host.Peerstore().PubKey(context.TODO(), ids.Host.ID())
// check if we even have a public key.
if ownKey == nil {
// public key is nil. We are either using insecure transport or something erratic happened.
// check if we're even operating in "secure mode"
if ids.Host.Peerstore().PrivKey(ids.Host.ID()) != nil {
if ids.Host.Peerstore().PrivKey(context.TODO(), ids.Host.ID()) != nil {
// private key is present. But NO public key. Something bad happened.
log.Errorf("did not have own public key in Peerstore")
}
@@ -698,10 +698,10 @@ func diff(a, b []protocol.ID) (added, removed []protocol.ID) {
func (ids *idService) consumeMessage(mes *pb.Identify, c network.Conn, isPush bool) {
p := c.RemotePeer()
supported, _ := ids.Host.Peerstore().GetProtocols(p)
supported, _ := ids.Host.Peerstore().GetProtocols(context.TODO(), p)
mesProtocols := protocol.ConvertFromStrings(mes.Protocols)
added, removed := diff(supported, mesProtocols)
ids.Host.Peerstore().SetProtocols(p, mesProtocols...)
ids.Host.Peerstore().SetProtocols(context.TODO(), p, mesProtocols...)
if isPush {
ids.emitters.evtPeerProtocolsUpdated.Emit(event.EvtPeerProtocolsUpdated{
Peer: p,
@@ -754,26 +754,27 @@ func (ids *idService) consumeMessage(mes *pb.Identify, c network.Conn, isPush bo
peerstore.RecentlyConnectedAddrTTL,
peerstore.ConnectedAddrTTL,
} {
ids.Host.Peerstore().UpdateAddrs(p, ttl, peerstore.TempAddrTTL)
ids.Host.Peerstore().UpdateAddrs(context.TODO(), p, ttl, peerstore.TempAddrTTL)
}
// add signed addrs if we have them and the peerstore supports them
cab, ok := peerstore.GetCertifiedAddrBook(ids.Host.Peerstore())
if ok && signedPeerRecord != nil && signedPeerRecord.PublicKey != nil {
id, err := peer.IDFromPublicKey(signedPeerRecord.PublicKey)
if err != nil {
log.Debugf("failed to derive peer ID from peer record: %s", err)
} else if id != c.RemotePeer() {
log.Debugf("received signed peer record for unexpected peer ID. expected %s, got %s", c.RemotePeer(), id)
} else if _, err := cab.ConsumePeerRecord(signedPeerRecord, ttl); err != nil {
} else if _, err := cab.ConsumePeerRecord(context.TODO(), signedPeerRecord, ttl); err != nil {
log.Debugf("error adding signed addrs to peerstore: %v", err)
}
} else {
ids.Host.Peerstore().AddAddrs(p, filterAddrs(lmaddrs, c.RemoteMultiaddr()), ttl)
ids.Host.Peerstore().AddAddrs(context.TODO(), p, filterAddrs(lmaddrs, c.RemoteMultiaddr()), ttl)
}
// Finally, expire all temporary addrs.
ids.Host.Peerstore().UpdateAddrs(p, peerstore.TempAddrTTL, 0)
ids.Host.Peerstore().UpdateAddrs(context.TODO(), p, peerstore.TempAddrTTL, 0)
ids.addrMu.Unlock()
log.Debugf("%s received listen addrs for %s: %s", c.LocalPeer(), c.RemotePeer(), lmaddrs)
@@ -782,8 +783,8 @@ func (ids *idService) consumeMessage(mes *pb.Identify, c network.Conn, isPush bo
pv := mes.GetProtocolVersion()
av := mes.GetAgentVersion()
ids.Host.Peerstore().Put(p, "ProtocolVersion", pv)
ids.Host.Peerstore().Put(p, "AgentVersion", av)
ids.Host.Peerstore().Put(context.TODO(), p, "ProtocolVersion", pv)
ids.Host.Peerstore().Put(context.TODO(), p, "AgentVersion", av)
// get the key from the other side. we may not have it (no-auth transport)
ids.consumeReceivedPubKey(c, mes.PublicKey)
@@ -816,7 +817,7 @@ func (ids *idService) consumeReceivedPubKey(c network.Conn, kb []byte) {
if rp == "" && np != "" {
// if local peerid is empty, then use the new, sent key.
err := ids.Host.Peerstore().AddPubKey(rp, newKey)
err := ids.Host.Peerstore().AddPubKey(context.TODO(), rp, newKey)
if err != nil {
log.Debugf("%s could not add key for %s to peerstore: %s", lp, rp, err)
}
@@ -828,10 +829,10 @@ func (ids *idService) consumeReceivedPubKey(c network.Conn, kb []byte) {
return
}
currKey := ids.Host.Peerstore().PubKey(rp)
currKey := ids.Host.Peerstore().PubKey(context.TODO(), rp)
if currKey == nil {
// no key? no auth transport. set this one.
err := ids.Host.Peerstore().AddPubKey(rp, newKey)
err := ids.Host.Peerstore().AddPubKey(context.TODO(), rp, newKey)
if err != nil {
log.Debugf("%s could not add key for %s to peerstore: %s", lp, rp, err)
}
@@ -953,7 +954,7 @@ func (nn *netNotifiee) Disconnected(_ network.Network, c network.Conn) {
// Undo the setting of addresses to peer.ConnectedAddrTTL we did
ids.addrMu.Lock()
defer ids.addrMu.Unlock()
ids.Host.Peerstore().UpdateAddrs(c.RemotePeer(), peerstore.ConnectedAddrTTL, peerstore.RecentlyConnectedAddrTTL)
ids.Host.Peerstore().UpdateAddrs(context.TODO(), c.RemotePeer(), peerstore.ConnectedAddrTTL, peerstore.RecentlyConnectedAddrTTL)
}
}

View File

@@ -118,7 +118,7 @@ func TestWrongSignedPeerRecord(t *testing.T) {
require.NoError(t, err)
time.Sleep(time.Second)
require.Empty(t, h1.Peerstore().Addrs(h3.ID()), "h1 should not know about h3 since it was relayed over h2")
require.Empty(t, h1.Peerstore().Addrs(context.Background(), h3.ID()), "h1 should not know about h3 since it was relayed over h2")
}
func TestInvalidSignedPeerRecord(t *testing.T) {
@@ -138,7 +138,7 @@ func TestInvalidSignedPeerRecord(t *testing.T) {
// ids2.Start()
h2.Connect(context.Background(), peer.AddrInfo{ID: h1.ID(), Addrs: h1.Addrs()})
require.Empty(t, h1.Peerstore().Addrs(h2.ID()))
require.Empty(t, h1.Peerstore().Addrs(context.Background(), h2.ID()))
s, err := h2.NewStream(context.Background(), h1.ID(), IDPush)
require.NoError(t, err)
@@ -171,5 +171,5 @@ func TestInvalidSignedPeerRecord(t *testing.T) {
cab, ok := h1.Peerstore().(peerstore.CertifiedAddrBook)
require.True(t, ok)
require.Nil(t, cab.GetPeerRecord(h2.ID()))
require.Nil(t, cab.GetPeerRecord(context.Background(), h2.ID()))
}

View File

@@ -42,7 +42,7 @@ func init() {
func testKnowsAddrs(t *testing.T, h host.Host, p peer.ID, expected []ma.Multiaddr) {
t.Helper()
require.True(t, assert.ElementsMatchf(t, expected, h.Peerstore().Addrs(p), fmt.Sprintf("%s did not have addr for %s", h.ID(), p)))
require.True(t, assert.ElementsMatchf(t, expected, h.Peerstore().Addrs(context.Background(), p), fmt.Sprintf("%s did not have addr for %s", h.ID(), p)))
}
func testHasCertifiedAddrs(t *testing.T, h host.Host, p peer.ID, expected []ma.Multiaddr) {
@@ -51,7 +51,7 @@ func testHasCertifiedAddrs(t *testing.T, h host.Host, p peer.ID, expected []ma.M
if !ok {
t.Error("expected peerstore to implement CertifiedAddrBook")
}
recordEnvelope := cab.GetPeerRecord(p)
recordEnvelope := cab.GetPeerRecord(context.Background(), p)
if recordEnvelope == nil {
if len(expected) == 0 {
return
@@ -70,14 +70,14 @@ func testHasCertifiedAddrs(t *testing.T, h host.Host, p peer.ID, expected []ma.M
}
func testHasAgentVersion(t *testing.T, h host.Host, p peer.ID) {
v, err := h.Peerstore().Get(p, "AgentVersion")
v, err := h.Peerstore().Get(context.Background(), p, "AgentVersion")
if v.(string) != "github.com/libp2p/go-libp2p" { // this is the default user agent
t.Error("agent version mismatch", err)
}
}
func testHasPublicKey(t *testing.T, h host.Host, p peer.ID, shouldBe ic.PubKey) {
k := h.Peerstore().PubKey(p)
k := h.Peerstore().PubKey(context.Background(), p)
if k == nil {
t.Error("no public key")
return
@@ -98,7 +98,7 @@ func testHasPublicKey(t *testing.T, h host.Host, p peer.ID, shouldBe ic.PubKey)
func getSignedRecord(t *testing.T, h host.Host, p peer.ID) *record.Envelope {
cab, ok := peerstore.GetCertifiedAddrBook(h.Peerstore())
require.True(t, ok)
rec := cab.GetPeerRecord(p)
rec := cab.GetPeerRecord(context.Background(), p)
return rec
}
@@ -109,7 +109,7 @@ func getSignedRecord(t *testing.T, h host.Host, p peer.ID) *record.Envelope {
func emitAddrChangeEvt(t *testing.T, h host.Host) {
t.Helper()
key := h.Peerstore().PrivKey(h.ID())
key := h.Peerstore().PrivKey(context.Background(), h.ID())
if key == nil {
t.Fatal("no private key for host")
}
@@ -124,7 +124,7 @@ func emitAddrChangeEvt(t *testing.T, h host.Host) {
cab, ok := peerstore.GetCertifiedAddrBook(h.Peerstore())
require.True(t, ok)
_, err = cab.ConsumePeerRecord(signed, peerstore.PermanentAddrTTL)
_, err = cab.ConsumePeerRecord(context.Background(), signed, peerstore.PermanentAddrTTL)
require.NoError(t, err)
evt := event.EvtLocalAddressesUpdated{}
@@ -179,8 +179,8 @@ func TestIDService(t *testing.T) {
// forgotten and replaced by the addrs h1 sends.
forgetMe, _ := ma.NewMultiaddr("/ip4/1.2.3.4/tcp/1234")
h2.Peerstore().AddAddr(h1p, forgetMe, peerstore.RecentlyConnectedAddrTTL)
h2pi := h2.Peerstore().PeerInfo(h2p)
h2.Peerstore().AddAddr(context.Background(), h1p, forgetMe, peerstore.RecentlyConnectedAddrTTL)
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2p)
require.NoError(t, h1.Connect(context.Background(), h2pi))
h1t2c := h1.Network().ConnsToPeer(h2p)
@@ -191,10 +191,10 @@ func TestIDService(t *testing.T) {
// the idService should be opened automatically, by the network.
// what we should see now is that both peers know about each others listen addresses.
t.Log("test peer1 has peer2 addrs correctly")
testKnowsAddrs(t, h1, h2p, h2.Addrs()) // has them
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(h2p)) // should have signed addrs also
testKnowsAddrs(t, h1, h2p, h2.Addrs()) // has them
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(context.Background(), h2p)) // should have signed addrs also
testHasAgentVersion(t, h1, h2p)
testHasPublicKey(t, h1, h2p, h2.Peerstore().PubKey(h2p)) // h1 should have h2's public key
testHasPublicKey(t, h1, h2p, h2.Peerstore().PubKey(context.Background(), h2p)) // h1 should have h2's public key
// now, this wait we do have to do. it's the wait for the Listening side
// to be done identifying the connection.
@@ -205,9 +205,9 @@ func TestIDService(t *testing.T) {
// and the protocol versions.
t.Log("test peer2 has peer1 addrs correctly")
testKnowsAddrs(t, h2, h1p, h1.Addrs()) // has them
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(h1p))
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(context.Background(), h1p))
testHasAgentVersion(t, h2, h1p)
testHasPublicKey(t, h2, h1p, h1.Peerstore().PubKey(h1p)) // h1 should have h2's public key
testHasPublicKey(t, h2, h1p, h1.Peerstore().PubKey(context.Background(), h1p)) // h1 should have h2's public key
// Need both sides to actually notice that the connection has been closed.
sentDisconnect1 := waitForDisconnectNotification(swarm1)
@@ -222,8 +222,8 @@ func TestIDService(t *testing.T) {
// addresses don't immediately expire on disconnect, so we should still have them
testKnowsAddrs(t, h2, h1p, h1.Addrs())
testKnowsAddrs(t, h1, h2p, h2.Addrs())
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(h2p))
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(h1p))
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(context.Background(), h2p))
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(context.Background(), h1p))
<-sentDisconnect1
<-sentDisconnect2
@@ -271,7 +271,7 @@ func TestLocalhostAddrFiltering(t *testing.T) {
}
p1addr1, _ := ma.NewMultiaddr("/ip4/1.2.3.4/tcp/1234")
p1addr2, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/2345")
ps1.AddAddrs(id1, []ma.Multiaddr{p1addr1, p1addr2}, peerstore.PermanentAddrTTL)
ps1.AddAddrs(context.Background(), id1, []ma.Multiaddr{p1addr1, p1addr2}, peerstore.PermanentAddrTTL)
p1, err := mn.AddPeerWithPeerstore(id1, ps1)
if err != nil {
t.Fatal(err)
@@ -285,7 +285,7 @@ func TestLocalhostAddrFiltering(t *testing.T) {
p2addr1, _ := ma.NewMultiaddr("/ip4/1.2.3.5/tcp/1234")
p2addr2, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/3456")
p2addrs := []ma.Multiaddr{p2addr1, p2addr2}
ps2.AddAddrs(id2, p2addrs, peerstore.PermanentAddrTTL)
ps2.AddAddrs(context.Background(), id2, p2addrs, peerstore.PermanentAddrTTL)
p2, err := mn.AddPeerWithPeerstore(id2, ps2)
if err != nil {
t.Fatal(err)
@@ -297,7 +297,7 @@ func TestLocalhostAddrFiltering(t *testing.T) {
t.Fatal(err)
}
p3addr1, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/4567")
ps3.AddAddrs(id3, []ma.Multiaddr{p3addr1}, peerstore.PermanentAddrTTL)
ps3.AddAddrs(context.Background(), id3, []ma.Multiaddr{p3addr1}, peerstore.PermanentAddrTTL)
p3, err := mn.AddPeerWithPeerstore(id3, ps3)
if err != nil {
t.Fatal(err)
@@ -340,7 +340,7 @@ func TestLocalhostAddrFiltering(t *testing.T) {
}
conn := conns[0]
ids2.IdentifyConn(conn)
addrs := p2.Peerstore().Addrs(id1)
addrs := p2.Peerstore().Addrs(context.Background(), id1)
if len(addrs) != 1 {
t.Fatalf("expected one addr, found %s", addrs)
}
@@ -351,7 +351,7 @@ func TestLocalhostAddrFiltering(t *testing.T) {
}
conn = conns[0]
ids3.IdentifyConn(conn)
addrs = p3.Peerstore().Addrs(id2)
addrs = p3.Peerstore().Addrs(context.Background(), id2)
if len(addrs) != 2 {
t.Fatalf("expected 2 addrs for %s, found %d: %s", id2, len(addrs), addrs)
}
@@ -455,7 +455,7 @@ func TestIdentifyPushOnAddrChange(t *testing.T) {
testKnowsAddrs(t, h1, h2p, []ma.Multiaddr{}) // nothing
testKnowsAddrs(t, h2, h1p, []ma.Multiaddr{}) // nothing
require.NoError(t, h1.Connect(ctx, h2.Peerstore().PeerInfo(h2p)))
require.NoError(t, h1.Connect(ctx, h2.Peerstore().PeerInfo(context.Background(), h2p)))
// h1 should immediately see a connection from h2
require.NotEmpty(t, h1.Network().ConnsToPeer(h2p))
// wait for h2 to Identify itself so we are sure h2 has seen the connection.
@@ -465,8 +465,8 @@ func TestIdentifyPushOnAddrChange(t *testing.T) {
require.NotEmpty(t, h2.Network().ConnsToPeer(h1p))
ids2.IdentifyConn(h2.Network().ConnsToPeer(h1p)[0])
testKnowsAddrs(t, h1, h2p, h2.Peerstore().Addrs(h2p))
testKnowsAddrs(t, h2, h1p, h1.Peerstore().Addrs(h1p))
testKnowsAddrs(t, h1, h2p, h2.Peerstore().Addrs(context.Background(), h2p))
testKnowsAddrs(t, h2, h1p, h1.Peerstore().Addrs(context.Background(), h1p))
// change addr on host 1 and ensure host2 gets a push
lad := ma.StringCast("/ip4/127.0.0.1/tcp/1234")
@@ -480,7 +480,7 @@ func TestIdentifyPushOnAddrChange(t *testing.T) {
// Wait for h2 to process the new addr
waitForAddrInStream(t, h2AddrStream, lad, 10*time.Second, "h2 did not receive addr change")
require.True(t, ma.Contains(h2.Peerstore().Addrs(h1p), lad))
require.True(t, ma.Contains(h2.Peerstore().Addrs(context.Background(), h1p), lad))
require.NotNil(t, getSignedRecord(t, h2, h1p))
// change addr on host2 and ensure host 1 gets a pus
@@ -493,7 +493,7 @@ func TestIdentifyPushOnAddrChange(t *testing.T) {
// Wait for h1 to process the new addr
waitForAddrInStream(t, h1AddrStream, lad, 10*time.Second, "h1 did not receive addr change")
require.True(t, ma.Contains(h1.Peerstore().Addrs(h2p), lad))
require.True(t, ma.Contains(h1.Peerstore().Addrs(context.Background(), h2p), lad))
require.NotNil(t, getSignedRecord(t, h1, h2p))
// change addr on host2 again
@@ -505,7 +505,7 @@ func TestIdentifyPushOnAddrChange(t *testing.T) {
// Wait for h1 to process the new addr
waitForAddrInStream(t, h1AddrStream, lad2, 10*time.Second, "h1 did not receive addr change")
require.True(t, ma.Contains(h1.Peerstore().Addrs(h2p), lad2))
require.True(t, ma.Contains(h1.Peerstore().Addrs(context.Background(), h2p), lad2))
require.NotNil(t, getSignedRecord(t, h1, h2p))
}
@@ -529,7 +529,7 @@ func TestUserAgent(t *testing.T) {
if err != nil {
t.Fatal(err)
}
av, err := h1.Peerstore().Get(h2.ID(), "AgentVersion")
av, err := h1.Peerstore().Get(context.Background(), h2.ID(), "AgentVersion")
if err != nil {
t.Fatal(err)
}
@@ -592,14 +592,14 @@ func TestSendPush(t *testing.T) {
// h1 starts listening on a new protocol and h2 finds out about that through a push
h1.SetStreamHandler("rand", func(network.Stream) {})
require.Eventually(t, func() bool {
sup, err := h2.Peerstore().SupportsProtocols(h1.ID(), []protocol.ID{"rand"}...)
sup, err := h2.Peerstore().SupportsProtocols(context.Background(), h1.ID(), []protocol.ID{"rand"}...)
return err == nil && len(sup) == 1 && sup[0] == "rand"
}, time.Second, 10*time.Millisecond)
// h1 stops listening on a protocol and h2 finds out about it via a push
h1.RemoveStreamHandler("rand")
require.Eventually(t, func() bool {
sup, err := h2.Peerstore().SupportsProtocols(h1.ID(), []protocol.ID{"rand"}...)
sup, err := h2.Peerstore().SupportsProtocols(context.Background(), h1.ID(), []protocol.ID{"rand"}...)
return err == nil && len(sup) == 0
}, time.Second, 10*time.Millisecond)
}
@@ -646,9 +646,9 @@ func TestLargeIdentifyMessage(t *testing.T) {
// (not via identify protocol). During the identify exchange, it will be
// forgotten and replaced by the addrs h1 sends.
forgetMe, _ := ma.NewMultiaddr("/ip4/1.2.3.4/tcp/1234")
h2.Peerstore().AddAddr(h1p, forgetMe, peerstore.RecentlyConnectedAddrTTL)
h2.Peerstore().AddAddr(context.Background(), h1p, forgetMe, peerstore.RecentlyConnectedAddrTTL)
h2pi := h2.Peerstore().PeerInfo(h2p)
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2p)
h2pi.Addrs = h2pi.Addrs[:1]
require.NoError(t, h1.Connect(context.Background(), h2pi))
@@ -660,10 +660,10 @@ func TestLargeIdentifyMessage(t *testing.T) {
// the idService should be opened automatically, by the network.
// what we should see now is that both peers know about each others listen addresses.
t.Log("test peer1 has peer2 addrs correctly")
testKnowsAddrs(t, h1, h2p, h2.Addrs()) // has them
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(h2p)) // should have signed addrs also
testKnowsAddrs(t, h1, h2p, h2.Addrs()) // has them
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(context.Background(), h2p)) // should have signed addrs also
testHasAgentVersion(t, h1, h2p)
testHasPublicKey(t, h1, h2p, h2.Peerstore().PubKey(h2p)) // h1 should have h2's public key
testHasPublicKey(t, h1, h2p, h2.Peerstore().PubKey(context.Background(), h2p)) // h1 should have h2's public key
// now, this wait we do have to do. it's the wait for the Listening side
// to be done identifying the connection.
@@ -676,9 +676,9 @@ func TestLargeIdentifyMessage(t *testing.T) {
// and the protocol versions.
t.Log("test peer2 has peer1 addrs correctly")
testKnowsAddrs(t, h2, h1p, h1.Addrs()) // has them
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(h1p))
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(context.Background(), h1p))
testHasAgentVersion(t, h2, h1p)
testHasPublicKey(t, h2, h1p, h1.Peerstore().PubKey(h1p)) // h1 should have h2's public key
testHasPublicKey(t, h2, h1p, h1.Peerstore().PubKey(context.Background(), h1p)) // h1 should have h2's public key
// Need both sides to actually notice that the connection has been closed.
sentDisconnect1 := waitForDisconnectNotification(swarm1)
@@ -693,8 +693,8 @@ func TestLargeIdentifyMessage(t *testing.T) {
// addresses don't immediately expire on disconnect, so we should still have them
testKnowsAddrs(t, h2, h1p, h1.Addrs())
testKnowsAddrs(t, h1, h2p, h2.Addrs())
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(h2p))
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(h1p))
testHasCertifiedAddrs(t, h1, h2p, h2.Peerstore().Addrs(context.Background(), h2p))
testHasCertifiedAddrs(t, h2, h1p, h1.Peerstore().Addrs(context.Background(), h1p))
<-sentDisconnect1
<-sentDisconnect2
@@ -747,7 +747,7 @@ func TestLargePushMessage(t *testing.T) {
testKnowsAddrs(t, h1, h2p, []ma.Multiaddr{}) // nothing
testKnowsAddrs(t, h2, h1p, []ma.Multiaddr{}) // nothing
h2pi := h2.Peerstore().PeerInfo(h2p)
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2p)
require.NoError(t, h1.Connect(ctx, h2pi))
// h1 should immediately see a connection from h2
require.NotEmpty(t, h1.Network().ConnsToPeer(h2p))
@@ -758,8 +758,8 @@ func TestLargePushMessage(t *testing.T) {
require.NotEmpty(t, h2.Network().ConnsToPeer(h1p))
ids2.IdentifyConn(h2.Network().ConnsToPeer(h1p)[0])
testKnowsAddrs(t, h1, h2p, h2.Peerstore().Addrs(h2p))
testKnowsAddrs(t, h2, h1p, h1.Peerstore().Addrs(h1p))
testKnowsAddrs(t, h1, h2p, h2.Peerstore().Addrs(context.Background(), h2p))
testKnowsAddrs(t, h2, h1p, h1.Peerstore().Addrs(context.Background(), h1p))
// change addr on host 1 and ensure host2 gets a push
lad := ma.StringCast("/ip4/127.0.0.1/tcp/1234")
@@ -768,7 +768,7 @@ func TestLargePushMessage(t *testing.T) {
emitAddrChangeEvt(t, h1)
require.Eventually(t, func() bool {
return ma.Contains(h2.Peerstore().Addrs(h1p), lad)
return ma.Contains(h2.Peerstore().Addrs(context.Background(), h1p), lad)
}, time.Second, 10*time.Millisecond)
require.NotNil(t, getSignedRecord(t, h2, h1p))
@@ -779,7 +779,7 @@ func TestLargePushMessage(t *testing.T) {
emitAddrChangeEvt(t, h2)
require.Eventually(t, func() bool {
return ma.Contains(h1.Peerstore().Addrs(h2p), lad)
return ma.Contains(h1.Peerstore().Addrs(context.Background(), h2p), lad)
}, time.Second, 10*time.Millisecond)
testHasCertifiedAddrs(t, h1, h2p, h2.Addrs())
@@ -790,7 +790,7 @@ func TestLargePushMessage(t *testing.T) {
emitAddrChangeEvt(t, h2)
require.Eventually(t, func() bool {
return ma.Contains(h1.Peerstore().Addrs(h2p), lad2)
return ma.Contains(h1.Peerstore().Addrs(context.Background(), h2p), lad2)
}, time.Second, 10*time.Millisecond)
testHasCertifiedAddrs(t, h2, h1p, h1.Addrs())
}
@@ -829,7 +829,7 @@ func TestIdentifyResponseReadTimeout(t *testing.T) {
sub, err := ids1.Host.EventBus().Subscribe(new(event.EvtPeerIdentificationFailed))
require.NoError(t, err)
h2pi := h2.Peerstore().PeerInfo(h2p)
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2p)
require.NoError(t, h1.Connect(ctx, h2pi))
select {
@@ -870,7 +870,7 @@ func TestIncomingIDStreamsTimeout(t *testing.T) {
ids2.Start()
h2p := h2.ID()
h2pi := h2.Peerstore().PeerInfo(h2p)
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2p)
require.NoError(t, h1.Connect(ctx, h2pi))
_, err = h1.NewStream(ctx, h2p, p)
@@ -904,7 +904,7 @@ func TestOutOfOrderConnectedNotifs(t *testing.T) {
idChan := bh1.IDService().IdentifyWait(c)
go func() {
<-idChan
protos, err := bh1.Peerstore().GetProtocols(h2.ID())
protos, err := bh1.Peerstore().GetProtocols(context.Background(), h2.ID())
if err != nil {
errCh <- err
}

View File

@@ -36,7 +36,7 @@ func TestStBackpressureStreamWrite(t *testing.T) {
<-ctx.Done()
})
h2pi := h2.Peerstore().PeerInfo(h2.ID())
h2pi := h2.Peerstore().PeerInfo(context.Background(), h2.ID())
log.Debugf("dialing %s", h2pi.Addrs)
if err := h1.Connect(ctx, h2pi); err != nil {
t.Fatal("Failed to connect:", err)

View File

@@ -64,7 +64,7 @@ func runRound(t *testing.T, hosts []host.Host) {
h1.SetStreamHandler(protocol.TestingID, EchoStreamHandler)
for _, h2 := range hosts {
h1.Peerstore().AddAddrs(h2.ID(), h2.Addrs(), time.Hour)
h1.Peerstore().AddAddrs(context.Background(), h2.ID(), h2.Addrs(), time.Hour)
}
}

View File

@@ -38,7 +38,7 @@ func createEchos(t *testing.T, count int, makeOpts ...func(int) libp2p.Option) [
continue
}
result[i].Host.Peerstore().AddAddrs(result[j].Host.ID(), result[j].Host.Addrs(), peerstore.PermanentAddrTTL)
result[i].Host.Peerstore().AddAddrs(context.Background(), result[j].Host.ID(), result[j].Host.Addrs(), peerstore.PermanentAddrTTL)
}
}

View File

@@ -163,7 +163,7 @@ func TestInterceptAccept(t *testing.T) {
// remove the certhash component from WebTransport addresses
require.Equal(t, stripCertHash(h2.Addrs()[0]), addrs.LocalMultiaddr())
})
h1.Peerstore().AddAddrs(h2.ID(), h2.Addrs(), time.Hour)
h1.Peerstore().AddAddrs(ctx, h2.ID(), h2.Addrs(), time.Hour)
_, err := h1.NewStream(ctx, h2.ID(), protocol.TestingID)
require.Error(t, err)
require.NotErrorIs(t, err, context.DeadlineExceeded)
@@ -194,7 +194,7 @@ func TestInterceptSecuredIncoming(t *testing.T) {
require.Equal(t, stripCertHash(h2.Addrs()[0]), addrs.LocalMultiaddr())
}),
)
h1.Peerstore().AddAddrs(h2.ID(), h2.Addrs(), time.Hour)
h1.Peerstore().AddAddrs(ctx, h2.ID(), h2.Addrs(), time.Hour)
_, err := h1.NewStream(ctx, h2.ID(), protocol.TestingID)
require.Error(t, err)
require.NotErrorIs(t, err, context.DeadlineExceeded)
@@ -228,7 +228,7 @@ func TestInterceptUpgradedIncoming(t *testing.T) {
require.Equal(t, h2.ID(), c.LocalPeer())
}),
)
h1.Peerstore().AddAddrs(h2.ID(), h2.Addrs(), time.Hour)
h1.Peerstore().AddAddrs(ctx, h2.ID(), h2.Addrs(), time.Hour)
_, err := h1.NewStream(ctx, h2.ID(), protocol.TestingID)
require.Error(t, err)
require.NotErrorIs(t, err, context.DeadlineExceeded)