diff --git a/.circleci/config.yml b/.circleci/config.yml index f6cc134f0..fa54d035a 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1,10 +1,14 @@ -version: 2 -jobs: - build: - docker: - - image: circleci/golang:1.12 - environment: - GO111MODULE: "on" - steps: - - checkout - - run: go test -v -race -bench . ./... +version: 2.1 +orbs: + ci-go: ipfs/ci-go@0.2.9 + +workflows: + version: 2 + test: + jobs: + - ci-go/build + - ci-go/lint + - ci-go/test + - ci-go/test: + race: true + name: "ci-go/test/race" diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 38bb1b842..000000000 --- a/.travis.yml +++ /dev/null @@ -1,31 +0,0 @@ -os: - - linux - -language: go - -go: - - 1.14.x - -env: - global: - - GOTFLAGS="-race" - - IPFS_REUSEPORT=false - matrix: - - BUILD_DEPTYPE=gomod - - -# disable travis install -install: - - true - -script: - - bash <(curl -s https://raw.githubusercontent.com/ipfs/ci-helpers/master/travis-ci/run-standard-tests.sh) - - -cache: - directories: - - $GOPATH/pkg/mod - - /home/travis/.cache/go-build - -notifications: - email: false diff --git a/dht.go b/dht.go index 2fef7dd60..e83eb47f8 100644 --- a/dht.go +++ b/dht.go @@ -5,6 +5,7 @@ import ( "context" "errors" "fmt" + "math" "sync" "time" @@ -18,18 +19,17 @@ import ( "go.opencensus.io/tag" "github.com/libp2p/go-libp2p-kad-dht/metrics" - opts "github.com/libp2p/go-libp2p-kad-dht/opts" pb "github.com/libp2p/go-libp2p-kad-dht/pb" "github.com/libp2p/go-libp2p-kad-dht/providers" + kb "github.com/libp2p/go-libp2p-kbucket" + record "github.com/libp2p/go-libp2p-record" + recpb "github.com/libp2p/go-libp2p-record/pb" "github.com/gogo/protobuf/proto" ds "github.com/ipfs/go-datastore" logging "github.com/ipfs/go-log" "github.com/jbenet/goprocess" goprocessctx "github.com/jbenet/goprocess/context" - kb "github.com/libp2p/go-libp2p-kbucket" - record "github.com/libp2p/go-libp2p-record" - recpb "github.com/libp2p/go-libp2p-record/pb" "github.com/multiformats/go-base32" "github.com/multiformats/go-multihash" ) @@ -38,6 +38,18 @@ var logger = logging.Logger("dht") const BaseConnMgrScore = 5 +type mode int + +const ( + modeServer mode = iota + 1 + modeClient +) + +const ( + kad1 protocol.ID = "/kad/1.0.0" + kad2 protocol.ID = "/kad/2.0.0" +) + // IpfsDHT is an implementation of Kademlia with S/Kademlia modifications. // It is used to implement the base Routing module. type IpfsDHT struct { @@ -65,14 +77,29 @@ type IpfsDHT struct { stripedPutLocks [256]sync.Mutex - protocols []protocol.ID // DHT protocols + // DHT protocols we query with. We'll only add peers to our routing + // table if they speak these protocols. + protocols []protocol.ID + + // DHT protocols we can respond to. + serverProtocols []protocol.ID + + auto bool + mode mode + modeLk sync.Mutex bucketSize int + alpha int // The concurrency parameter per path + beta int // The number of peers closest to a target that must have responded for a query path to terminate + + queryPeerFilter QueryFilterFunc + routingTablePeerFilter RouteTableFilterFunc autoRefresh bool rtRefreshQueryTimeout time.Duration - rtRefreshPeriod time.Duration + rtRefreshInterval time.Duration triggerRtRefresh chan chan<- error + triggerSelfLookup chan chan<- error maxRecordAge time.Duration @@ -80,6 +107,13 @@ type IpfsDHT struct { // "forked" DHTs (e.g., DHTs with custom protocols and/or private // networks). enableProviders, enableValues bool + + // maxLastSuccessfulOutboundThreshold is the max threshold/upper limit for the value of "lastSuccessfulOutboundQuery" + // of the peer in the bucket above which we will evict it to make place for a new peer if the bucket + // is full + maxLastSuccessfulOutboundThreshold time.Duration + + fixLowPeersChan chan struct{} } // Assert that IPFS assumptions about interfaces aren't broken. These aren't a @@ -93,33 +127,72 @@ var ( ) // New creates a new DHT with the specified host and options. -func New(ctx context.Context, h host.Host, options ...opts.Option) (*IpfsDHT, error) { - var cfg opts.Options - cfg.BucketSize = KValue - if err := cfg.Apply(append([]opts.Option{opts.Defaults}, options...)...); err != nil { +// Please note that being connected to a DHT peer does not necessarily imply that it's also in the DHT Routing Table. +// If the Routing Table has more than "minRTRefreshThreshold" peers, we consider a peer as a Routing Table candidate ONLY when +// we successfully get a query response from it OR if it send us a query. +func New(ctx context.Context, h host.Host, options ...Option) (*IpfsDHT, error) { + var cfg config + if err := cfg.apply(append([]Option{defaults}, options...)...); err != nil { return nil, err } - dht := makeDHT(ctx, h, &cfg) - dht.autoRefresh = cfg.RoutingTable.AutoRefresh - dht.rtRefreshPeriod = cfg.RoutingTable.RefreshPeriod - dht.rtRefreshQueryTimeout = cfg.RoutingTable.RefreshQueryTimeout - - dht.maxRecordAge = cfg.MaxRecordAge - dht.enableProviders = cfg.EnableProviders - dht.enableValues = cfg.EnableValues - // register for network notifs. - dht.host.Network().Notify((*netNotifiee)(dht)) + if err := cfg.validate(); err != nil { + return nil, err + } + dht, err := makeDHT(ctx, h, cfg) + if err != nil { + return nil, fmt.Errorf("failed to create DHT, err=%s", err) + } - dht.proc.AddChild(dht.ProviderManager.Process()) - dht.Validator = cfg.Validator + dht.autoRefresh = cfg.routingTable.autoRefresh + dht.rtRefreshInterval = cfg.routingTable.refreshInterval + dht.rtRefreshQueryTimeout = cfg.routingTable.refreshQueryTimeout + + dht.maxRecordAge = cfg.maxRecordAge + dht.enableProviders = cfg.enableProviders + dht.enableValues = cfg.enableValues + + dht.Validator = cfg.validator + + switch cfg.mode { + case ModeAuto: + dht.auto = true + dht.mode = modeClient + case ModeClient: + dht.auto = false + dht.mode = modeClient + case ModeServer: + dht.auto = false + dht.mode = modeServer + default: + return nil, fmt.Errorf("invalid dht mode %d", cfg.mode) + } - if !cfg.Client { - for _, p := range cfg.Protocols { - h.SetStreamHandler(p, dht.handleNewStream) + if dht.mode == modeServer { + if err := dht.moveToServerMode(); err != nil { + return nil, err } } + + // register for event bus and network notifications + sn, err := newSubscriberNotifiee(dht) + if err != nil { + return nil, err + } + dht.proc.Go(sn.subscribe) + // handle providers + dht.proc.AddChild(dht.ProviderManager.Process()) + + dht.startSelfLookup() dht.startRefreshing() + + // go-routine to make sure we ALWAYS have RT peer addresses in the peerstore + // since RT membership is decoupled from connectivity + go dht.persistRTPeersInPeerStore() + + // listens to the fix low peers chan and tries to fix the Routing Table + dht.proc.Go(dht.fixLowPeersRoutine) + return dht, nil } @@ -127,7 +200,7 @@ func New(ctx context.Context, h host.Host, options ...opts.Option) (*IpfsDHT, er // IpfsDHT's initialized with this function will respond to DHT requests, // whereas IpfsDHT's initialized with NewDHTClient will not. func NewDHT(ctx context.Context, h host.Host, dstore ds.Batching) *IpfsDHT { - dht, err := New(ctx, h, opts.Datastore(dstore)) + dht, err := New(ctx, h, Datastore(dstore)) if err != nil { panic(err) } @@ -139,105 +212,158 @@ func NewDHT(ctx context.Context, h host.Host, dstore ds.Batching) *IpfsDHT { // requests. If you need a peer to respond to DHT requests, use NewDHT instead. // NewDHTClient creates a new DHT object with the given peer as the 'local' host func NewDHTClient(ctx context.Context, h host.Host, dstore ds.Batching) *IpfsDHT { - dht, err := New(ctx, h, opts.Datastore(dstore), opts.Client(true)) + dht, err := New(ctx, h, Datastore(dstore), Mode(ModeClient)) if err != nil { panic(err) } return dht } -func makeDHT(ctx context.Context, h host.Host, cfg *opts.Options) *IpfsDHT { - self := kb.ConvertPeerID(h.ID()) - rt := kb.NewRoutingTable(cfg.BucketSize, self, cfg.RoutingTable.LatencyTolerance, h.Peerstore()) - cmgr := h.ConnManager() +func makeDHT(ctx context.Context, h host.Host, cfg config) (*IpfsDHT, error) { + var protocols, serverProtocols []protocol.ID + + // check if custom test protocols were set + if cfg.v1CompatibleMode { + // In compat mode, query/serve using the old protocol. + // + // DO NOT accept requests on the new protocol. Otherwise: + // 1. We'll end up in V2 routing tables. + // 2. We'll have V1 peers in our routing table. + // + // In other words, we'll pollute the V2 network. + protocols = []protocol.ID{cfg.protocolPrefix + kad1} + serverProtocols = []protocol.ID{cfg.protocolPrefix + kad1} + } else { + // In v2 mode, serve on both protocols, but only + // query/accept peers in v2 mode. + protocols = []protocol.ID{cfg.protocolPrefix + kad2} + serverProtocols = []protocol.ID{cfg.protocolPrefix + kad2, cfg.protocolPrefix + kad1} + } - rt.PeerAdded = func(p peer.ID) { - commonPrefixLen := kb.CommonPrefixLen(self, kb.ConvertPeerID(p)) - cmgr.TagPeer(p, "kbucket", BaseConnMgrScore+commonPrefixLen) + dht := &IpfsDHT{ + datastore: cfg.datastore, + self: h.ID(), + peerstore: h.Peerstore(), + host: h, + strmap: make(map[peer.ID]*messageSender), + birth: time.Now(), + protocols: protocols, + serverProtocols: serverProtocols, + bucketSize: cfg.bucketSize, + alpha: cfg.concurrency, + beta: cfg.resiliency, + triggerRtRefresh: make(chan chan<- error), + triggerSelfLookup: make(chan chan<- error), + queryPeerFilter: cfg.queryPeerFilter, + routingTablePeerFilter: cfg.routingTable.peerFilter, + fixLowPeersChan: make(chan struct{}), } - rt.PeerRemoved = func(p peer.ID) { - cmgr.UntagPeer(p, "kbucket") + // construct routing table + rt, err := makeRoutingTable(dht, cfg) + if err != nil { + return nil, fmt.Errorf("failed to construct routing table,err=%s", err) } + dht.routingTable = rt - dht := &IpfsDHT{ - datastore: cfg.Datastore, - self: h.ID(), - peerstore: h.Peerstore(), - host: h, - strmap: make(map[peer.ID]*messageSender), - birth: time.Now(), - routingTable: rt, - protocols: cfg.Protocols, - bucketSize: cfg.BucketSize, - triggerRtRefresh: make(chan chan<- error), - } - - // create a DHT proc with the given teardown - dht.proc = goprocess.WithTeardown(func() error { - // remove ourselves from network notifs. - dht.host.Network().StopNotify((*netNotifiee)(dht)) - return nil - }) + // create a DHT proc with the given context + dht.proc = goprocessctx.WithContext(ctx) // create a tagged context derived from the original context ctxTags := dht.newContextWithLocalTags(ctx) // the DHT context should be done when the process is closed dht.ctx = goprocessctx.WithProcessClosing(ctxTags, dht.proc) - dht.ProviderManager = providers.NewProviderManager(dht.ctx, h.ID(), cfg.Datastore) + dht.ProviderManager = providers.NewProviderManager(dht.ctx, h.ID(), cfg.datastore) - return dht + return dht, nil } -// TODO Implement RT seeding as described in https://github.com/libp2p/go-libp2p-kad-dht/pull/384#discussion_r320994340 OR -// come up with an alternative solution. -// issue is being tracked at https://github.com/libp2p/go-libp2p-kad-dht/issues/387 -/*func (dht *IpfsDHT) rtRecovery(proc goprocess.Process) { - writeResp := func(errorChan chan error, err error) { +func makeRoutingTable(dht *IpfsDHT, cfg config) (*kb.RoutingTable, error) { + // The threshold is calculated based on the expected amount of time that should pass before we + // query a peer as part of our refresh cycle. + // To grok the Math Wizardy that produced these exact equations, please be patient as a document explaining it will + // be published soon. + l1 := math.Log(float64(1) / float64(defaultBucketSize)) //(Log(1/K)) + l2 := math.Log(float64(1) - (float64(cfg.concurrency) / float64(defaultBucketSize))) // Log(1 - (alpha / K)) + maxLastSuccessfulOutboundThreshold := l1 / l2 * float64(cfg.routingTable.refreshInterval) + + self := kb.ConvertPeerID(dht.host.ID()) + + rt, err := kb.NewRoutingTable(cfg.bucketSize, self, time.Minute, dht.host.Peerstore(), maxLastSuccessfulOutboundThreshold) + cmgr := dht.host.ConnManager() + + rt.PeerAdded = func(p peer.ID) { + commonPrefixLen := kb.CommonPrefixLen(self, kb.ConvertPeerID(p)) + cmgr.TagPeer(p, "kbucket", BaseConnMgrScore+commonPrefixLen) + } + rt.PeerRemoved = func(p peer.ID) { + cmgr.UntagPeer(p, "kbucket") + + // try to fix the RT + dht.fixRTIfNeeded() + } + + return rt, err +} + +// fixLowPeers tries to get more peers into the routing table if we're below the threshold +func (dht *IpfsDHT) fixLowPeersRoutine(proc goprocess.Process) { + for { select { + case <-dht.fixLowPeersChan: case <-proc.Closing(): - case errorChan <- errChan: + return + } + if dht.routingTable.Size() > minRTRefreshThreshold { + continue + } + + for _, p := range dht.host.Network().Peers() { + dht.peerFound(dht.Context(), p, false) + } + + if dht.autoRefresh { + select { + case dht.triggerRtRefresh <- nil: + default: + } } - close(errorChan) } +} + +// TODO This is hacky, horrible and the programmer needs to have his mother called a hamster. +// SHOULD be removed once https://github.com/libp2p/go-libp2p/issues/800 goes in. +func (dht *IpfsDHT) persistRTPeersInPeerStore() { + tickr := time.NewTicker(peerstore.RecentlyConnectedAddrTTL / 3) + defer tickr.Stop() + for { select { - case req := <-dht.rtRecoveryChan: - if dht.routingTable.Size() == 0 { - logger.Infof("rt recovery proc: received request with reqID=%s, RT is empty. initiating recovery", req.id) - // TODO Call Seeder with default bootstrap peers here once #383 is merged - if dht.routingTable.Size() > 0 { - logger.Infof("rt recovery proc: successfully recovered RT for reqID=%s, RT size is now %d", req.id, dht.routingTable.Size()) - go writeResp(req.errorChan, nil) - } else { - logger.Errorf("rt recovery proc: failed to recover RT for reqID=%s, RT is still empty", req.id) - go writeResp(req.errorChan, errors.New("RT empty after seed attempt")) - } - } else { - logger.Infof("rt recovery proc: RT is not empty, no need to act on request with reqID=%s", req.id) - go writeResp(req.errorChan, nil) + case <-tickr.C: + ps := dht.routingTable.ListPeers() + for _, p := range ps { + dht.peerstore.UpdateAddrs(p, peerstore.RecentlyConnectedAddrTTL, peerstore.RecentlyConnectedAddrTTL) } - case <-proc.Closing(): + case <-dht.ctx.Done(): return } } -}*/ +} // putValueToPeer stores the given key/value pair at the peer 'p' func (dht *IpfsDHT) putValueToPeer(ctx context.Context, p peer.ID, rec *recpb.Record) error { - pmes := pb.NewMessage(pb.Message_PUT_VALUE, rec.Key, 0) pmes.Record = rec rpmes, err := dht.sendRequest(ctx, p, pmes) if err != nil { - logger.Debugf("putValueToPeer: %v. (peer: %s, key: %s)", err, p.Pretty(), loggableKey(string(rec.Key))) + logger.Debugw("failed to put value to peer", "to", p, "key", loggableKeyBytes(rec.Key), "error", err) return err } if !bytes.Equal(rpmes.GetRecord().Value, pmes.GetRecord().Value) { - logger.Warningf("putValueToPeer: value not put correctly. (%v != %v)", pmes, rpmes) + logger.Infow("value not put correctly", "put-message", pmes, "get-message", rpmes) return errors.New("value not put correctly") } @@ -251,7 +377,6 @@ var errInvalidRecord = errors.New("received invalid record") // NOTE: It will update the dht's peerstore with any new addresses // it finds for the given peer. func (dht *IpfsDHT) getValueOrPeers(ctx context.Context, p peer.ID, key string) (*recpb.Record, []*peer.AddrInfo, error) { - pmes, err := dht.getValueSingle(ctx, p, key) if err != nil { return nil, nil, err @@ -262,12 +387,12 @@ func (dht *IpfsDHT) getValueOrPeers(ctx context.Context, p peer.ID, key string) if record := pmes.GetRecord(); record != nil { // Success! We were given the value - logger.Debug("getValueOrPeers: got value") + logger.Debug("got value") // make sure record is valid. err = dht.Validator.Validate(string(record.GetKey()), record.GetValue()) if err != nil { - logger.Info("Received invalid record! (discarded)") + logger.Debug("received invalid record (discarded)") // return a sentinal to signify an invalid record was received err = errInvalidRecord record = new(recpb.Record) @@ -276,50 +401,31 @@ func (dht *IpfsDHT) getValueOrPeers(ctx context.Context, p peer.ID, key string) } if len(peers) > 0 { - logger.Debug("getValueOrPeers: peers") return nil, peers, nil } - logger.Warning("getValueOrPeers: routing.ErrNotFound") return nil, nil, routing.ErrNotFound } // getValueSingle simply performs the get value RPC with the given parameters func (dht *IpfsDHT) getValueSingle(ctx context.Context, p peer.ID, key string) (*pb.Message, error) { - meta := logging.LoggableMap{ - "key": key, - "peer": p, - } - - eip := logger.EventBegin(ctx, "getValueSingle", meta) - defer eip.Done() - pmes := pb.NewMessage(pb.Message_GET_VALUE, []byte(key), 0) - resp, err := dht.sendRequest(ctx, p, pmes) - switch err { - case nil: - return resp, nil - case ErrReadTimeout: - logger.Warningf("getValueSingle: read timeout %s %s", p.Pretty(), key) - fallthrough - default: - eip.SetError(err) - return nil, err - } + return dht.sendRequest(ctx, p, pmes) } // getLocal attempts to retrieve the value from the datastore func (dht *IpfsDHT) getLocal(key string) (*recpb.Record, error) { - logger.Debugf("getLocal %s", key) + logger.Debugw("finding value in datastore", "key", loggableKeyString(key)) + rec, err := dht.getRecordFromDatastore(mkDsKey(key)) if err != nil { - logger.Warningf("getLocal: %s", err) + logger.Warnw("get local failed", "key", key, "error", err) return nil, err } // Double check the key. Can't hurt. if rec != nil && string(rec.GetKey()) != key { - logger.Errorf("BUG getLocal: found a DHT record that didn't match it's key: %s != %s", rec.GetKey(), key) + logger.Errorw("BUG: found a DHT record that didn't match it's key", "expected", key, "got", rec.GetKey()) return nil, nil } @@ -328,21 +434,53 @@ func (dht *IpfsDHT) getLocal(key string) (*recpb.Record, error) { // putLocal stores the key value pair in the datastore func (dht *IpfsDHT) putLocal(key string, rec *recpb.Record) error { - logger.Debugf("putLocal: %v %v", key, rec) data, err := proto.Marshal(rec) if err != nil { - logger.Warningf("putLocal: %s", err) + logger.Warnw("failed to put marshal record for local put", "error", err, "key", key) return err } return dht.datastore.Put(mkDsKey(key), data) } -// Update signals the routingTable to Update its last-seen status -// on the given peer. -func (dht *IpfsDHT) Update(ctx context.Context, p peer.ID) { - logger.Event(ctx, "updatePeer", p) - dht.routingTable.Update(p) +// peerFound signals the routingTable that we've found a peer that +// might support the DHT protocol. +func (dht *IpfsDHT) peerFound(ctx context.Context, p peer.ID, queryPeer bool) { + logger.Debugw("peer found", "peer", p) + b, err := dht.validRTPeer(p) + if err != nil { + logger.Errorw("failed to validate if peer is a DHT peer", "peer", p, "error", err) + } else if b { + _, err := dht.routingTable.TryAddPeer(p, queryPeer) + if err != nil { + // peer not added. + return + } + + // If we discovered the peer because of a query, we need to ensure we override the "zero" lastSuccessfulOutboundQuery + // value that must have been set in the Routing Table for this peer when it was first added during a connection. + if queryPeer { + dht.routingTable.UpdateLastSuccessfulOutboundQuery(p, time.Now()) + } + } +} + +// peerStoppedDHT signals the routing table that a peer is unable to responsd to DHT queries anymore. +func (dht *IpfsDHT) peerStoppedDHT(ctx context.Context, p peer.ID) { + logger.Debugw("peer stopped dht", "peer", p) + // A peer that does not support the DHT protocol is dead for us. + // There's no point in talking to anymore till it starts supporting the DHT protocol again. + dht.routingTable.RemovePeer(p) + + // since we lost a peer from the RT, we should do this here + dht.fixRTIfNeeded() +} + +func (dht *IpfsDHT) fixRTIfNeeded() { + select { + case dht.fixLowPeersChan <- struct{}{}: + default: + } } // FindLocal looks for a peer with a given ID connected to this dht and returns the peer and the table it was found in. @@ -357,43 +495,13 @@ func (dht *IpfsDHT) FindLocal(id peer.ID) peer.AddrInfo { // findPeerSingle asks peer 'p' if they know where the peer with id 'id' is func (dht *IpfsDHT) findPeerSingle(ctx context.Context, p peer.ID, id peer.ID) (*pb.Message, error) { - eip := logger.EventBegin(ctx, "findPeerSingle", - logging.LoggableMap{ - "peer": p, - "target": id, - }) - defer eip.Done() - pmes := pb.NewMessage(pb.Message_FIND_NODE, []byte(id), 0) - resp, err := dht.sendRequest(ctx, p, pmes) - switch err { - case nil: - return resp, nil - case ErrReadTimeout: - logger.Warningf("read timeout: %s %s", p.Pretty(), id) - fallthrough - default: - eip.SetError(err) - return nil, err - } + return dht.sendRequest(ctx, p, pmes) } func (dht *IpfsDHT) findProvidersSingle(ctx context.Context, p peer.ID, key multihash.Multihash) (*pb.Message, error) { - eip := logger.EventBegin(ctx, "findProvidersSingle", p, multihashLoggableKey(key)) - defer eip.Done() - pmes := pb.NewMessage(pb.Message_GET_PROVIDERS, key, 0) - resp, err := dht.sendRequest(ctx, p, pmes) - switch err { - case nil: - return resp, nil - case ErrReadTimeout: - logger.Warningf("read timeout: %s %s", p.Pretty(), key) - fallthrough - default: - eip.SetError(err) - return nil, err - } + return dht.sendRequest(ctx, p, pmes) } // nearestPeersToQuery returns the routing tables closest peers. @@ -403,12 +511,12 @@ func (dht *IpfsDHT) nearestPeersToQuery(pmes *pb.Message, count int) []peer.ID { } // betterPeersToQuery returns nearestPeersToQuery with some additional filtering -func (dht *IpfsDHT) betterPeersToQuery(pmes *pb.Message, p peer.ID, count int) []peer.ID { +func (dht *IpfsDHT) betterPeersToQuery(pmes *pb.Message, from peer.ID, count int) []peer.ID { closer := dht.nearestPeersToQuery(pmes, count) // no node? nil if closer == nil { - logger.Warning("betterPeersToQuery: no closer peers to send:", p) + logger.Infow("no closer peers to send", from) return nil } @@ -421,7 +529,7 @@ func (dht *IpfsDHT) betterPeersToQuery(pmes *pb.Message, p peer.ID, count int) [ return nil } // Dont send a peer back themselves - if clp == p { + if clp == from { continue } @@ -432,6 +540,69 @@ func (dht *IpfsDHT) betterPeersToQuery(pmes *pb.Message, p peer.ID, count int) [ return filtered } +func (dht *IpfsDHT) setMode(m mode) error { + dht.modeLk.Lock() + defer dht.modeLk.Unlock() + + if m == dht.mode { + return nil + } + + switch m { + case modeServer: + return dht.moveToServerMode() + case modeClient: + return dht.moveToClientMode() + default: + return fmt.Errorf("unrecognized dht mode: %d", m) + } +} + +// moveToServerMode advertises (via libp2p identify updates) that we are able to respond to DHT queries and sets the appropriate stream handlers. +// Note: We may support responding to queries with protocols aside from our primary ones in order to support +// interoperability with older versions of the DHT protocol. +func (dht *IpfsDHT) moveToServerMode() error { + dht.mode = modeServer + for _, p := range dht.serverProtocols { + dht.host.SetStreamHandler(p, dht.handleNewStream) + } + return nil +} + +// moveToClientMode stops advertising (and rescinds advertisements via libp2p identify updates) that we are able to +// respond to DHT queries and removes the appropriate stream handlers. We also kill all inbound streams that were +// utilizing the handled protocols. +// Note: We may support responding to queries with protocols aside from our primary ones in order to support +// interoperability with older versions of the DHT protocol. +func (dht *IpfsDHT) moveToClientMode() error { + dht.mode = modeClient + for _, p := range dht.serverProtocols { + dht.host.RemoveStreamHandler(p) + } + + pset := make(map[protocol.ID]bool) + for _, p := range dht.serverProtocols { + pset[p] = true + } + + for _, c := range dht.host.Network().Conns() { + for _, s := range c.GetStreams() { + if pset[s.Protocol()] { + if s.Stat().Direction == network.DirInbound { + _ = s.Reset() + } + } + } + } + return nil +} + +func (dht *IpfsDHT) getMode() mode { + dht.modeLk.Lock() + defer dht.modeLk.Unlock() + return dht.mode +} + // Context return dht's context func (dht *IpfsDHT) Context() context.Context { return dht.ctx @@ -452,15 +623,6 @@ func (dht *IpfsDHT) Close() error { return dht.proc.Close() } -func (dht *IpfsDHT) protocolStrs() []string { - pstrs := make([]string, len(dht.protocols)) - for idx, proto := range dht.protocols { - pstrs[idx] = string(proto) - } - - return pstrs -} - func mkDsKey(s string) ds.Key { return ds.NewKey(base32.RawStdEncoding.EncodeToString([]byte(s))) } diff --git a/dht_bootstrap.go b/dht_bootstrap.go index d943d06bf..fb3961dc9 100644 --- a/dht_bootstrap.go +++ b/dht_bootstrap.go @@ -3,12 +3,13 @@ package dht import ( "context" "fmt" + "github.com/libp2p/go-libp2p-core/peer" "time" multierror "github.com/hashicorp/go-multierror" process "github.com/jbenet/goprocess" processctx "github.com/jbenet/goprocess/context" - "github.com/libp2p/go-libp2p-core/routing" + kbucket "github.com/libp2p/go-libp2p-kbucket" "github.com/multiformats/go-multiaddr" _ "github.com/multiformats/go-multiaddr-dns" ) @@ -17,7 +18,10 @@ var DefaultBootstrapPeers []multiaddr.Multiaddr // Minimum number of peers in the routing table. If we drop below this and we // see a new peer, we trigger a bootstrap round. -var minRTRefreshThreshold = 4 +var minRTRefreshThreshold = 10 + +// timeout for pinging one peer +const peerPingTimeout = 10 * time.Second func init() { for _, s := range []string{ @@ -35,18 +39,63 @@ func init() { } } +// startSelfLookup starts a go-routine that listens for requests to trigger a self walk on a dedicated channel +// and then sends the error status back on the error channel sent along with the request. +// if multiple callers "simultaneously" ask for a self walk, it performs ONLY one self walk and sends the same error status to all of them. +func (dht *IpfsDHT) startSelfLookup() { + dht.proc.Go(func(proc process.Process) { + ctx := processctx.WithProcessClosing(dht.ctx, proc) + for { + var waiting []chan<- error + select { + case res := <-dht.triggerSelfLookup: + if res != nil { + waiting = append(waiting, res) + } + case <-ctx.Done(): + return + } + + // batch multiple refresh requests if they're all waiting at the same time. + waiting = append(waiting, collectWaitingChannels(dht.triggerSelfLookup)...) + + // Do a self walk + queryCtx, cancel := context.WithTimeout(ctx, dht.rtRefreshQueryTimeout) + _, err := dht.GetClosestPeers(queryCtx, string(dht.self)) + if err == kbucket.ErrLookupFailure { + err = nil + } else if err != nil { + err = fmt.Errorf("failed to query self during routing table refresh: %s", err) + } + cancel() + + // send back the error status + for _, w := range waiting { + w <- err + close(w) + } + if err != nil { + logger.Warnw("self lookup failed", "error", err) + } + } + }) +} + // Start the refresh worker. -func (dht *IpfsDHT) startRefreshing() error { +func (dht *IpfsDHT) startRefreshing() { // scan the RT table periodically & do a random walk for cpl's that haven't been queried since the given period dht.proc.Go(func(proc process.Process) { - ctx := processctx.OnClosingContext(proc) + ctx := processctx.WithProcessClosing(dht.ctx, proc) - refreshTicker := time.NewTicker(dht.rtRefreshPeriod) + refreshTicker := time.NewTicker(dht.rtRefreshInterval) defer refreshTicker.Stop() // refresh if option is set if dht.autoRefresh { - dht.doRefresh(ctx) + err := dht.doRefresh(ctx) + if err != nil { + logger.Warn("failed when refreshing routing table", err) + } } else { // disable the "auto-refresh" ticker so that no more ticks are sent to this channel refreshTicker.Stop() @@ -65,17 +114,7 @@ func (dht *IpfsDHT) startRefreshing() error { } // Batch multiple refresh requests if they're all waiting at the same time. - collectWaiting: - for { - select { - case res := <-dht.triggerRtRefresh: - if res != nil { - waiting = append(waiting, res) - } - default: - break collectWaiting - } - } + waiting = append(waiting, collectWaitingChannels(dht.triggerRtRefresh)...) err := dht.doRefresh(ctx) for _, w := range waiting { @@ -83,19 +122,61 @@ func (dht *IpfsDHT) startRefreshing() error { close(w) } if err != nil { - logger.Warning(err) + logger.Warnw("failed when refreshing routing table", "error", err) + } + + // ping Routing Table peers that haven't been hear of/from in the interval they should have been. + for _, ps := range dht.routingTable.GetPeerInfos() { + // ping the peer if it's due for a ping and evict it if the ping fails + if time.Since(ps.LastSuccessfulOutboundQuery) > dht.maxLastSuccessfulOutboundThreshold { + livelinessCtx, cancel := context.WithTimeout(ctx, peerPingTimeout) + if err := dht.host.Connect(livelinessCtx, peer.AddrInfo{ID: ps.Id}); err != nil { + logger.Debugw("evicting peer after failed ping", "peer", ps.Id, "error", err) + dht.routingTable.RemovePeer(ps.Id) + } + cancel() + } } + } }) +} - return nil +func collectWaitingChannels(source chan chan<- error) []chan<- error { + var waiting []chan<- error + for { + select { + case res := <-source: + if res != nil { + waiting = append(waiting, res) + } + default: + return waiting + } + } } func (dht *IpfsDHT) doRefresh(ctx context.Context) error { var merr error - if err := dht.selfWalk(ctx); err != nil { - merr = multierror.Append(merr, err) + + // wait for the self walk result + selfWalkres := make(chan error, 1) + + select { + case dht.triggerSelfLookup <- selfWalkres: + case <-ctx.Done(): + return ctx.Err() } + + select { + case err := <-selfWalkres: + if err != nil { + merr = multierror.Append(merr, err) + } + case <-ctx.Done(): + return ctx.Err() + } + if err := dht.refreshCpls(ctx); err != nil { merr = multierror.Append(merr, err) } @@ -123,47 +204,34 @@ func (dht *IpfsDHT) refreshCpls(ctx context.Context) error { trackedCpls := dht.routingTable.GetTrackedCplsForRefresh() var merr error - for _, tcpl := range trackedCpls { - if time.Since(tcpl.LastRefreshAt) <= dht.rtRefreshPeriod { + for cpl, lastRefreshedAt := range trackedCpls { + if time.Since(lastRefreshedAt) <= dht.rtRefreshInterval { continue } + // gen rand peer with the cpl - randPeer, err := dht.routingTable.GenRandPeerID(tcpl.Cpl) + randPeer, err := dht.routingTable.GenRandPeerID(uint(cpl)) if err != nil { - logger.Errorf("failed to generate peerID for cpl %d, err: %s", tcpl.Cpl, err) + logger.Errorw("failed to generate peer ID", "cpl", cpl, "error", err) continue } // walk to the generated peer walkFnc := func(c context.Context) error { - _, err := dht.FindPeer(c, randPeer) - if err == routing.ErrNotFound { - return nil - } + _, err := dht.GetClosestPeers(c, string(randPeer)) return err } - if err := doQuery(tcpl.Cpl, randPeer.String(), walkFnc); err != nil { + if err := doQuery(uint(cpl), randPeer.String(), walkFnc); err != nil { merr = multierror.Append( merr, - fmt.Errorf("failed to do a random walk for cpl %d: %s", tcpl.Cpl, err), + fmt.Errorf("failed to do a random walk for cpl %d: %w", cpl, err), ) } } return merr } -// Traverse the DHT toward the self ID -func (dht *IpfsDHT) selfWalk(ctx context.Context) error { - queryCtx, cancel := context.WithTimeout(ctx, dht.rtRefreshQueryTimeout) - defer cancel() - _, err := dht.FindPeer(queryCtx, dht.self) - if err == routing.ErrNotFound { - return nil - } - return fmt.Errorf("failed to query self during routing table refresh: %s", err) -} - // Bootstrap tells the DHT to get into a bootstrapped state satisfying the // IpfsRouter interface. // @@ -179,6 +247,10 @@ func (dht *IpfsDHT) Bootstrap(_ context.Context) error { // error and close. The channel is buffered and safe to ignore. func (dht *IpfsDHT) RefreshRoutingTable() <-chan error { res := make(chan error, 1) - dht.triggerRtRefresh <- res + select { + case dht.triggerRtRefresh <- res: + case <-dht.ctx.Done(): + res <- dht.ctx.Err() + } return res } diff --git a/dht_bootstrap_test.go b/dht_bootstrap_test.go new file mode 100644 index 000000000..a7330c940 --- /dev/null +++ b/dht_bootstrap_test.go @@ -0,0 +1,49 @@ +package dht + +import ( + "context" + "testing" + "time" + + "github.com/libp2p/go-libp2p-core/event" + + kb "github.com/libp2p/go-libp2p-kbucket" + + "github.com/stretchr/testify/require" +) + +func TestSelfWalkOnAddressChange(t *testing.T) { + ctx := context.Background() + // create three DHT instances with auto refresh disabled + d1 := setupDHT(ctx, t, false, DisableAutoRefresh()) + d2 := setupDHT(ctx, t, false, DisableAutoRefresh()) + d3 := setupDHT(ctx, t, false, DisableAutoRefresh()) + + var connectedTo *IpfsDHT + // connect d1 to whoever is "further" + if kb.CommonPrefixLen(kb.ConvertPeerID(d1.self), kb.ConvertPeerID(d2.self)) <= + kb.CommonPrefixLen(kb.ConvertPeerID(d1.self), kb.ConvertPeerID(d3.self)) { + connect(t, ctx, d1, d3) + connectedTo = d3 + } else { + connect(t, ctx, d1, d2) + connectedTo = d2 + } + + // then connect d2 AND d3 + connect(t, ctx, d2, d3) + + // d1 should have ONLY 1 peer in it's RT + waitForWellFormedTables(t, []*IpfsDHT{d1}, 1, 1, 2*time.Second) + require.Equal(t, connectedTo.self, d1.routingTable.ListPeers()[0]) + + // now emit the address change event + em, err := d1.host.EventBus().Emitter(&event.EvtLocalAddressesUpdated{}) + require.NoError(t, err) + require.NoError(t, em.Emit(event.EvtLocalAddressesUpdated{})) + waitForWellFormedTables(t, []*IpfsDHT{d1}, 2, 2, 2*time.Second) + // it should now have both peers in the RT + ps := d1.routingTable.ListPeers() + require.Contains(t, ps, d2.self) + require.Contains(t, ps, d3.self) +} diff --git a/dht_filters.go b/dht_filters.go new file mode 100644 index 000000000..fe2885922 --- /dev/null +++ b/dht_filters.go @@ -0,0 +1,139 @@ +package dht + +import ( + "bytes" + "net" + + "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" + netroute "github.com/libp2p/go-netroute" + + ma "github.com/multiformats/go-multiaddr" + manet "github.com/multiformats/go-multiaddr-net" +) + +// QueryFilterFunc is a filter applied when considering peers to dial when querying +type QueryFilterFunc func(dht *IpfsDHT, ai peer.AddrInfo) bool + +// RouteTableFilterFunc is a filter applied when considering connections to keep in +// the local route table. +type RouteTableFilterFunc func(dht *IpfsDHT, conns []network.Conn) bool + +// PublicQueryFilter returns true if the peer is suspected of being publicly accessible +func PublicQueryFilter(_ *IpfsDHT, ai peer.AddrInfo) bool { + if len(ai.Addrs) == 0 { + return false + } + + var hasPublicAddr bool + for _, a := range ai.Addrs { + if !isRelayAddr(a) && manet.IsPublicAddr(a) { + hasPublicAddr = true + } + } + return hasPublicAddr +} + +var _ QueryFilterFunc = PublicQueryFilter + +// PublicRoutingTableFilter allows a peer to be added to the routing table if the connections to that peer indicate +// that it is on a public network +func PublicRoutingTableFilter(dht *IpfsDHT, conns []network.Conn) bool { + if len(conns) == 0 { + return false + } + + // Do we have a public address for this peer? + id := conns[0].RemotePeer() + known := dht.peerstore.PeerInfo(id) + for _, a := range known.Addrs { + if !isRelayAddr(a) && manet.IsPublicAddr(a) { + return true + } + } + + return false +} + +var _ RouteTableFilterFunc = PublicRoutingTableFilter + +// PrivateQueryFilter doens't currently restrict which peers we are willing to query from the local DHT. +func PrivateQueryFilter(dht *IpfsDHT, ai peer.AddrInfo) bool { + return len(ai.Addrs) > 0 +} + +var _ QueryFilterFunc = PrivateQueryFilter + +// PrivateRoutingTableFilter allows a peer to be added to the routing table if the connections to that peer indicate +// that it is on a private network +func PrivateRoutingTableFilter(dht *IpfsDHT, conns []network.Conn) bool { + router, _ := netroute.New() + myAdvertisedIPs := make([]net.IP, 0) + for _, a := range dht.Host().Addrs() { + if manet.IsPublicAddr(a) && !isRelayAddr(a) { + ip, err := manet.ToIP(a) + if err != nil { + continue + } + myAdvertisedIPs = append(myAdvertisedIPs, ip) + } + } + + for _, c := range conns { + ra := c.RemoteMultiaddr() + if manet.IsPrivateAddr(ra) && !isRelayAddr(ra) { + return true + } + + if manet.IsPublicAddr(ra) { + ip, err := manet.ToIP(ra) + if err != nil { + continue + } + + // if the ip is the same as one of the local host's public advertised IPs - then consider it local + for _, i := range myAdvertisedIPs { + if i.Equal(ip) { + return true + } + if ip.To4() == nil { + if i.To4() == nil && isEUI(ip) && sameV6Net(i, ip) { + return true + } + } + } + + // if there's no gateway - a direct host in the OS routing table - then consider it local + // This is relevant in particular to ipv6 networks where the addresses may all be public, + // but the nodes are aware of direct links between each other. + if router != nil { + _, gw, _, err := router.Route(ip) + if gw == nil && err == nil { + return true + } + } + } + } + + return false +} + +var _ RouteTableFilterFunc = PrivateRoutingTableFilter + +func isEUI(ip net.IP) bool { + // per rfc 2373 + return len(ip) == net.IPv6len && ip[11] == 0xff && ip[12] == 0xfe +} + +func sameV6Net(a, b net.IP) bool { + return len(a) == net.IPv6len && len(b) == net.IPv6len && bytes.Equal(a[0:8], b[0:8]) //nolint +} + +func isRelayAddr(a ma.Multiaddr) bool { + for _, p := range a.Protocols() { + if p.Code == ma.P_CIRCUIT { + return true + } + } + return false +} diff --git a/dht_filters_test.go b/dht_filters_test.go new file mode 100644 index 000000000..af4453bce --- /dev/null +++ b/dht_filters_test.go @@ -0,0 +1,23 @@ +package dht + +import ( + "testing" + + "github.com/multiformats/go-multiaddr" +) + +func TestIsRelay(t *testing.T) { + a, _ := multiaddr.NewMultiaddr("/ip4/127.0.0.1/tcp/5002/p2p/QmdPU7PfRyKehdrP5A3WqmjyD6bhVpU1mLGKppa2FjGDjZ/p2p-circuit/p2p/QmVT6GYwjeeAF5TR485Yc58S3xRF5EFsZ5YAF4VcP3URHt") + if !isRelayAddr(a) { + t.Fatalf("thought %s was not a relay", a) + } + a, _ = multiaddr.NewMultiaddr("/p2p-circuit/p2p/QmVT6GYwjeeAF5TR485Yc58S3xRF5EFsZ5YAF4VcP3URHt") + if !isRelayAddr(a) { + t.Fatalf("thought %s was not a relay", a) + } + a, _ = multiaddr.NewMultiaddr("/ip4/127.0.0.1/tcp/5002/p2p/QmdPU7PfRyKehdrP5A3WqmjyD6bhVpU1mLGKppa2FjGDjZ") + if isRelayAddr(a) { + t.Fatalf("thought %s was a relay", a) + } + +} diff --git a/dht_net.go b/dht_net.go index 18249099f..0fea86ecb 100644 --- a/dht_net.go +++ b/dht_net.go @@ -22,7 +22,7 @@ import ( "go.opencensus.io/tag" ) -var dhtReadMessageTimeout = time.Minute +var dhtReadMessageTimeout = 10 * time.Second var dhtStreamIdleTimeout = 1 * time.Minute var ErrReadTimeout = fmt.Errorf("timed out reading response") @@ -62,7 +62,7 @@ func (w *bufferedDelimitedWriter) Flush() error { // handleNewStream implements the network.StreamHandler func (dht *IpfsDHT) handleNewStream(s network.Stream) { - defer s.Reset() + defer s.Reset() //nolint if dht.handleNewMessage(s) { // Gracefully close the stream for writes. s.Close() @@ -76,10 +76,15 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { mPeer := s.Conn().RemotePeer() - timer := time.AfterFunc(dhtStreamIdleTimeout, func() { s.Reset() }) + timer := time.AfterFunc(dhtStreamIdleTimeout, func() { _ = s.Reset() }) defer timer.Stop() for { + if dht.getMode() != modeServer { + logger.Errorf("ignoring incoming dht message while not in server mode") + return false + } + var req pb.Message msgbytes, err := r.ReadMsg() msgLen := len(msgbytes) @@ -94,7 +99,7 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { logger.Debugf("error reading message: %#v", err) } if msgLen > 0 { - stats.RecordWithTags(ctx, + _ = stats.RecordWithTags(ctx, []tag.Mutator{tag.Upsert(metrics.KeyMessageType, "UNKNOWN")}, metrics.ReceivedMessages.M(1), metrics.ReceivedMessageErrors.M(1), @@ -107,7 +112,7 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { r.ReleaseMsg(msgbytes) if err != nil { logger.Debugf("error unmarshalling message: %#v", err) - stats.RecordWithTags(ctx, + _ = stats.RecordWithTags(ctx, []tag.Mutator{tag.Upsert(metrics.KeyMessageType, "UNKNOWN")}, metrics.ReceivedMessages.M(1), metrics.ReceivedMessageErrors.M(1), @@ -131,18 +136,37 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { handler := dht.handlerForMsgType(req.GetType()) if handler == nil { stats.Record(ctx, metrics.ReceivedMessageErrors.M(1)) - logger.Warningf("can't handle received message of type %v", req.GetType()) + logger.Warnw("can't handle received message", "from", mPeer, "type", req.GetType()) return false } + // a peer has queried us, let's add it to RT + dht.peerFound(dht.ctx, mPeer, true) + + logger.Debugw("handling message", + "type", req.GetType(), + "key", req.GetKey(), + "from", mPeer, + ) resp, err := handler(ctx, mPeer, &req) if err != nil { stats.Record(ctx, metrics.ReceivedMessageErrors.M(1)) - logger.Debugf("error handling message: %v", err) + logger.Debugw( + "error handling message", + "type", req.GetType(), + "key", req.GetKey(), + "from", mPeer, + "error", err) return false } - dht.updateFromMessage(ctx, mPeer, &req) + logger.Debugw( + "handled message", + "type", req.GetType(), + "key", req.GetKey(), + "from", mPeer, + "time", time.Since(startTime), + ) if resp == nil { continue @@ -152,11 +176,25 @@ func (dht *IpfsDHT) handleNewMessage(s network.Stream) bool { err = writeMsg(s, resp) if err != nil { stats.Record(ctx, metrics.ReceivedMessageErrors.M(1)) - logger.Debugf("error writing response: %v", err) + logger.Debugw( + "error writing response", + "type", req.GetType(), + "key", req.GetKey(), + "from", mPeer, + "error", err) return false } elapsedTime := time.Since(startTime) + + logger.Debugw( + "responded to message", + "type", req.GetType(), + "key", req.GetKey(), + "from", mPeer, + "time", elapsedTime, + ) + latencyMillis := float64(elapsedTime) / float64(time.Millisecond) stats.Record(ctx, metrics.InboundRequestLatency.M(latencyMillis)) } @@ -173,6 +211,7 @@ func (dht *IpfsDHT) sendRequest(ctx context.Context, p peer.ID, pmes *pb.Message metrics.SentRequests.M(1), metrics.SentRequestErrors.M(1), ) + logger.Debugw("request failed to open message sender", "error", err, "to", p) return nil, err } @@ -184,19 +223,16 @@ func (dht *IpfsDHT) sendRequest(ctx context.Context, p peer.ID, pmes *pb.Message metrics.SentRequests.M(1), metrics.SentRequestErrors.M(1), ) + logger.Debugw("request failed", "error", err, "to", p) return nil, err } - // update the peer (on valid msgs only) - dht.updateFromMessage(ctx, p, rpmes) - stats.Record(ctx, metrics.SentRequests.M(1), metrics.SentBytes.M(int64(pmes.Size())), metrics.OutboundRequestLatency.M(float64(time.Since(start))/float64(time.Millisecond)), ) dht.peerstore.RecordLatency(p, time.Since(start)) - logger.Event(ctx, "dhtReceivedMessage", dht.self, p, rpmes) return rpmes, nil } @@ -210,6 +246,7 @@ func (dht *IpfsDHT) sendMessage(ctx context.Context, p peer.ID, pmes *pb.Message metrics.SentMessages.M(1), metrics.SentMessageErrors.M(1), ) + logger.Debugw("message failed to open message sender", "error", err, "to", p) return err } @@ -218,6 +255,7 @@ func (dht *IpfsDHT) sendMessage(ctx context.Context, p peer.ID, pmes *pb.Message metrics.SentMessages.M(1), metrics.SentMessageErrors.M(1), ) + logger.Debugw("message failed", "error", err, "to", p) return err } @@ -225,17 +263,6 @@ func (dht *IpfsDHT) sendMessage(ctx context.Context, p peer.ID, pmes *pb.Message metrics.SentMessages.M(1), metrics.SentBytes.M(int64(pmes.Size())), ) - - logger.Event(ctx, "dhtSentMessage", dht.self, p, pmes) - return nil -} - -func (dht *IpfsDHT) updateFromMessage(ctx context.Context, p peer.ID, mes *pb.Message) error { - // Make sure that this node is actually a DHT server, not just a client. - protos, err := dht.peerstore.SupportsProtocols(p, dht.protocolStrs()...) - if err == nil && len(protos) > 0 { - dht.Update(ctx, p) - } return nil } @@ -288,7 +315,7 @@ type messageSender struct { func (ms *messageSender) invalidate() { ms.invalid = true if ms.s != nil { - ms.s.Reset() + _ = ms.s.Reset() ms.s = nil } } @@ -314,6 +341,9 @@ func (ms *messageSender) prep(ctx context.Context) error { return nil } + // We only want to speak to peers using our primary protocols. We do not want to query any peer that only speaks + // one of the secondary "server" protocols that we happen to support (e.g. older nodes that we can respond to for + // backwards compatibility reasons). nstr, err := ms.dht.host.NewStream(ctx, ms.p, ms.dht.protocols...) if err != nil { return err @@ -343,20 +373,18 @@ func (ms *messageSender) SendMessage(ctx context.Context, pmes *pb.Message) erro } if err := ms.writeMsg(pmes); err != nil { - ms.s.Reset() + _ = ms.s.Reset() ms.s = nil if retry { - logger.Info("error writing message, bailing: ", err) + logger.Debugw("error writing message", "error", err) return err } - logger.Info("error writing message, trying again: ", err) + logger.Debugw("error writing message", "error", err, "retrying", true) retry = true continue } - logger.Event(ctx, "dhtSentMessage", ms.dht.self, ms.p, pmes) - if ms.singleMes > streamReuseTries { go helpers.FullClose(ms.s) ms.s = nil @@ -381,34 +409,32 @@ func (ms *messageSender) SendRequest(ctx context.Context, pmes *pb.Message) (*pb } if err := ms.writeMsg(pmes); err != nil { - ms.s.Reset() + _ = ms.s.Reset() ms.s = nil if retry { - logger.Info("error writing message, bailing: ", err) + logger.Debugw("error writing message", "error", err) return nil, err } - logger.Info("error writing message, trying again: ", err) + logger.Debugw("error writing message", "error", err, "retrying", true) retry = true continue } mes := new(pb.Message) if err := ms.ctxReadMsg(ctx, mes); err != nil { - ms.s.Reset() + _ = ms.s.Reset() ms.s = nil if retry { - logger.Info("error reading message, bailing: ", err) + logger.Debugw("error reading message", "error", err) return nil, err } - logger.Info("error reading message, trying again: ", err) + logger.Debugw("error reading message", "error", err, "retrying", true) retry = true continue } - logger.Event(ctx, "dhtSentMessage", ms.dht.self, ms.p, pmes) - if ms.singleMes > streamReuseTries { go helpers.FullClose(ms.s) ms.s = nil diff --git a/dht_options.go b/dht_options.go new file mode 100644 index 000000000..a5c906795 --- /dev/null +++ b/dht_options.go @@ -0,0 +1,325 @@ +package dht + +import ( + "fmt" + "time" + + ds "github.com/ipfs/go-datastore" + dssync "github.com/ipfs/go-datastore/sync" + "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/protocol" + record "github.com/libp2p/go-libp2p-record" +) + +// ModeOpt describes what mode the dht should operate in +type ModeOpt int + +const ( + // ModeAuto utilizes EvtLocalReachabilityChanged events sent over the event bus to dynamically switch the DHT + // between Client and Server modes based on network conditions + ModeAuto ModeOpt = iota + // ModeClient operates the DHT as a client only, it cannot respond to incoming queries + ModeClient + // ModeServer operates the DHT as a server, it can both send and respond to queries + ModeServer +) + +const DefaultPrefix protocol.ID = "/ipfs" + +// Options is a structure containing all the options that can be used when constructing a DHT. +type config struct { + datastore ds.Batching + validator record.Validator + mode ModeOpt + protocolPrefix protocol.ID + bucketSize int + concurrency int + resiliency int + maxRecordAge time.Duration + enableProviders bool + enableValues bool + queryPeerFilter QueryFilterFunc + + routingTable struct { + refreshQueryTimeout time.Duration + refreshInterval time.Duration + autoRefresh bool + latencyTolerance time.Duration + checkInterval time.Duration + peerFilter RouteTableFilterFunc + } + + // set to true if we're operating in v1 dht compatible mode + v1CompatibleMode bool +} + +func emptyQueryFilter(_ *IpfsDHT, ai peer.AddrInfo) bool { return true } +func emptyRTFilter(_ *IpfsDHT, conns []network.Conn) bool { return true } + +// apply applies the given options to this Option +func (c *config) apply(opts ...Option) error { + for i, opt := range opts { + if err := opt(c); err != nil { + return fmt.Errorf("dht option %d failed: %s", i, err) + } + } + return nil +} + +// Option DHT option type. +type Option func(*config) error + +const defaultBucketSize = 20 + +// defaults are the default DHT options. This option will be automatically +// prepended to any options you pass to the DHT constructor. +var defaults = func(o *config) error { + o.validator = record.NamespacedValidator{ + "pk": record.PublicKeyValidator{}, + } + o.datastore = dssync.MutexWrap(ds.NewMapDatastore()) + o.protocolPrefix = DefaultPrefix + o.enableProviders = true + o.enableValues = true + o.queryPeerFilter = emptyQueryFilter + + o.routingTable.latencyTolerance = time.Minute + o.routingTable.refreshQueryTimeout = 1 * time.Minute + o.routingTable.refreshInterval = 10 * time.Minute + o.routingTable.autoRefresh = true + o.routingTable.peerFilter = emptyRTFilter + o.maxRecordAge = time.Hour * 36 + + o.bucketSize = defaultBucketSize + o.concurrency = 3 + o.resiliency = 3 + + o.v1CompatibleMode = true + + return nil +} + +func (c *config) validate() error { + if c.protocolPrefix == DefaultPrefix { + if c.bucketSize != defaultBucketSize { + return fmt.Errorf("protocol prefix %s must use bucket size %d", DefaultPrefix, defaultBucketSize) + } + if !c.enableProviders { + return fmt.Errorf("protocol prefix %s must have providers enabled", DefaultPrefix) + } + if !c.enableValues { + return fmt.Errorf("protocol prefix %s must have values enabled", DefaultPrefix) + } + if nsval, ok := c.validator.(record.NamespacedValidator); !ok { + return fmt.Errorf("protocol prefix %s must use a namespaced validator", DefaultPrefix) + } else if len(nsval) > 2 || nsval["pk"] == nil || nsval["ipns"] == nil { + return fmt.Errorf("protocol prefix %s must support only the /pk and /ipns namespaces", DefaultPrefix) + } + return nil + } + return nil +} + +// RoutingTableLatencyTolerance sets the maximum acceptable latency for peers +// in the routing table's cluster. +func RoutingTableLatencyTolerance(latency time.Duration) Option { + return func(c *config) error { + c.routingTable.latencyTolerance = latency + return nil + } +} + +// RoutingTableRefreshQueryTimeout sets the timeout for routing table refresh +// queries. +func RoutingTableRefreshQueryTimeout(timeout time.Duration) Option { + return func(c *config) error { + c.routingTable.refreshQueryTimeout = timeout + return nil + } +} + +// RoutingTableRefreshPeriod sets the period for refreshing buckets in the +// routing table. The DHT will refresh buckets every period by: +// +// 1. First searching for nearby peers to figure out how many buckets we should try to fill. +// 1. Then searching for a random key in each bucket that hasn't been queried in +// the last refresh period. +func RoutingTableRefreshPeriod(period time.Duration) Option { + return func(c *config) error { + c.routingTable.refreshInterval = period + return nil + } +} + +// Datastore configures the DHT to use the specified datastore. +// +// Defaults to an in-memory (temporary) map. +func Datastore(ds ds.Batching) Option { + return func(c *config) error { + c.datastore = ds + return nil + } +} + +// Mode configures which mode the DHT operates in (Client, Server, Auto). +// +// Defaults to ModeAuto. +func Mode(m ModeOpt) Option { + return func(c *config) error { + c.mode = m + return nil + } +} + +// Validator configures the DHT to use the specified validator. +// +// Defaults to a namespaced validator that can only validate public keys. +func Validator(v record.Validator) Option { + return func(c *config) error { + c.validator = v + return nil + } +} + +// NamespacedValidator adds a validator namespaced under `ns`. This option fails +// if the DHT is not using a `record.NamespacedValidator` as it's validator (it +// uses one by default but this can be overridden with the `Validator` option). +// +// Example: Given a validator registered as `NamespacedValidator("ipns", +// myValidator)`, all records with keys starting with `/ipns/` will be validated +// with `myValidator`. +func NamespacedValidator(ns string, v record.Validator) Option { + return func(c *config) error { + nsval, ok := c.validator.(record.NamespacedValidator) + if !ok { + return fmt.Errorf("can only add namespaced validators to a NamespacedValidator") + } + nsval[ns] = v + return nil + } +} + +// ProtocolPrefix sets an application specific prefix to be attached to all DHT protocols. For example, +// /myapp/kad/1.0.0 instead of /ipfs/kad/1.0.0. Prefix should be of the form /myapp. +// +// Defaults to dht.DefaultPrefix +func ProtocolPrefix(prefix protocol.ID) Option { + return func(c *config) error { + c.protocolPrefix = prefix + return nil + } +} + +// BucketSize configures the bucket size (k in the Kademlia paper) of the routing table. +// +// The default value is 20. +func BucketSize(bucketSize int) Option { + return func(c *config) error { + c.bucketSize = bucketSize + return nil + } +} + +// Concurrency configures the number of concurrent requests (alpha in the Kademlia paper) for a given query path. +// +// The default value is 3. +func Concurrency(alpha int) Option { + return func(c *config) error { + c.concurrency = alpha + return nil + } +} + +// Resiliency configures the number of peers closest to a target that must have responded in order for a given query +// path to complete. +// +// The default value is 3. +func Resiliency(beta int) Option { + return func(c *config) error { + c.resiliency = beta + return nil + } +} + +// MaxRecordAge specifies the maximum time that any node will hold onto a record ("PutValue record") +// from the time its received. This does not apply to any other forms of validity that +// the record may contain. +// For example, a record may contain an ipns entry with an EOL saying its valid +// until the year 2020 (a great time in the future). For that record to stick around +// it must be rebroadcasted more frequently than once every 'MaxRecordAge' +func MaxRecordAge(maxAge time.Duration) Option { + return func(c *config) error { + c.maxRecordAge = maxAge + return nil + } +} + +// DisableAutoRefresh completely disables 'auto-refresh' on the DHT routing +// table. This means that we will neither refresh the routing table periodically +// nor when the routing table size goes below the minimum threshold. +func DisableAutoRefresh() Option { + return func(c *config) error { + c.routingTable.autoRefresh = false + return nil + } +} + +// DisableProviders disables storing and retrieving provider records. +// +// Defaults to enabled. +// +// WARNING: do not change this unless you're using a forked DHT (i.e., a private +// network and/or distinct DHT protocols with the `Protocols` option). +func DisableProviders() Option { + return func(c *config) error { + c.enableProviders = false + return nil + } +} + +// DisableProviders disables storing and retrieving value records (including +// public keys). +// +// Defaults to enabled. +// +// WARNING: do not change this unless you're using a forked DHT (i.e., a private +// network and/or distinct DHT protocols with the `Protocols` option). +func DisableValues() Option { + return func(c *config) error { + c.enableValues = false + return nil + } +} + +// QueryFilter sets a function that approves which peers may be dialed in a query +func QueryFilter(filter QueryFilterFunc) Option { + return func(c *config) error { + c.queryPeerFilter = filter + return nil + } +} + +// RoutingTableFilter sets a function that approves which peers may be added to the routing table. The host should +// already have at least one connection to the peer under consideration. +func RoutingTableFilter(filter RouteTableFilterFunc) Option { + return func(c *config) error { + c.routingTable.peerFilter = filter + return nil + } +} + +// V1CompatibleMode sets the DHT to operate in V1 compatible mode. In this mode, +// the DHT node will act like a V1 DHT node (use the V1 protocol names) but will +// use the V2 query and routing table logic. +// +// For now, this option defaults to true for backwards compatibility. In the +// near future, it will switch to false. +// +// This option is perma-unstable and may be removed in the future. +func V1CompatibleMode(enable bool) Option { + return func(c *config) error { + c.v1CompatibleMode = enable + return nil + } +} diff --git a/dht_test.go b/dht_test.go index 47dd0368a..74eb52724 100644 --- a/dht_test.go +++ b/dht_test.go @@ -13,6 +13,8 @@ import ( "testing" "time" + "github.com/libp2p/go-libp2p-core/event" + "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/peerstore" "github.com/libp2p/go-libp2p-core/routing" @@ -22,7 +24,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - opts "github.com/libp2p/go-libp2p-kad-dht/opts" pb "github.com/libp2p/go-libp2p-kad-dht/pb" "github.com/ipfs/go-cid" @@ -31,9 +32,10 @@ import ( record "github.com/libp2p/go-libp2p-record" swarmt "github.com/libp2p/go-libp2p-swarm/testing" "github.com/libp2p/go-libp2p-testing/ci" - travisci "github.com/libp2p/go-libp2p-testing/ci/travis" bhost "github.com/libp2p/go-libp2p/p2p/host/basic" ma "github.com/multiformats/go-multiaddr" + + detectrace "github.com/ipfs/go-detect-race" ) var testCaseCids []cid.Cid @@ -125,15 +127,25 @@ func (testAtomicPutValidator) Select(_ string, bs [][]byte) (int, error) { return index, nil } -func setupDHT(ctx context.Context, t *testing.T, client bool, options ...opts.Option) *IpfsDHT { +var testPrefix = ProtocolPrefix("/test") + +func setupDHT(ctx context.Context, t *testing.T, client bool, options ...Option) *IpfsDHT { + baseOpts := []Option{ + testPrefix, + NamespacedValidator("v", blankValidator{}), + DisableAutoRefresh(), + } + + if client { + baseOpts = append(baseOpts, Mode(ModeClient)) + } else { + baseOpts = append(baseOpts, Mode(ModeServer)) + } + d, err := New( ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), - append([]opts.Option{ - opts.Client(client), - opts.NamespacedValidator("v", blankValidator{}), - opts.DisableAutoRefresh(), - }, options...)..., + append(baseOpts, options...)..., ) if err != nil { t.Fatal(err) @@ -233,7 +245,8 @@ func bootstrap(t *testing.T, ctx context.Context, dhts []*IpfsDHT) { // Check to make sure we always signal the RefreshRoutingTable channel. func TestRefreshMultiple(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + // TODO: What's with this test? How long should it take and why does RefreshRoutingTable not take a context? + ctx, cancel := context.WithTimeout(context.Background(), 50*time.Second) defer cancel() dhts := setupDHTS(t, ctx, 5) @@ -295,7 +308,7 @@ func TestValueGetSet(t *testing.T) { } t.Log("requesting value on dhts: ", dhts[1].self) - ctxT, cancel = context.WithTimeout(ctx, time.Second*2) + ctxT, cancel = context.WithTimeout(ctx, time.Second*2*60) defer cancel() val, err := dhts[1].GetValue(ctxT, "/v/hello") @@ -314,13 +327,13 @@ func TestValueGetSet(t *testing.T) { t.Log("requesting value (offline) on dhts: ", dhts[2].self) vala, err := dhts[2].GetValue(ctxT, "/v/hello", Quorum(0)) - if vala != nil { - t.Fatalf("offline get should have failed, got %s", string(vala)) - } - if err != routing.ErrNotFound { - t.Fatalf("offline get should have failed with ErrNotFound, got: %s", err) + if err != nil { + t.Fatal(err) } + if string(vala) != "world" { + t.Fatalf("Expected 'world' got '%s'", string(vala)) + } t.Log("requesting value (online) on dhts: ", dhts[2].self) val, err = dhts[2].GetValue(ctxT, "/v/hello") if err != nil { @@ -451,7 +464,7 @@ func TestSearchValue(t *testing.T) { ctxT, cancel = context.WithTimeout(ctx, time.Second*2) defer cancel() - valCh, err := dhtA.SearchValue(ctxT, "/v/hello", Quorum(-1)) + valCh, err := dhtA.SearchValue(ctxT, "/v/hello", Quorum(0)) if err != nil { t.Fatal(err) } @@ -758,21 +771,19 @@ func TestRefresh(t *testing.T) { ctxT, cancelT := context.WithTimeout(ctx, 5*time.Second) defer cancelT() - go func() { - for ctxT.Err() == nil { - bootstrap(t, ctxT, dhts) - - // wait a bit. - select { - case <-time.After(50 * time.Millisecond): - continue // being explicit - case <-ctxT.Done(): - return - } + for ctxT.Err() == nil { + bootstrap(t, ctxT, dhts) + + // wait a bit. + select { + case <-time.After(50 * time.Millisecond): + continue // being explicit + case <-ctxT.Done(): + return } - }() + } - waitForWellFormedTables(t, dhts, 7, 10, 20*time.Second) + waitForWellFormedTables(t, dhts, 7, 10, 10*time.Second) cancelT() if u.Debug { @@ -788,8 +799,9 @@ func TestRefreshBelowMinRTThreshold(t *testing.T) { dhtA, err := New( ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), - opts.Client(false), - opts.NamespacedValidator("v", blankValidator{}), + testPrefix, + Mode(ModeServer), + NamespacedValidator("v", blankValidator{}), ) if err != nil { t.Fatal(err) @@ -841,83 +853,6 @@ func TestRefreshBelowMinRTThreshold(t *testing.T) { assert.Equal(t, dhtE.self, dhtA.routingTable.Find(dhtE.self), "A's routing table should have peer E!") } -// Check to make sure we re-fill the routing table from connected peers when it -// completely empties. -func TestEmptyTable(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - nDHTs := 50 - dhts := setupDHTS(t, ctx, nDHTs) - defer func() { - for _, dht := range dhts { - dht.Close() - defer dht.host.Close() - } - }() - - t.Logf("dhts are not connected. %d", nDHTs) - for _, dht := range dhts { - rtlen := dht.routingTable.Size() - if rtlen > 0 { - t.Errorf("routing table for %s should have 0 peers. has %d", dht.self, rtlen) - } - } - - for i := 1; i < nDHTs; i++ { - connectNoSync(t, ctx, dhts[0], dhts[i]) - } - - // Wait till the routing table stabilizes. - oldSize := dhts[0].routingTable.Size() - for { - time.Sleep(time.Millisecond) - newSize := dhts[0].routingTable.Size() - if oldSize == newSize { - break - } - oldSize = newSize - } - - // remove any one peer from the RT so we don't end up disconnecting all of them if the RT - // already has all peers we are connected to - dhts[0].routingTable.Remove(dhts[0].routingTable.ListPeers()[0]) - - if u.Debug { - printRoutingTables(dhts[:1]) - } - - // Disconnect from all peers that _were_ in the routing table. - routingTablePeers := make(map[peer.ID]bool, nDHTs) - for _, p := range dhts[0].RoutingTable().ListPeers() { - routingTablePeers[p] = true - } - - oldDHTs := dhts[1:] - dhts = dhts[:1] - for _, dht := range oldDHTs { - if routingTablePeers[dht.Host().ID()] { - dhts[0].Host().Network().ClosePeer(dht.host.ID()) - dht.Close() - dht.host.Close() - } else { - dhts = append(dhts, dht) - } - } - - // we should now _re-add_ some peers to the routing table - for i := 0; i < 100; i++ { - if dhts[0].routingTable.Size() > 0 { - return - } - time.Sleep(time.Millisecond) - } - if u.Debug { - printRoutingTables(dhts[:1]) - } - t.Fatal("routing table shouldn't have been empty") -} - func TestPeriodicRefresh(t *testing.T) { if ci.IsRunning() { t.Skip("skipping on CI. highly timing dependent") @@ -963,10 +898,16 @@ func TestPeriodicRefresh(t *testing.T) { } t.Logf("bootstrapping them so they find each other. %d", nDHTs) + var wg sync.WaitGroup for _, dht := range dhts { - dht.RefreshRoutingTable() + wg.Add(1) + go func(d *IpfsDHT) { + <-d.RefreshRoutingTable() + wg.Done() + }(dht) } + wg.Wait() // this is async, and we dont know when it's finished with one cycle, so keep checking // until the routing tables look better, or some long timeout for the failure case. waitForWellFormedTables(t, dhts, 7, 10, 20*time.Second) @@ -1228,68 +1169,12 @@ func TestFindPeer(t *testing.T) { } } -func TestFindPeersConnectedToPeer(t *testing.T) { - t.Skip("not quite correct (see note)") - - if testing.Short() { - t.SkipNow() - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - dhts := setupDHTS(t, ctx, 4) - defer func() { - for i := 0; i < 4; i++ { - dhts[i].Close() - dhts[i].host.Close() - } - }() - - // topology: - // 0-1, 1-2, 1-3, 2-3 - connect(t, ctx, dhts[0], dhts[1]) - connect(t, ctx, dhts[1], dhts[2]) - connect(t, ctx, dhts[1], dhts[3]) - connect(t, ctx, dhts[2], dhts[3]) - - // fmt.Println("0 is", peers[0]) - // fmt.Println("1 is", peers[1]) - // fmt.Println("2 is", peers[2]) - // fmt.Println("3 is", peers[3]) - - ctxT, cancel := context.WithTimeout(ctx, time.Second) - defer cancel() - pchan, err := dhts[0].FindPeersConnectedToPeer(ctxT, dhts[2].PeerID()) - if err != nil { - t.Fatal(err) - } - - // shouldFind := []peer.ID{peers[1], peers[3]} - var found []*peer.AddrInfo - for nextp := range pchan { - found = append(found, nextp) - } - - // fmt.Printf("querying 0 (%s) FindPeersConnectedToPeer 2 (%s)\n", peers[0], peers[2]) - // fmt.Println("should find 1, 3", shouldFind) - // fmt.Println("found", found) - - // testPeerListsMatch(t, shouldFind, found) - - logger.Warning("TestFindPeersConnectedToPeer is not quite correct") - if len(found) == 0 { - t.Fatal("didn't find any peers.") - } -} - func TestConnectCollision(t *testing.T) { - // t.Skip("skipping test to debug another") if testing.Short() { t.SkipNow() } - if travisci.IsRunning() { - t.Skip("Skipping on Travis-CI.") + if ci.IsRunning() { + t.Skip("Skipping on CI.") } runTimes := 10 @@ -1389,7 +1274,7 @@ func TestAtomicPut(t *testing.T) { wg.Add(1) go func(v []byte) { defer wg.Done() - putRecord(v) + _ = putRecord(v) // we expect some of these to fail }(v) } wg.Wait() @@ -1481,6 +1366,10 @@ func TestFindPeerQueryMinimal(t *testing.T) { } func TestFindPeerQuery(t *testing.T) { + if detectrace.WithRace() { + t.Skip("skipping due to race detector max goroutines") + } + if testing.Short() { t.Skip("skipping test in short mode") } @@ -1490,6 +1379,7 @@ func TestFindPeerQuery(t *testing.T) { testFindPeerQuery(t, 20, 80, 16) } +// NOTE: You must have ATLEAST (minRTRefreshThreshold+1) test peers before using this. func testFindPeerQuery(t *testing.T, bootstrappers, // Number of nodes connected to the querying node leafs, // Number of nodes that might be connected to from the bootstrappers @@ -1512,18 +1402,28 @@ func testFindPeerQuery(t *testing.T, for i := 0; i < bootstrappers; i++ { for j := 0; j < bootstrapperLeafConns; j++ { v := mrand.Intn(leafs) - connect(t, ctx, others[i], others[bootstrappers+v]) + connectNoSync(t, ctx, others[i], others[bootstrappers+v]) } } for i := 0; i < bootstrappers; i++ { - connect(t, ctx, guy, others[i]) + connectNoSync(t, ctx, guy, others[i]) + } + + // give some time for things to settle down + waitForWellFormedTables(t, dhts, minRTRefreshThreshold, minRTRefreshThreshold, 5*time.Second) + + for _, d := range dhts { + if len(d.RoutingTable().ListPeers()) > 0 { + if err := <-d.RefreshRoutingTable(); err != nil { + t.Fatal(err) + } + } } var reachableIds []peer.ID for i, d := range dhts { lp := len(d.host.Network().Peers()) - //t.Log(i, lp) if i != 0 && lp > 0 { reachableIds = append(reachableIds, d.PeerID()) } @@ -1533,11 +1433,6 @@ func testFindPeerQuery(t *testing.T, val := "foobar" rtval := kb.ConvertKey(val) - rtablePeers := guy.routingTable.NearestPeers(rtval, AlphaValue) - assert.Len(t, rtablePeers, minInt(bootstrappers, AlphaValue)) - - assert.Len(t, guy.host.Network().Peers(), bootstrappers) - out, err := guy.GetClosestPeers(ctx, val) require.NoError(t, err) @@ -1548,7 +1443,7 @@ func testFindPeerQuery(t *testing.T, sort.Sort(peer.IDSlice(outpeers)) - exp := kb.SortClosestPeers(reachableIds, rtval)[:minInt(KValue, len(reachableIds))] + exp := kb.SortClosestPeers(reachableIds, rtval)[:minInt(guy.bucketSize, len(reachableIds))] t.Logf("got %d peers", len(outpeers)) got := kb.SortClosestPeers(outpeers, rtval) @@ -1573,7 +1468,8 @@ func TestFindClosestPeers(t *testing.T) { connect(t, ctx, dhts[i], dhts[(i+1)%len(dhts)]) } - peers, err := dhts[1].GetClosestPeers(ctx, "foo") + querier := dhts[1] + peers, err := querier.GetClosestPeers(ctx, "foo") if err != nil { t.Fatal(err) } @@ -1583,9 +1479,46 @@ func TestFindClosestPeers(t *testing.T) { out = append(out, p) } - if len(out) != KValue { - t.Fatalf("got wrong number of peers (got %d, expected %d)", len(out), KValue) + if len(out) < querier.beta { + t.Fatalf("got wrong number of peers (got %d, expected at least %d)", len(out), querier.beta) + } +} + +func TestFixLowPeers(t *testing.T) { + ctx := context.Background() + + dhts := setupDHTS(t, ctx, minRTRefreshThreshold+5) + + defer func() { + for _, d := range dhts { + d.Close() + d.Host().Close() + } + }() + + mainD := dhts[0] + + // connect it to everyone else + for _, d := range dhts[1:] { + mainD.peerstore.AddAddrs(d.self, d.peerstore.Addrs(d.self), peerstore.TempAddrTTL) + require.NoError(t, mainD.Host().Connect(ctx, peer.AddrInfo{ID: d.self})) + } + + waitForWellFormedTables(t, []*IpfsDHT{mainD}, minRTRefreshThreshold, minRTRefreshThreshold+4, 5*time.Second) + + // run a refresh on all of them + for _, d := range dhts { + err := <-d.RefreshRoutingTable() + require.NoError(t, err) + } + + // now remove peers from RT so threshold gets hit + for _, d := range dhts[3:] { + mainD.routingTable.RemovePeer(d.self) } + + // but we will still get enough peers in the RT because of fix low Peers + waitForWellFormedTables(t, []*IpfsDHT{mainD}, minRTRefreshThreshold, minRTRefreshThreshold, 5*time.Second) } func TestProvideDisabled(t *testing.T) { @@ -1599,13 +1532,16 @@ func TestProvideDisabled(t *testing.T) { defer cancel() var ( - optsA, optsB []opts.Option + optsA, optsB []Option ) + optsA = append(optsA, ProtocolPrefix("/provMaybeDisabled")) + optsB = append(optsB, ProtocolPrefix("/provMaybeDisabled")) + if !enabledA { - optsA = append(optsA, opts.DisableProviders()) + optsA = append(optsA, DisableProviders()) } if !enabledB { - optsB = append(optsB, opts.DisableProviders()) + optsB = append(optsB, DisableProviders()) } dhtA := setupDHT(ctx, t, false, optsA...) @@ -1655,16 +1591,53 @@ func TestProvideDisabled(t *testing.T) { } } +func TestHandleRemotePeerProtocolChanges(t *testing.T) { + ctx := context.Background() + os := []Option{ + testPrefix, + Mode(ModeServer), + NamespacedValidator("v", blankValidator{}), + DisableAutoRefresh(), + } + + // start host 1 that speaks dht v1 + dhtA, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), os...) + require.NoError(t, err) + defer dhtA.Close() + + // start host 2 that also speaks dht v1 + dhtB, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), os...) + require.NoError(t, err) + defer dhtB.Close() + + connect(t, ctx, dhtA, dhtB) + + // now assert both have each other in their RT + require.True(t, waitForWellFormedTables(t, []*IpfsDHT{dhtA, dhtB}, 1, 1, 10*time.Second), "both RT should have one peer each") + + // dhtB becomes a client + require.NoError(t, dhtB.setMode(modeClient)) + + // which means that dhtA should evict it from it's RT + require.True(t, waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 0, 0, 10*time.Second), "dHTA routing table should have 0 peers") + + // dhtB becomes a server + require.NoError(t, dhtB.setMode(modeServer)) + + // which means dhtA should have it in the RT again because of fixLowPeers + require.True(t, waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 1, 1, 10*time.Second), "dHTA routing table should have 1 peers") +} + func TestGetSetPluggedProtocol(t *testing.T) { t.Run("PutValue/GetValue - same protocol", func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - os := []opts.Option{ - opts.Protocols("/esh/dht"), - opts.Client(false), - opts.NamespacedValidator("v", blankValidator{}), - opts.DisableAutoRefresh(), + os := []Option{ + ProtocolPrefix("/esh"), + Mode(ModeServer), + NamespacedValidator("v", blankValidator{}), + DisableAutoRefresh(), } dhtA, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), os...) @@ -1699,21 +1672,21 @@ func TestGetSetPluggedProtocol(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() - dhtA, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), []opts.Option{ - opts.Protocols("/esh/dht"), - opts.Client(false), - opts.NamespacedValidator("v", blankValidator{}), - opts.DisableAutoRefresh(), + dhtA, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), []Option{ + ProtocolPrefix("/esh"), + Mode(ModeServer), + NamespacedValidator("v", blankValidator{}), + DisableAutoRefresh(), }...) if err != nil { t.Fatal(err) } - dhtB, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), []opts.Option{ - opts.Protocols("/lsr/dht"), - opts.Client(false), - opts.NamespacedValidator("v", blankValidator{}), - opts.DisableAutoRefresh(), + dhtB, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), []Option{ + ProtocolPrefix("/lsr"), + Mode(ModeServer), + NamespacedValidator("v", blankValidator{}), + DisableAutoRefresh(), }...) if err != nil { t.Fatal(err) @@ -1731,9 +1704,9 @@ func TestGetSetPluggedProtocol(t *testing.T) { t.Fatalf("put should not have been able to find any peers in routing table, err:'%v'", err) } - _, err = dhtB.GetValue(ctx, "/v/cat") - if err == nil || !strings.Contains(err.Error(), "failed to find any peer in table") { - t.Fatalf("get should not have been able to find any peers in routing table, err:'%v'", err) + v, err := dhtB.GetValue(ctx, "/v/cat") + if v != nil || err != routing.ErrNotFound { + t.Fatalf("get should have failed from not being able to find the value, err: '%v'", err) } }) } @@ -1755,3 +1728,185 @@ func TestClientModeAtInit(t *testing.T) { err := pinger.Ping(context.Background(), client.PeerID()) assert.True(t, errors.Is(err, multistream.ErrNotSupported)) } + +func TestModeChange(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + clientOnly := setupDHT(ctx, t, true) + clientToServer := setupDHT(ctx, t, true) + clientOnly.Host().Peerstore().AddAddrs(clientToServer.PeerID(), clientToServer.Host().Addrs(), peerstore.AddressTTL) + err := clientOnly.Ping(ctx, clientToServer.PeerID()) + assert.True(t, errors.Is(err, multistream.ErrNotSupported)) + err = clientToServer.setMode(modeServer) + assert.Nil(t, err) + err = clientOnly.Ping(ctx, clientToServer.PeerID()) + assert.Nil(t, err) + err = clientToServer.setMode(modeClient) + assert.Nil(t, err) + err = clientOnly.Ping(ctx, clientToServer.PeerID()) + assert.NotNil(t, err) +} + +func TestDynamicModeSwitching(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + prober := setupDHT(ctx, t, true) // our test harness + node := setupDHT(ctx, t, true, Mode(ModeAuto)) // the node under test + prober.Host().Peerstore().AddAddrs(node.PeerID(), node.Host().Addrs(), peerstore.AddressTTL) + if _, err := prober.Host().Network().DialPeer(ctx, node.PeerID()); err != nil { + t.Fatal(err) + } + + emitter, err := node.host.EventBus().Emitter(new(event.EvtLocalReachabilityChanged)) + if err != nil { + t.Fatal(err) + } + + assertDHTClient := func() { + err = prober.Ping(ctx, node.PeerID()) + assert.True(t, errors.Is(err, multistream.ErrNotSupported)) + if l := len(prober.RoutingTable().ListPeers()); l != 0 { + t.Errorf("expected routing table length to be 0; instead is %d", l) + } + } + + assertDHTServer := func() { + err = prober.Ping(ctx, node.PeerID()) + assert.Nil(t, err) + // the node should be in the RT for the prober + // because the prober will call fixLowPeers when the node updates it's protocols + if l := len(prober.RoutingTable().ListPeers()); l != 1 { + t.Errorf("expected routing table length to be 1; instead is %d", l) + } + } + + err = emitter.Emit(event.EvtLocalReachabilityChanged{Reachability: network.ReachabilityPrivate}) + if err != nil { + t.Fatal(err) + } + time.Sleep(500 * time.Millisecond) + + assertDHTClient() + + err = emitter.Emit(event.EvtLocalReachabilityChanged{Reachability: network.ReachabilityPublic}) + if err != nil { + t.Fatal(err) + } + time.Sleep(500 * time.Millisecond) + + assertDHTServer() + + err = emitter.Emit(event.EvtLocalReachabilityChanged{Reachability: network.ReachabilityUnknown}) + if err != nil { + t.Fatal(err) + } + time.Sleep(500 * time.Millisecond) + + assertDHTClient() +} + +func TestProtocolUpgrade(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + os := []Option{ + testPrefix, + Mode(ModeServer), + NamespacedValidator("v", blankValidator{}), + DisableAutoRefresh(), + } + + // This test verifies that we can have a node serving both old and new DHTs that will respond as a server to the old + // DHT, but only act as a client of the new DHT. In it's capacity as a server it should also only tell queriers + // about other DHT servers in the new DHT. + + dhtA, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), + append([]Option{V1CompatibleMode(false)}, os...)...) + if err != nil { + t.Fatal(err) + } + + dhtB, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), + append([]Option{V1CompatibleMode(false)}, os...)...) + if err != nil { + t.Fatal(err) + } + + dhtC, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), + append([]Option{V1CompatibleMode(true)}, os...)...) + if err != nil { + t.Fatal(err) + } + + connect(t, ctx, dhtA, dhtB) + connectNoSync(t, ctx, dhtA, dhtC) + wait(t, ctx, dhtC, dhtA) + + if sz := dhtA.RoutingTable().Size(); sz != 1 { + t.Fatalf("Expected routing table to be of size %d got %d", 1, sz) + } + + ctxT, cancel := context.WithTimeout(ctx, time.Second) + defer cancel() + if err := dhtB.PutValue(ctxT, "/v/bat", []byte("screech")); err != nil { + t.Fatal(err) + } + + value, err := dhtC.GetValue(ctxT, "/v/bat") + if err != nil { + t.Fatal(err) + } + + if string(value) != "screech" { + t.Fatalf("Expected 'screech' got '%s'", string(value)) + } + + if err := dhtC.PutValue(ctxT, "/v/cat", []byte("meow")); err != nil { + t.Fatal(err) + } + + value, err = dhtB.GetValue(ctxT, "/v/cat") + if err != nil { + t.Fatal(err) + } + + if string(value) != "meow" { + t.Fatalf("Expected 'meow' got '%s'", string(value)) + } + + // Add record into local DHT only + rec := record.MakePutRecord("/v/crow", []byte("caw")) + rec.TimeReceived = u.FormatRFC3339(time.Now()) + err = dhtC.putLocal(string(rec.Key), rec) + if err != nil { + t.Fatal(err) + } + + _, err = dhtB.GetValue(ctxT, "/v/crow") + switch err { + case nil: + t.Fatalf("should not have been able to find value for %s", "/v/crow") + case routing.ErrNotFound: + default: + t.Fatal(err) + } + + // Add record into local DHT only + rec = record.MakePutRecord("/v/bee", []byte("buzz")) + rec.TimeReceived = u.FormatRFC3339(time.Now()) + err = dhtB.putLocal(string(rec.Key), rec) + if err != nil { + t.Fatal(err) + } + + value, err = dhtC.GetValue(ctxT, "/v/bee") + if err != nil { + t.Fatal(err) + } + + if string(value) != "buzz" { + t.Fatalf("Expected 'buzz' got '%s'", string(value)) + } +} diff --git a/dial_queue.go b/dial_queue.go deleted file mode 100644 index de966bfd3..000000000 --- a/dial_queue.go +++ /dev/null @@ -1,360 +0,0 @@ -package dht - -import ( - "context" - "math" - "sync" - "time" - - "github.com/libp2p/go-libp2p-core/peer" - queue "github.com/libp2p/go-libp2p-peerstore/queue" -) - -const ( - // DefaultDialQueueMinParallelism is the default value for the minimum number of worker dial goroutines that will - // be alive at any time. - DefaultDialQueueMinParallelism = 6 - // DefaultDialQueueMaxParallelism is the default value for the maximum number of worker dial goroutines that can - // be alive at any time. - DefaultDialQueueMaxParallelism = 20 - // DefaultDialQueueMaxIdle is the default value for the period that a worker dial goroutine waits before signalling - // a worker pool downscaling. - DefaultDialQueueMaxIdle = 5 * time.Second - // DefaultDialQueueScalingMutePeriod is the default value for the amount of time to ignore further worker pool - // scaling events, after one is processed. Its role is to reduce jitter. - DefaultDialQueueScalingMutePeriod = 1 * time.Second - // DefaultDialQueueScalingFactor is the default factor by which the current number of workers will be multiplied - // or divided when upscaling and downscaling events occur, respectively. - DefaultDialQueueScalingFactor = 1.5 -) - -type dialQueue struct { - *dqParams - - nWorkers uint - out *queue.ChanQueue - startOnce sync.Once - - waitingCh chan waitingCh - dieCh chan struct{} - growCh chan struct{} - shrinkCh chan struct{} -} - -type dqParams struct { - ctx context.Context - target string - dialFn func(context.Context, peer.ID) error - in *queue.ChanQueue - config dqConfig -} - -type dqConfig struct { - // minParallelism is the minimum number of worker dial goroutines that will be alive at any time. - minParallelism uint - // maxParallelism is the maximum number of worker dial goroutines that can be alive at any time. - maxParallelism uint - // scalingFactor is the factor by which the current number of workers will be multiplied or divided when upscaling - // and downscaling events occur, respectively. - scalingFactor float64 - // mutePeriod is the amount of time to ignore further worker pool scaling events, after one is processed. - // Its role is to reduce jitter. - mutePeriod time.Duration - // maxIdle is the period that a worker dial goroutine waits before signalling a worker pool downscaling. - maxIdle time.Duration -} - -// dqDefaultConfig returns the default configuration for dial queues. See const documentation to learn the default values. -func dqDefaultConfig() dqConfig { - return dqConfig{ - minParallelism: DefaultDialQueueMinParallelism, - maxParallelism: DefaultDialQueueMaxParallelism, - scalingFactor: DefaultDialQueueScalingFactor, - maxIdle: DefaultDialQueueMaxIdle, - mutePeriod: DefaultDialQueueScalingMutePeriod, - } -} - -type waitingCh struct { - ch chan<- peer.ID - ts time.Time -} - -// newDialQueue returns an _unstarted_ adaptive dial queue that spawns a dynamically sized set of goroutines to -// preemptively stage dials for later handoff to the DHT protocol for RPC. It identifies backpressure on both -// ends (dial consumers and dial producers), and takes compensating action by adjusting the worker pool. To -// activate the dial queue, call Start(). -// -// Why? Dialing is expensive. It's orders of magnitude slower than running an RPC on an already-established -// connection, as it requires establishing a TCP connection, multistream handshake, crypto handshake, mux handshake, -// and protocol negotiation. -// -// We start with config.minParallelism number of workers, and scale up and down based on demand and supply of -// dialled peers. -// -// The following events trigger scaling: -// - we scale up when we can't immediately return a successful dial to a new consumer. -// - we scale down when we've been idle for a while waiting for new dial attempts. -// - we scale down when we complete a dial and realise nobody was waiting for it. -// -// Dialler throttling (e.g. FD limit exceeded) is a concern, as we can easily spin up more workers to compensate, and -// end up adding fuel to the fire. Since we have no deterministic way to detect this for now, we hard-limit concurrency -// to config.maxParallelism. -func newDialQueue(params *dqParams) (*dialQueue, error) { - dq := &dialQueue{ - dqParams: params, - out: queue.NewChanQueue(params.ctx, queue.NewXORDistancePQ(params.target)), - growCh: make(chan struct{}, 1), - shrinkCh: make(chan struct{}, 1), - waitingCh: make(chan waitingCh), - dieCh: make(chan struct{}, params.config.maxParallelism), - } - - return dq, nil -} - -// Start initiates action on this dial queue. It should only be called once; subsequent calls are ignored. -func (dq *dialQueue) Start() { - dq.startOnce.Do(func() { - go dq.control() - }) -} - -func (dq *dialQueue) control() { - var ( - dialled <-chan peer.ID - waiting []waitingCh - lastScalingEvt = time.Now() - ) - - defer func() { - for _, w := range waiting { - close(w.ch) - } - waiting = nil - }() - - // start workers - - tgt := int(dq.dqParams.config.minParallelism) - for i := 0; i < tgt; i++ { - go dq.worker() - } - dq.nWorkers = uint(tgt) - - // control workers - - for { - // First process any backlog of dial jobs and waiters -- making progress is the priority. - // This block is copied below; couldn't find a more concise way of doing this. - select { - case <-dq.ctx.Done(): - return - case w := <-dq.waitingCh: - waiting = append(waiting, w) - dialled = dq.out.DeqChan - continue // onto the top. - case p, ok := <-dialled: - if !ok { - return // we're done if the ChanQueue is closed, which happens when the context is closed. - } - w := waiting[0] - logger.Debugf("delivering dialled peer to DHT; took %dms.", time.Since(w.ts)/time.Millisecond) - w.ch <- p - close(w.ch) - waiting = waiting[1:] - if len(waiting) == 0 { - // no more waiters, so stop consuming dialled jobs. - dialled = nil - } - continue // onto the top. - default: - // there's nothing to process, so proceed onto the main select block. - } - - select { - case <-dq.ctx.Done(): - return - case w := <-dq.waitingCh: - waiting = append(waiting, w) - dialled = dq.out.DeqChan - case p, ok := <-dialled: - if !ok { - return // we're done if the ChanQueue is closed, which happens when the context is closed. - } - w := waiting[0] - logger.Debugf("delivering dialled peer to DHT; took %dms.", time.Since(w.ts)/time.Millisecond) - w.ch <- p - close(w.ch) - waiting = waiting[1:] - if len(waiting) == 0 { - // no more waiters, so stop consuming dialled jobs. - dialled = nil - } - case <-dq.growCh: - if time.Since(lastScalingEvt) < dq.config.mutePeriod { - continue - } - dq.grow() - lastScalingEvt = time.Now() - case <-dq.shrinkCh: - if time.Since(lastScalingEvt) < dq.config.mutePeriod { - continue - } - dq.shrink() - lastScalingEvt = time.Now() - } - } -} - -func (dq *dialQueue) Consume() <-chan peer.ID { - ch := make(chan peer.ID, 1) - - select { - case p, ok := <-dq.out.DeqChan: - // short circuit and return a dialled peer if it's immediately available, or abort if DeqChan is closed. - if ok { - ch <- p - } - close(ch) - return ch - case <-dq.ctx.Done(): - // return a closed channel with no value if we're done. - close(ch) - return ch - default: - } - - // we have no finished dials to return, trigger a scale up. - select { - case dq.growCh <- struct{}{}: - default: - } - - // park the channel until a dialled peer becomes available. - select { - case dq.waitingCh <- waitingCh{ch, time.Now()}: - // all good - case <-dq.ctx.Done(): - // return a closed channel with no value if we're done. - close(ch) - } - return ch -} - -func (dq *dialQueue) grow() { - // no mutex needed as this is only called from the (single-threaded) control loop. - defer func(prev uint) { - if prev == dq.nWorkers { - return - } - logger.Debugf("grew dial worker pool: %d => %d", prev, dq.nWorkers) - }(dq.nWorkers) - - if dq.nWorkers == dq.config.maxParallelism { - return - } - // choosing not to worry about uint wrapping beyond max value. - target := uint(math.Floor(float64(dq.nWorkers) * dq.config.scalingFactor)) - if target > dq.config.maxParallelism { - target = dq.config.maxParallelism - } - for ; dq.nWorkers < target; dq.nWorkers++ { - go dq.worker() - } -} - -func (dq *dialQueue) shrink() { - // no mutex needed as this is only called from the (single-threaded) control loop. - defer func(prev uint) { - if prev == dq.nWorkers { - return - } - logger.Debugf("shrunk dial worker pool: %d => %d", prev, dq.nWorkers) - }(dq.nWorkers) - - if dq.nWorkers == dq.config.minParallelism { - return - } - target := uint(math.Floor(float64(dq.nWorkers) / dq.config.scalingFactor)) - if target < dq.config.minParallelism { - target = dq.config.minParallelism - } - // send as many die signals as workers we have to prune. - for ; dq.nWorkers > target; dq.nWorkers-- { - select { - case dq.dieCh <- struct{}{}: - default: - logger.Debugf("too many die signals queued up.") - } - } -} - -func (dq *dialQueue) worker() { - // This idle timer tracks if the environment is slow. If we're waiting to long to acquire a peer to dial, - // it means that the DHT query is progressing slow and we should shrink the worker pool. - idleTimer := time.NewTimer(24 * time.Hour) // placeholder init value which will be overridden immediately. - defer idleTimer.Stop() - for { - // trap exit signals first. - select { - case <-dq.ctx.Done(): - return - case <-dq.dieCh: - return - default: - } - - idleTimer.Stop() - select { - case <-idleTimer.C: - default: - // NOTE: There is a slight race here. We could be in the - // middle of firing the timer and not read anything from the channel. - // - // However, that's not really a huge issue. We'll think - // we're idle but that's fine. - } - idleTimer.Reset(dq.config.maxIdle) - - select { - case <-dq.dieCh: - return - case <-dq.ctx.Done(): - return - case <-idleTimer.C: - // no new dial requests during our idle period; time to scale down. - case p, ok := <-dq.in.DeqChan: - if !ok { - return - } - - t := time.Now() - if err := dq.dialFn(dq.ctx, p); err != nil { - logger.Debugf("discarding dialled peer because of error: %v", err) - continue - } - logger.Debugf("dialling %v took %dms (as observed by the dht subsystem).", p, time.Since(t)/time.Millisecond) - waiting := len(dq.waitingCh) - - // by the time we're done dialling, it's possible that the context is closed, in which case there will - // be nobody listening on dq.out.EnqChan and we could block forever. - select { - case dq.out.EnqChan <- p: - case <-dq.ctx.Done(): - return - } - if waiting > 0 { - // we have somebody to deliver this value to, so no need to shrink. - continue - } - } - - // scaling down; control only arrives here if the idle timer fires, or if there are no goroutines - // waiting for the value we just produced. - select { - case dq.shrinkCh <- struct{}{}: - default: - } - } -} diff --git a/dial_queue_test.go b/dial_queue_test.go deleted file mode 100644 index e97bb9519..000000000 --- a/dial_queue_test.go +++ /dev/null @@ -1,247 +0,0 @@ -package dht - -import ( - "context" - "sync" - "sync/atomic" - "testing" - "time" - - "github.com/libp2p/go-libp2p-core/peer" - queue "github.com/libp2p/go-libp2p-peerstore/queue" -) - -func TestDialQueueGrowsOnSlowDials(t *testing.T) { - in := queue.NewChanQueue(context.Background(), queue.NewXORDistancePQ("test")) - hang := make(chan struct{}) - - var cnt int32 - dialFn := func(ctx context.Context, p peer.ID) error { - atomic.AddInt32(&cnt, 1) - <-hang - return nil - } - - // Enqueue 20 jobs. - for i := 0; i < 20; i++ { - in.EnqChan <- peer.ID(i) - } - - // remove the mute period to grow faster. - config := dqDefaultConfig() - config.maxIdle = 10 * time.Minute - config.mutePeriod = 0 - dq, err := newDialQueue(&dqParams{ - ctx: context.Background(), - target: "test", - in: in, - dialFn: dialFn, - config: config, - }) - if err != nil { - t.Error("unexpected error when constructing the dial queue", err) - } - - dq.Start() - - for i := 0; i < 4; i++ { - _ = dq.Consume() - time.Sleep(100 * time.Millisecond) - } - - for i := 0; i < 20; i++ { - if atomic.LoadInt32(&cnt) > int32(DefaultDialQueueMinParallelism) { - return - } - time.Sleep(100 * time.Millisecond) - } - - t.Errorf("expected 19 concurrent dials, got %d", atomic.LoadInt32(&cnt)) - -} - -func TestDialQueueShrinksWithNoConsumers(t *testing.T) { - // reduce interference from the other shrink path. - - in := queue.NewChanQueue(context.Background(), queue.NewXORDistancePQ("test")) - hang := make(chan struct{}) - - wg := new(sync.WaitGroup) - wg.Add(13) - dialFn := func(ctx context.Context, p peer.ID) error { - wg.Done() - <-hang - return nil - } - - config := dqDefaultConfig() - config.maxIdle = 10 * time.Minute - config.mutePeriod = 0 - dq, err := newDialQueue(&dqParams{ - ctx: context.Background(), - target: "test", - in: in, - dialFn: dialFn, - config: config, - }) - if err != nil { - t.Error("unexpected error when constructing the dial queue", err) - } - - dq.Start() - - // acquire 3 consumers, everytime we acquire a consumer, we will grow the pool because no dial job is completed - // and immediately returnable. - for i := 0; i < 3; i++ { - _ = dq.Consume() - } - - // Enqueue 13 jobs, one per worker we'll grow to. - for i := 0; i < 13; i++ { - in.EnqChan <- peer.ID(i) - } - - waitForWg(t, wg, 2*time.Second) - - // Release a few dialFn, but not all of them because downscaling happens when workers detect there are no - // consumers to consume their values. So the other three will be these witnesses. - for i := 0; i < 3; i++ { - hang <- struct{}{} - } - - // allow enough time for signalling and dispatching values to outstanding consumers. - time.Sleep(1 * time.Second) - - // unblock the rest. - for i := 0; i < 10; i++ { - hang <- struct{}{} - } - - wg = new(sync.WaitGroup) - // we should now only have 6 workers, because all the shrink events will have been honoured. - wg.Add(6) - - // enqueue more jobs. - for i := 0; i < 6; i++ { - in.EnqChan <- peer.ID(i) - } - - // let's check we have 6 workers hanging. - waitForWg(t, wg, 2*time.Second) -} - -// Inactivity = workers are idle because the DHT query is progressing slow and is producing too few peers to dial. -func TestDialQueueShrinksWithWhenIdle(t *testing.T) { - in := queue.NewChanQueue(context.Background(), queue.NewXORDistancePQ("test")) - hang := make(chan struct{}) - - var wg sync.WaitGroup - wg.Add(13) - dialFn := func(ctx context.Context, p peer.ID) error { - wg.Done() - <-hang - return nil - } - - // Enqueue 13 jobs. - for i := 0; i < 13; i++ { - in.EnqChan <- peer.ID(i) - } - - config := dqDefaultConfig() - config.maxIdle = 1 * time.Second - config.mutePeriod = 0 - dq, err := newDialQueue(&dqParams{ - ctx: context.Background(), - target: "test", - in: in, - dialFn: dialFn, - config: config, - }) - if err != nil { - t.Error("unexpected error when constructing the dial queue", err) - } - - dq.Start() - - // keep up to speed with backlog by releasing the dial function every time we acquire a channel. - for i := 0; i < 13; i++ { - ch := dq.Consume() - hang <- struct{}{} - <-ch - time.Sleep(100 * time.Millisecond) - } - - // wait for MaxIdlePeriod. - time.Sleep(1500 * time.Millisecond) - - // we should now only have 6 workers, because all the shrink events will have been honoured. - wg.Add(6) - - // enqueue more jobs - for i := 0; i < 10; i++ { - in.EnqChan <- peer.ID(i) - } - - // let's check we have 6 workers hanging. - waitForWg(t, &wg, 2*time.Second) -} - -func TestDialQueueMutePeriodHonored(t *testing.T) { - in := queue.NewChanQueue(context.Background(), queue.NewXORDistancePQ("test")) - hang := make(chan struct{}) - var wg sync.WaitGroup - wg.Add(6) - dialFn := func(ctx context.Context, p peer.ID) error { - wg.Done() - <-hang - return nil - } - - // Enqueue a bunch of jobs. - for i := 0; i < 20; i++ { - in.EnqChan <- peer.ID(i) - } - - config := dqDefaultConfig() - config.mutePeriod = 2 * time.Second - dq, err := newDialQueue(&dqParams{ - ctx: context.Background(), - target: "test", - in: in, - dialFn: dialFn, - config: config, - }) - if err != nil { - t.Error("unexpected error when constructing the dial queue", err) - } - - dq.Start() - - // pick up three consumers. - for i := 0; i < 3; i++ { - _ = dq.Consume() - time.Sleep(100 * time.Millisecond) - } - - time.Sleep(500 * time.Millisecond) - - // we'll only have 6 workers because the grow signals have been ignored. - waitForWg(t, &wg, 2*time.Second) -} - -func waitForWg(t *testing.T, wg *sync.WaitGroup, wait time.Duration) { - t.Helper() - - done := make(chan struct{}) - go func() { - defer close(done) - wg.Wait() - }() - - select { - case <-time.After(wait): - t.Error("timeout while waiting for WaitGroup") - case <-done: - } -} diff --git a/events.go b/events.go new file mode 100644 index 000000000..38ec6ca6c --- /dev/null +++ b/events.go @@ -0,0 +1,237 @@ +package dht + +import ( + "context" + "encoding/json" + "sync" + + "github.com/google/uuid" + + "github.com/libp2p/go-libp2p-core/peer" + kbucket "github.com/libp2p/go-libp2p-kbucket" +) + +type KeyKadID struct { + Key string + Kad kbucket.ID +} + +func NewKeyKadID(k string) *KeyKadID { + return &KeyKadID{ + Key: k, + Kad: kbucket.ConvertKey(k), + } +} + +type PeerKadID struct { + Peer peer.ID + Kad kbucket.ID +} + +func NewPeerKadID(p peer.ID) *PeerKadID { + return &PeerKadID{ + Peer: p, + Kad: kbucket.ConvertPeerID(p), + } +} + +func NewPeerKadIDSlice(p []peer.ID) []*PeerKadID { + r := make([]*PeerKadID, len(p)) + for i := range p { + r[i] = NewPeerKadID(p[i]) + } + return r +} + +func OptPeerKadID(p peer.ID) *PeerKadID { + if p == "" { + return nil + } else { + return NewPeerKadID(p) + } +} + +func NewLookupEvent( + node peer.ID, + id uuid.UUID, + key string, + request *LookupUpdateEvent, + response *LookupUpdateEvent, + terminate *LookupTerminateEvent, +) *LookupEvent { + return &LookupEvent{ + Node: NewPeerKadID(node), + ID: id, + Key: NewKeyKadID(key), + Request: request, + Response: response, + Terminate: terminate, + } +} + +// LookupEvent is emitted for every notable event that happens during a DHT lookup. +// LookupEvent supports JSON marshalling because all of its fields do, recursively. +type LookupEvent struct { + // Node is the ID of the node performing the lookup. + Node *PeerKadID + // ID is a unique identifier for the lookup instance. + ID uuid.UUID + // Key is the Kademlia key used as a lookup target. + Key *KeyKadID + // Request, if not nil, describes a state update event, associated with an outgoing query request. + Request *LookupUpdateEvent + // Response, if not nil, describes a state update event, associated with an outgoing query response. + Response *LookupUpdateEvent + // Terminate, if not nil, describe a termination event. + Terminate *LookupTerminateEvent +} + +func NewLookupUpdateEvent( + cause peer.ID, + source peer.ID, + heard []peer.ID, + waiting []peer.ID, + queried []peer.ID, + unreachable []peer.ID, +) *LookupUpdateEvent { + return &LookupUpdateEvent{ + Cause: OptPeerKadID(cause), + Source: OptPeerKadID(source), + Heard: NewPeerKadIDSlice(heard), + Waiting: NewPeerKadIDSlice(waiting), + Queried: NewPeerKadIDSlice(queried), + Unreachable: NewPeerKadIDSlice(unreachable), + } +} + +// LookupUpdateEvent describes a lookup state update event. +type LookupUpdateEvent struct { + // Cause is the peer whose response (or lack of response) caused the update event. + // If Cause is nil, this is the first update event in the lookup, caused by the seeding. + Cause *PeerKadID + // Source is the peer who informed us about the peer IDs in this update (below). + Source *PeerKadID + // Heard is a set of peers whose state in the lookup's peerset is being set to "heard". + Heard []*PeerKadID + // Waiting is a set of peers whose state in the lookup's peerset is being set to "waiting". + Waiting []*PeerKadID + // Queried is a set of peers whose state in the lookup's peerset is being set to "queried". + Queried []*PeerKadID + // Unreachable is a set of peers whose state in the lookup's peerset is being set to "unreachable". + Unreachable []*PeerKadID +} + +// LookupTerminateEvent describes a lookup termination event. +type LookupTerminateEvent struct { + // Reason is the reason for lookup termination. + Reason LookupTerminationReason +} + +func NewLookupTerminateEvent(reason LookupTerminationReason) *LookupTerminateEvent { + return &LookupTerminateEvent{Reason: reason} +} + +// LookupTerminationReason captures reasons for terminating a lookup. +type LookupTerminationReason int + +func (r LookupTerminationReason) MarshalJSON() ([]byte, error) { + return json.Marshal(r.String()) +} + +func (r LookupTerminationReason) String() string { + switch r { + case LookupStopped: + return "stopped" + case LookupCancelled: + return "cancelled" + case LookupStarvation: + return "starvation" + case LookupCompleted: + return "completed" + } + panic("unreachable") +} + +const ( + // LookupStopped indicates that the lookup was aborted by the user's stopFn. + LookupStopped LookupTerminationReason = iota + // LookupCancelled indicates that the lookup was aborted by the context. + LookupCancelled + // LookupStarvation indicates that the lookup terminated due to lack of unqueried peers. + LookupStarvation + // LookupCompleted indicates that the lookup terminated successfully, reaching the Kademlia end condition. + LookupCompleted +) + +type routingLookupKey struct{} + +// TODO: lookupEventChannel copies the implementation of eventChanel. +// The two should be refactored to use a common event channel implementation. +// A common implementation needs to rethink the signature of RegisterForEvents, +// because returning a typed channel cannot be made polymorphic without creating +// additional "adapter" channels. This will be easier to handle when Go +// introduces generics. +type lookupEventChannel struct { + mu sync.Mutex + ctx context.Context + ch chan<- *LookupEvent +} + +// waitThenClose is spawned in a goroutine when the channel is registered. This +// safely cleans up the channel when the context has been canceled. +func (e *lookupEventChannel) waitThenClose() { + <-e.ctx.Done() + e.mu.Lock() + close(e.ch) + // 1. Signals that we're done. + // 2. Frees memory (in case we end up hanging on to this for a while). + e.ch = nil + e.mu.Unlock() +} + +// send sends an event on the event channel, aborting if either the passed or +// the internal context expire. +func (e *lookupEventChannel) send(ctx context.Context, ev *LookupEvent) { + e.mu.Lock() + // Closed. + if e.ch == nil { + e.mu.Unlock() + return + } + // in case the passed context is unrelated, wait on both. + select { + case e.ch <- ev: + case <-e.ctx.Done(): + case <-ctx.Done(): + } + e.mu.Unlock() +} + +// RegisterForLookupEvents registers a lookup event channel with the given context. +// The returned context can be passed to DHT queries to receive lookup events on +// the returned channels. +// +// The passed context MUST be canceled when the caller is no longer interested +// in query events. +func RegisterForLookupEvents(ctx context.Context) (context.Context, <-chan *LookupEvent) { + ch := make(chan *LookupEvent, LookupEventBufferSize) + ech := &lookupEventChannel{ch: ch, ctx: ctx} + go ech.waitThenClose() + return context.WithValue(ctx, routingLookupKey{}, ech), ch +} + +// Number of events to buffer. +var LookupEventBufferSize = 16 + +// PublishLookupEvent publishes a query event to the query event channel +// associated with the given context, if any. +func PublishLookupEvent(ctx context.Context, ev *LookupEvent) { + ich := ctx.Value(routingLookupKey{}) + if ich == nil { + return + } + + // We *want* to panic here. + ech := ich.(*lookupEventChannel) + ech.send(ctx, ev) +} diff --git a/ext_test.go b/ext_test.go index b01d0b5dc..467ac1ff5 100644 --- a/ext_test.go +++ b/ext_test.go @@ -8,17 +8,24 @@ import ( "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/peerstore" + "github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p-core/routing" - opts "github.com/libp2p/go-libp2p-kad-dht/opts" - ggio "github.com/gogo/protobuf/io" - u "github.com/ipfs/go-ipfs-util" pb "github.com/libp2p/go-libp2p-kad-dht/pb" record "github.com/libp2p/go-libp2p-record" + swarmt "github.com/libp2p/go-libp2p-swarm/testing" + bhost "github.com/libp2p/go-libp2p/p2p/host/basic" mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" + + ggio "github.com/gogo/protobuf/io" + u "github.com/ipfs/go-ipfs-util" + "github.com/stretchr/testify/require" ) -func TestHang(t *testing.T) { +// Test that one hung request to a peer doesn't prevent another request +// using that same peer from obeying its context. +func TestHungRequest(t *testing.T) { ctx := context.Background() mn, err := mocknet.FullMeshConnected(ctx, 2) if err != nil { @@ -26,37 +33,46 @@ func TestHang(t *testing.T) { } hosts := mn.Hosts() - os := []opts.Option{opts.DisableAutoRefresh()} + os := []Option{testPrefix, DisableAutoRefresh()} d, err := New(ctx, hosts[0], os...) if err != nil { t.Fatal(err) } - // Hang on every request. - hosts[1].SetStreamHandler(d.protocols[0], func(s network.Stream) { - defer s.Reset() - <-ctx.Done() - }) - d.Update(ctx, hosts[1].ID()) + for _, proto := range d.serverProtocols { + // Hang on every request. + hosts[1].SetStreamHandler(proto, func(s network.Stream) { + defer s.Reset() //nolint + <-ctx.Done() + }) + } + + require.NoError(t, hosts[0].Peerstore().AddProtocols(hosts[1].ID(), protocol.ConvertToStrings(d.serverProtocols)...)) + d.peerFound(ctx, hosts[1].ID(), true) ctx1, cancel1 := context.WithTimeout(ctx, 1*time.Second) defer cancel1() - peers, err := d.GetClosestPeers(ctx1, testCaseCids[0].KeyString()) - if err != nil { - t.Fatal(err) - } + done := make(chan error, 1) + go func() { + _, err := d.GetClosestPeers(ctx1, testCaseCids[0].KeyString()) + done <- err + }() time.Sleep(100 * time.Millisecond) ctx2, cancel2 := context.WithTimeout(ctx, 100*time.Millisecond) defer cancel2() - _ = d.Provide(ctx2, testCaseCids[0], true) - if ctx2.Err() != context.DeadlineExceeded { + err = d.Provide(ctx2, testCaseCids[0], true) + if err != context.DeadlineExceeded { t.Errorf("expected to fail with deadline exceeded, got: %s", ctx2.Err()) } select { - case <-peers: - t.Error("GetClosestPeers should not have returned yet") + case <-done: + t.Errorf("GetClosestPeers should not have returned yet") default: + err = <-done + if err != context.DeadlineExceeded { + t.Errorf("expected the deadline to be exceeded, got %s", err) + } } } @@ -67,24 +83,29 @@ func TestGetFailures(t *testing.T) { } ctx := context.Background() - mn, err := mocknet.FullMeshConnected(ctx, 2) + + host1 := bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)) + host2 := bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)) + + d, err := New(ctx, host1, testPrefix, DisableAutoRefresh(), Mode(ModeServer)) if err != nil { t.Fatal(err) } - hosts := mn.Hosts() - os := []opts.Option{opts.DisableAutoRefresh()} - d, err := New(ctx, hosts[0], os...) + // Reply with failures to every message + for _, proto := range d.serverProtocols { + host2.SetStreamHandler(proto, func(s network.Stream) { + time.Sleep(400 * time.Millisecond) + s.Close() + }) + } + + host1.Peerstore().AddAddrs(host2.ID(), host2.Addrs(), peerstore.ConnectedAddrTTL) + _, err = host1.Network().DialPeer(ctx, host2.ID()) if err != nil { t.Fatal(err) } - d.Update(ctx, hosts[1].ID()) - - // Reply with failures to every message - hosts[1].SetStreamHandler(d.protocols[0], func(s network.Stream) { - time.Sleep(400 * time.Millisecond) - s.Close() - }) + time.Sleep(1 * time.Second) // This one should time out ctx1, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond) @@ -103,25 +124,26 @@ func TestGetFailures(t *testing.T) { t.Log("Timeout test passed.") - // Reply with failures to every message - hosts[1].SetStreamHandler(d.protocols[0], func(s network.Stream) { - defer s.Close() + for _, proto := range d.serverProtocols { + // Reply with failures to every message + host2.SetStreamHandler(proto, func(s network.Stream) { + defer s.Close() - pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := ggio.NewDelimitedWriter(s) + pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) + pbw := ggio.NewDelimitedWriter(s) - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } + pmes := new(pb.Message) + if err := pbr.ReadMsg(pmes); err != nil { + // user gave up + return + } - resp := &pb.Message{ - Type: pmes.Type, - } - if err := pbw.WriteMsg(resp); err != nil { - panic(err) - } - }) + resp := &pb.Message{ + Type: pmes.Type, + } + _ = pbw.WriteMsg(resp) + }) + } // This one should fail with NotFound. // long context timeout to ensure we dont end too early. @@ -156,7 +178,7 @@ func TestGetFailures(t *testing.T) { Record: rec, } - s, err := hosts[1].NewStream(context.Background(), hosts[0].ID(), d.protocols[0]) + s, err := host2.NewStream(context.Background(), host1.ID(), d.protocols...) if err != nil { t.Fatal(err) } @@ -195,49 +217,51 @@ func TestNotFound(t *testing.T) { } hosts := mn.Hosts() - os := []opts.Option{opts.DisableAutoRefresh()} + os := []Option{testPrefix, DisableAutoRefresh()} d, err := New(ctx, hosts[0], os...) if err != nil { t.Fatal(err) } for _, p := range hosts { - d.Update(ctx, p.ID()) + d.peerFound(ctx, p.ID(), true) } // Reply with random peers to every message for _, host := range hosts { host := host // shadow loop var - host.SetStreamHandler(d.protocols[0], func(s network.Stream) { - defer s.Close() - - pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := ggio.NewDelimitedWriter(s) - - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } + for _, proto := range d.serverProtocols { + host.SetStreamHandler(proto, func(s network.Stream) { + defer s.Close() - switch pmes.GetType() { - case pb.Message_GET_VALUE: - resp := &pb.Message{Type: pmes.Type} + pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) + pbw := ggio.NewDelimitedWriter(s) - ps := []peer.AddrInfo{} - for i := 0; i < 7; i++ { - p := hosts[rand.Intn(len(hosts))].ID() - pi := host.Peerstore().PeerInfo(p) - ps = append(ps, pi) + pmes := new(pb.Message) + if err := pbr.ReadMsg(pmes); err != nil { + panic(err) } - resp.CloserPeers = pb.PeerInfosToPBPeers(d.host.Network(), ps) - if err := pbw.WriteMsg(resp); err != nil { - panic(err) + switch pmes.GetType() { + case pb.Message_GET_VALUE: + resp := &pb.Message{Type: pmes.Type} + + ps := []peer.AddrInfo{} + for i := 0; i < 7; i++ { + p := hosts[rand.Intn(len(hosts))].ID() + pi := host.Peerstore().PeerInfo(p) + ps = append(ps, pi) + } + + resp.CloserPeers = pb.PeerInfosToPBPeers(d.host.Network(), ps) + if err := pbw.WriteMsg(resp); err != nil { + panic(err) + } + default: + panic("Shouldnt recieve this.") } - default: - panic("Shouldnt recieve this.") - } - }) + }) + } } // long timeout to ensure timing is not at play. @@ -275,46 +299,48 @@ func TestLessThanKResponses(t *testing.T) { } hosts := mn.Hosts() - os := []opts.Option{opts.DisableAutoRefresh()} + os := []Option{testPrefix, DisableAutoRefresh()} d, err := New(ctx, hosts[0], os...) if err != nil { t.Fatal(err) } for i := 1; i < 5; i++ { - d.Update(ctx, hosts[i].ID()) + d.peerFound(ctx, hosts[i].ID(), true) } // Reply with random peers to every message for _, host := range hosts { host := host // shadow loop var - host.SetStreamHandler(d.protocols[0], func(s network.Stream) { - defer s.Close() + for _, proto := range d.serverProtocols { + host.SetStreamHandler(proto, func(s network.Stream) { + defer s.Close() - pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := ggio.NewDelimitedWriter(s) + pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) + pbw := ggio.NewDelimitedWriter(s) - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } - - switch pmes.GetType() { - case pb.Message_GET_VALUE: - pi := host.Peerstore().PeerInfo(hosts[1].ID()) - resp := &pb.Message{ - Type: pmes.Type, - CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), + pmes := new(pb.Message) + if err := pbr.ReadMsg(pmes); err != nil { + panic(err) } - if err := pbw.WriteMsg(resp); err != nil { - panic(err) + switch pmes.GetType() { + case pb.Message_GET_VALUE: + pi := host.Peerstore().PeerInfo(hosts[1].ID()) + resp := &pb.Message{ + Type: pmes.Type, + CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), + } + + if err := pbw.WriteMsg(resp); err != nil { + panic(err) + } + default: + panic("Shouldnt recieve this.") } - default: - panic("Shouldnt recieve this.") - } - }) + }) + } } ctx, cancel := context.WithTimeout(ctx, time.Second*30) @@ -345,42 +371,44 @@ func TestMultipleQueries(t *testing.T) { t.Fatal(err) } hosts := mn.Hosts() - os := []opts.Option{opts.DisableAutoRefresh()} + os := []Option{testPrefix, DisableAutoRefresh()} d, err := New(ctx, hosts[0], os...) if err != nil { t.Fatal(err) } - d.Update(ctx, hosts[1].ID()) + d.peerFound(ctx, hosts[1].ID(), true) - // It would be nice to be able to just get a value and succeed but then - // we'd need to deal with selectors and validators... - hosts[1].SetStreamHandler(d.protocols[0], func(s network.Stream) { - defer s.Close() - - pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) - pbw := ggio.NewDelimitedWriter(s) + for _, proto := range d.serverProtocols { + // It would be nice to be able to just get a value and succeed but then + // we'd need to deal with selectors and validators... + hosts[1].SetStreamHandler(proto, func(s network.Stream) { + defer s.Close() - pmes := new(pb.Message) - if err := pbr.ReadMsg(pmes); err != nil { - panic(err) - } + pbr := ggio.NewDelimitedReader(s, network.MessageSizeMax) + pbw := ggio.NewDelimitedWriter(s) - switch pmes.GetType() { - case pb.Message_GET_VALUE: - pi := hosts[1].Peerstore().PeerInfo(hosts[0].ID()) - resp := &pb.Message{ - Type: pmes.Type, - CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), + pmes := new(pb.Message) + if err := pbr.ReadMsg(pmes); err != nil { + panic(err) } - if err := pbw.WriteMsg(resp); err != nil { - panic(err) + switch pmes.GetType() { + case pb.Message_GET_VALUE: + pi := hosts[1].Peerstore().PeerInfo(hosts[0].ID()) + resp := &pb.Message{ + Type: pmes.Type, + CloserPeers: pb.PeerInfosToPBPeers(d.host.Network(), []peer.AddrInfo{pi}), + } + + if err := pbw.WriteMsg(resp); err != nil { + panic(err) + } + default: + panic("Shouldnt recieve this.") } - default: - panic("Shouldnt recieve this.") - } - }) + }) + } // long timeout to ensure timing is not at play. ctx, cancel := context.WithTimeout(ctx, time.Second*20) diff --git a/go.mod b/go.mod index 3b1df20a3..6be298416 100644 --- a/go.mod +++ b/go.mod @@ -4,29 +4,33 @@ go 1.14 require ( github.com/gogo/protobuf v1.3.1 - github.com/hashicorp/go-multierror v1.0.0 + github.com/google/uuid v1.1.1 + github.com/hashicorp/go-multierror v1.1.0 github.com/hashicorp/golang-lru v0.5.4 github.com/ipfs/go-cid v0.0.5 github.com/ipfs/go-datastore v0.4.4 + github.com/ipfs/go-detect-race v0.0.1 github.com/ipfs/go-ipfs-util v0.0.1 - github.com/ipfs/go-log v1.0.2 - github.com/ipfs/go-todocounter v0.0.2 - github.com/jbenet/goprocess v0.1.3 - github.com/libp2p/go-libp2p v0.6.1 - github.com/libp2p/go-libp2p-core v0.5.0 - github.com/libp2p/go-libp2p-kbucket v0.2.3 - github.com/libp2p/go-libp2p-peerstore v0.2.0 + github.com/ipfs/go-log v1.0.3 + github.com/jbenet/goprocess v0.1.4 + github.com/libp2p/go-eventbus v0.1.0 + github.com/libp2p/go-libp2p v0.7.4 + github.com/libp2p/go-libp2p-core v0.5.1 + github.com/libp2p/go-libp2p-kbucket v0.3.3 + github.com/libp2p/go-libp2p-peerstore v0.2.2 github.com/libp2p/go-libp2p-record v0.1.2 - github.com/libp2p/go-libp2p-routing v0.1.0 - github.com/libp2p/go-libp2p-swarm v0.2.2 + github.com/libp2p/go-libp2p-swarm v0.2.3 github.com/libp2p/go-libp2p-testing v0.1.1 github.com/libp2p/go-msgio v0.0.4 + github.com/libp2p/go-netroute v0.1.2 github.com/mr-tron/base58 v1.1.3 github.com/multiformats/go-base32 v0.0.3 github.com/multiformats/go-multiaddr v0.2.1 github.com/multiformats/go-multiaddr-dns v0.2.0 + github.com/multiformats/go-multiaddr-net v0.1.4 github.com/multiformats/go-multihash v0.0.13 github.com/multiformats/go-multistream v0.1.1 github.com/stretchr/testify v1.5.1 + github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 go.opencensus.io v0.22.3 ) diff --git a/go.sum b/go.sum index 23b6dfe20..7a938c7da 100644 --- a/go.sum +++ b/go.sum @@ -65,18 +65,22 @@ github.com/google/go-cmp v0.2.0 h1:+dTQ8DZQJz0Mb/HjFlkptS1FeQ4cWSnN941F8aEG4SQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0 h1:crn/baboCvb5fXaQ0IJ1SGTsTVrWpDsCWC8EGETZijY= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/gopacket v1.1.17 h1:rMrlX2ZY2UbvT+sdz3+6J+pp2z+msCq9MxTU6ymxbBY= +github.com/google/gopacket v1.1.17/go.mod h1:UdDNZ1OO62aGYVnPhxT1U6aI7ukYtA/kB8vaU0diBUM= github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gorilla/websocket v1.4.1 h1:q7AeDBpnBk8AogcD4DSag/Ukw/KV+YhzLj2bP5HvKCM= github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= +github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gxed/hashland/keccakpg v0.0.1 h1:wrk3uMNaMxbXiHibbPO4S0ymqJMm41WiudyFSs7UnsU= github.com/gxed/hashland/keccakpg v0.0.1/go.mod h1:kRzw3HkwxFU1mpmPP8v1WyQzwdGfmKFJ6tItnhQ67kU= github.com/gxed/hashland/murmur3 v0.0.1 h1:SheiaIt0sda5K+8FLz952/1iWS9zrnKsEJaOJu4ZbSc= github.com/gxed/hashland/murmur3 v0.0.1/go.mod h1:KjXop02n4/ckmZSnY2+HKcLud/tcmvhST0bie/0lS48= github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= -github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-multierror v1.1.0 h1:B9UzwGQJehnUY1yNrnwREHc3fGbC2xefo8g4TbElacI= +github.com/hashicorp/go-multierror v1.1.0/go.mod h1:spPvp8C1qA32ftKqdAHm4hHTbPw+vmowP0z+KUhOZdA= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1 h1:0hERBMJE1eitiLkihrMvRVBYAkpHzc/J3QdDN+dAcgU= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= @@ -101,10 +105,6 @@ github.com/ipfs/go-cid v0.0.5 h1:o0Ix8e/ql7Zb5UVUJEUfjsWCIY8t48++9lR8qi6oiJU= github.com/ipfs/go-cid v0.0.5/go.mod h1:plgt+Y5MnOey4vO4UlUazGqdbEXuFYitED67FexhXog= github.com/ipfs/go-datastore v0.0.1 h1:AW/KZCScnBWlSb5JbnEnLKFWXL224LBEh/9KXXOrUms= github.com/ipfs/go-datastore v0.0.1/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= -github.com/ipfs/go-datastore v0.1.0 h1:TOxI04l8CmO4zGtesENhzm4PwkFwJXY3rKiYaaMf9fI= -github.com/ipfs/go-datastore v0.1.0/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= -github.com/ipfs/go-datastore v0.1.1 h1:F4k0TkTAZGLFzBOrVKDAvch6JZtuN4NHkfdcEZL50aI= -github.com/ipfs/go-datastore v0.1.1/go.mod h1:w38XXW9kVFNp57Zj5knbKWM2T+KOZCGDRVNdgPHtbHw= github.com/ipfs/go-datastore v0.4.0/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.1 h1:W4ZfzyhNi3xmuU5dQhjfuRn/wFuqEE1KnOmmQiOevEY= github.com/ipfs/go-datastore v0.4.1/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= @@ -114,10 +114,8 @@ github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= github.com/ipfs/go-ds-badger v0.0.2/go.mod h1:Y3QpeSFWQf6MopLTiZD+VT6IC1yZqaGmjvRcKeSGij8= github.com/ipfs/go-ds-badger v0.0.5/go.mod h1:g5AuuCGmr7efyzQhLL8MzwqcauPojGPUaHzfGTzuE3s= -github.com/ipfs/go-ds-badger v0.0.7/go.mod h1:qt0/fWzZDoPW6jpQeqUjR5kBfhDNB65jd9YlmAvpQBk= github.com/ipfs/go-ds-badger v0.2.1/go.mod h1:Tx7l3aTph3FMFrRS838dcSJh+jjA7cX9DrGVwx/NOwE= github.com/ipfs/go-ds-leveldb v0.0.1/go.mod h1:feO8V3kubwsEF22n0YRQCffeb79OOYIykR4L04tMOYc= -github.com/ipfs/go-ds-leveldb v0.1.0/go.mod h1:hqAW8y4bwX5LWcCtku2rFNX3vjDZCy5LZCg+cSZvYb8= github.com/ipfs/go-ds-leveldb v0.4.1/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-util v0.0.1 h1:Wz9bL2wB2YBJqggkA4dD7oSmqB4cAnpNbGrlHJulv50= @@ -126,14 +124,18 @@ github.com/ipfs/go-log v0.0.1 h1:9XTUN/rW64BCG1YhPK9Hoy3q8nr4gOmHHBpgFdfw6Lc= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-log v1.0.2 h1:s19ZwJxH8rPWzypjcDpqPLIyV7BnbLqvpli3iZoqYK0= github.com/ipfs/go-log v1.0.2/go.mod h1:1MNjMxe0u6xvJZgeqbJ8vdo2TKaGwZ1a0Bpza+sr2Sk= +github.com/ipfs/go-log v1.0.3 h1:Gg7SUYSZ7BrqaKMwM+hRgcAkKv4QLfzP4XPQt5Sx/OI= +github.com/ipfs/go-log v1.0.3/go.mod h1:OsLySYkwIbiSUR/yBTdv1qPtcE4FW3WPWk/ewz9Ru+A= github.com/ipfs/go-log/v2 v2.0.2 h1:xguurydRdfKMJjKyxNXNU8lYP0VZH1NUwJRwUorjuEw= github.com/ipfs/go-log/v2 v2.0.2/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= -github.com/ipfs/go-todocounter v0.0.2 h1:9UBngSQhylg2UDcxSAtpkT+rEWFr26hDPXVStE8LFyc= -github.com/ipfs/go-todocounter v0.0.2/go.mod h1:l5aErvQc8qKE2r7NDMjmq5UNAvuZy0rC8BHOplkWvZ4= +github.com/ipfs/go-log/v2 v2.0.3 h1:Q2gXcBoCALyLN/pUQlz1qgu0x3uFV6FzP9oXhpfyJpc= +github.com/ipfs/go-log/v2 v2.0.3/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= github.com/jackpal/gateway v1.0.5 h1:qzXWUJfuMdlLMtt0a3Dgt+xkWQiA5itDEITVJtuSwMc= github.com/jackpal/gateway v1.0.5/go.mod h1:lTpwd4ACLXmpyiCTRtfiNyVnUmqT9RivzCDQetPfnjA= github.com/jackpal/go-nat-pmp v1.0.1 h1:i0LektDkO1QlrTm/cSuP+PyBCDnYvjPLGl4LdWEMiaA= github.com/jackpal/go-nat-pmp v1.0.1/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= +github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= +github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jbenet/go-cienv v0.0.0-20150120210510-1bb1476777ec h1:DQqZhhDvrTrEQ3Qod5yfavcA064e53xlQ+xajiorXgM= github.com/jbenet/go-cienv v0.0.0-20150120210510-1bb1476777ec/go.mod h1:rGaEvXB4uRSZMmzKNLoXvTu1sfx+1kv/DojUlPrSZGs= github.com/jbenet/go-cienv v0.1.0 h1:Vc/s0QbQtoxX8MwwSLWWh+xNNZvM3Lw7NsTcHrvvhMc= @@ -144,6 +146,8 @@ github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8 h1:bspPhN+oKYFk5f github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8/go.mod h1:Ly/wlsjFq/qrU3Rar62tu1gASgGw6chQbSh/XgIIXCY= github.com/jbenet/goprocess v0.1.3 h1:YKyIEECS/XvcfHtBzxtjBBbWK+MbvA6dG8ASiqwvr10= github.com/jbenet/goprocess v0.1.3/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= +github.com/jbenet/goprocess v0.1.4 h1:DRGOFReOMqqDNXwW70QkacFW0YN9QnwLV0Vqk+3oU0o= +github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= @@ -178,13 +182,22 @@ github.com/libp2p/go-flow-metrics v0.0.3 h1:8tAs/hSdNvUiLgtlSy3mxwxWP4I9y/jlkPFT github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-libp2p v0.6.1 h1:mxabyJf4l6AmotDOKObwSfBNBWjL5VYXysVFLUMAuB8= github.com/libp2p/go-libp2p v0.6.1/go.mod h1:CTFnWXogryAHjXAKEbOf1OWY+VeAP3lDMZkfEI5sT54= +github.com/libp2p/go-libp2p v0.7.0/go.mod h1:hZJf8txWeCduQRDC/WSqBGMxaTHCOYHt2xSU1ivxn0k= +github.com/libp2p/go-libp2p v0.7.4 h1:xVj1oSlN0C+FlxqiLuHC8WruMvq24xxfeVxmNhTG0r0= +github.com/libp2p/go-libp2p v0.7.4/go.mod h1:oXsBlTLF1q7pxr+9w6lqzS1ILpyHsaBPniVO7zIHGMw= github.com/libp2p/go-libp2p-autonat v0.1.1 h1:WLBZcIRsjZlWdAZj9CiBSvU2wQXoUOiS1Zk1tM7DTJI= github.com/libp2p/go-libp2p-autonat v0.1.1/go.mod h1:OXqkeGOY2xJVWKAGV2inNF5aKN/djNA3fdpCWloIudE= +github.com/libp2p/go-libp2p-autonat v0.2.0 h1:Kok+0M/4jiz6TTmxtBqAa5tLyHb/U+G/7o/JEeW7Wok= +github.com/libp2p/go-libp2p-autonat v0.2.0/go.mod h1:DX+9teU4pEEoZUqR1PiMlqliONQdNbfzE1C718tcViI= +github.com/libp2p/go-libp2p-autonat v0.2.1 h1:T0CRQhrvTBKfBSYw6Xo2K3ixtNpAnRCraxof3AAfgQA= +github.com/libp2p/go-libp2p-autonat v0.2.1/go.mod h1:MWtAhV5Ko1l6QBsHQNSuM6b1sRkXrpk0/LqCr+vCVxI= github.com/libp2p/go-libp2p-blankhost v0.1.1/go.mod h1:pf2fvdLJPsC1FsVrNP3DUUvMzUts2dsLLBEpo1vW1ro= github.com/libp2p/go-libp2p-blankhost v0.1.4 h1:I96SWjR4rK9irDHcHq3XHN6hawCRTPUADzkJacgZLvk= github.com/libp2p/go-libp2p-blankhost v0.1.4/go.mod h1:oJF0saYsAXQCSfDq254GMNmLNz6ZTHTOvtF4ZydUvwU= github.com/libp2p/go-libp2p-circuit v0.1.4 h1:Phzbmrg3BkVzbqd4ZZ149JxCuUWu2wZcXf/Kr6hZJj8= github.com/libp2p/go-libp2p-circuit v0.1.4/go.mod h1:CY67BrEjKNDhdTk8UgBX1Y/H5c3xkAcs3gnksxY7osU= +github.com/libp2p/go-libp2p-circuit v0.2.1 h1:BDiBcQxX/ZJJ/yDl3sqZt1bjj4PkZCEi7IEpwxXr13k= +github.com/libp2p/go-libp2p-circuit v0.2.1/go.mod h1:BXPwYDN5A8z4OEY9sOfr2DUQMLQvKt/6oku45YUmjIo= github.com/libp2p/go-libp2p-core v0.0.1 h1:HSTZtFIq/W5Ue43Zw+uWZyy2Vl5WtF0zDjKN8/DT/1I= github.com/libp2p/go-libp2p-core v0.0.1/go.mod h1:g/VxnTZ/1ygHxH3dKok7Vno1VfpvGcGip57wjTU4fco= github.com/libp2p/go-libp2p-core v0.0.4/go.mod h1:jyuCQP356gzfCFtRKyvAbNkyeuxb7OlyhWZ3nls5d2I= @@ -203,12 +216,16 @@ github.com/libp2p/go-libp2p-core v0.3.1/go.mod h1:thvWy0hvaSBhnVBaW37BvzgVV68OUh github.com/libp2p/go-libp2p-core v0.4.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= github.com/libp2p/go-libp2p-core v0.5.0 h1:FBQ1fpq2Fo/ClyjojVJ5AKXlKhvNc/B6U0O+7AN1ffE= github.com/libp2p/go-libp2p-core v0.5.0/go.mod h1:49XGI+kc38oGVwqSBhDEwytaAxgZasHhFfQKibzTls0= +github.com/libp2p/go-libp2p-core v0.5.1 h1:6Cu7WljPQtGY2krBlMoD8L/zH3tMUsCbqNFH7cZwCoI= +github.com/libp2p/go-libp2p-core v0.5.1/go.mod h1:uN7L2D4EvPCvzSH5SrhR72UWbnSGpt5/a35Sm4upn4Y= github.com/libp2p/go-libp2p-crypto v0.1.0 h1:k9MFy+o2zGDNGsaoZl0MA3iZ75qXxr9OOoAZF+sD5OQ= github.com/libp2p/go-libp2p-crypto v0.1.0/go.mod h1:sPUokVISZiy+nNuTTH/TY+leRSxnFj/2GLjtOTW90hI= github.com/libp2p/go-libp2p-discovery v0.2.0 h1:1p3YSOq7VsgaL+xVHPi8XAmtGyas6D2J6rWBEfz/aiY= github.com/libp2p/go-libp2p-discovery v0.2.0/go.mod h1:s4VGaxYMbw4+4+tsoQTqh7wfxg97AEdo4GYBt6BadWg= -github.com/libp2p/go-libp2p-kbucket v0.2.3 h1:XtNfN4WUy0cfeJoJgWCf1lor4Pp3kBkFJ9vQ+Zs+VUM= -github.com/libp2p/go-libp2p-kbucket v0.2.3/go.mod h1:opWrBZSWnBYPc315q497huxY3sz1t488X6OiXUEYWKA= +github.com/libp2p/go-libp2p-discovery v0.3.0 h1:+JnYBRLzZQtRq0mK3xhyjBwHytLmJXMTZkQfbw+UrGA= +github.com/libp2p/go-libp2p-discovery v0.3.0/go.mod h1:o03drFnz9BVAZdzC/QUQ+NeQOu38Fu7LJGEOK2gQltw= +github.com/libp2p/go-libp2p-kbucket v0.3.3 h1:V2Zwv6QnCK6Who0iiJW2eUKwdlTYGJ2HnLViaolDOcs= +github.com/libp2p/go-libp2p-kbucket v0.3.3/go.mod h1:IWFdYRBOYzaLEHnvrfzEkr+UcuveCXIoeO8QeFZSI6A= github.com/libp2p/go-libp2p-loggables v0.1.0 h1:h3w8QFfCt2UJl/0/NW4K829HX/0S4KD31PQ7m8UXXO8= github.com/libp2p/go-libp2p-loggables v0.1.0/go.mod h1:EyumB2Y6PrYjr55Q3/tiJ/o3xoDasoRYM7nOzEpoa90= github.com/libp2p/go-libp2p-mplex v0.2.0/go.mod h1:Ejl9IyjvXJ0T9iqUTE1jpYATQ9NM3g+OtR+EMMODbKo= @@ -216,8 +233,12 @@ github.com/libp2p/go-libp2p-mplex v0.2.1 h1:E1xaJBQnbSiTHGI1gaBKmKhu1TUKkErKJnE8 github.com/libp2p/go-libp2p-mplex v0.2.1/go.mod h1:SC99Rxs8Vuzrf/6WhmH41kNn13TiYdAWNYHrwImKLnE= github.com/libp2p/go-libp2p-mplex v0.2.2 h1:+Ld7YDAfVERQ0E+qqjE7o6fHwKuM0SqTzYiwN1lVVSA= github.com/libp2p/go-libp2p-mplex v0.2.2/go.mod h1:74S9eum0tVQdAfFiKxAyKzNdSuLqw5oadDq7+L/FELo= +github.com/libp2p/go-libp2p-mplex v0.2.3 h1:2zijwaJvpdesST2MXpI5w9wWFRgYtMcpRX7rrw0jmOo= +github.com/libp2p/go-libp2p-mplex v0.2.3/go.mod h1:CK3p2+9qH9x+7ER/gWWDYJ3QW5ZxWDkm+dVvjfuG3ek= github.com/libp2p/go-libp2p-nat v0.0.5 h1:/mH8pXFVKleflDL1YwqMg27W9GD8kjEx7NY0P6eGc98= github.com/libp2p/go-libp2p-nat v0.0.5/go.mod h1:1qubaE5bTZMJE+E/uu2URroMbzdubFz1ChgiN79yKPE= +github.com/libp2p/go-libp2p-nat v0.0.6 h1:wMWis3kYynCbHoyKLPBEMu4YRLltbm8Mk08HGSfvTkU= +github.com/libp2p/go-libp2p-nat v0.0.6/go.mod h1:iV59LVhB3IkFvS6S6sauVTSOrNEANnINbI/fkaLimiw= github.com/libp2p/go-libp2p-netutil v0.1.0 h1:zscYDNVEcGxyUpMd0JReUZTrpMfia8PmLKcKF72EAMQ= github.com/libp2p/go-libp2p-netutil v0.1.0/go.mod h1:3Qv/aDqtMLTUyQeundkKsA+YCThNdbQD54k3TqjpbFU= github.com/libp2p/go-libp2p-peer v0.2.0 h1:EQ8kMjaCUwt/Y5uLgjT8iY2qg0mGUT0N1zUjer50DsY= @@ -226,24 +247,28 @@ github.com/libp2p/go-libp2p-peerstore v0.1.0 h1:MKh7pRNPHSh1fLPj8u/M/s/napdmeNpo github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= github.com/libp2p/go-libp2p-peerstore v0.1.3 h1:wMgajt1uM2tMiqf4M+4qWKVyyFc8SfA+84VV9glZq1M= github.com/libp2p/go-libp2p-peerstore v0.1.3/go.mod h1:BJ9sHlm59/80oSkpWgr1MyY1ciXAXV397W6h1GH/uKI= -github.com/libp2p/go-libp2p-peerstore v0.1.4 h1:d23fvq5oYMJ/lkkbO4oTwBp/JP+I/1m5gZJobNXCE/k= -github.com/libp2p/go-libp2p-peerstore v0.1.4/go.mod h1:+4BDbDiiKf4PzpANZDAT+knVdLxvqh7hXOujessqdzs= github.com/libp2p/go-libp2p-peerstore v0.2.0 h1:XcgJhI8WyUOCbHyRLNEX5542YNj8hnLSJ2G1InRjDhk= github.com/libp2p/go-libp2p-peerstore v0.2.0/go.mod h1:N2l3eVIeAitSg3Pi2ipSrJYnqhVnMNQZo9nkSCuAbnQ= +github.com/libp2p/go-libp2p-peerstore v0.2.1 h1:u+gOfsKgu73ZkGWhvckRm03z9C+iS9TrLqpANweELGs= +github.com/libp2p/go-libp2p-peerstore v0.2.1/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= +github.com/libp2p/go-libp2p-peerstore v0.2.2 h1:iqc/m03jHn5doXN3+kS6JKvqQRHEltiXljQB85iVHWE= +github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6na5f0/k= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= github.com/libp2p/go-libp2p-record v0.1.2 h1:M50VKzWnmUrk/M5/Dz99qO9Xh4vs8ijsK+7HkJvRP+0= github.com/libp2p/go-libp2p-record v0.1.2/go.mod h1:pal0eNcT5nqZaTV7UGhqeGqxFgGdsU/9W//C8dqjQDk= -github.com/libp2p/go-libp2p-routing v0.1.0 h1:hFnj3WR3E2tOcKaGpyzfP4gvFZ3t8JkQmbapN0Ct+oU= -github.com/libp2p/go-libp2p-routing v0.1.0/go.mod h1:zfLhI1RI8RLEzmEaaPwzonRvXeeSHddONWkcTcB54nE= github.com/libp2p/go-libp2p-secio v0.1.0/go.mod h1:tMJo2w7h3+wN4pgU2LSYeiKPrfqBgkOsdiKK77hE7c8= github.com/libp2p/go-libp2p-secio v0.2.0 h1:ywzZBsWEEz2KNTn5RtzauEDq5RFEefPsttXYwAWqHng= github.com/libp2p/go-libp2p-secio v0.2.0/go.mod h1:2JdZepB8J5V9mBp79BmwsaPQhRPNN2NrnB2lKQcdy6g= github.com/libp2p/go-libp2p-secio v0.2.1 h1:eNWbJTdyPA7NxhP7J3c5lT97DC5d+u+IldkgCYFTPVA= github.com/libp2p/go-libp2p-secio v0.2.1/go.mod h1:cWtZpILJqkqrSkiYcDBh5lA3wbT2Q+hz3rJQq3iftD8= +github.com/libp2p/go-libp2p-secio v0.2.2 h1:rLLPvShPQAcY6eNurKNZq3eZjPWfU9kXF2eI9jIYdrg= +github.com/libp2p/go-libp2p-secio v0.2.2/go.mod h1:wP3bS+m5AUnFA+OFO7Er03uO1mncHG0uVwGrwvjYlNY= github.com/libp2p/go-libp2p-swarm v0.1.0/go.mod h1:wQVsCdjsuZoc730CgOvh5ox6K8evllckjebkdiY5ta4= github.com/libp2p/go-libp2p-swarm v0.2.2 h1:T4hUpgEs2r371PweU3DuH7EOmBIdTBCwWs+FLcgx3bQ= github.com/libp2p/go-libp2p-swarm v0.2.2/go.mod h1:fvmtQ0T1nErXym1/aa1uJEyN7JzaTNyBcHImCxRpPKU= +github.com/libp2p/go-libp2p-swarm v0.2.3 h1:uVkCb8Blfg7HQ/f30TyHn1g/uCwXsAET7pU0U59gx/A= +github.com/libp2p/go-libp2p-swarm v0.2.3/go.mod h1:P2VO/EpxRyDxtChXz/VPVXyTnszHvokHKRhfkEgFKNM= github.com/libp2p/go-libp2p-testing v0.0.2/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.3/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.4/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= @@ -257,8 +282,11 @@ github.com/libp2p/go-libp2p-transport-upgrader v0.2.0 h1:5EhPgQhXZNyfL22ERZTUoVp github.com/libp2p/go-libp2p-transport-upgrader v0.2.0/go.mod h1:mQcrHj4asu6ArfSoMuyojOdjx73Q47cYD7s5+gZOlns= github.com/libp2p/go-libp2p-yamux v0.2.0 h1:TSPZ5cMMz/wdoYsye/wU1TE4G3LDGMoeEN0xgnCKU/I= github.com/libp2p/go-libp2p-yamux v0.2.0/go.mod h1:Db2gU+XfLpm6E4rG5uGCFX6uXA8MEXOxFcRoXUODaK8= +github.com/libp2p/go-libp2p-yamux v0.2.2/go.mod h1:lIohaR0pT6mOt0AZ0L2dFze9hds9Req3OfS+B+dv4qw= github.com/libp2p/go-libp2p-yamux v0.2.5 h1:MuyItOqz03oi8npvjgMJxgnhllJLZnO/dKVOpTZ9+XI= github.com/libp2p/go-libp2p-yamux v0.2.5/go.mod h1:Zpgj6arbyQrmZ3wxSZxfBmbdnWtbZ48OpsfmQVTErwA= +github.com/libp2p/go-libp2p-yamux v0.2.7 h1:vzKu0NVtxvEIDGCv6mjKRcK0gipSgaXmJZ6jFv0d/dk= +github.com/libp2p/go-libp2p-yamux v0.2.7/go.mod h1:X28ENrBMU/nm4I3Nx4sZ4dgjZ6VhLEn0XhIoZ5viCwU= github.com/libp2p/go-maddr-filter v0.0.4 h1:hx8HIuuwk34KePddrp2mM5ivgPkZ09JH4AvsALRbFUs= github.com/libp2p/go-maddr-filter v0.0.4/go.mod h1:6eT12kSQMA9x2pvFQa+xesMKUBlj9VImZbj3B9FBH/Q= github.com/libp2p/go-maddr-filter v0.0.5 h1:CW3AgbMO6vUvT4kf87y4N+0P8KUl2aqLYhrGyDUbLSg= @@ -269,11 +297,17 @@ github.com/libp2p/go-mplex v0.1.0 h1:/nBTy5+1yRyY82YaO6HXQRnO5IAGsXTjEJaR3LdTPc0 github.com/libp2p/go-mplex v0.1.0/go.mod h1:SXgmdki2kwCUlCCbfGLEgHjC4pFqhTp0ZoV6aiKgxDU= github.com/libp2p/go-mplex v0.1.1 h1:huPH/GGRJzmsHR9IZJJsrSwIM5YE2gL4ssgl1YWb/ps= github.com/libp2p/go-mplex v0.1.1/go.mod h1:Xgz2RDCi3co0LeZfgjm4OgUF15+sVR8SRcu3SFXI1lk= +github.com/libp2p/go-mplex v0.1.2 h1:qOg1s+WdGLlpkrczDqmhYzyk3vCfsQ8+RxRTQjOZWwI= +github.com/libp2p/go-mplex v0.1.2/go.mod h1:Xgz2RDCi3co0LeZfgjm4OgUF15+sVR8SRcu3SFXI1lk= github.com/libp2p/go-msgio v0.0.2/go.mod h1:63lBBgOTDKQL6EWazRMCwXsEeEeK9O2Cd+0+6OOuipQ= github.com/libp2p/go-msgio v0.0.4 h1:agEFehY3zWJFUHK6SEMR7UYmk2z6kC3oeCM7ybLhguA= github.com/libp2p/go-msgio v0.0.4/go.mod h1:63lBBgOTDKQL6EWazRMCwXsEeEeK9O2Cd+0+6OOuipQ= github.com/libp2p/go-nat v0.0.4 h1:KbizNnq8YIf7+Hn7+VFL/xE0eDrkPru2zIO9NMwL8UQ= github.com/libp2p/go-nat v0.0.4/go.mod h1:Nmw50VAvKuk38jUBcmNh6p9lUJLoODbJRvYAa/+KSDo= +github.com/libp2p/go-nat v0.0.5 h1:qxnwkco8RLKqVh1NmjQ+tJ8p8khNLFxuElYG/TwqW4Q= +github.com/libp2p/go-nat v0.0.5/go.mod h1:B7NxsVNPZmRLvMOwiEO1scOSyjA56zxYAGv1yQgRkEU= +github.com/libp2p/go-netroute v0.1.2 h1:UHhB35chwgvcRI392znJA3RCBtZ3MpE3ahNCN5MR4Xg= +github.com/libp2p/go-netroute v0.1.2/go.mod h1:jZLDV+1PE8y5XxBySEBgbuVAXbhtuHSdmLPL2n9MKbk= github.com/libp2p/go-openssl v0.0.2 h1:9pP2d3Ubaxkv7ZisLjx9BFwgOGnQdQYnfcH29HNY3ls= github.com/libp2p/go-openssl v0.0.2/go.mod h1:v8Zw2ijCSWBQi8Pq5GAixw6DbFfa9u6VIYDXnvOXkc0= github.com/libp2p/go-openssl v0.0.3 h1:wjlG7HvQkt4Fq4cfH33Ivpwp0omaElYEi9z26qaIkIk= @@ -284,6 +318,10 @@ github.com/libp2p/go-reuseport v0.0.1 h1:7PhkfH73VXfPJYKQ6JwS5I/eVcoyYi9IMNGc6FW github.com/libp2p/go-reuseport v0.0.1/go.mod h1:jn6RmB1ufnQwl0Q1f+YxAj8isJgDCQzaaxIFYDhcYEA= github.com/libp2p/go-reuseport-transport v0.0.2 h1:WglMwyXyBu61CMkjCCtnmqNqnjib0GIEjMiHTwR/KN4= github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA20obEtsW9BLkuOUyQAOCbs= +github.com/libp2p/go-reuseport-transport v0.0.3 h1:zzOeXnTooCkRvoH+bSXEfXhn76+LAiwoneM0gnXjF2M= +github.com/libp2p/go-reuseport-transport v0.0.3/go.mod h1:Spv+MPft1exxARzP2Sruj2Wb5JSyHNncjf1Oi2dEbzM= +github.com/libp2p/go-sockaddr v0.0.2 h1:tCuXfpA9rq7llM/v834RKc/Xvovy/AqM9kHvTV/jY/Q= +github.com/libp2p/go-sockaddr v0.0.2/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-stream-muxer v0.0.1 h1:Ce6e2Pyu+b5MC1k3eeFtAax0pW4gc6MosYSLV05UeLw= github.com/libp2p/go-stream-muxer v0.0.1/go.mod h1:bAo8x7YkSpadMTbtTaxGVHWUQsR/l5MEaHbKaliuT14= github.com/libp2p/go-stream-muxer-multistream v0.2.0 h1:714bRJ4Zy9mdhyTLJ+ZKiROmAFwUHpeRidG+q7LTQOg= @@ -292,12 +330,19 @@ github.com/libp2p/go-tcp-transport v0.1.0 h1:IGhowvEqyMFknOar4FWCKSWE0zL36UFKQti github.com/libp2p/go-tcp-transport v0.1.0/go.mod h1:oJ8I5VXryj493DEJ7OsBieu8fcg2nHGctwtInJVpipc= github.com/libp2p/go-tcp-transport v0.1.1 h1:yGlqURmqgNA2fvzjSgZNlHcsd/IulAnKM8Ncu+vlqnw= github.com/libp2p/go-tcp-transport v0.1.1/go.mod h1:3HzGvLbx6etZjnFlERyakbaYPdfjg2pWP97dFZworkY= +github.com/libp2p/go-tcp-transport v0.2.0 h1:YoThc549fzmNJIh7XjHVtMIFaEDRtIrtWciG5LyYAPo= +github.com/libp2p/go-tcp-transport v0.2.0/go.mod h1:vX2U0CnWimU4h0SGSEsg++AzvBcroCGYw28kh94oLe0= github.com/libp2p/go-ws-transport v0.2.0 h1:MJCw2OrPA9+76YNRvdo1wMnSOxb9Bivj6sVFY1Xrj6w= github.com/libp2p/go-ws-transport v0.2.0/go.mod h1:9BHJz/4Q5A9ludYWKoGCFC5gUElzlHoKzu0yY9p/klM= +github.com/libp2p/go-ws-transport v0.3.0 h1:mjo6pL5aVR9rCjl9wNq3DupbaQlyR61pzoOT2MdtxaA= +github.com/libp2p/go-ws-transport v0.3.0/go.mod h1:bpgTJmRZAvVHrgHybCVyqoBmyLQ1fiZuEaBYusP5zsk= github.com/libp2p/go-yamux v1.2.2 h1:s6J6o7+ajoQMjHe7BEnq+EynOj5D2EoG8CuQgL3F2vg= github.com/libp2p/go-yamux v1.2.2/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= +github.com/libp2p/go-yamux v1.3.0/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= github.com/libp2p/go-yamux v1.3.3 h1:mWuzZRCAeTBFdynLlsYgA/EIeMOLr8XY04wa52NRhsE= github.com/libp2p/go-yamux v1.3.3/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= +github.com/libp2p/go-yamux v1.3.5 h1:ibuz4naPAully0pN6J/kmUARiqLpnDQIzI/8GCOrljg= +github.com/libp2p/go-yamux v1.3.5/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329 h1:2gxZ0XQIU/5z3Z3bUBu+FXuk2pFbkN6tcwi/pjyaDic= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -309,6 +354,7 @@ github.com/mattn/go-isatty v0.0.5 h1:tHXDdz1cpzGaovsTB+TVB8q90WEokoVmfMqoVcrLUgw github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b/go.mod h1:01TrycV0kFyexm33Z7vhZRXopbI8J3TDReVlkTgMUxE= github.com/miekg/dns v1.1.12/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= +github.com/miekg/dns v1.1.28/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16 h1:5W7KhL8HVF3XCFOweFD3BNESdnO8ewyYTFT2R+/b8FQ= @@ -361,6 +407,8 @@ github.com/multiformats/go-multiaddr-net v0.1.2 h1:P7zcBH9FRETdPkDrylcXVjQLQ2t1J github.com/multiformats/go-multiaddr-net v0.1.2/go.mod h1:QsWt3XK/3hwvNxZJp92iMQKME1qHfpYmyIjFVsSOY6Y= github.com/multiformats/go-multiaddr-net v0.1.3 h1:q/IYAvoPKuRzGeERn3uacWgm0LIWkLZBAvO5DxSzq3g= github.com/multiformats/go-multiaddr-net v0.1.3/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= +github.com/multiformats/go-multiaddr-net v0.1.4 h1:g6gwydsfADqFvrHoMkS0n9Ok9CG6F7ytOH/bJDkhIOY= +github.com/multiformats/go-multiaddr-net v0.1.4/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= github.com/multiformats/go-multibase v0.0.1 h1:PN9/v21eLywrFWdFNsFKaU04kLJzuYzmrJR+ubhT9qA= github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= github.com/multiformats/go-multihash v0.0.1 h1:HHwN1K12I+XllBCrqKnhX949Orn4oawPkegHMu2vDqQ= @@ -438,6 +486,7 @@ github.com/stretchr/testify v1.5.1 h1:nOGnQDM7FYENwehXlg/kFVnos3rEvtKTjRvOWSzb6H github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/syndtr/goleveldb v1.0.0/go.mod h1:ZVVdQEZoIme9iO1Ch2Jdy24qqXrMMOU6lpPAyBWyWuQ= github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= +github.com/wangjia184/sortedset v0.0.0-20160527075905-f5d03557ba30/go.mod h1:YkocrP2K2tcw938x9gCOmT5G5eCD6jsTz0SZuyAqwIE= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc h1:9lDbC6Rz4bwmou+oE6Dt4Cb2BGMur5eR/GYptkKUVHo= @@ -461,6 +510,8 @@ go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.4.0 h1:cxzIVoETapQEqDhQu3QfnvXAV4AlzcvUCxkVUFw3+EU= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= +go.uber.org/goleak v1.0.0 h1:qsup4IcBdlmsnGfqyLl4Ntn3C2XCCuKAE7DwHpScyUo= +go.uber.org/goleak v1.0.0/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.1.0 h1:HoEmRHQPVSqub6w2z2d2EOVs2fjyFRGyofhKuyDq0QI= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/zap v1.10.0 h1:ORx85nbTijNz8ljznvCMR1ZBIPKFn3jQrag10X2AsuM= @@ -487,6 +538,9 @@ golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -500,6 +554,8 @@ golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3 h1:0GoQqolDA55aaLxZyTzK/Y2eP golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859 h1:R/3boaszxrf1GEUWTVDzSKVwLmSJpwZ1yqXm8j0v2QI= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190923162816-aa69164e4478 h1:l5EDrHhldLYb3ZRHDUhXF7Om7MvYXnkV9/iQNo1lX6g= +golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f h1:wMNYb4v58l5UBM7MYRLPG6ZhfOqbKu7X5eyFl8ZhKvA= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -517,11 +573,13 @@ golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223 h1:DH4skfRX4EBpamg7iV4ZlCpbl golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190228124157-a34e9553db1e h1:ZytStCyV048ZqDsWHiYDdoI2Vd4msMcrDECFxS+tL9c= golang.org/x/sys v0.0.0-20190228124157-a34e9553db1e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190405154228-4b34438f7a67/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d h1:+R4KGOnez64A81RvjARKc4UT5/tI9ujCIVX+P5KiHuI= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb h1:fgwFCsaw9buMuxNd6+DQfAuSFqbNiQZpcgJQAgJsK6k= golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae h1:/WDfKMnPU+m5M4xB+6x4kaepxRw6jWvR5iDRdvjHgy8= @@ -538,8 +596,13 @@ golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd h1:/e+gpKk9r3dJobndpTytxS2gOy6m5uvpg+ISQoEcusQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216052735-49a3e744a425 h1:VvQyQJN0tSuecqgcIxMWnnfG5kSmgy9KZR9sW3W5QeA= +golang.org/x/tools v0.0.0-20191216052735-49a3e744a425/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7 h1:9zdDQZ7Thm29KFXgAX/+yaf3eVbP7djjWp/dXAppNCc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898 h1:/atklqdjdhuosWIl6AIbOeHJjicWYPqR9bpxqxYG2pA= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= diff --git a/handlers.go b/handlers.go index f4d835d9f..42445b70f 100644 --- a/handlers.go +++ b/handlers.go @@ -7,7 +7,6 @@ import ( "fmt" "time" - "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/peerstore" pstore "github.com/libp2p/go-libp2p-peerstore" @@ -53,11 +52,6 @@ func (dht *IpfsDHT) handlerForMsgType(t pb.Message_MessageType) dhtHandler { } func (dht *IpfsDHT) handleGetValue(ctx context.Context, p peer.ID, pmes *pb.Message) (_ *pb.Message, err error) { - ctx = logger.Start(ctx, "handleGetValue") - logger.SetTag(ctx, "peer", p) - defer func() { logger.FinishWithErr(ctx, err) }() - logger.Debugf("%s handleGetValue for key: %s", dht.self, pmes.GetKey()) - // setup response resp := pb.NewMessage(pmes.GetType(), pmes.GetKey(), pmes.GetClusterLevel()) @@ -82,10 +76,11 @@ func (dht *IpfsDHT) handleGetValue(ctx context.Context, p peer.ID, pmes *pb.Mess for _, pi := range closerinfos { logger.Debugf("handleGetValue returning closer peer: '%s'", pi.ID) if len(pi.Addrs) < 1 { - logger.Warningf(`no addresses on peer being sent! - [local:%s] - [sending:%s] - [remote:%s]`, dht.self, pi.ID, p) + logger.Warnw("no addresses on peer being sent", + "local", dht.self, + "to", p, + "sending", pi.ID, + ) } } @@ -155,13 +150,9 @@ func cleanRecord(rec *recpb.Record) { // Store a value in this peer local storage func (dht *IpfsDHT) handlePutValue(ctx context.Context, p peer.ID, pmes *pb.Message) (_ *pb.Message, err error) { - ctx = logger.Start(ctx, "handlePutValue") - logger.SetTag(ctx, "peer", p) - defer func() { logger.FinishWithErr(ctx, err) }() - rec := pmes.GetRecord() if rec == nil { - logger.Infof("Got nil record from: %s", p.Pretty()) + logger.Debugw("got nil record from", "from", p) return nil, errors.New("nil record") } @@ -173,7 +164,7 @@ func (dht *IpfsDHT) handlePutValue(ctx context.Context, p peer.ID, pmes *pb.Mess // Make sure the record is valid (not expired, valid signature etc) if err = dht.Validator.Validate(string(rec.GetKey()), rec.GetValue()); err != nil { - logger.Warningf("Bad dht record in PUT from: %s. %s", p.Pretty(), err) + logger.Infow("bad dht record in PUT", "from", p, "key", rec.GetKey(), "error", err) return nil, err } @@ -202,11 +193,11 @@ func (dht *IpfsDHT) handlePutValue(ctx context.Context, p peer.ID, pmes *pb.Mess recs := [][]byte{rec.GetValue(), existing.GetValue()} i, err := dht.Validator.Select(string(rec.GetKey()), recs) if err != nil { - logger.Warningf("Bad dht record in PUT from %s: %s", p.Pretty(), err) + logger.Warnw("dht record passed validation but failed select", "from", p, "key", rec.GetKey(), "error", err) return nil, err } if i != 0 { - logger.Infof("DHT record in PUT from %s is older than existing record. Ignoring", p.Pretty()) + logger.Infow("DHT record in PUT older than existing record (ignoring)", "peer", p, "key", rec.GetKey()) return nil, errors.New("old record") } } @@ -220,7 +211,6 @@ func (dht *IpfsDHT) handlePutValue(ctx context.Context, p peer.ID, pmes *pb.Mess } err = dht.datastore.Put(dskey, data) - logger.Debugf("%s handlePutValue %v", dht.self, dskey) return pmes, err } @@ -232,14 +222,14 @@ func (dht *IpfsDHT) getRecordFromDatastore(dskey ds.Key) (*recpb.Record, error) return nil, nil } if err != nil { - logger.Errorf("Got error retrieving record with key %s from datastore: %s", dskey, err) + logger.Errorw("error retrieving record from datastore", "key", dskey, "error", err) return nil, err } rec := new(recpb.Record) err = proto.Unmarshal(buf, rec) if err != nil { // Bad data in datastore, log it but don't return an error, we'll just overwrite it - logger.Errorf("Bad record data stored in datastore with key %s: could not unmarshal record", dskey) + logger.Errorw("failed to unmarshal record from datastore", "key", dskey, "error", err) return nil, nil } @@ -247,7 +237,7 @@ func (dht *IpfsDHT) getRecordFromDatastore(dskey ds.Key) (*recpb.Record, error) if err != nil { // Invalid record in datastore, probably expired but don't return an error, // we'll just overwrite it - logger.Debugf("Local record verify failed: %s (discarded)", err) + logger.Debugw("local record verify failed", "key", rec.GetKey(), "error", err) return nil, nil } @@ -259,10 +249,7 @@ func (dht *IpfsDHT) handlePing(_ context.Context, p peer.ID, pmes *pb.Message) ( return pmes, nil } -func (dht *IpfsDHT) handleFindPeer(ctx context.Context, p peer.ID, pmes *pb.Message) (_ *pb.Message, _err error) { - ctx = logger.Start(ctx, "handleFindPeer") - defer func() { logger.FinishWithErr(ctx, _err) }() - logger.SetTag(ctx, "peer", p) +func (dht *IpfsDHT) handleFindPeer(ctx context.Context, from peer.ID, pmes *pb.Message) (_ *pb.Message, _err error) { resp := pb.NewMessage(pmes.GetType(), nil, pmes.GetClusterLevel()) var closest []peer.ID @@ -271,28 +258,30 @@ func (dht *IpfsDHT) handleFindPeer(ctx context.Context, p peer.ID, pmes *pb.Mess if targetPid == dht.self { closest = []peer.ID{dht.self} } else { - closest = dht.betterPeersToQuery(pmes, p, dht.bucketSize) + closest = dht.betterPeersToQuery(pmes, from, dht.bucketSize) // Never tell a peer about itself. - if targetPid != p { - // If we're connected to the target peer, report their - // peer info. This makes FindPeer work even if the - // target peer isn't in our routing table. + if targetPid != from { + // Add the target peer to the set of closest peers if + // not already present in our routing table. // - // Alternatively, we could just check our peerstore. - // However, we don't want to return out of date - // information. We can change this in the future when we - // add a progressive, asynchronous `SearchPeer` function - // and improve peer routing in the host. - switch dht.host.Network().Connectedness(targetPid) { - case network.Connected, network.CanConnect: + // Later, when we lookup known addresses for all peers + // in this set, we'll prune this peer if we don't + // _actually_ know where it is. + found := false + for _, p := range closest { + if targetPid == p { + found = true + break + } + } + if !found { closest = append(closest, targetPid) } } } if closest == nil { - logger.Infof("%s handleFindPeer %s: could not find anything.", dht.self, p) return resp, nil } @@ -311,26 +300,18 @@ func (dht *IpfsDHT) handleFindPeer(ctx context.Context, p peer.ID, pmes *pb.Mess } func (dht *IpfsDHT) handleGetProviders(ctx context.Context, p peer.ID, pmes *pb.Message) (_ *pb.Message, _err error) { - ctx = logger.Start(ctx, "handleGetProviders") - defer func() { logger.FinishWithErr(ctx, _err) }() - logger.SetTag(ctx, "peer", p) - resp := pb.NewMessage(pmes.GetType(), pmes.GetKey(), pmes.GetClusterLevel()) key := pmes.GetKey() if len(key) > 80 { return nil, fmt.Errorf("handleGetProviders key size too large") } - logger.SetTag(ctx, "key", key) - - // debug logging niceness. - reqDesc := fmt.Sprintf("%s handleGetProviders(%s, %s): ", dht.self, p, key) - logger.Debugf("%s begin", reqDesc) - defer logger.Debugf("%s end", reqDesc) // check if we have this value, to add ourselves as provider. has, err := dht.datastore.Has(convertToDsKey(key)) if err != nil && err != ds.ErrNotFound { - logger.Debugf("unexpected datastore error: %v\n", err) + // FIXME: This doesn't work reliably. If we want this check, we + // need a _blockstore_. + logger.Errorw("error checking datastore for block", "key", key, "error", err) has = false } @@ -338,14 +319,12 @@ func (dht *IpfsDHT) handleGetProviders(ctx context.Context, p peer.ID, pmes *pb. providers := dht.ProviderManager.GetProviders(ctx, key) if has { providers = append(providers, dht.self) - logger.Debugf("%s have the value. added self as provider", reqDesc) } if len(providers) > 0 { // TODO: pstore.PeerInfos should move to core (=> peerstore.AddrInfos). infos := pstore.PeerInfos(dht.peerstore, providers) resp.ProviderPeers = pb.PeerInfosToPBPeers(dht.host.Network(), infos) - logger.Debugf("%s have %d providers: %s", reqDesc, len(providers), infos) } // Also send closer peers. @@ -354,24 +333,18 @@ func (dht *IpfsDHT) handleGetProviders(ctx context.Context, p peer.ID, pmes *pb. // TODO: pstore.PeerInfos should move to core (=> peerstore.AddrInfos). infos := pstore.PeerInfos(dht.peerstore, closer) resp.CloserPeers = pb.PeerInfosToPBPeers(dht.host.Network(), infos) - logger.Debugf("%s have %d closer peers: %s", reqDesc, len(closer), infos) } return resp, nil } func (dht *IpfsDHT) handleAddProvider(ctx context.Context, p peer.ID, pmes *pb.Message) (_ *pb.Message, _err error) { - ctx = logger.Start(ctx, "handleAddProvider") - defer func() { logger.FinishWithErr(ctx, _err) }() - logger.SetTag(ctx, "peer", p) - key := pmes.GetKey() if len(key) > 80 { return nil, fmt.Errorf("handleAddProviders key size too large") } - logger.SetTag(ctx, "key", key) - logger.Debugf("%s adding %s as a provider for '%s'\n", dht.self, p, key) + logger.Debugf("adding provider", "from", p, "key", key) // add provider should use the address given in the message pinfos := pb.PBPeersToPeerInfos(pmes.GetProviderPeers()) @@ -379,16 +352,15 @@ func (dht *IpfsDHT) handleAddProvider(ctx context.Context, p peer.ID, pmes *pb.M if pi.ID != p { // we should ignore this provider record! not from originator. // (we should sign them and check signature later...) - logger.Debugf("handleAddProvider received provider %s from %s. Ignore.", pi.ID, p) + logger.Debugw("received provider from wrong peer", "from", p, "peer", pi.ID) continue } if len(pi.Addrs) < 1 { - logger.Debugf("%s got no valid addresses for provider %s. Ignore.", dht.self, p) + logger.Debugw("no valid addresses for provider", "from", p) continue } - logger.Debugf("received provider %s for %s (addrs: %s)", p, key, pi.Addrs) if pi.ID != dht.self { // don't add own addrs. // add the received addresses to our peerstore. dht.peerstore.AddAddrs(pi.ID, pi.Addrs, peerstore.ProviderAddrTTL) diff --git a/handlers_test.go b/handlers_test.go index dafa1ee05..6e098f815 100644 --- a/handlers_test.go +++ b/handlers_test.go @@ -89,7 +89,7 @@ func BenchmarkHandleFindPeer(b *testing.B) { panic(err) } - d.routingTable.Update(id) + d.peerFound(ctx, id, true) peers = append(peers, id) a, err := ma.NewMultiaddr(fmt.Sprintf("/ip4/127.0.0.1/tcp/%d", 2000+i)) @@ -110,7 +110,10 @@ func BenchmarkHandleFindPeer(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - d.handleFindPeer(ctx, peers[0], reqs[i]) + _, err = d.handleFindPeer(ctx, peers[0], reqs[i]) + if err != nil { + b.Error(err) + } } } diff --git a/lookup.go b/lookup.go index 48c23e205..a602a9a8f 100644 --- a/lookup.go +++ b/lookup.go @@ -7,14 +7,12 @@ import ( "time" "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/routing" "github.com/ipfs/go-cid" - logging "github.com/ipfs/go-log" pb "github.com/libp2p/go-libp2p-kad-dht/pb" kb "github.com/libp2p/go-libp2p-kbucket" - notif "github.com/libp2p/go-libp2p-routing/notifications" "github.com/multiformats/go-base32" - "github.com/multiformats/go-multihash" ) func tryFormatLoggableKey(k string) (string, error) { @@ -46,89 +44,77 @@ func tryFormatLoggableKey(k string) (string, error) { return fmt.Sprintf("/%s/%s", proto, encStr), nil } -func loggableKey(k string) logging.LoggableMap { - newKey, err := tryFormatLoggableKey(k) - if err != nil { - logger.Debug(err) - } else { - k = newKey - } +type loggableKeyBytes []byte - return logging.LoggableMap{ - "key": k, +func (lk loggableKeyString) String() string { + k := string(lk) + newKey, err := tryFormatLoggableKey(k) + if err == nil { + return newKey } + return k } -func multihashLoggableKey(mh multihash.Multihash) logging.LoggableMap { - return logging.LoggableMap{ - "multihash": base32.RawStdEncoding.EncodeToString(mh), +type loggableKeyString string + +func (lk loggableKeyBytes) String() string { + k := string(lk) + newKey, err := tryFormatLoggableKey(k) + if err == nil { + return newKey } + return k } // Kademlia 'node lookup' operation. Returns a channel of the K closest peers // to the given key +// +// If the context is canceled, this function will return the context error along +// with the closest K peers it has found so far. func (dht *IpfsDHT) GetClosestPeers(ctx context.Context, key string) (<-chan peer.ID, error) { - e := logger.EventBegin(ctx, "getClosestPeers", loggableKey(key)) - tablepeers := dht.routingTable.NearestPeers(kb.ConvertKey(key), AlphaValue) - if len(tablepeers) == 0 { - return nil, kb.ErrLookupFailure - } + //TODO: I can break the interface! return []peer.ID + lookupRes, err := dht.runLookupWithFollowup(ctx, key, + func(ctx context.Context, p peer.ID) ([]*peer.AddrInfo, error) { + // For DHT query command + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.SendingQuery, + ID: p, + }) + + pmes, err := dht.findPeerSingle(ctx, p, peer.ID(key)) + if err != nil { + logger.Debugf("error getting closer peers: %s", err) + return nil, err + } + peers := pb.PBPeersToPeerInfos(pmes.GetCloserPeers()) - out := make(chan peer.ID, dht.bucketSize) + // For DHT query command + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.PeerResponse, + ID: p, + Responses: peers, + }) - // since the query doesnt actually pass our context down - // we have to hack this here. whyrusleeping isnt a huge fan of goprocess - parent := ctx - query := dht.newQuery(key, func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) { - // For DHT query command - notif.PublishQueryEvent(parent, ¬if.QueryEvent{ - Type: notif.SendingQuery, - ID: p, - }) - - pmes, err := dht.findPeerSingle(ctx, p, peer.ID(key)) - if err != nil { - logger.Debugf("error getting closer peers: %s", err) - return nil, err - } - peers := pb.PBPeersToPeerInfos(pmes.GetCloserPeers()) - - // For DHT query command - notif.PublishQueryEvent(parent, ¬if.QueryEvent{ - Type: notif.PeerResponse, - ID: p, - Responses: peers, - }) - - return &dhtQueryResult{closerPeers: peers}, nil - }) - - go func() { - defer close(out) - defer e.Done() - timedCtx, cancel := context.WithTimeout(ctx, time.Minute) - defer cancel() - // run it! - res, err := query.Run(timedCtx, tablepeers) - if err != nil { - logger.Debugf("closestPeers query run error: %s", err) - } + return peers, err + }, + func() bool { return false }, + ) - if res != nil && res.queriedSet != nil { - // refresh the cpl for this key as the query was successful - dht.routingTable.ResetCplRefreshedAtForID(kb.ConvertKey(key), time.Now()) + if err != nil { + return nil, err + } - sorted := kb.SortClosestPeers(res.queriedSet.Peers(), kb.ConvertKey(key)) - l := len(sorted) - if l > dht.bucketSize { - sorted = sorted[:dht.bucketSize] - } + out := make(chan peer.ID, dht.bucketSize) + defer close(out) - for _, p := range sorted { - out <- p - } - } - }() + for _, p := range lookupRes.peers { + out <- p + } + + if ctx.Err() == nil && lookupRes.completed { + // refresh the cpl for this key as the query was successful + dht.routingTable.ResetCplRefreshedAtForID(kb.ConvertKey(key), time.Now()) + } - return out, nil + return out, ctx.Err() } diff --git a/nofile_test_posix.go b/nofile_test_posix.go index efd1b383c..0bcee894f 100644 --- a/nofile_test_posix.go +++ b/nofile_test_posix.go @@ -6,7 +6,7 @@ import "syscall" func curFileLimit() uint64 { var n syscall.Rlimit - syscall.Getrlimit(syscall.RLIMIT_NOFILE, &n) + _ = syscall.Getrlimit(syscall.RLIMIT_NOFILE, &n) // used for testing, ignore error. // cast because some platforms use int64 (e.g., freebsd) return uint64(n.Cur) } diff --git a/notif.go b/notif.go deleted file mode 100644 index 04000e31e..000000000 --- a/notif.go +++ /dev/null @@ -1,144 +0,0 @@ -package dht - -import ( - "context" - - "github.com/libp2p/go-libp2p-core/helpers" - "github.com/libp2p/go-libp2p-core/network" - - ma "github.com/multiformats/go-multiaddr" - mstream "github.com/multiformats/go-multistream" -) - -// netNotifiee defines methods to be used with the IpfsDHT -type netNotifiee IpfsDHT - -func (nn *netNotifiee) DHT() *IpfsDHT { - return (*IpfsDHT)(nn) -} - -func (nn *netNotifiee) Connected(n network.Network, v network.Conn) { - dht := nn.DHT() - select { - case <-dht.Process().Closing(): - return - default: - } - - p := v.RemotePeer() - protos, err := dht.peerstore.SupportsProtocols(p, dht.protocolStrs()...) - if err == nil && len(protos) != 0 { - // We lock here for consistency with the lock in testConnection. - // This probably isn't necessary because (dis)connect - // notifications are serialized but it's nice to be consistent. - dht.plk.Lock() - defer dht.plk.Unlock() - if dht.host.Network().Connectedness(p) == network.Connected { - refresh := dht.routingTable.Size() <= minRTRefreshThreshold - dht.Update(dht.Context(), p) - if refresh && dht.autoRefresh { - select { - case dht.triggerRtRefresh <- nil: - default: - } - } - } - return - } - - // Note: Unfortunately, the peerstore may not yet know that this peer is - // a DHT server. So, if it didn't return a positive response above, test - // manually. - go nn.testConnection(v) -} - -func (nn *netNotifiee) testConnection(v network.Conn) { - dht := nn.DHT() - p := v.RemotePeer() - - // Forcibly use *this* connection. Otherwise, if we have two connections, we could: - // 1. Test it twice. - // 2. Have it closed from under us leaving the second (open) connection untested. - s, err := v.NewStream() - if err != nil { - // Connection error - return - } - defer helpers.FullClose(s) - - selected, err := mstream.SelectOneOf(dht.protocolStrs(), s) - if err != nil { - // Doesn't support the protocol - return - } - // Remember this choice (makes subsequent negotiations faster) - dht.peerstore.AddProtocols(p, selected) - - // We lock here as we race with disconnect. If we didn't lock, we could - // finish processing a connect after handling the associated disconnect - // event and add the peer to the routing table after removing it. - dht.plk.Lock() - defer dht.plk.Unlock() - if dht.host.Network().Connectedness(p) == network.Connected { - refresh := dht.routingTable.Size() <= minRTRefreshThreshold - dht.Update(dht.Context(), p) - if refresh && dht.autoRefresh { - select { - case dht.triggerRtRefresh <- nil: - default: - } - } - } -} - -func (nn *netNotifiee) Disconnected(n network.Network, v network.Conn) { - dht := nn.DHT() - select { - case <-dht.Process().Closing(): - return - default: - } - - p := v.RemotePeer() - - // Lock and check to see if we're still connected. We lock to make sure - // we don't concurrently process a connect event. - dht.plk.Lock() - defer dht.plk.Unlock() - if dht.host.Network().Connectedness(p) == network.Connected { - // We're still connected. - return - } - - dht.routingTable.Remove(p) - if dht.routingTable.Size() < minRTRefreshThreshold { - // TODO: Actively bootstrap. For now, just try to add the currently connected peers. - for _, p := range dht.host.Network().Peers() { - // Don't bother probing, we do that on connect. - protos, err := dht.peerstore.SupportsProtocols(p, dht.protocolStrs()...) - if err == nil && len(protos) != 0 { - dht.Update(dht.Context(), p) - } - } - } - - dht.smlk.Lock() - defer dht.smlk.Unlock() - ms, ok := dht.strmap[p] - if !ok { - return - } - delete(dht.strmap, p) - - // Do this asynchronously as ms.lk can block for a while. - go func() { - ms.lk.Lock(context.Background()) - defer ms.lk.Unlock() - ms.invalidate() - }() -} - -func (nn *netNotifiee) OpenedStream(n network.Network, v network.Stream) {} -func (nn *netNotifiee) ClosedStream(n network.Network, v network.Stream) {} -func (nn *netNotifiee) Listen(n network.Network, a ma.Multiaddr) {} -func (nn *netNotifiee) ListenClose(n network.Network, a ma.Multiaddr) {} diff --git a/notify_test.go b/notify_test.go deleted file mode 100644 index 3a15a8e82..000000000 --- a/notify_test.go +++ /dev/null @@ -1,81 +0,0 @@ -package dht - -import ( - "context" - "fmt" - "testing" - "time" - - tu "github.com/libp2p/go-libp2p-testing/etc" -) - -func TestNotifieeMultipleConn(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - d1 := setupDHT(ctx, t, false) - d2 := setupDHT(ctx, t, false) - - nn1 := (*netNotifiee)(d1) - nn2 := (*netNotifiee)(d2) - - connect(t, ctx, d1, d2) - c12 := d1.host.Network().ConnsToPeer(d2.self)[0] - c21 := d2.host.Network().ConnsToPeer(d1.self)[0] - - // Pretend to reestablish/re-kill connection - nn1.Connected(d1.host.Network(), c12) - nn2.Connected(d2.host.Network(), c21) - - if !checkRoutingTable(d1, d2) { - t.Fatal("no routes") - } - nn1.Disconnected(d1.host.Network(), c12) - nn2.Disconnected(d2.host.Network(), c21) - - if !checkRoutingTable(d1, d2) { - t.Fatal("no routes") - } - - for _, conn := range d1.host.Network().ConnsToPeer(d2.self) { - conn.Close() - } - for _, conn := range d2.host.Network().ConnsToPeer(d1.self) { - conn.Close() - } - - tu.WaitFor(ctx, func() error { - if checkRoutingTable(d1, d2) { - return fmt.Errorf("should not have routes") - } - return nil - }) -} - -func TestNotifieeFuzz(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*2) - defer cancel() - - d1 := setupDHT(ctx, t, false) - d2 := setupDHT(ctx, t, false) - - for i := 0; i < 10; i++ { - connectNoSync(t, ctx, d1, d2) - for _, conn := range d1.host.Network().ConnsToPeer(d2.self) { - conn.Close() - } - } - tu.WaitFor(ctx, func() error { - if checkRoutingTable(d1, d2) { - return fmt.Errorf("should not have routes") - } - return nil - }) - connect(t, ctx, d1, d2) -} - -func checkRoutingTable(a, b *IpfsDHT) bool { - // loop until connection notification has been received. - // under high load, this may not happen as immediately as we would like. - return a.routingTable.Find(b.self) != "" && b.routingTable.Find(a.self) != "" -} diff --git a/opts/options.go b/opts/options.go index 7d2eb619a..a8619ade8 100644 --- a/opts/options.go +++ b/opts/options.go @@ -1,221 +1,68 @@ +// Deprecated: Options are now defined in the root package. + package dhtopts import ( - "fmt" "time" ds "github.com/ipfs/go-datastore" - dssync "github.com/ipfs/go-datastore/sync" - "github.com/libp2p/go-libp2p-core/protocol" - "github.com/libp2p/go-libp2p-record" -) - -// Deprecated: The old format did not support more than one message per stream, and is not supported -// or relevant with stream pooling. ProtocolDHT should be used instead. -const ProtocolDHTOld protocol.ID = "/ipfs/dht" - -var ( - ProtocolDHT protocol.ID = "/ipfs/kad/1.0.0" - DefaultProtocols = []protocol.ID{ProtocolDHT} + dht "github.com/libp2p/go-libp2p-kad-dht" + record "github.com/libp2p/go-libp2p-record" ) -// Options is a structure containing all the options that can be used when constructing a DHT. -type Options struct { - Datastore ds.Batching - Validator record.Validator - Client bool - Protocols []protocol.ID - BucketSize int - MaxRecordAge time.Duration - EnableProviders bool - EnableValues bool - - RoutingTable struct { - RefreshQueryTimeout time.Duration - RefreshPeriod time.Duration - AutoRefresh bool - LatencyTolerance time.Duration - } -} - -// Apply applies the given options to this Option -func (o *Options) Apply(opts ...Option) error { - for i, opt := range opts { - if err := opt(o); err != nil { - return fmt.Errorf("dht option %d failed: %s", i, err) - } - } - return nil -} - -// Option DHT option type. -type Option func(*Options) error - -// Defaults are the default DHT options. This option will be automatically -// prepended to any options you pass to the DHT constructor. -var Defaults = func(o *Options) error { - o.Validator = record.NamespacedValidator{ - "pk": record.PublicKeyValidator{}, - } - o.Datastore = dssync.MutexWrap(ds.NewMapDatastore()) - o.Protocols = DefaultProtocols - o.EnableProviders = true - o.EnableValues = true - - o.RoutingTable.LatencyTolerance = time.Minute - o.RoutingTable.RefreshQueryTimeout = 10 * time.Second - o.RoutingTable.RefreshPeriod = 1 * time.Hour - o.RoutingTable.AutoRefresh = true - o.MaxRecordAge = time.Hour * 36 - - return nil -} +type Option = dht.Option -// RoutingTableLatencyTolerance sets the maximum acceptable latency for peers -// in the routing table's cluster. -func RoutingTableLatencyTolerance(latency time.Duration) Option { - return func(o *Options) error { - o.RoutingTable.LatencyTolerance = latency - return nil - } +// Deprecated: use dht.RoutingTableLatencyTolerance +func RoutingTableLatencyTolerance(latency time.Duration) dht.Option { + return dht.RoutingTableLatencyTolerance(latency) } -// RoutingTableRefreshQueryTimeout sets the timeout for routing table refresh -// queries. -func RoutingTableRefreshQueryTimeout(timeout time.Duration) Option { - return func(o *Options) error { - o.RoutingTable.RefreshQueryTimeout = timeout - return nil - } +// Deprecated: use dht.RoutingTableRefreshQueryTimeout +func RoutingTableRefreshQueryTimeout(timeout time.Duration) dht.Option { + return dht.RoutingTableRefreshQueryTimeout(timeout) } -// RoutingTableRefreshPeriod sets the period for refreshing buckets in the -// routing table. The DHT will refresh buckets every period by: -// -// 1. First searching for nearby peers to figure out how many buckets we should try to fill. -// 1. Then searching for a random key in each bucket that hasn't been queried in -// the last refresh period. -func RoutingTableRefreshPeriod(period time.Duration) Option { - return func(o *Options) error { - o.RoutingTable.RefreshPeriod = period - return nil - } +// Deprecated: use dht.RoutingTableRefreshPeriod +func RoutingTableRefreshPeriod(period time.Duration) dht.Option { + return dht.RoutingTableRefreshPeriod(period) } -// Datastore configures the DHT to use the specified datastore. -// -// Defaults to an in-memory (temporary) map. -func Datastore(ds ds.Batching) Option { - return func(o *Options) error { - o.Datastore = ds - return nil - } -} +// Deprecated: use dht.Datastore +func Datastore(ds ds.Batching) dht.Option { return dht.Datastore(ds) } // Client configures whether or not the DHT operates in client-only mode. // -// Defaults to false. -func Client(only bool) Option { - return func(o *Options) error { - o.Client = only - return nil +// Defaults to false (which is ModeAuto). +// Deprecated: use dht.Mode(ModeClient) +func Client(only bool) dht.Option { + if only { + return dht.Mode(dht.ModeClient) } + return dht.Mode(dht.ModeAuto) } -// Validator configures the DHT to use the specified validator. -// -// Defaults to a namespaced validator that can only validate public keys. -func Validator(v record.Validator) Option { - return func(o *Options) error { - o.Validator = v - return nil - } -} +// Deprecated: use dht.Mode +func Mode(m dht.ModeOpt) dht.Option { return dht.Mode(m) } -// NamespacedValidator adds a validator namespaced under `ns`. This option fails -// if the DHT is not using a `record.NamespacedValidator` as it's validator (it -// uses one by default but this can be overridden with the `Validator` option). -// -// Example: Given a validator registered as `NamespacedValidator("ipns", -// myValidator)`, all records with keys starting with `/ipns/` will be validated -// with `myValidator`. -func NamespacedValidator(ns string, v record.Validator) Option { - return func(o *Options) error { - nsval, ok := o.Validator.(record.NamespacedValidator) - if !ok { - return fmt.Errorf("can only add namespaced validators to a NamespacedValidator") - } - nsval[ns] = v - return nil - } -} +// Deprecated: use dht.Validator +func Validator(v record.Validator) dht.Option { return dht.Validator(v) } -// Protocols sets the protocols for the DHT -// -// Defaults to dht.DefaultProtocols -func Protocols(protocols ...protocol.ID) Option { - return func(o *Options) error { - o.Protocols = protocols - return nil - } +// Deprecated: use dht.NamespacedValidator +func NamespacedValidator(ns string, v record.Validator) dht.Option { + return dht.NamespacedValidator(ns, v) } -// BucketSize configures the bucket size of the routing table. -// -// The default value is 20. -func BucketSize(bucketSize int) Option { - return func(o *Options) error { - o.BucketSize = bucketSize - return nil - } -} +// Deprecated: use dht.BucketSize +func BucketSize(bucketSize int) dht.Option { return dht.BucketSize(bucketSize) } -// MaxRecordAge specifies the maximum time that any node will hold onto a record ("PutValue record") -// from the time its received. This does not apply to any other forms of validity that -// the record may contain. -// For example, a record may contain an ipns entry with an EOL saying its valid -// until the year 2020 (a great time in the future). For that record to stick around -// it must be rebroadcasted more frequently than once every 'MaxRecordAge' -func MaxRecordAge(maxAge time.Duration) Option { - return func(o *Options) error { - o.MaxRecordAge = maxAge - return nil - } -} +// Deprecated: use dht.MaxRecordAge +func MaxRecordAge(maxAge time.Duration) dht.Option { return dht.MaxRecordAge(maxAge) } -// DisableAutoRefresh completely disables 'auto-refresh' on the DHT routing -// table. This means that we will neither refresh the routing table periodically -// nor when the routing table size goes below the minimum threshold. -func DisableAutoRefresh() Option { - return func(o *Options) error { - o.RoutingTable.AutoRefresh = false - return nil - } -} +// Deprecated: use dht.DisableAutoRefresh +func DisableAutoRefresh() dht.Option { return dht.DisableAutoRefresh() } -// DisableProviders disables storing and retrieving provider records. -// -// Defaults to enabled. -// -// WARNING: do not change this unless you're using a forked DHT (i.e., a private -// network and/or distinct DHT protocols with the `Protocols` option). -func DisableProviders() Option { - return func(o *Options) error { - o.EnableProviders = false - return nil - } -} +// Deprecated: use dht.DisableProviders +func DisableProviders() dht.Option { return dht.DisableProviders() } -// DisableProviders disables storing and retrieving value records (including -// public keys). -// -// Defaults to enabled. -// -// WARNING: do not change this unless you're using a forked DHT (i.e., a private -// network and/or distinct DHT protocols with the `Protocols` option). -func DisableValues() Option { - return func(o *Options) error { - o.EnableValues = false - return nil - } -} +// Deprecated: use dht.DisableValues +func DisableValues() dht.Option { return dht.DisableValues() } diff --git a/pb/message.go b/pb/message.go index da49f268c..a7e9d14f2 100644 --- a/pb/message.go +++ b/pb/message.go @@ -110,7 +110,7 @@ func (m *Message_Peer) Addresses() []ma.Multiaddr { for _, addr := range m.Addrs { maddr, err := ma.NewMultiaddrBytes(addr) if err != nil { - log.Warningf("error decoding Multiaddr for peer: %s", m.GetId()) + log.Debugw("error decoding multiaddr for peer", "peer", m.GetId(), "error", err) continue } diff --git a/protocol.go b/protocol.go new file mode 100644 index 000000000..855592e25 --- /dev/null +++ b/protocol.go @@ -0,0 +1,12 @@ +package dht + +import ( + "github.com/libp2p/go-libp2p-core/protocol" +) + +var ( + // ProtocolDHT is the default DHT protocol. + ProtocolDHT protocol.ID = "/ipfs/kad/1.0.0" + // DefualtProtocols spoken by the DHT. + DefaultProtocols = []protocol.ID{ProtocolDHT} +) diff --git a/qpeerset/qpeerset.go b/qpeerset/qpeerset.go new file mode 100644 index 000000000..971b5489d --- /dev/null +++ b/qpeerset/qpeerset.go @@ -0,0 +1,173 @@ +package qpeerset + +import ( + "math/big" + "sort" + + "github.com/libp2p/go-libp2p-core/peer" + ks "github.com/whyrusleeping/go-keyspace" +) + +// PeerState describes the state of a peer ID during the lifecycle of an individual lookup. +type PeerState int + +const ( + // PeerHeard is applied to peers which have not been queried yet. + PeerHeard PeerState = iota + // PeerWaiting is applied to peers that are currently being queried. + PeerWaiting + // PeerQueried is applied to peers who have been queried and a response was retrieved successfully. + PeerQueried + // PeerUnreachable is applied to peers who have been queried and a response was not retrieved successfully. + PeerUnreachable +) + +// QueryPeerset maintains the state of a Kademlia asynchronous lookup. +// The lookup state is a set of peers, each labeled with a peer state. +type QueryPeerset struct { + // the key being searched for + key ks.Key + + // all known peers + all []queryPeerState + + // sorted is true if all is currently in sorted order + sorted bool +} + +type queryPeerState struct { + id peer.ID + distance *big.Int + state PeerState + referredBy peer.ID +} + +type sortedQueryPeerset QueryPeerset + +func (sqp *sortedQueryPeerset) Len() int { + return len(sqp.all) +} + +func (sqp *sortedQueryPeerset) Swap(i, j int) { + sqp.all[i], sqp.all[j] = sqp.all[j], sqp.all[i] +} + +func (sqp *sortedQueryPeerset) Less(i, j int) bool { + di, dj := sqp.all[i].distance, sqp.all[j].distance + return di.Cmp(dj) == -1 +} + +// NewQueryPeerset creates a new empty set of peers. +// key is the target key of the lookup that this peer set is for. +func NewQueryPeerset(key string) *QueryPeerset { + return &QueryPeerset{ + key: ks.XORKeySpace.Key([]byte(key)), + all: []queryPeerState{}, + sorted: false, + } +} + +func (qp *QueryPeerset) find(p peer.ID) int { + for i := range qp.all { + if qp.all[i].id == p { + return i + } + } + return -1 +} + +func (qp *QueryPeerset) distanceToKey(p peer.ID) *big.Int { + return ks.XORKeySpace.Key([]byte(p)).Distance(qp.key) +} + +// TryAdd adds the peer p to the peer set. +// If the peer is already present, no action is taken. +// Otherwise, the peer is added with state set to PeerHeard. +// TryAdd returns true iff the peer was not already present. +func (qp *QueryPeerset) TryAdd(p, referredBy peer.ID) bool { + if qp.find(p) >= 0 { + return false + } else { + qp.all = append(qp.all, + queryPeerState{id: p, distance: qp.distanceToKey(p), state: PeerHeard, referredBy: referredBy}) + qp.sorted = false + return true + } +} + +func (qp *QueryPeerset) sort() { + if qp.sorted { + return + } + sort.Sort((*sortedQueryPeerset)(qp)) + qp.sorted = true +} + +// SetState sets the state of peer p to s. +// If p is not in the peerset, SetState panics. +func (qp *QueryPeerset) SetState(p peer.ID, s PeerState) { + qp.all[qp.find(p)].state = s +} + +// GetState returns the state of peer p. +// If p is not in the peerset, GetState panics. +func (qp *QueryPeerset) GetState(p peer.ID) PeerState { + return qp.all[qp.find(p)].state +} + +// GetReferrer returns the peer that referred us to the peer p. +// If p is not in the peerset, GetReferrer panics. +func (qp *QueryPeerset) GetReferrer(p peer.ID) peer.ID { + return qp.all[qp.find(p)].referredBy +} + +// NumWaiting returns the number of peers in state PeerWaiting. +func (qp *QueryPeerset) NumWaiting() int { + return len(qp.GetWaitingPeers()) +} + +// GetWaitingPeers returns a slice of all peers in state PeerWaiting, in an undefined order. +func (qp *QueryPeerset) GetWaitingPeers() (result []peer.ID) { + for _, p := range qp.all { + if p.state == PeerWaiting { + result = append(result, p.id) + } + } + return +} + +// GetClosestNotUnreachable returns the closest to the key peers, which are not in state PeerUnreachable. +// It returns count peers or less, if fewer peers meet the condition. +func (qp *QueryPeerset) GetClosestNotUnreachable(count int) (result []peer.ID) { + qp.sort() + for _, p := range qp.all { + if p.state != PeerUnreachable { + result = append(result, p.id) + } + } + if len(result) >= count { + return result[:count] + } + return result +} + +// NumHeard returns the number of peers in state PeerHeard. +func (qp *QueryPeerset) NumHeard() int { + return len(qp.GetHeardPeers()) +} + +// GetHeardPeers returns a slice of all peers in state PeerHeard, in an undefined order. +func (qp *QueryPeerset) GetHeardPeers() (result []peer.ID) { + for _, p := range qp.all { + if p.state == PeerHeard { + result = append(result, p.id) + } + } + return +} + +// GetSortedHeard returns a slice of all peers in state PeerHeard, ordered by ascending distance to the target key. +func (qp *QueryPeerset) GetSortedHeard() (result []peer.ID) { + qp.sort() + return qp.GetHeardPeers() +} diff --git a/qpeerset/qpeerset_test.go b/qpeerset/qpeerset_test.go new file mode 100644 index 000000000..4d0f7db0a --- /dev/null +++ b/qpeerset/qpeerset_test.go @@ -0,0 +1,86 @@ +package qpeerset + +import ( + "testing" + + "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/test" + + kb "github.com/libp2p/go-libp2p-kbucket" + + "github.com/stretchr/testify/require" +) + +func TestQPeerSet(t *testing.T) { + key := "test" + qp := NewQueryPeerset(key) + + // -----------------Ordering between peers for the Test ----- + // KEY < peer3 < peer1 < peer4 < peer2 + // ---------------------------------------------------------- + peer2 := test.RandPeerIDFatal(t) + var peer4 peer.ID + for { + peer4 = test.RandPeerIDFatal(t) + if kb.Closer(peer4, peer2, key) { + break + } + } + + var peer1 peer.ID + for { + peer1 = test.RandPeerIDFatal(t) + if kb.Closer(peer1, peer4, key) { + break + } + } + + var peer3 peer.ID + for { + peer3 = test.RandPeerIDFatal(t) + if kb.Closer(peer3, peer1, key) { + break + } + } + + oracle := test.RandPeerIDFatal(t) + + // find fails + require.Equal(t, -1, qp.find(peer2)) + + // add peer2,assert state & then another add fails + require.True(t, qp.TryAdd(peer2, oracle)) + require.Equal(t, PeerHeard, qp.GetState(peer2)) + require.False(t, qp.TryAdd(peer2, oracle)) + require.Equal(t, 0, qp.NumWaiting()) + + // add peer4 + require.True(t, qp.TryAdd(peer4, oracle)) + cl := qp.GetClosestNotUnreachable(2) + require.Equal(t, []peer.ID{peer4, peer2}, cl) + cl = qp.GetClosestNotUnreachable(3) + require.Equal(t, []peer.ID{peer4, peer2}, cl) + cl = qp.GetClosestNotUnreachable(1) + require.Equal(t, []peer.ID{peer4}, cl) + + // mark as unreachable & try to get it + qp.SetState(peer4, PeerUnreachable) + cl = qp.GetClosestNotUnreachable(1) + require.Equal(t, []peer.ID{peer2}, cl) + + // add peer1 + require.True(t, qp.TryAdd(peer1, oracle)) + cl = qp.GetClosestNotUnreachable(1) + require.Equal(t, []peer.ID{peer1}, cl) + cl = qp.GetClosestNotUnreachable(2) + require.Equal(t, []peer.ID{peer1, peer2}, cl) + + // mark as waiting and assert + qp.SetState(peer2, PeerWaiting) + require.Equal(t, []peer.ID{peer2}, qp.GetWaitingPeers()) + + require.Equal(t, []peer.ID{peer1}, qp.GetHeardPeers()) + require.True(t, qp.TryAdd(peer3, oracle)) + require.Equal(t, []peer.ID{peer3, peer1}, qp.GetSortedHeard()) + require.Equal(t, 2, qp.NumHeard()) +} diff --git a/query.go b/query.go index 325101145..d65c5cab1 100644 --- a/query.go +++ b/query.go @@ -3,317 +3,499 @@ package dht import ( "context" "errors" + "fmt" + "math" "sync" + "time" + + "github.com/google/uuid" "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peer" - - logging "github.com/ipfs/go-log" - todoctr "github.com/ipfs/go-todocounter" - process "github.com/jbenet/goprocess" - ctxproc "github.com/jbenet/goprocess/context" - kb "github.com/libp2p/go-libp2p-kbucket" - pstore "github.com/libp2p/go-libp2p-core/peerstore" "github.com/libp2p/go-libp2p-core/routing" - queue "github.com/libp2p/go-libp2p-peerstore/queue" - notif "github.com/libp2p/go-libp2p-routing/notifications" + + "github.com/libp2p/go-libp2p-kad-dht/qpeerset" + kb "github.com/libp2p/go-libp2p-kbucket" ) // ErrNoPeersQueried is returned when we failed to connect to any peers. var ErrNoPeersQueried = errors.New("failed to query any peers") -var maxQueryConcurrency = AlphaValue +type queryFn func(context.Context, peer.ID) ([]*peer.AddrInfo, error) +type stopFn func() bool + +// query represents a single DHT query. +type query struct { + // unique identifier for the lookup instance + id uuid.UUID + + // target key for the lookup + key string + + // the query context. + ctx context.Context + + dht *IpfsDHT + + // seedPeers is the set of peers that seed the query + seedPeers []peer.ID + + // peerTimes contains the duration of each successful query to a peer + peerTimes map[peer.ID]time.Duration + + // queryPeers is the set of peers known by this query and their respective states. + queryPeers *qpeerset.QueryPeerset + + // terminated is set when the first worker thread encounters the termination condition. + // Its role is to make sure that once termination is determined, it is sticky. + terminated bool + + // waitGroup ensures lookup does not end until all query goroutines complete. + waitGroup sync.WaitGroup + + // the function that will be used to query a single peer. + queryFn queryFn -type dhtQuery struct { - dht *IpfsDHT - key string // the key we're querying for - qfunc queryFunc // the function to execute per peer - concurrency int // the concurrency parameter + // stopFn is used to determine if we should stop the WHOLE disjoint query. + stopFn stopFn } -type dhtQueryResult struct { - peer *peer.AddrInfo // FindPeer - closerPeers []*peer.AddrInfo // * - success bool +type lookupWithFollowupResult struct { + peers []peer.ID // the top K not unreachable peers at the end of the query + state []qpeerset.PeerState // the peer states at the end of the query - finalSet *peer.Set - queriedSet *peer.Set + // indicates that neither the lookup nor the followup has been prematurely terminated by an external condition such + // as context cancellation or the stop function being called. + completed bool } -// constructs query -func (dht *IpfsDHT) newQuery(k string, f queryFunc) *dhtQuery { - return &dhtQuery{ - key: k, - dht: dht, - qfunc: f, - concurrency: maxQueryConcurrency, +// runLookupWithFollowup executes the lookup on the target using the given query function and stopping when either the +// context is cancelled or the stop function returns true. Note: if the stop function is not sticky, i.e. it does not +// return true every time after the first time it returns true, it is not guaranteed to cause a stop to occur just +// because it momentarily returns true. +// +// After the lookup is complete the query function is run (unless stopped) against all of the top K peers from the +// lookup that have not already been successfully queried. +func (dht *IpfsDHT) runLookupWithFollowup(ctx context.Context, target string, queryFn queryFn, stopFn stopFn) (*lookupWithFollowupResult, error) { + // run the query + lookupRes, err := dht.runQuery(ctx, target, queryFn, stopFn) + if err != nil { + return nil, err } -} -// QueryFunc is a function that runs a particular query with a given peer. -// It returns either: -// - the value -// - a list of peers potentially better able to serve the query -// - an error -type queryFunc func(context.Context, peer.ID) (*dhtQueryResult, error) - -// Run runs the query at hand. pass in a list of peers to use first. -func (q *dhtQuery) Run(ctx context.Context, peers []peer.ID) (*dhtQueryResult, error) { - if len(peers) == 0 { - logger.Warning("Running query with no peers!") - return nil, kb.ErrLookupFailure + // query all of the top K peers we've either Heard about or have outstanding queries we're Waiting on. + // This ensures that all of the top K results have been queried which adds to resiliency against churn for query + // functions that carry state (e.g. FindProviders and GetValue) as well as establish connections that are needed + // by stateless query functions (e.g. GetClosestPeers and therefore Provide and PutValue) + queryPeers := make([]peer.ID, 0, len(lookupRes.peers)) + for i, p := range lookupRes.peers { + if state := lookupRes.state[i]; state == qpeerset.PeerHeard || state == qpeerset.PeerWaiting { + queryPeers = append(queryPeers, p) + } } - select { - case <-ctx.Done(): - return nil, ctx.Err() - default: + if len(queryPeers) == 0 { + return lookupRes, nil } - ctx, cancel := context.WithCancel(ctx) - defer cancel() + // return if the lookup has been externally stopped + if ctx.Err() != nil || stopFn() { + lookupRes.completed = false + return lookupRes, nil + } + + doneCh := make(chan struct{}, len(queryPeers)) + followUpCtx, cancelFollowUp := context.WithCancel(ctx) + defer cancelFollowUp() + for _, p := range queryPeers { + qp := p + go func() { + _, _ = queryFn(followUpCtx, qp) + doneCh <- struct{}{} + }() + } - runner := newQueryRunner(q) - return runner.Run(ctx, peers) + // wait for all queries to complete before returning, aborting ongoing queries if we've been externally stopped +processFollowUp: + for i := 0; i < len(queryPeers); i++ { + select { + case <-doneCh: + if stopFn() { + cancelFollowUp() + if i < len(queryPeers)-1 { + lookupRes.completed = false + } + break processFollowUp + } + case <-ctx.Done(): + lookupRes.completed = false + break processFollowUp + } + } + + return lookupRes, nil } -type dhtQueryRunner struct { - query *dhtQuery // query to run - peersSeen *peer.Set // all peers queried. prevent querying same peer 2x - peersQueried *peer.Set // peers successfully connected to and queried - peersDialed *dialQueue // peers we have dialed to - peersToQuery *queue.ChanQueue // peers remaining to be queried - peersRemaining todoctr.Counter // peersToQuery + currently processing +func (dht *IpfsDHT) runQuery(ctx context.Context, target string, queryFn queryFn, stopFn stopFn) (*lookupWithFollowupResult, error) { + // pick the K closest peers to the key in our Routing table and shuffle them. + targetKadID := kb.ConvertKey(target) + seedPeers := dht.routingTable.NearestPeers(targetKadID, dht.bucketSize) + if len(seedPeers) == 0 { + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.QueryError, + Extra: kb.ErrLookupFailure.Error(), + }) + return nil, kb.ErrLookupFailure + } - result *dhtQueryResult // query result + q := &query{ + id: uuid.New(), + key: target, + ctx: ctx, + dht: dht, + queryPeers: qpeerset.NewQueryPeerset(target), + seedPeers: seedPeers, + peerTimes: make(map[peer.ID]time.Duration), + terminated: false, + queryFn: queryFn, + stopFn: stopFn, + } - rateLimit chan struct{} // processing semaphore - log logging.EventLogger + // run the query + q.run() - runCtx context.Context + if ctx.Err() == nil { + q.recordValuablePeers() + } - proc process.Process - sync.RWMutex + res := q.constructLookupResult(targetKadID) + return res, nil } -func newQueryRunner(q *dhtQuery) *dhtQueryRunner { - proc := process.WithParent(process.Background()) - ctx := ctxproc.OnClosingContext(proc) - peersToQuery := queue.NewChanQueue(ctx, queue.NewXORDistancePQ(string(q.key))) - r := &dhtQueryRunner{ - query: q, - peersRemaining: todoctr.NewSyncCounter(), - peersSeen: peer.NewSet(), - peersQueried: peer.NewSet(), - rateLimit: make(chan struct{}, q.concurrency), - peersToQuery: peersToQuery, - proc: proc, - } - dq, err := newDialQueue(&dqParams{ - ctx: ctx, - target: q.key, - in: peersToQuery, - dialFn: r.dialPeer, - config: dqDefaultConfig(), - }) - if err != nil { - panic(err) +func (q *query) recordPeerIsValuable(p peer.ID) { + q.dht.routingTable.UpdateLastSuccessfulOutboundQuery(p, time.Now()) +} + +func (q *query) recordValuablePeers() { + // Valuable peers algorithm: + // Label the seed peer that responded to a query in the shortest amount of time as the "most valuable peer" (MVP) + // Each seed peer that responded to a query within some range (i.e. 2x) of the MVP's time is a valuable peer + // Mark the MVP and all the other valuable peers as valuable + mvpDuration := time.Duration(math.MaxInt64) + for _, p := range q.seedPeers { + if queryTime, ok := q.peerTimes[p]; ok && queryTime < mvpDuration { + mvpDuration = queryTime + } + } + + for _, p := range q.seedPeers { + if queryTime, ok := q.peerTimes[p]; ok && queryTime < mvpDuration*2 { + q.recordPeerIsValuable(p) + } } - r.peersDialed = dq - return r } -func (r *dhtQueryRunner) Run(ctx context.Context, peers []peer.ID) (*dhtQueryResult, error) { - // Make sure to clean everything up when we return from this function. - ctx, cancel := context.WithCancel(ctx) - defer cancel() +// constructLookupResult takes the query information and uses it to construct the lookup result +func (q *query) constructLookupResult(target kb.ID) *lookupWithFollowupResult { + // determine if the query terminated early + completed := true - r.log = logger - r.runCtx = ctx + // Lookup and starvation are both valid ways for a lookup to complete. (Starvation does not imply failure.) + // Lookup termination (as defined in isLookupTermination) is not possible in small networks. + // Starvation is a successful query termination in small networks. + if !(q.isLookupTermination() || q.isStarvationTermination()) { + completed = false + } - // setup concurrency rate limiting - for i := 0; i < r.query.concurrency; i++ { - r.rateLimit <- struct{}{} + // extract the top K not unreachable peers + var peers []peer.ID + peerState := make(map[peer.ID]qpeerset.PeerState) + qp := q.queryPeers.GetClosestNotUnreachable(q.dht.bucketSize) + for _, p := range qp { + state := q.queryPeers.GetState(p) + peerState[p] = state + peers = append(peers, p) } - // add all the peers we got first. - for _, p := range peers { - r.addPeerToQuery(p) + // get the top K overall peers + sortedPeers := kb.SortClosestPeers(peers, target) + if len(sortedPeers) > q.dht.bucketSize { + sortedPeers = sortedPeers[:q.dht.bucketSize] + } + + // return the top K not unreachable peers as well as their states at the end of the query + res := &lookupWithFollowupResult{ + peers: sortedPeers, + state: make([]qpeerset.PeerState, len(sortedPeers)), + completed: completed, } - // start the dial queue only after we've added the initial set of peers. - // this is to avoid race conditions that could cause the peersRemaining todoctr - // to be done too early if the initial dial fails before others make it into the queue. - r.peersDialed.Start() + for i, p := range sortedPeers { + res.state[i] = peerState[p] + } - // go do this thing. - // do it as a child proc to make sure Run exits - // ONLY AFTER spawn workers has exited. - r.proc.Go(r.spawnWorkers) + return res +} - // wait until they're done. - var err error +type queryUpdate struct { + cause peer.ID + heard []peer.ID + queried []peer.ID + unreachable []peer.ID + queryDuration time.Duration +} - // now, if the context finishes, close the proc. - // we have to do it here because the logic before is setup, which - // should run without closing the proc. - ctxproc.CloseAfterContext(r.proc, ctx) +func (q *query) run() { + pathCtx, cancelPath := context.WithCancel(q.ctx) + defer cancelPath() - select { - case <-r.peersRemaining.Done(): - r.proc.Close() - if r.peersQueried.Size() == 0 { - err = ErrNoPeersQueried - } else { - err = routing.ErrNotFound + alpha := q.dht.alpha + + ch := make(chan *queryUpdate, alpha) + ch <- &queryUpdate{cause: q.dht.self, heard: q.seedPeers} + + // return only once all outstanding queries have completed. + defer q.waitGroup.Wait() + for { + var cause peer.ID + select { + case update := <-ch: + q.updateState(pathCtx, update) + cause = update.cause + case <-pathCtx.Done(): + q.terminate(pathCtx, cancelPath, LookupCancelled) + } + + // termination is triggered on end-of-lookup conditions or starvation of unused peers + if ready, reason := q.isReadyToTerminate(); ready { + q.terminate(pathCtx, cancelPath, reason) + } + + if q.terminated { + return + } + + // if all "threads" are busy, wait until someone finishes + if q.queryPeers.NumWaiting() >= alpha { + continue } - case <-r.proc.Closed(): - err = r.runCtx.Err() + // spawn new queries, up to the parallelism allowance + // calculate the maximum number of queries we could be spawning. + // Note: NumWaiting will be updated in spawnQuery + maxNumQueriesToSpawn := alpha - q.queryPeers.NumWaiting() + // try spawning the queries, if there are no available peers to query then we won't spawn them + for j := 0; j < maxNumQueriesToSpawn; j++ { + q.spawnQuery(pathCtx, cause, ch) + } } +} - r.RLock() - defer r.RUnlock() +// spawnQuery starts one query, if an available heard peer is found +func (q *query) spawnQuery(ctx context.Context, cause peer.ID, ch chan<- *queryUpdate) { + if peers := q.queryPeers.GetSortedHeard(); len(peers) == 0 { + return + } else { + PublishLookupEvent(ctx, + NewLookupEvent( + q.dht.self, + q.id, + q.key, + NewLookupUpdateEvent( + cause, + q.queryPeers.GetReferrer(peers[0]), + nil, // heard + []peer.ID{peers[0]}, // waiting + nil, // queried + nil, // unreachable + ), + nil, + nil, + ), + ) + q.queryPeers.SetState(peers[0], qpeerset.PeerWaiting) + q.waitGroup.Add(1) + go q.queryPeer(ctx, ch, peers[0]) + } +} - if r.result != nil && r.result.success { - return r.result, nil +func (q *query) isReadyToTerminate() (bool, LookupTerminationReason) { + // give the application logic a chance to terminate + if q.stopFn() { + return true, LookupStopped + } + if q.isStarvationTermination() { + return true, LookupStarvation + } + if q.isLookupTermination() { + return true, LookupCompleted } + return false, -1 +} - return &dhtQueryResult{ - finalSet: r.peersSeen, - queriedSet: r.peersQueried, - }, err +// From the set of all nodes that are not unreachable, +// if the closest beta nodes are all queried, the lookup can terminate. +func (q *query) isLookupTermination() bool { + peers := q.queryPeers.GetClosestNotUnreachable(q.dht.beta) + for _, p := range peers { + if q.queryPeers.GetState(p) != qpeerset.PeerQueried { + return false + } + } + return true } -func (r *dhtQueryRunner) addPeerToQuery(next peer.ID) { - // if new peer is ourselves... - if next == r.query.dht.self { - r.log.Debug("addPeerToQuery skip self") +func (q *query) isStarvationTermination() bool { + return q.queryPeers.NumHeard() == 0 && q.queryPeers.NumWaiting() == 0 +} + +func (q *query) terminate(ctx context.Context, cancel context.CancelFunc, reason LookupTerminationReason) { + if q.terminated { return + } else { + PublishLookupEvent(ctx, + NewLookupEvent( + q.dht.self, + q.id, + q.key, + nil, + nil, + NewLookupTerminateEvent(reason), + ), + ) + cancel() // abort outstanding queries + q.terminated = true } +} - if !r.peersSeen.TryAdd(next) { +// queryPeer queries a single peer and reports its findings on the channel. +// queryPeer does not access the query state in queryPeers! +func (q *query) queryPeer(ctx context.Context, ch chan<- *queryUpdate, p peer.ID) { + defer q.waitGroup.Done() + dialCtx, queryCtx := ctx, ctx + + startQuery := time.Now() + // dial the peer + if err := q.dht.dialPeer(dialCtx, p); err != nil { + // remove the peer if there was a dial failure..but not because of a context cancellation + if dialCtx.Err() == nil { + q.dht.peerStoppedDHT(q.dht.ctx, p) + } + ch <- &queryUpdate{cause: p, unreachable: []peer.ID{p}} return } - notif.PublishQueryEvent(r.runCtx, ¬if.QueryEvent{ - Type: notif.AddingPeer, - ID: next, - }) - - r.peersRemaining.Increment(1) - select { - case r.peersToQuery.EnqChan <- next: - case <-r.proc.Closing(): + // send query RPC to the remote peer + newPeers, err := q.queryFn(queryCtx, p) + if err != nil { + if queryCtx.Err() == nil { + q.dht.peerStoppedDHT(q.dht.ctx, p) + } + ch <- &queryUpdate{cause: p, unreachable: []peer.ID{p}} + return } -} -func (r *dhtQueryRunner) spawnWorkers(proc process.Process) { - for { - select { - case <-r.peersRemaining.Done(): - return + queryDuration := time.Since(startQuery) - case <-r.proc.Closing(): - return + // query successful, try to add to RT + q.dht.peerFound(q.dht.ctx, p, true) - case <-r.rateLimit: - ch := r.peersDialed.Consume() - select { - case p, ok := <-ch: - if !ok { - // this signals context cancellation. - return - } - // do it as a child func to make sure Run exits - // ONLY AFTER spawn workers has exited. - proc.Go(func(proc process.Process) { - r.queryPeer(proc, p) - }) - case <-r.proc.Closing(): - return - case <-r.peersRemaining.Done(): - return - } + // process new peers + saw := []peer.ID{} + for _, next := range newPeers { + if next.ID == q.dht.self { // don't add self. + logger.Debugf("PEERS CLOSER -- worker for: %v found self", p) + continue + } + + // add any other know addresses for the candidate peer. + curInfo := q.dht.peerstore.PeerInfo(next.ID) + next.Addrs = append(next.Addrs, curInfo.Addrs...) + + // add their addresses to the dialer's peerstore + if q.dht.queryPeerFilter(q.dht, *next) { + q.dht.peerstore.AddAddrs(next.ID, next.Addrs, pstore.TempAddrTTL) + saw = append(saw, next.ID) + } + } + + ch <- &queryUpdate{cause: p, heard: saw, queried: []peer.ID{p}, queryDuration: queryDuration} +} + +func (q *query) updateState(ctx context.Context, up *queryUpdate) { + if q.terminated { + panic("update should not be invoked after the logical lookup termination") + } + PublishLookupEvent(ctx, + NewLookupEvent( + q.dht.self, + q.id, + q.key, + nil, + NewLookupUpdateEvent( + up.cause, + up.cause, + up.heard, // heard + nil, // waiting + up.queried, // queried + up.unreachable, // unreachable + ), + nil, + ), + ) + for _, p := range up.heard { + if p == q.dht.self { // don't add self. + continue + } + q.queryPeers.TryAdd(p, up.cause) + } + for _, p := range up.queried { + if p == q.dht.self { // don't add self. + continue + } + if st := q.queryPeers.GetState(p); st == qpeerset.PeerWaiting { + q.queryPeers.SetState(p, qpeerset.PeerQueried) + q.peerTimes[p] = up.queryDuration + } else { + panic(fmt.Errorf("kademlia protocol error: tried to transition to the queried state from state %v", st)) + } + } + for _, p := range up.unreachable { + if p == q.dht.self { // don't add self. + continue + } + if st := q.queryPeers.GetState(p); st == qpeerset.PeerWaiting { + q.queryPeers.SetState(p, qpeerset.PeerUnreachable) + } else { + panic(fmt.Errorf("kademlia protocol error: tried to transition to the unreachable state from state %v", st)) } } } -func (r *dhtQueryRunner) dialPeer(ctx context.Context, p peer.ID) error { +func (dht *IpfsDHT) dialPeer(ctx context.Context, p peer.ID) error { // short-circuit if we're already connected. - if r.query.dht.host.Network().Connectedness(p) == network.Connected { + if dht.host.Network().Connectedness(p) == network.Connected { return nil } logger.Debug("not connected. dialing.") - notif.PublishQueryEvent(r.runCtx, ¬if.QueryEvent{ - Type: notif.DialingPeer, + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.DialingPeer, ID: p, }) pi := peer.AddrInfo{ID: p} - if err := r.query.dht.host.Connect(ctx, pi); err != nil { + if err := dht.host.Connect(ctx, pi); err != nil { logger.Debugf("error connecting: %s", err) - notif.PublishQueryEvent(r.runCtx, ¬if.QueryEvent{ - Type: notif.QueryError, + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.QueryError, Extra: err.Error(), ID: p, }) - // This peer is dropping out of the race. - r.peersRemaining.Decrement(1) return err } logger.Debugf("connected. dial success.") return nil } - -func (r *dhtQueryRunner) queryPeer(proc process.Process, p peer.ID) { - // ok let's do this! - - // create a context from our proc. - ctx := ctxproc.OnClosingContext(proc) - - // make sure we do this when we exit - defer func() { - // signal we're done processing peer p - r.peersRemaining.Decrement(1) - r.rateLimit <- struct{}{} - }() - - // finally, run the query against this peer - res, err := r.query.qfunc(ctx, p) - - r.peersQueried.Add(p) - - if err != nil { - logger.Debugf("ERROR worker for: %v %v", p, err) - } else if res.success { - logger.Debugf("SUCCESS worker for: %v %s", p, res) - r.Lock() - r.result = res - r.Unlock() - if res.peer != nil { - r.query.dht.peerstore.AddAddrs(res.peer.ID, res.peer.Addrs, pstore.TempAddrTTL) - } - go r.proc.Close() // signal to everyone that we're done. - // must be async, as we're one of the children, and Close blocks. - - } else if len(res.closerPeers) > 0 { - logger.Debugf("PEERS CLOSER -- worker for: %v (%d closer peers)", p, len(res.closerPeers)) - for _, next := range res.closerPeers { - if next.ID == r.query.dht.self { // don't add self. - logger.Debugf("PEERS CLOSER -- worker for: %v found self", p) - continue - } - - // add their addresses to the dialer's peerstore - r.query.dht.peerstore.AddAddrs(next.ID, next.Addrs, pstore.TempAddrTTL) - r.addPeerToQuery(next.ID) - logger.Debugf("PEERS CLOSER -- worker for: %v added %v (%v)", p, next.ID, next.Addrs) - } - } else { - logger.Debugf("QUERY worker for: %v - not found, and no closer peers.", p) - } -} diff --git a/query_test.go b/query_test.go new file mode 100644 index 000000000..e79af514f --- /dev/null +++ b/query_test.go @@ -0,0 +1,118 @@ +package dht + +import ( + "context" + "fmt" + "testing" + "time" + + tu "github.com/libp2p/go-libp2p-testing/etc" + + "github.com/stretchr/testify/require" +) + +// TODO Debug test failures due to timing issue on windows +// Tests are timing dependent as can be seen in the 2 seconds timed context that we use in "tu.WaitFor". +// While the tests work fine on OSX and complete in under a second, +// they repeatedly fail to complete in the stipulated time on Windows. +// However, increasing the timeout makes them pass on Windows. + +func TestRTEvictionOnFailedQuery(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + + d1 := setupDHT(ctx, t, false) + d2 := setupDHT(ctx, t, false) + + for i := 0; i < 10; i++ { + connect(t, ctx, d1, d2) + for _, conn := range d1.host.Network().ConnsToPeer(d2.self) { + conn.Close() + } + } + + // peers should be in the RT because of fixLowPeers + require.NoError(t, tu.WaitFor(ctx, func() error { + if !checkRoutingTable(d1, d2) { + return fmt.Errorf("should have routes") + } + return nil + })) + + // close both hosts so query fails + require.NoError(t, d1.host.Close()) + require.NoError(t, d2.host.Close()) + // peers will still be in the RT because we have decoupled membership from connectivity + require.NoError(t, tu.WaitFor(ctx, func() error { + if !checkRoutingTable(d1, d2) { + return fmt.Errorf("should have routes") + } + return nil + })) + + // failed queries should remove the peers from the RT + _, err := d1.GetClosestPeers(ctx, "test") + require.NoError(t, err) + + _, err = d2.GetClosestPeers(ctx, "test") + require.NoError(t, err) + + require.NoError(t, tu.WaitFor(ctx, func() error { + if checkRoutingTable(d1, d2) { + return fmt.Errorf("should not have routes") + } + return nil + })) +} + +func TestRTAdditionOnSuccessfulQuery(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + + d1 := setupDHT(ctx, t, false) + d2 := setupDHT(ctx, t, false) + d3 := setupDHT(ctx, t, false) + + connect(t, ctx, d1, d2) + connect(t, ctx, d2, d3) + // validate RT states + + // d1 has d2 + require.NoError(t, tu.WaitFor(ctx, func() error { + if !checkRoutingTable(d1, d2) { + return fmt.Errorf("should have routes") + } + return nil + })) + // d2 has d3 + require.NoError(t, tu.WaitFor(ctx, func() error { + if !checkRoutingTable(d2, d3) { + return fmt.Errorf("should have routes") + } + return nil + })) + + // however, d1 does not know about d3 + require.NoError(t, tu.WaitFor(ctx, func() error { + if checkRoutingTable(d1, d3) { + return fmt.Errorf("should not have routes") + } + return nil + })) + + // but when d3 queries d2, d1 and d3 discover each other + _, err := d3.GetClosestPeers(ctx, "something") + require.NoError(t, err) + require.NoError(t, tu.WaitFor(ctx, func() error { + if !checkRoutingTable(d1, d3) { + return fmt.Errorf("should have routes") + } + return nil + })) +} + +func checkRoutingTable(a, b *IpfsDHT) bool { + // loop until connection notification has been received. + // under high load, this may not happen as immediately as we would like. + return a.routingTable.Find(b.self) != "" && b.routingTable.Find(a.self) != "" +} diff --git a/records.go b/records.go index 7a1ee0ca0..0677b5c40 100644 --- a/records.go +++ b/records.go @@ -55,7 +55,7 @@ func (dht *IpfsDHT) GetPublicKey(ctx context.Context, p peer.ID) (ci.PubKey, err // Found the public key err := dht.peerstore.AddPubKey(p, r.pubk) if err != nil { - logger.Warningf("Failed to add public key to peerstore for %v", p) + logger.Errorw("failed to add public key to peerstore", "peer", p) } return r.pubk, nil } diff --git a/records_test.go b/records_test.go index 458de2b91..092f5b3c0 100644 --- a/records_test.go +++ b/records_test.go @@ -14,8 +14,6 @@ import ( "github.com/libp2p/go-libp2p-core/routing" record "github.com/libp2p/go-libp2p-record" tnet "github.com/libp2p/go-libp2p-testing/net" - - dhtopt "github.com/libp2p/go-libp2p-kad-dht/opts" ) // Check that GetPublicKey() correctly extracts a public key @@ -318,13 +316,16 @@ func TestValuesDisabled(t *testing.T) { defer cancel() var ( - optsA, optsB []dhtopt.Option + optsA, optsB []Option ) + optsA = append(optsA, ProtocolPrefix("/valuesMaybeDisabled")) + optsB = append(optsB, ProtocolPrefix("/valuesMaybeDisabled")) + if !enabledA { - optsA = append(optsA, dhtopt.DisableValues()) + optsA = append(optsA, DisableValues()) } if !enabledB { - optsB = append(optsB, dhtopt.DisableValues()) + optsB = append(optsB, DisableValues()) } dhtA := setupDHT(ctx, t, false, optsA...) diff --git a/routing.go b/routing.go index 33e82d3f7..201b346dc 100644 --- a/routing.go +++ b/routing.go @@ -4,7 +4,6 @@ import ( "bytes" "context" "fmt" - "runtime" "sync" "time" @@ -15,19 +14,13 @@ import ( "github.com/ipfs/go-cid" u "github.com/ipfs/go-ipfs-util" - logging "github.com/ipfs/go-log" pb "github.com/libp2p/go-libp2p-kad-dht/pb" + "github.com/libp2p/go-libp2p-kad-dht/qpeerset" kb "github.com/libp2p/go-libp2p-kbucket" record "github.com/libp2p/go-libp2p-record" "github.com/multiformats/go-multihash" ) -// asyncQueryBuffer is the size of buffered channels in async queries. This -// buffer allows multiple queries to execute simultaneously, return their -// results and continue querying closer peers. Note that different query -// results will wait for the channel to drain. -var asyncQueryBuffer = 10 - // This file implements the Routing interface for the IpfsDHT struct. // Basic Put/Get @@ -39,15 +32,7 @@ func (dht *IpfsDHT) PutValue(ctx context.Context, key string, value []byte, opts return routing.ErrNotSupported } - eip := logger.EventBegin(ctx, "PutValue") - defer func() { - eip.Append(loggableKey(key)) - if err != nil { - eip.SetError(err) - } - eip.Done() - }() - logger.Debugf("PutValue %s", key) + logger.Debugw("putting value", "key", loggableKeyString(key)) // don't even allow local users to put bad values. if err := dht.Validator.Validate(key, value); err != nil { @@ -119,15 +104,6 @@ func (dht *IpfsDHT) GetValue(ctx context.Context, key string, opts ...routing.Op return nil, routing.ErrNotSupported } - eip := logger.EventBegin(ctx, "GetValue") - defer func() { - eip.Append(loggableKey(key)) - if err != nil { - eip.SetError(err) - } - eip.Done() - }() - // apply defaultQuorum if relevant var cfg routing.Options if err := cfg.Apply(opts...); err != nil { @@ -168,97 +144,62 @@ func (dht *IpfsDHT) SearchValue(ctx context.Context, key string, opts ...routing responsesNeeded := 0 if !cfg.Offline { - responsesNeeded = getQuorum(&cfg, -1) + responsesNeeded = getQuorum(&cfg, defaultQuorum) } - valCh, err := dht.getValues(ctx, key, responsesNeeded) - if err != nil { - return nil, err - } + stopCh := make(chan struct{}) + valCh, lookupRes := dht.getValues(ctx, key, stopCh) out := make(chan []byte) go func() { defer close(out) - - maxVals := responsesNeeded - if maxVals < 0 { - maxVals = defaultQuorum * 4 // we want some upper bound on how - // much correctional entries we will send + best, peersWithBest, aborted := dht.searchValueQuorum(ctx, key, valCh, stopCh, out, responsesNeeded) + if best == nil || aborted { + return } - // vals is used collect entries we got so far and send corrections to peers - // when we exit this function - vals := make([]RecvdVal, 0, maxVals) - var best *RecvdVal - - defer func() { - if len(vals) <= 1 || best == nil { + updatePeers := make([]peer.ID, 0, dht.bucketSize) + select { + case l := <-lookupRes: + if l == nil { return } - fixupRec := record.MakePutRecord(key, best.Val) - for _, v := range vals { - // if someone sent us a different 'less-valid' record, lets correct them - if !bytes.Equal(v.Val, best.Val) { - go func(v RecvdVal) { - if v.From == dht.self { - err := dht.putLocal(key, fixupRec) - if err != nil { - logger.Error("Error correcting local dht entry:", err) - } - return - } - ctx, cancel := context.WithTimeout(dht.Context(), time.Second*30) - defer cancel() - err := dht.putValueToPeer(ctx, v.From, fixupRec) - if err != nil { - logger.Debug("Error correcting DHT entry: ", err) - } - }(v) + + for _, p := range l.peers { + if _, ok := peersWithBest[p]; !ok { + updatePeers = append(updatePeers, p) } } - }() + case <-ctx.Done(): + return + } - for { - select { - case v, ok := <-valCh: - if !ok { - return - } + dht.updatePeerValues(dht.Context(), key, best, updatePeers) + }() - if len(vals) < maxVals { - vals = append(vals, v) - } + return out, nil +} - if v.Val == nil { - continue - } - // Select best value - if best != nil { - if bytes.Equal(best.Val, v.Val) { - continue - } - sel, err := dht.Validator.Select(key, [][]byte{best.Val, v.Val}) - if err != nil { - logger.Warning("Failed to select dht key: ", err) - continue - } - if sel != 1 { - continue - } - } - best = &v +func (dht *IpfsDHT) searchValueQuorum(ctx context.Context, key string, valCh <-chan RecvdVal, stopCh chan struct{}, + out chan<- []byte, nvals int) ([]byte, map[peer.ID]struct{}, bool) { + numResponses := 0 + return dht.processValues(ctx, key, valCh, + func(ctx context.Context, v RecvdVal, better bool) bool { + numResponses++ + if better { select { case out <- v.Val: case <-ctx.Done(): - return + return false } - case <-ctx.Done(): - return } - } - }() - return out, nil + if nvals > 0 && numResponses > nvals { + close(stopCh) + return true + } + return false + }) } // GetValues gets nvals values corresponding to the given key. @@ -267,144 +208,182 @@ func (dht *IpfsDHT) GetValues(ctx context.Context, key string, nvals int) (_ []R return nil, routing.ErrNotSupported } - eip := logger.EventBegin(ctx, "GetValues") - eip.Append(loggableKey(key)) - defer eip.Done() - - valCh, err := dht.getValues(ctx, key, nvals) - if err != nil { - eip.SetError(err) - return nil, err - } + queryCtx, cancel := context.WithCancel(ctx) + defer cancel() + valCh, _ := dht.getValues(queryCtx, key, nil) out := make([]RecvdVal, 0, nvals) for val := range valCh { out = append(out, val) + if len(out) == nvals { + cancel() + } } return out, ctx.Err() } -func (dht *IpfsDHT) getValues(ctx context.Context, key string, nvals int) (<-chan RecvdVal, error) { - vals := make(chan RecvdVal, 1) - - done := func(err error) (<-chan RecvdVal, error) { - defer close(vals) - return vals, err - } - - // If we have it local, don't bother doing an RPC! - lrec, err := dht.getLocal(key) - if err != nil { - // something is wrong with the datastore. - return done(err) - } - if lrec != nil { - // TODO: this is tricky, we don't always want to trust our own value - // what if the authoritative source updated it? - logger.Debug("have it locally") - vals <- RecvdVal{ - Val: lrec.GetValue(), - From: dht.self, +func (dht *IpfsDHT) processValues(ctx context.Context, key string, vals <-chan RecvdVal, + newVal func(ctx context.Context, v RecvdVal, better bool) bool) (best []byte, peersWithBest map[peer.ID]struct{}, aborted bool) { +loop: + for { + if aborted { + return } - if nvals == 0 || nvals == 1 { - return done(nil) - } + select { + case v, ok := <-vals: + if !ok { + break loop + } - nvals-- - } else if nvals == 0 { - return done(routing.ErrNotFound) + // Select best value + if best != nil { + if bytes.Equal(best, v.Val) { + peersWithBest[v.From] = struct{}{} + aborted = newVal(ctx, v, false) + continue + } + sel, err := dht.Validator.Select(key, [][]byte{best, v.Val}) + if err != nil { + logger.Warnw("failed to select best value", "key", key, "error", err) + continue + } + if sel != 1 { + aborted = newVal(ctx, v, false) + continue + } + } + peersWithBest = make(map[peer.ID]struct{}) + peersWithBest[v.From] = struct{}{} + best = v.Val + aborted = newVal(ctx, v, true) + case <-ctx.Done(): + return + } } - // get closest peers in the routing table - rtp := dht.routingTable.NearestPeers(kb.ConvertKey(key), AlphaValue) - logger.Debugf("peers in rt: %d %s", len(rtp), rtp) - if len(rtp) == 0 { - logger.Warning("No peers from routing table!") - return done(kb.ErrLookupFailure) + return +} + +func (dht *IpfsDHT) updatePeerValues(ctx context.Context, key string, val []byte, peers []peer.ID) { + fixupRec := record.MakePutRecord(key, val) + for _, p := range peers { + go func(p peer.ID) { + //TODO: Is this possible? + if p == dht.self { + err := dht.putLocal(key, fixupRec) + if err != nil { + logger.Error("Error correcting local dht entry:", err) + } + return + } + ctx, cancel := context.WithTimeout(ctx, time.Second*30) + defer cancel() + err := dht.putValueToPeer(ctx, p, fixupRec) + if err != nil { + logger.Debug("Error correcting DHT entry: ", err) + } + }(p) } +} - var valslock sync.Mutex - var got int +func (dht *IpfsDHT) getValues(ctx context.Context, key string, stopQuery chan struct{}) (<-chan RecvdVal, <-chan *lookupWithFollowupResult) { + valCh := make(chan RecvdVal, 1) + lookupResCh := make(chan *lookupWithFollowupResult, 1) - // setup the Query - parent := ctx - query := dht.newQuery(key, func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) { - routing.PublishQueryEvent(parent, &routing.QueryEvent{ - Type: routing.SendingQuery, - ID: p, - }) + logger.Debugw("finding value", "key", loggableKeyString(key)) - rec, peers, err := dht.getValueOrPeers(ctx, p, key) - switch err { - case routing.ErrNotFound: - // in this case, they responded with nothing, - // still send a notification so listeners can know the - // request has completed 'successfully' - routing.PublishQueryEvent(parent, &routing.QueryEvent{ - Type: routing.PeerResponse, - ID: p, - }) - return nil, err - default: - return nil, err - - case nil, errInvalidRecord: - // in either of these cases, we want to keep going + if rec, err := dht.getLocal(key); rec != nil && err == nil { + select { + case valCh <- RecvdVal{ + Val: rec.GetValue(), + From: dht.self, + }: + case <-ctx.Done(): } + } - res := &dhtQueryResult{closerPeers: peers} - - if rec.GetValue() != nil || err == errInvalidRecord { - rv := RecvdVal{ - Val: rec.GetValue(), - From: p, - } - valslock.Lock() - select { - case vals <- rv: - case <-ctx.Done(): - valslock.Unlock() - return nil, ctx.Err() - } - got++ + go func() { + defer close(valCh) + defer close(lookupResCh) + lookupRes, err := dht.runLookupWithFollowup(ctx, key, + func(ctx context.Context, p peer.ID) ([]*peer.AddrInfo, error) { + // For DHT query command + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.SendingQuery, + ID: p, + }) + + rec, peers, err := dht.getValueOrPeers(ctx, p, key) + switch err { + case routing.ErrNotFound: + // in this case, they responded with nothing, + // still send a notification so listeners can know the + // request has completed 'successfully' + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.PeerResponse, + ID: p, + }) + return nil, err + default: + return nil, err + case nil, errInvalidRecord: + // in either of these cases, we want to keep going + } - // If we have collected enough records, we're done - if nvals == got { - res.success = true - } - valslock.Unlock() - } + // TODO: What should happen if the record is invalid? + // Pre-existing code counted it towards the quorum, but should it? + if rec != nil && rec.GetValue() != nil { + rv := RecvdVal{ + Val: rec.GetValue(), + From: p, + } - routing.PublishQueryEvent(parent, &routing.QueryEvent{ - Type: routing.PeerResponse, - ID: p, - Responses: peers, - }) + select { + case valCh <- rv: + case <-ctx.Done(): + return nil, ctx.Err() + } + } - return res, nil - }) + // For DHT query command + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.PeerResponse, + ID: p, + Responses: peers, + }) - go func() { - reqCtx, cancel := context.WithTimeout(ctx, time.Minute) - defer cancel() + return peers, err + }, + func() bool { + select { + case <-stopQuery: + return true + default: + return false + } + }, + ) - _, err = query.Run(reqCtx, rtp) + if err != nil { + return + } + lookupResCh <- lookupRes - // We do have some values but we either ran out of peers to query or - // searched for a whole minute. - // - // We'll just call this a success. - if got > 0 && (err == routing.ErrNotFound || reqCtx.Err() == context.DeadlineExceeded) { - // refresh the cpl for this key as the query was successful - dht.routingTable.ResetCplRefreshedAtForID(kb.ConvertKey(key), time.Now()) - err = nil + if ctx.Err() == nil { + dht.refreshRTIfNoShortcut(kb.ConvertKey(key), lookupRes) } - done(err) }() - return vals, nil + return valCh, lookupResCh +} + +func (dht *IpfsDHT) refreshRTIfNoShortcut(key kb.ID, lookupRes *lookupWithFollowupResult) { + if lookupRes.completed { + // refresh the cpl for this key as the query was successful + dht.routingTable.ResetCplRefreshedAtForID(key, time.Now()) + } } // Provider abstraction for indirect stores. @@ -416,14 +395,9 @@ func (dht *IpfsDHT) Provide(ctx context.Context, key cid.Cid, brdcst bool) (err if !dht.enableProviders { return routing.ErrNotSupported } + logger.Debugw("finding provider", "cid", key) + keyMH := key.Hash() - eip := logger.EventBegin(ctx, "Provide", multihashLoggableKey(keyMH), logging.LoggableMap{"broadcast": brdcst}) - defer func() { - if err != nil { - eip.SetError(err) - } - eip.Done() - }() // add self locally dht.ProviderManager.AddProvider(ctx, keyMH, dht.self) @@ -452,8 +426,19 @@ func (dht *IpfsDHT) Provide(ctx context.Context, key cid.Cid, brdcst bool) (err defer cancel() } + var exceededDeadline bool peers, err := dht.GetClosestPeers(closerCtx, string(keyMH)) - if err != nil { + switch err { + case context.DeadlineExceeded: + // If the _inner_ deadline has been exceeded but the _outer_ + // context is still fine, provide the value to the closest peers + // we managed to find, even if they're not the _actual_ closest peers. + if ctx.Err() != nil { + return ctx.Err() + } + exceededDeadline = true + case nil: + default: return err } @@ -475,7 +460,10 @@ func (dht *IpfsDHT) Provide(ctx context.Context, key cid.Cid, brdcst bool) (err }(p) } wg.Wait() - return nil + if exceededDeadline { + return context.DeadlineExceeded + } + return ctx.Err() } func (dht *IpfsDHT) makeProvRecord(key []byte) (*pb.Message, error) { pi := peer.AddrInfo{ @@ -508,26 +496,41 @@ func (dht *IpfsDHT) FindProviders(ctx context.Context, c cid.Cid) ([]peer.AddrIn // FindProvidersAsync is the same thing as FindProviders, but returns a channel. // Peers will be returned on the channel as soon as they are found, even before -// the search query completes. +// the search query completes. If count is zero then the query will run until it +// completes. Note: not reading from the returned channel may block the query +// from progressing. func (dht *IpfsDHT) FindProvidersAsync(ctx context.Context, key cid.Cid, count int) <-chan peer.AddrInfo { - peerOut := make(chan peer.AddrInfo, count) if !dht.enableProviders { + peerOut := make(chan peer.AddrInfo) close(peerOut) return peerOut } + chSize := count + if count == 0 { + chSize = 1 + } + peerOut := make(chan peer.AddrInfo, chSize) + keyMH := key.Hash() - logger.Event(ctx, "findProviders", multihashLoggableKey(keyMH)) go dht.findProvidersAsyncRoutine(ctx, keyMH, count, peerOut) return peerOut } func (dht *IpfsDHT) findProvidersAsyncRoutine(ctx context.Context, key multihash.Multihash, count int, peerOut chan peer.AddrInfo) { - defer logger.EventBegin(ctx, "findProvidersAsync", multihashLoggableKey(key)).Done() + logger.Debugw("finding providers", "key", key) + defer close(peerOut) - ps := peer.NewLimitedSet(count) + findAll := count == 0 + var ps *peer.Set + if findAll { + ps = peer.NewSet() + } else { + ps = peer.NewLimitedSet(count) + } + provs := dht.ProviderManager.GetProviders(ctx, key) for _, p := range provs { // NOTE: Assuming that this list of peers is unique @@ -542,239 +545,131 @@ func (dht *IpfsDHT) findProvidersAsyncRoutine(ctx context.Context, key multihash // If we have enough peers locally, don't bother with remote RPC // TODO: is this a DOS vector? - if ps.Size() >= count { + if !findAll && ps.Size() >= count { return } } - peers := dht.routingTable.NearestPeers(kb.ConvertKey(string(key)), AlphaValue) - if len(peers) == 0 { - routing.PublishQueryEvent(ctx, &routing.QueryEvent{ - Type: routing.QueryError, - Extra: kb.ErrLookupFailure.Error(), - }) - return - } + lookupRes, err := dht.runLookupWithFollowup(ctx, string(key), + func(ctx context.Context, p peer.ID) ([]*peer.AddrInfo, error) { + // For DHT query command + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.SendingQuery, + ID: p, + }) - // setup the Query - parent := ctx - query := dht.newQuery(string(key), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) { - routing.PublishQueryEvent(parent, &routing.QueryEvent{ - Type: routing.SendingQuery, - ID: p, - }) - pmes, err := dht.findProvidersSingle(ctx, p, key) - if err != nil { - return nil, err - } + pmes, err := dht.findProvidersSingle(ctx, p, key) + if err != nil { + return nil, err + } - logger.Debugf("%d provider entries", len(pmes.GetProviderPeers())) - provs := pb.PBPeersToPeerInfos(pmes.GetProviderPeers()) - logger.Debugf("%d provider entries decoded", len(provs)) + logger.Debugf("%d provider entries", len(pmes.GetProviderPeers())) + provs := pb.PBPeersToPeerInfos(pmes.GetProviderPeers()) + logger.Debugf("%d provider entries decoded", len(provs)) - // Add unique providers from request, up to 'count' - for _, prov := range provs { - if prov.ID != dht.self { - dht.peerstore.AddAddrs(prov.ID, prov.Addrs, peerstore.TempAddrTTL) - } - logger.Debugf("got provider: %s", prov) - if ps.TryAdd(prov.ID) { - logger.Debugf("using provider: %s", prov) - select { - case peerOut <- *prov: - case <-ctx.Done(): - logger.Debug("context timed out sending more providers") - return nil, ctx.Err() + // Add unique providers from request, up to 'count' + for _, prov := range provs { + if prov.ID != dht.self { + dht.peerstore.AddAddrs(prov.ID, prov.Addrs, peerstore.TempAddrTTL) + } + logger.Debugf("got provider: %s", prov) + if ps.TryAdd(prov.ID) { + logger.Debugf("using provider: %s", prov) + select { + case peerOut <- *prov: + case <-ctx.Done(): + logger.Debug("context timed out sending more providers") + return nil, ctx.Err() + } + } + if !findAll && ps.Size() >= count { + logger.Debugf("got enough providers (%d/%d)", ps.Size(), count) + return nil, nil } } - if ps.Size() >= count { - logger.Debugf("got enough providers (%d/%d)", ps.Size(), count) - return &dhtQueryResult{success: true}, nil - } - } - // Give closer peers back to the query to be queried - closer := pmes.GetCloserPeers() - clpeers := pb.PBPeersToPeerInfos(closer) - logger.Debugf("got closer peers: %d %s", len(clpeers), clpeers) + // Give closer peers back to the query to be queried + closer := pmes.GetCloserPeers() + peers := pb.PBPeersToPeerInfos(closer) + logger.Debugf("got closer peers: %d %s", len(peers), peers) - routing.PublishQueryEvent(parent, &routing.QueryEvent{ - Type: routing.PeerResponse, - ID: p, - Responses: clpeers, - }) - return &dhtQueryResult{closerPeers: clpeers}, nil - }) + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.PeerResponse, + ID: p, + Responses: peers, + }) - _, err := query.Run(ctx, peers) - if err != nil { - logger.Debugf("Query error: %s", err) - // Special handling for issue: https://github.com/ipfs/go-ipfs/issues/3032 - if fmt.Sprint(err) == "" { - logger.Error("reproduced bug 3032:") - logger.Errorf("Errors type information: %#v", err) - logger.Errorf("go version: %s", runtime.Version()) - logger.Error("please report this information to: https://github.com/ipfs/go-ipfs/issues/3032") - - // replace problematic error with something that won't crash the daemon - err = fmt.Errorf("") - } - routing.PublishQueryEvent(ctx, &routing.QueryEvent{ - Type: routing.QueryError, - Extra: err.Error(), - }) - } + return peers, nil + }, + func() bool { + return !findAll && ps.Size() >= count + }, + ) - // refresh the cpl for this key after the query is run - dht.routingTable.ResetCplRefreshedAtForID(kb.ConvertKey(string(key)), time.Now()) + if err != nil && ctx.Err() == nil { + dht.refreshRTIfNoShortcut(kb.ConvertKey(string(key)), lookupRes) + } } // FindPeer searches for a peer with given ID. func (dht *IpfsDHT) FindPeer(ctx context.Context, id peer.ID) (_ peer.AddrInfo, err error) { - eip := logger.EventBegin(ctx, "FindPeer", id) - defer func() { - if err != nil { - eip.SetError(err) - } - eip.Done() - }() + logger.Debugw("finding peer", "peer", id) // Check if were already connected to them if pi := dht.FindLocal(id); pi.ID != "" { return pi, nil } - peers := dht.routingTable.NearestPeers(kb.ConvertPeerID(id), AlphaValue) - if len(peers) == 0 { - return peer.AddrInfo{}, kb.ErrLookupFailure - } - - // Sanity... - for _, p := range peers { - if p == id { - logger.Debug("found target peer in list of closest peers...") - return dht.peerstore.PeerInfo(p), nil - } - } - - // setup the Query - parent := ctx - query := dht.newQuery(string(id), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) { - routing.PublishQueryEvent(parent, &routing.QueryEvent{ - Type: routing.SendingQuery, - ID: p, - }) - - pmes, err := dht.findPeerSingle(ctx, p, id) - if err != nil { - return nil, err - } - - closer := pmes.GetCloserPeers() - clpeerInfos := pb.PBPeersToPeerInfos(closer) + lookupRes, err := dht.runLookupWithFollowup(ctx, string(id), + func(ctx context.Context, p peer.ID) ([]*peer.AddrInfo, error) { + // For DHT query command + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.SendingQuery, + ID: p, + }) - // see if we got the peer here - for _, npi := range clpeerInfos { - if npi.ID == id { - return &dhtQueryResult{ - peer: npi, - success: true, - }, nil + pmes, err := dht.findPeerSingle(ctx, p, id) + if err != nil { + logger.Debugf("error getting closer peers: %s", err) + return nil, err } - } + peers := pb.PBPeersToPeerInfos(pmes.GetCloserPeers()) - routing.PublishQueryEvent(parent, &routing.QueryEvent{ - Type: routing.PeerResponse, - ID: p, - Responses: clpeerInfos, - }) + // For DHT query command + routing.PublishQueryEvent(ctx, &routing.QueryEvent{ + Type: routing.PeerResponse, + ID: p, + Responses: peers, + }) - return &dhtQueryResult{closerPeers: clpeerInfos}, nil - }) + return peers, err + }, + func() bool { + return dht.host.Network().Connectedness(id) == network.Connected + }, + ) - // run it! - result, err := query.Run(ctx, peers) if err != nil { return peer.AddrInfo{}, err } - // refresh the cpl for this key since the lookup was successful - dht.routingTable.ResetCplRefreshedAtForID(kb.ConvertPeerID(id), time.Now()) - - logger.Debugf("FindPeer %v %v", id, result.success) - if result.peer.ID == "" { - return peer.AddrInfo{}, routing.ErrNotFound + dialedPeerDuringQuery := false + for i, p := range lookupRes.peers { + if p == id { + // Note: we consider PeerUnreachable to be a valid state because the peer may not support the DHT protocol + // and therefore the peer would fail the query. The fact that a peer that is returned can be a non-DHT + // server peer and is not identified as such is a bug. + dialedPeerDuringQuery = lookupRes.state[i] != qpeerset.PeerHeard + break + } } - return *result.peer, nil -} - -// FindPeersConnectedToPeer searches for peers directly connected to a given peer. -func (dht *IpfsDHT) FindPeersConnectedToPeer(ctx context.Context, id peer.ID) (<-chan *peer.AddrInfo, error) { - - peerchan := make(chan *peer.AddrInfo, asyncQueryBuffer) - peersSeen := make(map[peer.ID]struct{}) - var peersSeenMx sync.Mutex - - peers := dht.routingTable.NearestPeers(kb.ConvertPeerID(id), AlphaValue) - if len(peers) == 0 { - return nil, kb.ErrLookupFailure + // Return peer information if we tried to dial the peer during the query or we are (or recently were) connected + // to the peer. + connectedness := dht.host.Network().Connectedness(id) + if dialedPeerDuringQuery || connectedness == network.Connected || connectedness == network.CanConnect { + return dht.peerstore.PeerInfo(id), nil } - // setup the Query - query := dht.newQuery(string(id), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) { - - pmes, err := dht.findPeerSingle(ctx, p, id) - if err != nil { - return nil, err - } - - var clpeers []*peer.AddrInfo - closer := pmes.GetCloserPeers() - for _, pbp := range closer { - pi := pb.PBPeerToPeerInfo(pbp) - - // skip peers already seen - peersSeenMx.Lock() - if _, found := peersSeen[pi.ID]; found { - peersSeenMx.Unlock() - continue - } - peersSeen[pi.ID] = struct{}{} - peersSeenMx.Unlock() - - // if peer is connected, send it to our client. - if pb.Connectedness(pbp.Connection) == network.Connected { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case peerchan <- pi: - } - } - - // if peer is the peer we're looking for, don't bother querying it. - // TODO maybe query it? - if pb.Connectedness(pbp.Connection) != network.Connected { - clpeers = append(clpeers, pi) - } - } - - return &dhtQueryResult{closerPeers: clpeers}, nil - }) - - // run it! run it asynchronously to gen peers as results are found. - // this does no error checking - go func() { - if _, err := query.Run(ctx, peers); err != nil { - logger.Debug(err) - } - - // refresh the cpl for this key - dht.routingTable.ResetCplRefreshedAtForID(kb.ConvertPeerID(id), time.Now()) - - // close the peerchan channel when done. - close(peerchan) - }() - - return peerchan, nil + return peer.AddrInfo{}, routing.ErrNotFound } diff --git a/routing_options.go b/routing_options.go index 591ce54c4..a1e5935b9 100644 --- a/routing_options.go +++ b/routing_options.go @@ -4,12 +4,13 @@ import "github.com/libp2p/go-libp2p-core/routing" type quorumOptionKey struct{} -const defaultQuorum = 16 +const defaultQuorum = 0 // Quorum is a DHT option that tells the DHT how many peers it needs to get -// values from before returning the best one. +// values from before returning the best one. Zero means the DHT query +// should complete instead of returning early. // -// Default: 16 +// Default: 0 func Quorum(n int) routing.Option { return func(opts *routing.Options) error { if opts.Other == nil { diff --git a/subscriber_notifee.go b/subscriber_notifee.go new file mode 100644 index 000000000..18cb045e0 --- /dev/null +++ b/subscriber_notifee.go @@ -0,0 +1,223 @@ +package dht + +import ( + "fmt" + + "github.com/libp2p/go-libp2p-core/event" + "github.com/libp2p/go-libp2p-core/network" + "github.com/libp2p/go-libp2p-core/peer" + "github.com/libp2p/go-libp2p-core/protocol" + + "github.com/libp2p/go-eventbus" + + ma "github.com/multiformats/go-multiaddr" + + "github.com/jbenet/goprocess" +) + +// subscriberNotifee implements network.Notifee and also manages the subscriber to the event bus. We consume peer +// identification events to trigger inclusion in the routing table, and we consume Disconnected events to eject peers +// from it. +type subscriberNotifee struct { + dht *IpfsDHT + subs event.Subscription +} + +func newSubscriberNotifiee(dht *IpfsDHT) (*subscriberNotifee, error) { + bufSize := eventbus.BufSize(256) + + evts := []interface{}{ + // register for event bus notifications of when peers successfully complete identification in order to update + // the routing table + new(event.EvtPeerIdentificationCompleted), + + // register for event bus protocol ID changes in order to update the routing table + new(event.EvtPeerProtocolsUpdated), + + // register for event bus notifications for when our local address/addresses change so we can + // advertise those to the network + new(event.EvtLocalAddressesUpdated), + } + + // register for event bus local routability changes in order to trigger switching between client and server modes + // only register for events if the DHT is operating in ModeAuto + if dht.auto { + evts = append(evts, new(event.EvtLocalReachabilityChanged)) + } + + subs, err := dht.host.EventBus().Subscribe(evts, bufSize) + if err != nil { + return nil, fmt.Errorf("dht could not subscribe to eventbus events; err: %s", err) + } + + nn := &subscriberNotifee{ + dht: dht, + subs: subs, + } + + // register for network notifications + dht.host.Network().Notify(nn) + + // Fill routing table with currently connected peers that are DHT servers + dht.plk.Lock() + defer dht.plk.Unlock() + for _, p := range dht.host.Network().Peers() { + dht.peerFound(dht.ctx, p, false) + } + + return nn, nil +} + +func (nn *subscriberNotifee) subscribe(proc goprocess.Process) { + dht := nn.dht + defer dht.host.Network().StopNotify(nn) + defer nn.subs.Close() + + for { + select { + case e, more := <-nn.subs.Out(): + if !more { + return + } + + switch evt := e.(type) { + case event.EvtLocalAddressesUpdated: + // when our address changes, we should proactively tell our closest peers about it so + // we become discoverable quickly. The Identify protocol will push a signed peer record + // with our new address to all peers we are connected to. However, we might not necessarily be connected + // to our closet peers & so in the true spirit of Zen, searching for ourself in the network really is the best way + // to to forge connections with those matter. + select { + case dht.triggerSelfLookup <- nil: + default: + } + case event.EvtPeerProtocolsUpdated: + handlePeerProtocolsUpdatedEvent(dht, evt) + case event.EvtPeerIdentificationCompleted: + handlePeerIdentificationCompletedEvent(dht, evt) + case event.EvtLocalReachabilityChanged: + if dht.auto { + handleLocalReachabilityChangedEvent(dht, evt) + } else { + // something has gone really wrong if we get an event we did not subscribe to + logger.Errorf("received LocalReachabilityChanged event that was not subscribed to") + } + default: + // something has gone really wrong if we get an event for another type + logger.Errorf("got wrong type from subscription: %T", e) + } + case <-proc.Closing(): + return + } + } +} + +func handlePeerIdentificationCompletedEvent(dht *IpfsDHT, e event.EvtPeerIdentificationCompleted) { + dht.plk.Lock() + defer dht.plk.Unlock() + if dht.host.Network().Connectedness(e.Peer) != network.Connected { + return + } + + // if the peer supports the DHT protocol, add it to our RT and kick a refresh if needed + valid, err := dht.validRTPeer(e.Peer) + if err != nil { + logger.Errorf("could not check peerstore for protocol support: err: %s", err) + return + } else if valid { + dht.peerFound(dht.ctx, e.Peer, false) + dht.fixRTIfNeeded() + } +} + +func handlePeerProtocolsUpdatedEvent(dht *IpfsDHT, e event.EvtPeerProtocolsUpdated) { + valid, err := dht.validRTPeer(e.Peer) + if err != nil { + logger.Errorf("could not check peerstore for protocol support: err: %s", err) + return + } + + if !valid { + dht.peerStoppedDHT(dht.ctx, e.Peer) + return + } + + // we just might have discovered a peer that supports the DHT protocol + dht.fixRTIfNeeded() +} + +func handleLocalReachabilityChangedEvent(dht *IpfsDHT, e event.EvtLocalReachabilityChanged) { + var target mode + + switch e.Reachability { + case network.ReachabilityPrivate, network.ReachabilityUnknown: + target = modeClient + case network.ReachabilityPublic: + target = modeServer + } + + logger.Infof("processed event %T; performing dht mode switch", e) + + err := dht.setMode(target) + // NOTE: the mode will be printed out as a decimal. + if err == nil { + logger.Infow("switched DHT mode successfully", "mode", target) + } else { + logger.Errorw("switching DHT mode failed", "mode", target, "error", err) + } +} + +// validRTPeer returns true if the peer supports the DHT protocol and false otherwise. Supporting the DHT protocol means +// supporting the primary protocols, we do not want to add peers that are speaking obsolete secondary protocols to our +// routing table +func (dht *IpfsDHT) validRTPeer(p peer.ID) (bool, error) { + protos, err := dht.peerstore.SupportsProtocols(p, protocol.ConvertToStrings(dht.protocols)...) + if err != nil { + return false, err + } + + return len(protos) > 0, nil +} + +func (nn *subscriberNotifee) Disconnected(n network.Network, v network.Conn) { + dht := nn.dht + select { + case <-dht.Process().Closing(): + return + default: + } + + p := v.RemotePeer() + + // Lock and check to see if we're still connected. We lock to make sure + // we don't concurrently process a connect event. + dht.plk.Lock() + defer dht.plk.Unlock() + if dht.host.Network().Connectedness(p) == network.Connected { + // We're still connected. + return + } + + dht.smlk.Lock() + defer dht.smlk.Unlock() + ms, ok := dht.strmap[p] + if !ok { + return + } + delete(dht.strmap, p) + + // Do this asynchronously as ms.lk can block for a while. + go func() { + if err := ms.lk.Lock(dht.Context()); err != nil { + return + } + defer ms.lk.Unlock() + ms.invalidate() + }() +} + +func (nn *subscriberNotifee) Connected(n network.Network, v network.Conn) {} +func (nn *subscriberNotifee) OpenedStream(n network.Network, v network.Stream) {} +func (nn *subscriberNotifee) ClosedStream(n network.Network, v network.Stream) {} +func (nn *subscriberNotifee) Listen(n network.Network, a ma.Multiaddr) {} +func (nn *subscriberNotifee) ListenClose(n network.Network, a ma.Multiaddr) {} diff --git a/util.go b/util.go deleted file mode 100644 index 6ab5c0067..000000000 --- a/util.go +++ /dev/null @@ -1,10 +0,0 @@ -package dht - -// Pool size is the number of nodes used for group find/set RPC calls -var PoolSize = 6 - -// K is the maximum number of requests to perform before returning failure. -var KValue = 20 - -// Alpha is the concurrency factor for asynchronous requests. -var AlphaValue = 3