routing.go 13.8 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
	cid "github.com/ipfs/go-cid"
Jeromy's avatar
Jeromy committed
12
	logging "github.com/ipfs/go-log"
George Antoniadis's avatar
George Antoniadis committed
13 14
	pb "github.com/libp2p/go-libp2p-kad-dht/pb"
	kb "github.com/libp2p/go-libp2p-kbucket"
15 16 17 18
	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
19 20 21
	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
22 23
)

24 25 26 27 28 29
// 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
30 31 32 33 34
// This file implements the Routing interface for the IpfsDHT struct.

// Basic Put/Get

// PutValue adds value corresponding to given Key.
35
// This is the top level "Store" operation of the DHT
36
func (dht *IpfsDHT) PutValue(ctx context.Context, key string, value []byte) error {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
37
	log.Debugf("PutValue %s", key)
Jeromy's avatar
Jeromy committed
38
	sk, err := dht.getOwnPrivateKey()
Jeromy's avatar
Jeromy committed
39 40 41
	if err != nil {
		return err
	}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
42

43 44 45 46 47
	sign, err := dht.Validator.IsSigned(key)
	if err != nil {
		return err
	}

Jeromy's avatar
Jeromy committed
48
	rec, err := record.MakePutRecord(sk, key, value, sign)
Jeromy's avatar
Jeromy committed
49
	if err != nil {
50
		log.Debug("creation of record failed!")
Jeromy's avatar
Jeromy committed
51 52 53
		return err
	}

Jeromy's avatar
Jeromy committed
54
	err = dht.putLocal(key, rec)
55 56 57 58
	if err != nil {
		return err
	}

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

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

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

// GetValue searches for the value corresponding to given Key.
87
func (dht *IpfsDHT) GetValue(ctx context.Context, key string) ([]byte, error) {
Jeromy's avatar
Jeromy committed
88 89 90 91
	ctx, cancel := context.WithTimeout(ctx, time.Minute)
	defer cancel()

	vals, err := dht.GetValues(ctx, key, 16)
92 93 94 95
	if err != nil {
		return nil, err
	}

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

	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
	}

	fixupRec, err := record.MakePutRecord(dht.peerstore.PrivKey(dht.self), key, best, true)
	if err != nil {
		// probably shouldnt actually 'error' here as we have found a value we like,
		// but this call failing probably isnt something we want to ignore
		return nil, err
	}

	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) {
126 127 128 129 130 131 132
				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
133 134
				ctx, cancel := context.WithTimeout(dht.Context(), time.Second*30)
				defer cancel()
135 136 137 138 139 140 141 142 143 144 145
				err := dht.putValueToPeer(ctx, v.From, key, fixupRec)
				if err != nil {
					log.Error("Error correcting DHT entry: ", err)
				}
			}(v)
		}
	}

	return best, nil
}

146
func (dht *IpfsDHT) GetValues(ctx context.Context, key string, nvals int) ([]routing.RecvdVal, error) {
Steven Allen's avatar
Steven Allen committed
147
	vals := make([]routing.RecvdVal, 0, nvals)
148 149
	var valslock sync.Mutex

Jeromy's avatar
Jeromy committed
150
	// If we have it local, dont bother doing an RPC!
151
	lrec, err := dht.getLocal(key)
Jeromy's avatar
Jeromy committed
152
	if err == nil {
153 154
		// TODO: this is tricky, we dont always want to trust our own value
		// what if the authoritative source updated it?
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
155
		log.Debug("have it locally")
156 157 158 159 160 161 162 163 164 165
		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
166 167
	}

168
	// get closest peers in the routing table
Jeromy's avatar
Jeromy committed
169
	rtp := dht.routingTable.NearestPeers(kb.ConvertKey(key), AlphaValue)
170
	log.Debugf("peers in rt: %d %s", len(rtp), rtp)
171
	if len(rtp) == 0 {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
172
		log.Warning("No peers from routing table!")
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
173
		return nil, kb.ErrLookupFailure
174 175
	}

176
	// setup the Query
Jeromy's avatar
Jeromy committed
177
	parent := ctx
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
178
	query := dht.newQuery(key, func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) {
Jeromy's avatar
Jeromy committed
179
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
Jeromy's avatar
Jeromy committed
180 181 182 183
			Type: notif.SendingQuery,
			ID:   p,
		})

184
		rec, peers, err := dht.getValueOrPeers(ctx, p, key)
185 186 187 188 189 190 191 192 193
		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,
			})
194
			return nil, err
195 196 197 198 199
		default:
			return nil, err

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

202 203
		res := &dhtQueryResult{closerPeers: peers}

204
		if rec.GetValue() != nil || err == errInvalidRecord {
205 206 207 208 209 210 211 212 213 214 215 216
			rv := routing.RecvdVal{
				Val:  rec.GetValue(),
				From: p,
			}
			valslock.Lock()
			vals = append(vals, rv)

			// If weve collected enough records, we're done
			if len(vals) >= nvals {
				res.success = true
			}
			valslock.Unlock()
217 218
		}

Jeromy's avatar
Jeromy committed
219
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
Jeromy's avatar
Jeromy committed
220 221
			Type:      notif.PeerResponse,
			ID:        p,
Jeromy's avatar
Jeromy committed
222
			Responses: peers,
Jeromy's avatar
Jeromy committed
223 224
		})

225 226
		return res, nil
	})
227

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
228
	// run it!
229 230 231 232 233
	_, err = query.Run(ctx, rtp)
	if len(vals) == 0 {
		if err != nil {
			return nil, err
		}
234 235
	}

236
	return vals, nil
237

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
238 239 240 241 242
}

// Value provider layer of indirection.
// This is what DSHTs (Coral and MainlineDHT) do to store large values in a DHT.

243
// Provide makes this node announce that it can provide a value for the given key
Jeromy's avatar
Jeromy committed
244 245
func (dht *IpfsDHT) Provide(ctx context.Context, key *cid.Cid, brdcst bool) error {
	defer log.EventBegin(ctx, "provide", key, logging.LoggableMap{"broadcast": brdcst}).Done()
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
246 247

	// add self locally
248
	dht.providers.AddProvider(ctx, key, dht.self)
Jeromy's avatar
Jeromy committed
249 250 251
	if !brdcst {
		return nil
	}
252

253
	peers, err := dht.GetClosestPeers(ctx, key.KeyString())
254 255
	if err != nil {
		return err
256 257
	}

258 259 260 261 262
	mes, err := dht.makeProvRecord(key)
	if err != nil {
		return err
	}

Jeromy's avatar
Jeromy committed
263
	wg := sync.WaitGroup{}
264
	for p := range peers {
Jeromy's avatar
Jeromy committed
265 266 267
		wg.Add(1)
		go func(p peer.ID) {
			defer wg.Done()
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
268
			log.Debugf("putProvider(%s, %s)", key, p)
269
			err := dht.sendMessage(ctx, p, mes)
Jeromy's avatar
Jeromy committed
270
			if err != nil {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
271
				log.Debug(err)
Jeromy's avatar
Jeromy committed
272 273
			}
		}(p)
274
	}
Jeromy's avatar
Jeromy committed
275
	wg.Wait()
276
	return nil
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
277
}
278
func (dht *IpfsDHT) makeProvRecord(skey *cid.Cid) (*pb.Message, error) {
279 280 281 282 283 284 285 286 287 288 289
	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.")
	}

290
	pmes := pb.NewMessage(pb.Message_ADD_PROVIDER, skey.KeyString(), 0)
291 292 293
	pmes.ProviderPeers = pb.RawPeerInfosToPBPeers([]pstore.PeerInfo{pi})
	return pmes, nil
}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
294

Brian Tiger Chow's avatar
Brian Tiger Chow committed
295
// FindProviders searches until the context expires.
296
func (dht *IpfsDHT) FindProviders(ctx context.Context, c *cid.Cid) ([]pstore.PeerInfo, error) {
Jeromy's avatar
Jeromy committed
297
	var providers []pstore.PeerInfo
298
	for p := range dht.FindProvidersAsync(ctx, c, KValue) {
Brian Tiger Chow's avatar
Brian Tiger Chow committed
299 300 301 302 303
		providers = append(providers, p)
	}
	return providers, nil
}

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
304 305 306
// 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.
307 308
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
309
	peerOut := make(chan pstore.PeerInfo, count)
Jeromy's avatar
Jeromy committed
310 311 312 313
	go dht.findProvidersAsyncRoutine(ctx, key, count, peerOut)
	return peerOut
}

314 315
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
316 317
	defer close(peerOut)

Jeromy's avatar
Jeromy committed
318
	ps := pset.NewLimited(count)
Jeromy's avatar
Jeromy committed
319 320
	provs := dht.providers.GetProviders(ctx, key)
	for _, p := range provs {
321
		// NOTE: Assuming that this list of peers is unique
Jeromy's avatar
Jeromy committed
322
		if ps.TryAdd(p) {
Jeromy's avatar
Jeromy committed
323
			pi := dht.peerstore.PeerInfo(p)
Jeromy's avatar
Jeromy committed
324
			select {
Jeromy's avatar
Jeromy committed
325
			case peerOut <- pi:
Jeromy's avatar
Jeromy committed
326 327 328
			case <-ctx.Done():
				return
			}
Jeromy's avatar
Jeromy committed
329
		}
Jeromy's avatar
Jeromy committed
330 331

		// If we have enough peers locally, dont bother with remote RPC
Jeromy's avatar
Jeromy committed
332
		// TODO: is this a DOS vector?
Jeromy's avatar
Jeromy committed
333
		if ps.Size() >= count {
Jeromy's avatar
Jeromy committed
334 335 336 337 338
			return
		}
	}

	// setup the Query
Jeromy's avatar
Jeromy committed
339
	parent := ctx
340
	query := dht.newQuery(key.KeyString(), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) {
Jeromy's avatar
Jeromy committed
341
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
342 343 344
			Type: notif.SendingQuery,
			ID:   p,
		})
345
		pmes, err := dht.findProvidersSingle(ctx, p, key)
Jeromy's avatar
Jeromy committed
346 347 348 349
		if err != nil {
			return nil, err
		}

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
350
		log.Debugf("%d provider entries", len(pmes.GetProviderPeers()))
351
		provs := pb.PBPeersToPeerInfos(pmes.GetProviderPeers())
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
352
		log.Debugf("%d provider entries decoded", len(provs))
Jeromy's avatar
Jeromy committed
353 354 355

		// Add unique providers from request, up to 'count'
		for _, prov := range provs {
356 357 358
			if prov.ID != dht.self {
				dht.peerstore.AddAddrs(prov.ID, prov.Addrs, pstore.TempAddrTTL)
			}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
359
			log.Debugf("got provider: %s", prov)
360
			if ps.TryAdd(prov.ID) {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
361
				log.Debugf("using provider: %s", prov)
Jeromy's avatar
Jeromy committed
362
				select {
Jeromy's avatar
Jeromy committed
363
				case peerOut <- *prov:
Jeromy's avatar
Jeromy committed
364
				case <-ctx.Done():
365
					log.Debug("context timed out sending more providers")
Jeromy's avatar
Jeromy committed
366 367
					return nil, ctx.Err()
				}
368
			}
Jeromy's avatar
Jeromy committed
369
			if ps.Size() >= count {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
370
				log.Debugf("got enough providers (%d/%d)", ps.Size(), count)
Jeromy's avatar
Jeromy committed
371
				return &dhtQueryResult{success: true}, nil
372 373 374
			}
		}

Jeromy's avatar
Jeromy committed
375 376
		// Give closer peers back to the query to be queried
		closer := pmes.GetCloserPeers()
377
		clpeers := pb.PBPeersToPeerInfos(closer)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
378
		log.Debugf("got closer peers: %d %s", len(clpeers), clpeers)
379

Jeromy's avatar
Jeromy committed
380
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
381 382
			Type:      notif.PeerResponse,
			ID:        p,
Jeromy's avatar
Jeromy committed
383
			Responses: clpeers,
384
		})
Jeromy's avatar
Jeromy committed
385 386 387
		return &dhtQueryResult{closerPeers: clpeers}, nil
	})

Jeromy's avatar
Jeromy committed
388
	peers := dht.routingTable.NearestPeers(kb.ConvertKey(key.KeyString()), AlphaValue)
Jeromy's avatar
Jeromy committed
389 390
	_, err := query.Run(ctx, peers)
	if err != nil {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
391
		log.Debugf("Query error: %s", err)
392 393 394 395 396 397 398 399 400 401
		// 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>")
		}
402 403 404 405
		notif.PublishQueryEvent(ctx, &notif.QueryEvent{
			Type:  notif.QueryError,
			Extra: err.Error(),
		})
Jeromy's avatar
Jeromy committed
406
	}
407 408
}

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
409
// FindPeer searches for a peer with given ID.
Jeromy's avatar
Jeromy committed
410
func (dht *IpfsDHT) FindPeer(ctx context.Context, id peer.ID) (pstore.PeerInfo, error) {
Jeromy's avatar
Jeromy committed
411
	defer log.EventBegin(ctx, "FindPeer", id).Done()
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
412

413
	// Check if were already connected to them
Jeromy's avatar
Jeromy committed
414
	if pi := dht.FindLocal(id); pi.ID != "" {
415
		return pi, nil
416 417
	}

Jeromy's avatar
Jeromy committed
418
	peers := dht.routingTable.NearestPeers(kb.ConvertPeerID(id), AlphaValue)
419
	if len(peers) == 0 {
Jeromy's avatar
Jeromy committed
420
		return pstore.PeerInfo{}, kb.ErrLookupFailure
421
	}
422

Jeromy's avatar
Jeromy committed
423
	// Sanity...
424
	for _, p := range peers {
425
		if p == id {
426
			log.Debug("found target peer in list of closest peers...")
427
			return dht.peerstore.PeerInfo(p), nil
428
		}
429
	}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
430

Jeromy's avatar
Jeromy committed
431
	// setup the Query
Jeromy's avatar
Jeromy committed
432
	parent := ctx
433
	query := dht.newQuery(string(id), func(ctx context.Context, p peer.ID) (*dhtQueryResult, error) {
Jeromy's avatar
Jeromy committed
434
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
435 436 437
			Type: notif.SendingQuery,
			ID:   p,
		})
Jeromy's avatar
Jeromy committed
438

439
		pmes, err := dht.findPeerSingle(ctx, p, id)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
440
		if err != nil {
441
			return nil, err
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
442
		}
443

Jeromy's avatar
Jeromy committed
444
		closer := pmes.GetCloserPeers()
445
		clpeerInfos := pb.PBPeersToPeerInfos(closer)
446

447
		// see if we got the peer here
448 449
		for _, npi := range clpeerInfos {
			if npi.ID == id {
Jeromy's avatar
Jeromy committed
450
				return &dhtQueryResult{
451
					peer:    npi,
Jeromy's avatar
Jeromy committed
452 453 454
					success: true,
				}, nil
			}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
455 456
		}

Jeromy's avatar
Jeromy committed
457
		notif.PublishQueryEvent(parent, &notif.QueryEvent{
458
			Type:      notif.PeerResponse,
459
			ID:        p,
Jeromy's avatar
Jeromy committed
460
			Responses: clpeerInfos,
461 462
		})

463
		return &dhtQueryResult{closerPeers: clpeerInfos}, nil
464
	})
465

Jeromy's avatar
Jeromy committed
466
	// run it!
467
	result, err := query.Run(ctx, peers)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
468
	if err != nil {
Jeromy's avatar
Jeromy committed
469
		return pstore.PeerInfo{}, err
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
470 471
	}

472
	log.Debugf("FindPeer %v %v", id, result.success)
473
	if result.peer.ID == "" {
Jeromy's avatar
Jeromy committed
474
		return pstore.PeerInfo{}, routing.ErrNotFound
475
	}
Jeromy's avatar
Jeromy committed
476

Jeromy's avatar
Jeromy committed
477
	return *result.peer, nil
478 479
}

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

Jeromy's avatar
Jeromy committed
483
	peerchan := make(chan *pstore.PeerInfo, asyncQueryBuffer)
Jeromy's avatar
Jeromy committed
484
	peersSeen := make(map[peer.ID]struct{})
485

Jeromy's avatar
Jeromy committed
486
	peers := dht.routingTable.NearestPeers(kb.ConvertPeerID(id), AlphaValue)
487
	if len(peers) == 0 {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
488
		return nil, kb.ErrLookupFailure
489 490 491
	}

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

494
		pmes, err := dht.findPeerSingle(ctx, p, id)
495 496 497 498
		if err != nil {
			return nil, err
		}

Jeromy's avatar
Jeromy committed
499
		var clpeers []*pstore.PeerInfo
500 501
		closer := pmes.GetCloserPeers()
		for _, pbp := range closer {
502
			pi := pb.PBPeerToPeerInfo(pbp)
503

504 505
			// skip peers already seen
			if _, found := peersSeen[pi.ID]; found {
506 507
				continue
			}
508
			peersSeen[pi.ID] = struct{}{}
509 510 511 512 513 514

			// if peer is connected, send it to our client.
			if pb.Connectedness(*pbp.Connection) == inet.Connected {
				select {
				case <-ctx.Done():
					return nil, ctx.Err()
515
				case peerchan <- pi:
516 517 518 519
				}
			}

			// if peer is the peer we're looking for, don't bother querying it.
520
			// TODO maybe query it?
521
			if pb.Connectedness(*pbp.Connection) != inet.Connected {
522
				clpeers = append(clpeers, pi)
523 524 525 526 527 528 529 530 531
			}
		}

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

	// run it! run it asynchronously to gen peers as results are found.
	// this does no error checking
	go func() {
532
		if _, err := query.Run(ctx, peers); err != nil {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
533
			log.Debug(err)
534 535 536 537 538 539 540 541
		}

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

	return peerchan, nil
}