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

3
import (
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
4
	"bytes"
Jeromy's avatar
Jeromy committed
5
	"context"
6
	"errors"
7
	"fmt"
8 9
	"sync"
	"time"
10

11 12 13 14 15 16 17
	"github.com/libp2p/go-libp2p-core/host"
	"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"

18
	"go.opencensus.io/tag"
19

20
	"github.com/libp2p/go-libp2p-kad-dht/metrics"
21 22
	opts "github.com/libp2p/go-libp2p-kad-dht/opts"
	pb "github.com/libp2p/go-libp2p-kad-dht/pb"
Aarsh Shah's avatar
Aarsh Shah committed
23
	"github.com/libp2p/go-libp2p-kad-dht/providers"
24

Aarsh Shah's avatar
Aarsh Shah committed
25
	"github.com/gogo/protobuf/proto"
26 27
	ds "github.com/ipfs/go-datastore"
	logging "github.com/ipfs/go-log"
Aarsh Shah's avatar
Aarsh Shah committed
28
	"github.com/jbenet/goprocess"
Henrique Dias's avatar
Henrique Dias committed
29
	goprocessctx "github.com/jbenet/goprocess/context"
George Antoniadis's avatar
George Antoniadis committed
30
	kb "github.com/libp2p/go-libp2p-kbucket"
Henrique Dias's avatar
Henrique Dias committed
31
	record "github.com/libp2p/go-libp2p-record"
George Antoniadis's avatar
George Antoniadis committed
32
	recpb "github.com/libp2p/go-libp2p-record/pb"
Steven Allen's avatar
Steven Allen committed
33
	"github.com/multiformats/go-base32"
Adin Schmahmann's avatar
Adin Schmahmann committed
34
	"github.com/multiformats/go-multihash"
35 36
)

Matt Joiner's avatar
Matt Joiner committed
37
var logger = logging.Logger("dht")
38

Henrique Dias's avatar
Henrique Dias committed
39 40
const BaseConnMgrScore = 5

41
// IpfsDHT is an implementation of Kademlia with S/Kademlia modifications.
42
// It is used to implement the base Routing module.
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
43
type IpfsDHT struct {
44 45 46
	host      host.Host           // the network services we need
	self      peer.ID             // Local peer (yourself)
	peerstore peerstore.Peerstore // Peer Registry
47

48
	datastore ds.Datastore // Local data
49

50
	routingTable *kb.RoutingTable // Array of routing tables for differently distanced nodes
51 52
	// ProviderManager stores & manages the provider records for this Dht peer.
	ProviderManager *providers.ProviderManager
53

Jeromy's avatar
Jeromy committed
54
	birth time.Time // When this peer started up
55

56
	Validator record.Validator
57

58 59
	ctx  context.Context
	proc goprocess.Process
60 61 62

	strmap map[peer.ID]*messageSender
	smlk   sync.Mutex
63

Steven Allen's avatar
Steven Allen committed
64
	plk sync.Mutex
65

66 67
	stripedPutLocks [256]sync.Mutex

68
	protocols []protocol.ID // DHT protocols
69 70

	bucketSize int
71

72 73 74
	autoRefresh           bool
	rtRefreshQueryTimeout time.Duration
	rtRefreshPeriod       time.Duration
Steven Allen's avatar
Steven Allen committed
75
	triggerRtRefresh      chan chan<- error
Aarsh Shah's avatar
Aarsh Shah committed
76 77

	maxRecordAge time.Duration
78

79 80 81
	// Allows disabling dht subsystems. These should _only_ be set on
	// "forked" DHTs (e.g., DHTs with custom protocols and/or private
	// networks).
82
	enableProviders, enableValues bool
83 84
}

Matt Joiner's avatar
Matt Joiner committed
85 86 87 88
// Assert that IPFS assumptions about interfaces aren't broken. These aren't a
// guarantee, but we can use them to aid refactoring.
var (
	_ routing.ContentRouting = (*IpfsDHT)(nil)
89
	_ routing.Routing        = (*IpfsDHT)(nil)
Matt Joiner's avatar
Matt Joiner committed
90 91 92 93 94
	_ routing.PeerRouting    = (*IpfsDHT)(nil)
	_ routing.PubKeyFetcher  = (*IpfsDHT)(nil)
	_ routing.ValueStore     = (*IpfsDHT)(nil)
)

95 96 97
// 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
98
	cfg.BucketSize = KValue
99 100 101
	if err := cfg.Apply(append([]opts.Option{opts.Defaults}, options...)...); err != nil {
		return nil, err
	}
102
	dht := makeDHT(ctx, h, &cfg)
103 104 105
	dht.autoRefresh = cfg.RoutingTable.AutoRefresh
	dht.rtRefreshPeriod = cfg.RoutingTable.RefreshPeriod
	dht.rtRefreshQueryTimeout = cfg.RoutingTable.RefreshQueryTimeout
106

Aarsh Shah's avatar
Aarsh Shah committed
107
	dht.maxRecordAge = cfg.MaxRecordAge
108 109
	dht.enableProviders = cfg.EnableProviders
	dht.enableValues = cfg.EnableValues
Aarsh Shah's avatar
Aarsh Shah committed
110

111 112 113
	// register for network notifs.
	dht.host.Network().Notify((*netNotifiee)(dht))

114
	dht.proc.AddChild(dht.ProviderManager.Process())
115 116 117
	dht.Validator = cfg.Validator

	if !cfg.Client {
118 119 120
		for _, p := range cfg.Protocols {
			h.SetStreamHandler(p, dht.handleNewStream)
		}
121
	}
122
	dht.startRefreshing()
123 124
	return dht, nil
}
125

126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145
// NewDHT creates a new DHT object with the given peer as the 'local' host.
// 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))
	if err != nil {
		panic(err)
	}
	return dht
}

// NewDHTClient creates a new DHT object with the given peer as the 'local'
// host. IpfsDHT clients initialized with this function will not respond to DHT
// 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))
	if err != nil {
		panic(err)
	}
146 147 148
	return dht
}

149
func makeDHT(ctx context.Context, h host.Host, cfg *opts.Options) *IpfsDHT {
Henrique Dias's avatar
Henrique Dias committed
150
	self := kb.ConvertPeerID(h.ID())
151
	rt := kb.NewRoutingTable(cfg.BucketSize, self, cfg.RoutingTable.LatencyTolerance, h.Peerstore())
152
	cmgr := h.ConnManager()
153

154
	rt.PeerAdded = func(p peer.ID) {
Henrique Dias's avatar
Henrique Dias committed
155
		commonPrefixLen := kb.CommonPrefixLen(self, kb.ConvertPeerID(p))
Henrique Dias's avatar
Henrique Dias committed
156
		cmgr.TagPeer(p, "kbucket", BaseConnMgrScore+commonPrefixLen)
157
	}
158

159 160 161 162
	rt.PeerRemoved = func(p peer.ID) {
		cmgr.UntagPeer(p, "kbucket")
	}

163
	dht := &IpfsDHT{
164
		datastore:        cfg.Datastore,
Aarsh Shah's avatar
Aarsh Shah committed
165 166 167 168 169 170
		self:             h.ID(),
		peerstore:        h.Peerstore(),
		host:             h,
		strmap:           make(map[peer.ID]*messageSender),
		birth:            time.Now(),
		routingTable:     rt,
171 172
		protocols:        cfg.Protocols,
		bucketSize:       cfg.BucketSize,
Steven Allen's avatar
Steven Allen committed
173
		triggerRtRefresh: make(chan chan<- error),
Jeromy's avatar
Jeromy committed
174
	}
175

Aarsh Shah's avatar
Aarsh Shah committed
176 177 178 179 180 181 182 183 184 185 186 187
	// 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 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)

188
	dht.ProviderManager = providers.NewProviderManager(dht.ctx, h.ID(), cfg.Datastore)
189 190

	return dht
Jeromy's avatar
Jeromy committed
191 192
}

Aarsh Shah's avatar
Aarsh Shah committed
193 194 195 196
// 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) {
197
	writeResp := func(errorChan chan error, err error) {
198 199
		select {
		case <-proc.Closing():
200
		case errorChan <- errChan:
201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225
		}
		close(errorChan)
	}

	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 <-proc.Closing():
			return
		}
	}
Aarsh Shah's avatar
Aarsh Shah committed
226
}*/
227

Jeromy's avatar
Jeromy committed
228
// putValueToPeer stores the given key/value pair at the peer 'p'
229
func (dht *IpfsDHT) putValueToPeer(ctx context.Context, p peer.ID, rec *recpb.Record) error {
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
230

231
	pmes := pb.NewMessage(pb.Message_PUT_VALUE, rec.Key, 0)
232
	pmes.Record = rec
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
233
	rpmes, err := dht.sendRequest(ctx, p, pmes)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
234
	if err != nil {
Matt Joiner's avatar
Matt Joiner committed
235
		logger.Debugf("putValueToPeer: %v. (peer: %s, key: %s)", err, p.Pretty(), loggableKey(string(rec.Key)))
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
236 237
		return err
	}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
238

239
	if !bytes.Equal(rpmes.GetRecord().Value, pmes.GetRecord().Value) {
Matt Joiner's avatar
Matt Joiner committed
240
		logger.Warningf("putValueToPeer: value not put correctly. (%v != %v)", pmes, rpmes)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
241 242
		return errors.New("value not put correctly")
	}
gpestana's avatar
gpestana committed
243

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
244
	return nil
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
245 246
}

247 248
var errInvalidRecord = errors.New("received invalid record")

249 250
// getValueOrPeers queries a particular peer p for the value for
// key. It returns either the value or a list of closer peers.
251
// NOTE: It will update the dht's peerstore with any new addresses
252
// it finds for the given peer.
253
func (dht *IpfsDHT) getValueOrPeers(ctx context.Context, p peer.ID, key string) (*recpb.Record, []*peer.AddrInfo, error) {
254

255
	pmes, err := dht.getValueSingle(ctx, p, key)
256
	if err != nil {
257
		return nil, nil, err
258 259
	}

260 261 262
	// Perhaps we were given closer peers
	peers := pb.PBPeersToPeerInfos(pmes.GetCloserPeers())

263
	if record := pmes.GetRecord(); record != nil {
264
		// Success! We were given the value
Matt Joiner's avatar
Matt Joiner committed
265
		logger.Debug("getValueOrPeers: got value")
266

267
		// make sure record is valid.
268
		err = dht.Validator.Validate(string(record.GetKey()), record.GetValue())
269
		if err != nil {
Matt Joiner's avatar
Matt Joiner committed
270
			logger.Info("Received invalid record! (discarded)")
271 272
			// return a sentinal to signify an invalid record was received
			err = errInvalidRecord
George Antoniadis's avatar
George Antoniadis committed
273
			record = new(recpb.Record)
274
		}
275
		return record, peers, err
276
	}
277

278
	if len(peers) > 0 {
Matt Joiner's avatar
Matt Joiner committed
279
		logger.Debug("getValueOrPeers: peers")
280 281 282
		return nil, peers, nil
	}

Matt Joiner's avatar
Matt Joiner committed
283
	logger.Warning("getValueOrPeers: routing.ErrNotFound")
284
	return nil, nil, routing.ErrNotFound
285 286
}

287
// getValueSingle simply performs the get value RPC with the given parameters
288 289 290 291 292 293
func (dht *IpfsDHT) getValueSingle(ctx context.Context, p peer.ID, key string) (*pb.Message, error) {
	meta := logging.LoggableMap{
		"key":  key,
		"peer": p,
	}

Matt Joiner's avatar
Matt Joiner committed
294
	eip := logger.EventBegin(ctx, "getValueSingle", meta)
ForrestWeston's avatar
ForrestWeston committed
295
	defer eip.Done()
296

297
	pmes := pb.NewMessage(pb.Message_GET_VALUE, []byte(key), 0)
298 299 300 301 302
	resp, err := dht.sendRequest(ctx, p, pmes)
	switch err {
	case nil:
		return resp, nil
	case ErrReadTimeout:
Matt Joiner's avatar
Matt Joiner committed
303
		logger.Warningf("getValueSingle: read timeout %s %s", p.Pretty(), key)
304 305
		fallthrough
	default:
ForrestWeston's avatar
ForrestWeston committed
306
		eip.SetError(err)
307 308
		return nil, err
	}
Jeromy's avatar
Jeromy committed
309 310
}

311
// getLocal attempts to retrieve the value from the datastore
312
func (dht *IpfsDHT) getLocal(key string) (*recpb.Record, error) {
Matt Joiner's avatar
Matt Joiner committed
313
	logger.Debugf("getLocal %s", key)
314
	rec, err := dht.getRecordFromDatastore(mkDsKey(key))
315
	if err != nil {
Matt Joiner's avatar
Matt Joiner committed
316
		logger.Warningf("getLocal: %s", err)
317 318
		return nil, err
	}
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
319

320
	// Double check the key. Can't hurt.
321
	if rec != nil && string(rec.GetKey()) != key {
Matt Joiner's avatar
Matt Joiner committed
322
		logger.Errorf("BUG getLocal: found a DHT record that didn't match it's key: %s != %s", rec.GetKey(), key)
Steven Allen's avatar
Steven Allen committed
323
		return nil, nil
324 325

	}
326
	return rec, nil
327 328
}

329
// putLocal stores the key value pair in the datastore
330
func (dht *IpfsDHT) putLocal(key string, rec *recpb.Record) error {
Matt Joiner's avatar
Matt Joiner committed
331
	logger.Debugf("putLocal: %v %v", key, rec)
332 333
	data, err := proto.Marshal(rec)
	if err != nil {
Matt Joiner's avatar
Matt Joiner committed
334
		logger.Warningf("putLocal: %s", err)
335 336 337
		return err
	}

338
	return dht.datastore.Put(mkDsKey(key), data)
339
}
340

341
// Update signals the routingTable to Update its last-seen status
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
342
// on the given peer.
343
func (dht *IpfsDHT) Update(ctx context.Context, p peer.ID) {
Matt Joiner's avatar
Matt Joiner committed
344
	logger.Event(ctx, "updatePeer", p)
345
	dht.routingTable.Update(p)
346
}
Jeromy's avatar
Jeromy committed
347

Jeromy's avatar
Jeromy committed
348
// FindLocal looks for a peer with a given ID connected to this dht and returns the peer and the table it was found in.
349
func (dht *IpfsDHT) FindLocal(id peer.ID) peer.AddrInfo {
350
	switch dht.host.Network().Connectedness(id) {
351
	case network.Connected, network.CanConnect:
352 353
		return dht.peerstore.PeerInfo(id)
	default:
354
		return peer.AddrInfo{}
Jeromy's avatar
Jeromy committed
355 356
	}
}
357

Jeromy's avatar
Jeromy committed
358
// findPeerSingle asks peer 'p' if they know where the peer with id 'id' is
359
func (dht *IpfsDHT) findPeerSingle(ctx context.Context, p peer.ID, id peer.ID) (*pb.Message, error) {
Matt Joiner's avatar
Matt Joiner committed
360
	eip := logger.EventBegin(ctx, "findPeerSingle",
361 362 363 364
		logging.LoggableMap{
			"peer":   p,
			"target": id,
		})
ForrestWeston's avatar
ForrestWeston committed
365
	defer eip.Done()
366

367
	pmes := pb.NewMessage(pb.Message_FIND_NODE, []byte(id), 0)
368 369 370 371 372
	resp, err := dht.sendRequest(ctx, p, pmes)
	switch err {
	case nil:
		return resp, nil
	case ErrReadTimeout:
Matt Joiner's avatar
Matt Joiner committed
373
		logger.Warningf("read timeout: %s %s", p.Pretty(), id)
374 375
		fallthrough
	default:
ForrestWeston's avatar
ForrestWeston committed
376
		eip.SetError(err)
377 378
		return nil, err
	}
379
}
380

Adin Schmahmann's avatar
Adin Schmahmann committed
381 382
func (dht *IpfsDHT) findProvidersSingle(ctx context.Context, p peer.ID, key multihash.Multihash) (*pb.Message, error) {
	eip := logger.EventBegin(ctx, "findProvidersSingle", p, multihashLoggableKey(key))
ForrestWeston's avatar
ForrestWeston committed
383
	defer eip.Done()
384

Adin Schmahmann's avatar
Adin Schmahmann committed
385
	pmes := pb.NewMessage(pb.Message_GET_PROVIDERS, key, 0)
386 387 388 389 390
	resp, err := dht.sendRequest(ctx, p, pmes)
	switch err {
	case nil:
		return resp, nil
	case ErrReadTimeout:
Adin Schmahmann's avatar
Adin Schmahmann committed
391
		logger.Warningf("read timeout: %s %s", p.Pretty(), key)
392 393
		fallthrough
	default:
ForrestWeston's avatar
ForrestWeston committed
394
		eip.SetError(err)
395 396
		return nil, err
	}
Jeromy's avatar
Jeromy committed
397 398
}

399
// nearestPeersToQuery returns the routing tables closest peers.
400
func (dht *IpfsDHT) nearestPeersToQuery(pmes *pb.Message, count int) []peer.ID {
401
	closer := dht.routingTable.NearestPeers(kb.ConvertKey(string(pmes.GetKey())), count)
402 403 404
	return closer
}

Aarsh Shah's avatar
Aarsh Shah committed
405
// betterPeersToQuery returns nearestPeersToQuery with some additional filtering
406
func (dht *IpfsDHT) betterPeersToQuery(pmes *pb.Message, p peer.ID, count int) []peer.ID {
407
	closer := dht.nearestPeersToQuery(pmes, count)
408 409 410

	// no node? nil
	if closer == nil {
Matt Joiner's avatar
Matt Joiner committed
411
		logger.Warning("betterPeersToQuery: no closer peers to send:", p)
412 413 414
		return nil
	}

Steven Allen's avatar
Steven Allen committed
415
	filtered := make([]peer.ID, 0, len(closer))
Jeromy's avatar
Jeromy committed
416 417 418
	for _, clp := range closer {

		// == to self? thats bad
Jeromy's avatar
Jeromy committed
419
		if clp == dht.self {
Matt Joiner's avatar
Matt Joiner committed
420
			logger.Error("BUG betterPeersToQuery: attempted to return self! this shouldn't happen...")
421 422
			return nil
		}
423
		// Dont send a peer back themselves
Jeromy's avatar
Jeromy committed
424
		if clp == p {
425 426 427
			continue
		}

Jeromy's avatar
Jeromy committed
428
		filtered = append(filtered, clp)
429 430
	}

431 432
	// ok seems like closer nodes
	return filtered
433 434
}

435 436 437 438 439 440 441 442 443 444
// Context return dht's context
func (dht *IpfsDHT) Context() context.Context {
	return dht.ctx
}

// Process return dht's process
func (dht *IpfsDHT) Process() goprocess.Process {
	return dht.proc
}

ZhengQi's avatar
ZhengQi committed
445 446 447 448 449
// RoutingTable return dht's routingTable
func (dht *IpfsDHT) RoutingTable() *kb.RoutingTable {
	return dht.routingTable
}

450 451 452 453
// Close calls Process Close
func (dht *IpfsDHT) Close() error {
	return dht.proc.Close()
}
454

455 456
func (dht *IpfsDHT) protocolStrs() []string {
	pstrs := make([]string, len(dht.protocols))
457 458
	for idx, proto := range dht.protocols {
		pstrs[idx] = string(proto)
459 460 461 462 463
	}

	return pstrs
}

464 465 466
func mkDsKey(s string) ds.Key {
	return ds.NewKey(base32.RawStdEncoding.EncodeToString([]byte(s)))
}
467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483

func (dht *IpfsDHT) PeerID() peer.ID {
	return dht.self
}

func (dht *IpfsDHT) PeerKey() []byte {
	return kb.ConvertPeerID(dht.self)
}

func (dht *IpfsDHT) Host() host.Host {
	return dht.host
}

func (dht *IpfsDHT) Ping(ctx context.Context, p peer.ID) error {
	req := pb.NewMessage(pb.Message_PING, nil, 0)
	resp, err := dht.sendRequest(ctx, p, req)
	if err != nil {
Steven Allen's avatar
Steven Allen committed
484
		return fmt.Errorf("sending request: %w", err)
485 486
	}
	if resp.Type != pb.Message_PING {
Steven Allen's avatar
Steven Allen committed
487
		return fmt.Errorf("got unexpected response type: %v", resp.Type)
488 489 490
	}
	return nil
}
491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506

// newContextWithLocalTags returns a new context.Context with the InstanceID and
// PeerID keys populated. It will also take any extra tags that need adding to
// the context as tag.Mutators.
func (dht *IpfsDHT) newContextWithLocalTags(ctx context.Context, extraTags ...tag.Mutator) context.Context {
	extraTags = append(
		extraTags,
		tag.Upsert(metrics.KeyPeerID, dht.self.Pretty()),
		tag.Upsert(metrics.KeyInstanceID, fmt.Sprintf("%p", dht)),
	)
	ctx, _ = tag.New(
		ctx,
		extraTags...,
	) // ignoring error as it is unrelated to the actual function of this code.
	return ctx
}