routing.go 14.3 KB
Newer Older
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
1 2 3
package dht

import (
4
	"bytes"
Jeromy's avatar
Jeromy committed
5
	"context"
6
	"fmt"
7
	"runtime"
Jeromy's avatar
Jeromy committed
8
	"sync"
Jeromy's avatar
Jeromy committed
9
	"time"
10

11
	proto "github.com/gogo/protobuf/proto"
12
	cid "github.com/ipfs/go-cid"
13
	u "github.com/ipfs/go-ipfs-util"
Jeromy's avatar
Jeromy committed
14
	logging "github.com/ipfs/go-log"
George Antoniadis's avatar
George Antoniadis committed
15 16
	pb "github.com/libp2p/go-libp2p-kad-dht/pb"
	kb "github.com/libp2p/go-libp2p-kbucket"
17 18 19 20
	inet "github.com/libp2p/go-libp2p-net"
	peer "github.com/libp2p/go-libp2p-peer"
	pset "github.com/libp2p/go-libp2p-peer/peerset"
	pstore "github.com/libp2p/go-libp2p-peerstore"
George Antoniadis's avatar
George Antoniadis committed
21 22 23
	record "github.com/libp2p/go-libp2p-record"
	routing "github.com/libp2p/go-libp2p-routing"
	notif "github.com/libp2p/go-libp2p-routing/notifications"
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
24 25
)

26 27 28 29 30 31
// 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

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
32 33 34 35 36
// This file implements the Routing interface for the IpfsDHT struct.

// Basic Put/Get

// PutValue adds value corresponding to given Key.
37
// This is the top level "Store" operation of the DHT
38 39 40 41 42 43 44 45 46
func (dht *IpfsDHT) PutValue(ctx context.Context, key string, value []byte) (err error) {
	eip := log.EventBegin(ctx, "PutValue")
	defer func() {
		eip.Append(loggableKey(key))
		if err != nil {
			eip.SetError(err)
		}
		eip.Done()
	}()
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
47
	log.Debugf("PutValue %s", key)
Jeromy's avatar
Jeromy committed
48

49
	rec := record.MakePutRecord(key, value)
50
	rec.TimeReceived = proto.String(u.FormatRFC3339(time.Now()))
Jeromy's avatar
Jeromy committed
51
	err = dht.putLocal(key, rec)
52 53 54 55
	if err != nil {
		return err
	}

56
	pchan, err := dht.GetClosestPeers(ctx, key)
57 58 59
	if err != nil {
		return err
	}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
60

61 62 63 64
	wg := sync.WaitGroup{}
	for p := range pchan {
		wg.Add(1)
		go func(p peer.ID) {
Jeromy's avatar
Jeromy committed
65 66
			ctx, cancel := context.WithCancel(ctx)
			defer cancel()
67
			defer wg.Done()
Jeromy's avatar
Jeromy committed
68 69 70 71 72
			notif.PublishQueryEvent(ctx, &notif.QueryEvent{
				Type: notif.Value,
				ID:   p,
			})

Jeromy's avatar
Jeromy committed
73
			err := dht.putValueToPeer(ctx, p, key, rec)
74
			if err != nil {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
75
				log.Debugf("failed putting value to peer: %s", err)
76 77 78 79 80
			}
		}(p)
	}
	wg.Wait()
	return nil
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
81 82 83
}

// GetValue searches for the value corresponding to given Key.
84 85 86 87 88 89 90 91 92
func (dht *IpfsDHT) GetValue(ctx context.Context, key string) (_ []byte, err error) {
	eip := log.EventBegin(ctx, "GetValue")
	defer func() {
		eip.Append(loggableKey(key))
		if err != nil {
			eip.SetError(err)
		}
		eip.Done()
	}()
Jeromy's avatar
Jeromy committed
93 94 95 96
	ctx, cancel := context.WithTimeout(ctx, time.Minute)
	defer cancel()

	vals, err := dht.GetValues(ctx, key, 16)
97 98 99 100
	if err != nil {
		return nil, err
	}

Steven Allen's avatar
Steven Allen committed
101
	recs := make([][]byte, 0, len(vals))
102
	for _, v := range vals {
103 104 105
		if v.Val != nil {
			recs = append(recs, v.Val)
		}
106
	}
107 108 109
	if len(recs) == 0 {
		return nil, routing.ErrNotFound
	}
110 111 112 113 114 115 116 117 118 119 120 121 122

	i, err := dht.Selector.BestRecord(key, recs)
	if err != nil {
		return nil, err
	}

	best := recs[i]
	log.Debugf("GetValue %v %v", key, best)
	if best == nil {
		log.Errorf("GetValue yielded correct record with nil value.")
		return nil, routing.ErrNotFound
	}

123
	fixupRec := record.MakePutRecord(key, best)
124 125 126 127
	for _, v := range vals {
		// if someone sent us a different 'less-valid' record, lets correct them
		if !bytes.Equal(v.Val, best) {
			go func(v routing.RecvdVal) {
128 129 130 131 132 133 134
				if v.From == dht.self {
					err := dht.putLocal(key, fixupRec)
					if err != nil {
						log.Error("Error correcting local dht entry:", err)
					}
					return
				}
Jeromy's avatar
Jeromy committed
135 136
				ctx, cancel := context.WithTimeout(dht.Context(), time.Second*30)
				defer cancel()
137 138 139 140 141 142 143 144 145 146 147
				err := dht.putValueToPeer(ctx, v.From, key, fixupRec)
				if err != nil {
					log.Error("Error correcting DHT entry: ", err)
				}
			}(v)
		}
	}

	return best, nil
}

148 149 150 151 152 153 154 155 156
func (dht *IpfsDHT) GetValues(ctx context.Context, key string, nvals int) (_ []routing.RecvdVal, err error) {
	eip := log.EventBegin(ctx, "GetValues")
	defer func() {
		eip.Append(loggableKey(key))
		if err != nil {
			eip.SetError(err)
		}
		eip.Done()
	}()
Steven Allen's avatar
Steven Allen committed
157
	vals := make([]routing.RecvdVal, 0, nvals)
158 159
	var valslock sync.Mutex

160
	// If we have it local, don't bother doing an RPC!
161
	lrec, err := dht.getLocal(key)
Jeromy's avatar
Jeromy committed
162
	if err == nil {
163
		// TODO: this is tricky, we don't always want to trust our own value
164
		// what if the authoritative source updated it?
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
165
		log.Debug("have it locally")
166 167 168 169 170 171 172 173 174 175
		vals = append(vals, routing.RecvdVal{
			Val:  lrec.GetValue(),
			From: dht.self,
		})

		if nvals <= 1 {
			return vals, nil
		}
	} else if nvals == 0 {
		return nil, err
Jeromy's avatar
Jeromy committed
176 177
	}

178
	// get closest peers in the routing table
Jeromy's avatar
Jeromy committed
179
	rtp := dht.routingTable.NearestPeers(kb.ConvertKey(key), AlphaValue)
180
	log.Debugf("peers in rt: %d %s", len(rtp), rtp)
181
	if len(rtp) == 0 {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
182
		log.Warning("No peers from routing table!")
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
183
		return nil, kb.ErrLookupFailure
184 185
	}

186
	// setup the Query
Jeromy's avatar
Jeromy committed
187
	parent := ctx
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
188
	query := dht.newQuery(key, func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) {
Jeromy's avatar
Jeromy committed
189
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
Jeromy's avatar
Jeromy committed
190 191 192 193
			Type: notif.SendingQuery,
			ID:   p,
		})

194
		rec, peers, err := dht.getValueOrPeers(ctx, p, key)
195 196 197 198 199 200 201 202 203
		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'
			notif.PublishQueryEvent(parent, &notif.QueryEvent{
				Type: notif.PeerResponse,
				ID:   p,
			})
204
			return nil, err
205 206 207 208 209
		default:
			return nil, err

		case nil, errInvalidRecord:
			// in either of these cases, we want to keep going
210
		}
211

212 213
		res := &dhtQueryResult{closerPeers: peers}

214
		if rec.GetValue() != nil || err == errInvalidRecord {
215 216 217 218 219 220 221
			rv := routing.RecvdVal{
				Val:  rec.GetValue(),
				From: p,
			}
			valslock.Lock()
			vals = append(vals, rv)

222
			// If we have collected enough records, we're done
223 224 225 226
			if len(vals) >= nvals {
				res.success = true
			}
			valslock.Unlock()
227 228
		}

Jeromy's avatar
Jeromy committed
229
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
Jeromy's avatar
Jeromy committed
230 231
			Type:      notif.PeerResponse,
			ID:        p,
Jeromy's avatar
Jeromy committed
232
			Responses: peers,
Jeromy's avatar
Jeromy committed
233 234
		})

235 236
		return res, nil
	})
237

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
238
	// run it!
239 240 241 242 243
	_, err = query.Run(ctx, rtp)
	if len(vals) == 0 {
		if err != nil {
			return nil, err
		}
244 245
	}

246
	return vals, nil
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
247 248
}

249 250 251
// Provider abstraction for indirect stores.
// Some DHTs store values directly, while an indirect store stores pointers to
// locations of the value, similarly to Coral and Mainline DHT.
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
252

253
// Provide makes this node announce that it can provide a value for the given key
254 255 256 257 258 259 260 261
func (dht *IpfsDHT) Provide(ctx context.Context, key *cid.Cid, brdcst bool) (err error) {
	eip := log.EventBegin(ctx, "Provide", key, logging.LoggableMap{"broadcast": brdcst})
	defer func() {
		if err != nil {
			eip.SetError(err)
		}
		eip.Done()
	}()
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
262 263

	// add self locally
264
	dht.providers.AddProvider(ctx, key, dht.self)
Jeromy's avatar
Jeromy committed
265 266 267
	if !brdcst {
		return nil
	}
268

269
	peers, err := dht.GetClosestPeers(ctx, key.KeyString())
270 271
	if err != nil {
		return err
272 273
	}

274 275 276 277 278
	mes, err := dht.makeProvRecord(key)
	if err != nil {
		return err
	}

Jeromy's avatar
Jeromy committed
279
	wg := sync.WaitGroup{}
280
	for p := range peers {
Jeromy's avatar
Jeromy committed
281 282 283
		wg.Add(1)
		go func(p peer.ID) {
			defer wg.Done()
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
284
			log.Debugf("putProvider(%s, %s)", key, p)
285
			err := dht.sendMessage(ctx, p, mes)
Jeromy's avatar
Jeromy committed
286
			if err != nil {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
287
				log.Debug(err)
Jeromy's avatar
Jeromy committed
288 289
			}
		}(p)
290
	}
Jeromy's avatar
Jeromy committed
291
	wg.Wait()
292
	return nil
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
293
}
294
func (dht *IpfsDHT) makeProvRecord(skey *cid.Cid) (*pb.Message, error) {
295 296 297 298 299 300 301 302 303 304 305
	pi := pstore.PeerInfo{
		ID:    dht.self,
		Addrs: dht.host.Addrs(),
	}

	// // only share WAN-friendly addresses ??
	// pi.Addrs = addrutil.WANShareableAddrs(pi.Addrs)
	if len(pi.Addrs) < 1 {
		return nil, fmt.Errorf("no known addresses for self. cannot put provider.")
	}

306
	pmes := pb.NewMessage(pb.Message_ADD_PROVIDER, skey.KeyString(), 0)
307 308 309
	pmes.ProviderPeers = pb.RawPeerInfosToPBPeers([]pstore.PeerInfo{pi})
	return pmes, nil
}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
310

Brian Tiger Chow's avatar
Brian Tiger Chow committed
311
// FindProviders searches until the context expires.
312
func (dht *IpfsDHT) FindProviders(ctx context.Context, c *cid.Cid) ([]pstore.PeerInfo, error) {
Jeromy's avatar
Jeromy committed
313
	var providers []pstore.PeerInfo
314
	for p := range dht.FindProvidersAsync(ctx, c, KValue) {
Brian Tiger Chow's avatar
Brian Tiger Chow committed
315 316 317 318 319
		providers = append(providers, p)
	}
	return providers, nil
}

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
320 321 322
// 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.
323 324
func (dht *IpfsDHT) FindProvidersAsync(ctx context.Context, key *cid.Cid, count int) <-chan pstore.PeerInfo {
	log.Event(ctx, "findProviders", key)
Jeromy's avatar
Jeromy committed
325
	peerOut := make(chan pstore.PeerInfo, count)
Jeromy's avatar
Jeromy committed
326 327 328 329
	go dht.findProvidersAsyncRoutine(ctx, key, count, peerOut)
	return peerOut
}

330 331
func (dht *IpfsDHT) findProvidersAsyncRoutine(ctx context.Context, key *cid.Cid, count int, peerOut chan pstore.PeerInfo) {
	defer log.EventBegin(ctx, "findProvidersAsync", key).Done()
Jeromy's avatar
Jeromy committed
332 333
	defer close(peerOut)

Jeromy's avatar
Jeromy committed
334
	ps := pset.NewLimited(count)
Jeromy's avatar
Jeromy committed
335 336
	provs := dht.providers.GetProviders(ctx, key)
	for _, p := range provs {
337
		// NOTE: Assuming that this list of peers is unique
Jeromy's avatar
Jeromy committed
338
		if ps.TryAdd(p) {
Jeromy's avatar
Jeromy committed
339
			pi := dht.peerstore.PeerInfo(p)
Jeromy's avatar
Jeromy committed
340
			select {
Jeromy's avatar
Jeromy committed
341
			case peerOut <- pi:
Jeromy's avatar
Jeromy committed
342 343 344
			case <-ctx.Done():
				return
			}
Jeromy's avatar
Jeromy committed
345
		}
Jeromy's avatar
Jeromy committed
346

347
		// If we have enough peers locally, don't bother with remote RPC
Jeromy's avatar
Jeromy committed
348
		// TODO: is this a DOS vector?
Jeromy's avatar
Jeromy committed
349
		if ps.Size() >= count {
Jeromy's avatar
Jeromy committed
350 351 352 353 354
			return
		}
	}

	// setup the Query
Jeromy's avatar
Jeromy committed
355
	parent := ctx
356
	query := dht.newQuery(key.KeyString(), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) {
Jeromy's avatar
Jeromy committed
357
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
358 359 360
			Type: notif.SendingQuery,
			ID:   p,
		})
361
		pmes, err := dht.findProvidersSingle(ctx, p, key)
Jeromy's avatar
Jeromy committed
362 363 364 365
		if err != nil {
			return nil, err
		}

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
366
		log.Debugf("%d provider entries", len(pmes.GetProviderPeers()))
367
		provs := pb.PBPeersToPeerInfos(pmes.GetProviderPeers())
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
368
		log.Debugf("%d provider entries decoded", len(provs))
Jeromy's avatar
Jeromy committed
369 370 371

		// Add unique providers from request, up to 'count'
		for _, prov := range provs {
372 373 374
			if prov.ID != dht.self {
				dht.peerstore.AddAddrs(prov.ID, prov.Addrs, pstore.TempAddrTTL)
			}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
375
			log.Debugf("got provider: %s", prov)
376
			if ps.TryAdd(prov.ID) {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
377
				log.Debugf("using provider: %s", prov)
Jeromy's avatar
Jeromy committed
378
				select {
Jeromy's avatar
Jeromy committed
379
				case peerOut <- *prov:
Jeromy's avatar
Jeromy committed
380
				case <-ctx.Done():
381
					log.Debug("context timed out sending more providers")
Jeromy's avatar
Jeromy committed
382 383
					return nil, ctx.Err()
				}
384
			}
Jeromy's avatar
Jeromy committed
385
			if ps.Size() >= count {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
386
				log.Debugf("got enough providers (%d/%d)", ps.Size(), count)
Jeromy's avatar
Jeromy committed
387
				return &dhtQueryResult{success: true}, nil
388 389 390
			}
		}

Jeromy's avatar
Jeromy committed
391 392
		// Give closer peers back to the query to be queried
		closer := pmes.GetCloserPeers()
393
		clpeers := pb.PBPeersToPeerInfos(closer)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
394
		log.Debugf("got closer peers: %d %s", len(clpeers), clpeers)
395

Jeromy's avatar
Jeromy committed
396
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
397 398
			Type:      notif.PeerResponse,
			ID:        p,
Jeromy's avatar
Jeromy committed
399
			Responses: clpeers,
400
		})
Jeromy's avatar
Jeromy committed
401 402 403
		return &dhtQueryResult{closerPeers: clpeers}, nil
	})

Jeromy's avatar
Jeromy committed
404
	peers := dht.routingTable.NearestPeers(kb.ConvertKey(key.KeyString()), AlphaValue)
Jeromy's avatar
Jeromy committed
405 406
	_, err := query.Run(ctx, peers)
	if err != nil {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
407
		log.Debugf("Query error: %s", err)
408 409 410 411 412 413 414 415 416 417
		// Special handling for issue: https://github.com/ipfs/go-ipfs/issues/3032
		if fmt.Sprint(err) == "<nil>" {
			log.Error("reproduced bug 3032:")
			log.Errorf("Errors type information: %#v", err)
			log.Errorf("go version: %s", runtime.Version())
			log.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("<nil>")
		}
418 419 420 421
		notif.PublishQueryEvent(ctx, &notif.QueryEvent{
			Type:  notif.QueryError,
			Extra: err.Error(),
		})
Jeromy's avatar
Jeromy committed
422
	}
423 424
}

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
425
// FindPeer searches for a peer with given ID.
426 427 428 429 430 431 432 433
func (dht *IpfsDHT) FindPeer(ctx context.Context, id peer.ID) (_ pstore.PeerInfo, err error) {
	eip := log.EventBegin(ctx, "FindPeer", id)
	defer func() {
		if err != nil {
			eip.SetError(err)
		}
		eip.Done()
	}()
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
434

435
	// Check if were already connected to them
Jeromy's avatar
Jeromy committed
436
	if pi := dht.FindLocal(id); pi.ID != "" {
437
		return pi, nil
438 439
	}

Jeromy's avatar
Jeromy committed
440
	peers := dht.routingTable.NearestPeers(kb.ConvertPeerID(id), AlphaValue)
441
	if len(peers) == 0 {
Jeromy's avatar
Jeromy committed
442
		return pstore.PeerInfo{}, kb.ErrLookupFailure
443
	}
444

Jeromy's avatar
Jeromy committed
445
	// Sanity...
446
	for _, p := range peers {
447
		if p == id {
448
			log.Debug("found target peer in list of closest peers...")
449
			return dht.peerstore.PeerInfo(p), nil
450
		}
451
	}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
452

Jeromy's avatar
Jeromy committed
453
	// setup the Query
Jeromy's avatar
Jeromy committed
454
	parent := ctx
455
	query := dht.newQuery(string(id), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) {
Jeromy's avatar
Jeromy committed
456
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
457 458 459
			Type: notif.SendingQuery,
			ID:   p,
		})
Jeromy's avatar
Jeromy committed
460

461
		pmes, err := dht.findPeerSingle(ctx, p, id)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
462
		if err != nil {
463
			return nil, err
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
464
		}
465

Jeromy's avatar
Jeromy committed
466
		closer := pmes.GetCloserPeers()
467
		clpeerInfos := pb.PBPeersToPeerInfos(closer)
468

469
		// see if we got the peer here
470 471
		for _, npi := range clpeerInfos {
			if npi.ID == id {
Jeromy's avatar
Jeromy committed
472
				return &dhtQueryResult{
473
					peer:    npi,
Jeromy's avatar
Jeromy committed
474 475 476
					success: true,
				}, nil
			}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
477 478
		}

Jeromy's avatar
Jeromy committed
479
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
480
			Type:      notif.PeerResponse,
481
			ID:        p,
Jeromy's avatar
Jeromy committed
482
			Responses: clpeerInfos,
483 484
		})

485
		return &dhtQueryResult{closerPeers: clpeerInfos}, nil
486
	})
487

Jeromy's avatar
Jeromy committed
488
	// run it!
489
	result, err := query.Run(ctx, peers)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
490
	if err != nil {
Jeromy's avatar
Jeromy committed
491
		return pstore.PeerInfo{}, err
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
492 493
	}

494
	log.Debugf("FindPeer %v %v", id, result.success)
495
	if result.peer.ID == "" {
Jeromy's avatar
Jeromy committed
496
		return pstore.PeerInfo{}, routing.ErrNotFound
497
	}
Jeromy's avatar
Jeromy committed
498

Jeromy's avatar
Jeromy committed
499
	return *result.peer, nil
500 501
}

502
// FindPeersConnectedToPeer searches for peers directly connected to a given peer.
Jeromy's avatar
Jeromy committed
503
func (dht *IpfsDHT) FindPeersConnectedToPeer(ctx context.Context, id peer.ID) (<-chan *pstore.PeerInfo, error) {
504

Jeromy's avatar
Jeromy committed
505
	peerchan := make(chan *pstore.PeerInfo, asyncQueryBuffer)
Jeromy's avatar
Jeromy committed
506
	peersSeen := make(map[peer.ID]struct{})
507
	var peersSeenMx sync.Mutex
508

Jeromy's avatar
Jeromy committed
509
	peers := dht.routingTable.NearestPeers(kb.ConvertPeerID(id), AlphaValue)
510
	if len(peers) == 0 {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
511
		return nil, kb.ErrLookupFailure
512 513 514
	}

	// setup the Query
515
	query := dht.newQuery(string(id), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) {
516

517
		pmes, err := dht.findPeerSingle(ctx, p, id)
518 519 520 521
		if err != nil {
			return nil, err
		}

Jeromy's avatar
Jeromy committed
522
		var clpeers []*pstore.PeerInfo
523 524
		closer := pmes.GetCloserPeers()
		for _, pbp := range closer {
525
			pi := pb.PBPeerToPeerInfo(pbp)
526

527
			// skip peers already seen
528
			peersSeenMx.Lock()
529
			if _, found := peersSeen[pi.ID]; found {
530
				peersSeenMx.Unlock()
531 532
				continue
			}
533
			peersSeen[pi.ID] = struct{}{}
534
			peersSeenMx.Unlock()
535 536 537 538 539 540

			// if peer is connected, send it to our client.
			if pb.Connectedness(*pbp.Connection) == inet.Connected {
				select {
				case <-ctx.Done():
					return nil, ctx.Err()
541
				case peerchan <- pi:
542 543 544 545
				}
			}

			// if peer is the peer we're looking for, don't bother querying it.
546
			// TODO maybe query it?
547
			if pb.Connectedness(*pbp.Connection) != inet.Connected {
548
				clpeers = append(clpeers, pi)
549 550 551 552 553 554 555 556 557
			}
		}

		return &dhtQueryResult{closerPeers: clpeers}, nil
	})

	// run it! run it asynchronously to gen peers as results are found.
	// this does no error checking
	go func() {
558
		if _, err := query.Run(ctx, peers); err != nil {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
559
			log.Debug(err)
560 561 562 563 564 565 566 567
		}

		// close the peerchan channel when done.
		close(peerchan)
	}()

	return peerchan, nil
}