chore: rename bootstrap to refresh

As pointed out by raul, bootstrapping and refreshing are not the same thing.
Bootstrapping is the initial setup (i.e., connect to some initial nodes to get
started). Refreshing is the process of refreshing the routing table.
This commit is contained in:
Steven Allen 2019-11-05 23:28:12 +00:00
parent 71c05a8cd0
commit ef319676a9
6 changed files with 83 additions and 76 deletions

18
dht.go
View File

@ -67,10 +67,10 @@ type IpfsDHT struct {
bucketSize int bucketSize int
autoBootstrap bool autoRefresh bool
bootstrapTimeout time.Duration rtRefreshQueryTimeout time.Duration
bootstrapPeriod time.Duration rtRefreshPeriod time.Duration
triggerBootstrap chan struct{} triggerRtRefresh chan struct{}
} }
// Assert that IPFS assumptions about interfaces aren't broken. These aren't a // Assert that IPFS assumptions about interfaces aren't broken. These aren't a
@ -91,9 +91,9 @@ func New(ctx context.Context, h host.Host, options ...opts.Option) (*IpfsDHT, er
return nil, err return nil, err
} }
dht := makeDHT(ctx, h, cfg.Datastore, cfg.Protocols, cfg.BucketSize) dht := makeDHT(ctx, h, cfg.Datastore, cfg.Protocols, cfg.BucketSize)
dht.autoBootstrap = cfg.AutoBootstrap dht.autoRefresh = cfg.AutoRefresh
dht.bootstrapPeriod = cfg.BootstrapPeriod dht.rtRefreshPeriod = cfg.RoutingTableRefreshPeriod
dht.bootstrapTimeout = cfg.BootstrapTimeout dht.rtRefreshQueryTimeout = cfg.RoutingTableRefreshQueryTimeout
// register for network notifs. // register for network notifs.
dht.host.Network().Notify((*netNotifiee)(dht)) dht.host.Network().Notify((*netNotifiee)(dht))
@ -112,7 +112,7 @@ func New(ctx context.Context, h host.Host, options ...opts.Option) (*IpfsDHT, er
h.SetStreamHandler(p, dht.handleNewStream) h.SetStreamHandler(p, dht.handleNewStream)
} }
} }
dht.startBootstrapping() dht.startRefreshing()
return dht, nil return dht, nil
} }
@ -163,7 +163,7 @@ func makeDHT(ctx context.Context, h host.Host, dstore ds.Batching, protocols []p
routingTable: rt, routingTable: rt,
protocols: protocols, protocols: protocols,
bucketSize: bucketSize, bucketSize: bucketSize,
triggerBootstrap: make(chan struct{}), triggerRtRefresh: make(chan struct{}),
} }
dht.ctx = dht.newContextWithLocalTags(ctx) dht.ctx = dht.newContextWithLocalTags(ctx)

View File

@ -13,7 +13,7 @@ import (
var DefaultBootstrapPeers []multiaddr.Multiaddr var DefaultBootstrapPeers []multiaddr.Multiaddr
var minRTBootstrapThreshold = 4 var minRTRefreshThreshold = 4
func init() { func init() {
for _, s := range []string{ for _, s := range []string{
@ -39,53 +39,53 @@ func init() {
} }
} }
// Start the bootstrap worker. // Start the refresh worker.
func (dht *IpfsDHT) startBootstrapping() error { func (dht *IpfsDHT) startRefreshing() error {
// scan the RT table periodically & do a random walk on k-buckets that haven't been queried since the given bucket period // scan the RT table periodically & do a random walk on k-buckets that haven't been queried since the given bucket period
dht.proc.Go(func(proc process.Process) { dht.proc.Go(func(proc process.Process) {
ctx := processctx.OnClosingContext(proc) ctx := processctx.OnClosingContext(proc)
scanInterval := time.NewTicker(dht.bootstrapPeriod) scanInterval := time.NewTicker(dht.rtRefreshPeriod)
defer scanInterval.Stop() defer scanInterval.Stop()
// run bootstrap if option is set // refresh if option is set
if dht.autoBootstrap { if dht.autoRefresh {
dht.doBootstrap(ctx) dht.doRefresh(ctx)
} else { } else {
// disable the "auto-bootstrap" ticker so that no more ticks are sent to this channel // disable the "auto-refresh" ticker so that no more ticks are sent to this channel
scanInterval.Stop() scanInterval.Stop()
} }
for { for {
select { select {
case <-scanInterval.C: case <-scanInterval.C:
case <-dht.triggerBootstrap: case <-dht.triggerRtRefresh:
logger.Infof("triggering a bootstrap: RT has %d peers", dht.routingTable.Size()) logger.Infof("triggering a refresh: RT has %d peers", dht.routingTable.Size())
case <-ctx.Done(): case <-ctx.Done():
return return
} }
dht.doBootstrap(ctx) dht.doRefresh(ctx)
} }
}) })
return nil return nil
} }
func (dht *IpfsDHT) doBootstrap(ctx context.Context) { func (dht *IpfsDHT) doRefresh(ctx context.Context) {
dht.selfWalk(ctx) dht.selfWalk(ctx)
dht.bootstrapBuckets(ctx) dht.refreshBuckets(ctx)
} }
// bootstrapBuckets scans the routing table, and does a random walk on k-buckets that haven't been queried since the given bucket period // refreshBuckets scans the routing table, and does a random walk on k-buckets that haven't been queried since the given bucket period
func (dht *IpfsDHT) bootstrapBuckets(ctx context.Context) { func (dht *IpfsDHT) refreshBuckets(ctx context.Context) {
doQuery := func(bucketId int, target string, f func(context.Context) error) error { doQuery := func(bucketId int, target string, f func(context.Context) error) error {
logger.Infof("starting bootstrap query for bucket %d to %s (routing table size was %d)", logger.Infof("starting refreshing bucket %d to %s (routing table size was %d)",
bucketId, target, dht.routingTable.Size()) bucketId, target, dht.routingTable.Size())
defer func() { defer func() {
logger.Infof("finished bootstrap query for bucket %d to %s (routing table size is now %d)", logger.Infof("finished refreshing bucket %d to %s (routing table size is now %d)",
bucketId, target, dht.routingTable.Size()) bucketId, target, dht.routingTable.Size())
}() }()
queryCtx, cancel := context.WithTimeout(ctx, dht.bootstrapTimeout) queryCtx, cancel := context.WithTimeout(ctx, dht.rtRefreshQueryTimeout)
defer cancel() defer cancel()
err := f(queryCtx) err := f(queryCtx)
if err == context.DeadlineExceeded && queryCtx.Err() == context.DeadlineExceeded && ctx.Err() == nil { if err == context.DeadlineExceeded && queryCtx.Err() == context.DeadlineExceeded && ctx.Err() == nil {
@ -102,7 +102,7 @@ func (dht *IpfsDHT) bootstrapBuckets(ctx context.Context) {
buckets = buckets[:16] buckets = buckets[:16]
} }
for bucketID, bucket := range buckets { for bucketID, bucket := range buckets {
if time.Since(bucket.RefreshedAt()) > dht.bootstrapPeriod { if time.Since(bucket.RefreshedAt()) > dht.rtRefreshPeriod {
// gen rand peer in the bucket // gen rand peer in the bucket
randPeerInBucket := dht.routingTable.GenRandPeerID(bucketID) randPeerInBucket := dht.routingTable.GenRandPeerID(bucketID)
@ -124,22 +124,28 @@ func (dht *IpfsDHT) bootstrapBuckets(ctx context.Context) {
// Traverse the DHT toward the self ID // Traverse the DHT toward the self ID
func (dht *IpfsDHT) selfWalk(ctx context.Context) { func (dht *IpfsDHT) selfWalk(ctx context.Context) {
queryCtx, cancel := context.WithTimeout(ctx, dht.bootstrapTimeout) queryCtx, cancel := context.WithTimeout(ctx, dht.rtRefreshQueryTimeout)
defer cancel() defer cancel()
_, err := dht.FindPeer(queryCtx, dht.self) _, err := dht.FindPeer(queryCtx, dht.self)
if err == routing.ErrNotFound { if err == routing.ErrNotFound {
return return
} }
logger.Warningf("failed to bootstrap self: %s", err) logger.Warningf("failed to query self during routing table refresh: %s", err)
} }
// Bootstrap tells the DHT to get into a bootstrapped state. // Bootstrap tells the DHT to get into a bootstrapped state satisfying the
// IpfsRouter interface.
// //
// Note: the context is ignored. // This just calls `RefreshRoutingTable`.
func (dht *IpfsDHT) Bootstrap(_ context.Context) error { func (dht *IpfsDHT) Bootstrap(_ context.Context) error {
select { dht.RefreshRoutingTable()
case dht.triggerBootstrap <- struct{}{}:
default:
}
return nil return nil
} }
// RefreshRoutingTable tells the DHT to refresh it's routing tables.
func (dht *IpfsDHT) RefreshRoutingTable() {
select {
case dht.triggerRtRefresh <- struct{}{}:
default:
}
}

View File

@ -113,7 +113,7 @@ func setupDHT(ctx context.Context, t *testing.T, client bool) *IpfsDHT {
bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)),
opts.Client(client), opts.Client(client),
opts.NamespacedValidator("v", blankValidator{}), opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(), opts.DisableAutoRefresh(),
) )
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
@ -191,7 +191,7 @@ func bootstrap(t *testing.T, ctx context.Context, dhts []*IpfsDHT) {
ctx, cancel := context.WithCancel(ctx) ctx, cancel := context.WithCancel(ctx)
defer cancel() defer cancel()
logger.Debugf("Bootstrapping DHTs...") logger.Debugf("refreshing DHTs routing tables...")
// tried async. sequential fares much better. compare: // tried async. sequential fares much better. compare:
// 100 async https://gist.github.com/jbenet/56d12f0578d5f34810b2 // 100 async https://gist.github.com/jbenet/56d12f0578d5f34810b2
@ -201,7 +201,7 @@ func bootstrap(t *testing.T, ctx context.Context, dhts []*IpfsDHT) {
start := rand.Intn(len(dhts)) // randomize to decrease bias. start := rand.Intn(len(dhts)) // randomize to decrease bias.
for i := range dhts { for i := range dhts {
dht := dhts[(start+i)%len(dhts)] dht := dhts[(start+i)%len(dhts)]
dht.Bootstrap(ctx) dht.RefreshRoutingTable()
} }
} }
@ -639,7 +639,7 @@ func printRoutingTables(dhts []*IpfsDHT) {
} }
} }
func TestBootstrap(t *testing.T) { func TestRefresh(t *testing.T) {
if testing.Short() { if testing.Short() {
t.SkipNow() t.SkipNow()
} }
@ -689,7 +689,7 @@ func TestBootstrap(t *testing.T) {
} }
} }
func TestBootstrapBelowMinRTThreshold(t *testing.T) { func TestRefreshBelowMinRTThreshold(t *testing.T) {
ctx := context.Background() ctx := context.Background()
// enable auto bootstrap on A // enable auto bootstrap on A
@ -721,7 +721,7 @@ func TestBootstrapBelowMinRTThreshold(t *testing.T) {
connect(t, ctx, dhtB, dhtC) connect(t, ctx, dhtB, dhtC)
// we ONLY init bootstrap on A // we ONLY init bootstrap on A
dhtA.Bootstrap(ctx) dhtA.RefreshRoutingTable()
// and wait for one round to complete i.e. A should be connected to both B & C // and wait for one round to complete i.e. A should be connected to both B & C
waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 2, 2, 20*time.Second) waitForWellFormedTables(t, []*IpfsDHT{dhtA}, 2, 2, 20*time.Second)
@ -749,7 +749,7 @@ func TestBootstrapBelowMinRTThreshold(t *testing.T) {
assert.Equal(t, dhtE.self, dhtA.routingTable.Find(dhtE.self), "A's routing table should have peer E!") assert.Equal(t, dhtE.self, dhtA.routingTable.Find(dhtE.self), "A's routing table should have peer E!")
} }
func TestPeriodicBootstrap(t *testing.T) { func TestPeriodicRefresh(t *testing.T) {
if ci.IsRunning() { if ci.IsRunning() {
t.Skip("skipping on CI. highly timing dependent") t.Skip("skipping on CI. highly timing dependent")
} }
@ -795,7 +795,7 @@ func TestPeriodicBootstrap(t *testing.T) {
t.Logf("bootstrapping them so they find each other. %d", nDHTs) t.Logf("bootstrapping them so they find each other. %d", nDHTs)
for _, dht := range dhts { for _, dht := range dhts {
go dht.Bootstrap(ctx) dht.RefreshRoutingTable()
} }
// this is async, and we dont know when it's finished with one cycle, so keep checking // this is async, and we dont know when it's finished with one cycle, so keep checking
@ -1428,7 +1428,7 @@ func TestGetSetPluggedProtocol(t *testing.T) {
opts.Protocols("/esh/dht"), opts.Protocols("/esh/dht"),
opts.Client(false), opts.Client(false),
opts.NamespacedValidator("v", blankValidator{}), opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(), opts.DisableAutoRefresh(),
} }
dhtA, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), os...) dhtA, err := New(ctx, bhost.New(swarmt.GenSwarm(t, ctx, swarmt.OptDisableReuseport)), os...)
@ -1467,7 +1467,7 @@ func TestGetSetPluggedProtocol(t *testing.T) {
opts.Protocols("/esh/dht"), opts.Protocols("/esh/dht"),
opts.Client(false), opts.Client(false),
opts.NamespacedValidator("v", blankValidator{}), opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(), opts.DisableAutoRefresh(),
}...) }...)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
@ -1477,7 +1477,7 @@ func TestGetSetPluggedProtocol(t *testing.T) {
opts.Protocols("/lsr/dht"), opts.Protocols("/lsr/dht"),
opts.Client(false), opts.Client(false),
opts.NamespacedValidator("v", blankValidator{}), opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(), opts.DisableAutoRefresh(),
}...) }...)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)

View File

@ -30,7 +30,7 @@ func TestGetFailures(t *testing.T) {
} }
hosts := mn.Hosts() hosts := mn.Hosts()
os := []opts.Option{opts.DisableAutoBootstrap()} os := []opts.Option{opts.DisableAutoRefresh()}
d, err := New(ctx, hosts[0], os...) d, err := New(ctx, hosts[0], os...)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
@ -152,7 +152,7 @@ func TestNotFound(t *testing.T) {
} }
hosts := mn.Hosts() hosts := mn.Hosts()
os := []opts.Option{opts.DisableAutoBootstrap()} os := []opts.Option{opts.DisableAutoRefresh()}
d, err := New(ctx, hosts[0], os...) d, err := New(ctx, hosts[0], os...)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
@ -232,7 +232,7 @@ func TestLessThanKResponses(t *testing.T) {
} }
hosts := mn.Hosts() hosts := mn.Hosts()
os := []opts.Option{opts.DisableAutoBootstrap()} os := []opts.Option{opts.DisableAutoRefresh()}
d, err := New(ctx, hosts[0], os...) d, err := New(ctx, hosts[0], os...)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
@ -302,7 +302,7 @@ func TestMultipleQueries(t *testing.T) {
t.Fatal(err) t.Fatal(err)
} }
hosts := mn.Hosts() hosts := mn.Hosts()
os := []opts.Option{opts.DisableAutoBootstrap()} os := []opts.Option{opts.DisableAutoRefresh()}
d, err := New(ctx, hosts[0], os...) d, err := New(ctx, hosts[0], os...)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)

View File

@ -32,11 +32,11 @@ func (nn *netNotifiee) Connected(n network.Network, v network.Conn) {
dht.plk.Lock() dht.plk.Lock()
defer dht.plk.Unlock() defer dht.plk.Unlock()
if dht.host.Network().Connectedness(p) == network.Connected { if dht.host.Network().Connectedness(p) == network.Connected {
bootstrap := dht.routingTable.Size() <= minRTBootstrapThreshold refresh := dht.routingTable.Size() <= minRTRefreshThreshold
dht.Update(dht.Context(), p) dht.Update(dht.Context(), p)
if bootstrap && dht.autoBootstrap { if refresh && dht.autoRefresh {
select { select {
case dht.triggerBootstrap <- struct{}{}: case dht.triggerRtRefresh <- struct{}{}:
default: default:
} }
} }
@ -78,11 +78,11 @@ func (nn *netNotifiee) testConnection(v network.Conn) {
dht.plk.Lock() dht.plk.Lock()
defer dht.plk.Unlock() defer dht.plk.Unlock()
if dht.host.Network().Connectedness(p) == network.Connected { if dht.host.Network().Connectedness(p) == network.Connected {
bootstrap := dht.routingTable.Size() <= minRTBootstrapThreshold refresh := dht.routingTable.Size() <= minRTRefreshThreshold
dht.Update(dht.Context(), p) dht.Update(dht.Context(), p)
if bootstrap && dht.autoBootstrap { if refresh && dht.autoRefresh {
select { select {
case dht.triggerBootstrap <- struct{}{}: case dht.triggerRtRefresh <- struct{}{}:
default: default:
} }
} }

View File

@ -27,9 +27,9 @@ type Options struct {
Protocols []protocol.ID Protocols []protocol.ID
BucketSize int BucketSize int
BootstrapTimeout time.Duration RoutingTableRefreshQueryTimeout time.Duration
BootstrapPeriod time.Duration RoutingTableRefreshPeriod time.Duration
AutoBootstrap bool AutoRefresh bool
} }
// Apply applies the given options to this Option // Apply applies the given options to this Option
@ -54,30 +54,31 @@ var Defaults = func(o *Options) error {
o.Datastore = dssync.MutexWrap(ds.NewMapDatastore()) o.Datastore = dssync.MutexWrap(ds.NewMapDatastore())
o.Protocols = DefaultProtocols o.Protocols = DefaultProtocols
o.BootstrapTimeout = 10 * time.Second o.RoutingTableRefreshQueryTimeout = 10 * time.Second
o.BootstrapPeriod = 1 * time.Hour o.RoutingTableRefreshPeriod = 1 * time.Hour
o.AutoBootstrap = true o.AutoRefresh = true
return nil return nil
} }
// BootstrapTimeout sets the timeout for bootstrap queries. // RoutingTableRefreshQueryTimeout sets the timeout for routing table refresh
func BootstrapTimeout(timeout time.Duration) Option { // queries.
func RoutingTableRefreshQueryTimeout(timeout time.Duration) Option {
return func(o *Options) error { return func(o *Options) error {
o.BootstrapTimeout = timeout o.RoutingTableRefreshQueryTimeout = timeout
return nil return nil
} }
} }
// BootstrapPeriod sets the period for bootstrapping. The DHT will bootstrap // RoutingTableRefreshPeriod sets the period for refreshing buckets in the
// every bootstrap period by: // 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. 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 // 1. Then searching for a random key in each bucket that hasn't been queried in
// the last bootstrap period. // the last refresh period.
func BootstrapPeriod(period time.Duration) Option { func RoutingTableRefreshPeriod(period time.Duration) Option {
return func(o *Options) error { return func(o *Options) error {
o.BootstrapPeriod = period o.RoutingTableRefreshPeriod = period
return nil return nil
} }
} }
@ -150,12 +151,12 @@ func BucketSize(bucketSize int) Option {
} }
} }
// DisableAutoBootstrap completely disables 'auto-bootstrap' on the Dht // DisableAutoRefresh completely disables 'auto-refresh' on the DHT routing
// This means that neither will we do periodic bootstrap nor will we // table. This means that we will neither refresh the routing table periodically
// bootstrap the Dht even if the Routing Table size goes below the minimum threshold // nor when the routing table size goes below the minimum threshold.
func DisableAutoBootstrap() Option { func DisableAutoRefresh() Option {
return func(o *Options) error { return func(o *Options) error {
o.AutoBootstrap = false o.AutoRefresh = false
return nil return nil
} }
} }