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
autoBootstrap bool
bootstrapTimeout time.Duration
bootstrapPeriod time.Duration
triggerBootstrap chan struct{}
autoRefresh bool
rtRefreshQueryTimeout time.Duration
rtRefreshPeriod time.Duration
triggerRtRefresh chan struct{}
}
// 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
}
dht := makeDHT(ctx, h, cfg.Datastore, cfg.Protocols, cfg.BucketSize)
dht.autoBootstrap = cfg.AutoBootstrap
dht.bootstrapPeriod = cfg.BootstrapPeriod
dht.bootstrapTimeout = cfg.BootstrapTimeout
dht.autoRefresh = cfg.AutoRefresh
dht.rtRefreshPeriod = cfg.RoutingTableRefreshPeriod
dht.rtRefreshQueryTimeout = cfg.RoutingTableRefreshQueryTimeout
// register for network notifs.
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)
}
}
dht.startBootstrapping()
dht.startRefreshing()
return dht, nil
}
@ -163,7 +163,7 @@ func makeDHT(ctx context.Context, h host.Host, dstore ds.Batching, protocols []p
routingTable: rt,
protocols: protocols,
bucketSize: bucketSize,
triggerBootstrap: make(chan struct{}),
triggerRtRefresh: make(chan struct{}),
}
dht.ctx = dht.newContextWithLocalTags(ctx)

View File

@ -13,7 +13,7 @@ import (
var DefaultBootstrapPeers []multiaddr.Multiaddr
var minRTBootstrapThreshold = 4
var minRTRefreshThreshold = 4
func init() {
for _, s := range []string{
@ -39,53 +39,53 @@ func init() {
}
}
// Start the bootstrap worker.
func (dht *IpfsDHT) startBootstrapping() error {
// Start the refresh worker.
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
dht.proc.Go(func(proc process.Process) {
ctx := processctx.OnClosingContext(proc)
scanInterval := time.NewTicker(dht.bootstrapPeriod)
scanInterval := time.NewTicker(dht.rtRefreshPeriod)
defer scanInterval.Stop()
// run bootstrap if option is set
if dht.autoBootstrap {
dht.doBootstrap(ctx)
// refresh if option is set
if dht.autoRefresh {
dht.doRefresh(ctx)
} 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()
}
for {
select {
case <-scanInterval.C:
case <-dht.triggerBootstrap:
logger.Infof("triggering a bootstrap: RT has %d peers", dht.routingTable.Size())
case <-dht.triggerRtRefresh:
logger.Infof("triggering a refresh: RT has %d peers", dht.routingTable.Size())
case <-ctx.Done():
return
}
dht.doBootstrap(ctx)
dht.doRefresh(ctx)
}
})
return nil
}
func (dht *IpfsDHT) doBootstrap(ctx context.Context) {
func (dht *IpfsDHT) doRefresh(ctx context.Context) {
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
func (dht *IpfsDHT) bootstrapBuckets(ctx context.Context) {
// 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) refreshBuckets(ctx context.Context) {
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())
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())
}()
queryCtx, cancel := context.WithTimeout(ctx, dht.bootstrapTimeout)
queryCtx, cancel := context.WithTimeout(ctx, dht.rtRefreshQueryTimeout)
defer cancel()
err := f(queryCtx)
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]
}
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
randPeerInBucket := dht.routingTable.GenRandPeerID(bucketID)
@ -124,22 +124,28 @@ func (dht *IpfsDHT) bootstrapBuckets(ctx context.Context) {
// Traverse the DHT toward the self ID
func (dht *IpfsDHT) selfWalk(ctx context.Context) {
queryCtx, cancel := context.WithTimeout(ctx, dht.bootstrapTimeout)
queryCtx, cancel := context.WithTimeout(ctx, dht.rtRefreshQueryTimeout)
defer cancel()
_, err := dht.FindPeer(queryCtx, dht.self)
if err == routing.ErrNotFound {
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 {
select {
case dht.triggerBootstrap <- struct{}{}:
default:
}
dht.RefreshRoutingTable()
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)),
opts.Client(client),
opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(),
opts.DisableAutoRefresh(),
)
if err != nil {
t.Fatal(err)
@ -191,7 +191,7 @@ func bootstrap(t *testing.T, ctx context.Context, dhts []*IpfsDHT) {
ctx, cancel := context.WithCancel(ctx)
defer cancel()
logger.Debugf("Bootstrapping DHTs...")
logger.Debugf("refreshing DHTs routing tables...")
// tried async. sequential fares much better. compare:
// 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.
for i := range 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() {
t.SkipNow()
}
@ -689,7 +689,7 @@ func TestBootstrap(t *testing.T) {
}
}
func TestBootstrapBelowMinRTThreshold(t *testing.T) {
func TestRefreshBelowMinRTThreshold(t *testing.T) {
ctx := context.Background()
// enable auto bootstrap on A
@ -721,7 +721,7 @@ func TestBootstrapBelowMinRTThreshold(t *testing.T) {
connect(t, ctx, dhtB, dhtC)
// 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
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!")
}
func TestPeriodicBootstrap(t *testing.T) {
func TestPeriodicRefresh(t *testing.T) {
if ci.IsRunning() {
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)
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
@ -1428,7 +1428,7 @@ func TestGetSetPluggedProtocol(t *testing.T) {
opts.Protocols("/esh/dht"),
opts.Client(false),
opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(),
opts.DisableAutoRefresh(),
}
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.Client(false),
opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(),
opts.DisableAutoRefresh(),
}...)
if err != nil {
t.Fatal(err)
@ -1477,7 +1477,7 @@ func TestGetSetPluggedProtocol(t *testing.T) {
opts.Protocols("/lsr/dht"),
opts.Client(false),
opts.NamespacedValidator("v", blankValidator{}),
opts.DisableAutoBootstrap(),
opts.DisableAutoRefresh(),
}...)
if err != nil {
t.Fatal(err)

View File

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

View File

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

View File

@ -27,9 +27,9 @@ type Options struct {
Protocols []protocol.ID
BucketSize int
BootstrapTimeout time.Duration
BootstrapPeriod time.Duration
AutoBootstrap bool
RoutingTableRefreshQueryTimeout time.Duration
RoutingTableRefreshPeriod time.Duration
AutoRefresh bool
}
// 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.Protocols = DefaultProtocols
o.BootstrapTimeout = 10 * time.Second
o.BootstrapPeriod = 1 * time.Hour
o.AutoBootstrap = true
o.RoutingTableRefreshQueryTimeout = 10 * time.Second
o.RoutingTableRefreshPeriod = 1 * time.Hour
o.AutoRefresh = true
return nil
}
// BootstrapTimeout sets the timeout for bootstrap queries.
func BootstrapTimeout(timeout time.Duration) Option {
// RoutingTableRefreshQueryTimeout sets the timeout for routing table refresh
// queries.
func RoutingTableRefreshQueryTimeout(timeout time.Duration) Option {
return func(o *Options) error {
o.BootstrapTimeout = timeout
o.RoutingTableRefreshQueryTimeout = timeout
return nil
}
}
// BootstrapPeriod sets the period for bootstrapping. The DHT will bootstrap
// every bootstrap period by:
// 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 bootstrap period.
func BootstrapPeriod(period time.Duration) Option {
// the last refresh period.
func RoutingTableRefreshPeriod(period time.Duration) Option {
return func(o *Options) error {
o.BootstrapPeriod = period
o.RoutingTableRefreshPeriod = period
return nil
}
}
@ -150,12 +151,12 @@ func BucketSize(bucketSize int) Option {
}
}
// DisableAutoBootstrap completely disables 'auto-bootstrap' on the Dht
// This means that neither will we do periodic bootstrap nor will we
// bootstrap the Dht even if the Routing Table size goes below the minimum threshold
func DisableAutoBootstrap() Option {
// 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.AutoBootstrap = false
o.AutoRefresh = false
return nil
}
}