mirror of
				https://github.com/libp2p/go-libp2p.git
				synced 2025-10-31 20:02:48 +08:00 
			
		
		
		
	autorelay: refactor relay finder and start autorelay after identify (#2120)
* Refactor relay_finder and start autorelay after identify * Clock fork * Remove multiple timers and use a single rate limiting chan for findNodes * Remove clock fork * Rename * Use scheduledWork.nextAllowedCallToPeerSource.Add(rf.conf.minInterval) * Fix flaky test that relied on time
This commit is contained in:
		| @@ -433,7 +433,9 @@ func (cfg *Config) NewNode() (host.Host, error) { | |||||||
| 		ho = routed.Wrap(h, router) | 		ho = routed.Wrap(h, router) | ||||||
| 	} | 	} | ||||||
| 	if ar != nil { | 	if ar != nil { | ||||||
| 		return autorelay.NewAutoRelayHost(ho, ar), nil | 		arh := autorelay.NewAutoRelayHost(ho, ar) | ||||||
|  | 		arh.Start() | ||||||
|  | 		ho = arh | ||||||
| 	} | 	} | ||||||
| 	return ho, nil | 	return ho, nil | ||||||
| } | } | ||||||
|   | |||||||
| @@ -49,12 +49,15 @@ func NewAutoRelay(bhost *basic.BasicHost, opts ...Option) (*AutoRelay, error) { | |||||||
| 	r.relayFinder = newRelayFinder(bhost, conf.peerSource, &conf) | 	r.relayFinder = newRelayFinder(bhost, conf.peerSource, &conf) | ||||||
| 	bhost.AddrsFactory = r.hostAddrs | 	bhost.AddrsFactory = r.hostAddrs | ||||||
|  |  | ||||||
|  | 	return r, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (r *AutoRelay) Start() { | ||||||
| 	r.refCount.Add(1) | 	r.refCount.Add(1) | ||||||
| 	go func() { | 	go func() { | ||||||
| 		defer r.refCount.Done() | 		defer r.refCount.Done() | ||||||
| 		r.background() | 		r.background() | ||||||
| 	}() | 	}() | ||||||
| 	return r, nil |  | ||||||
| } | } | ||||||
|  |  | ||||||
| func (r *AutoRelay) background() { | func (r *AutoRelay) background() { | ||||||
|   | |||||||
| @@ -2,14 +2,12 @@ package autorelay_test | |||||||
|  |  | ||||||
| import ( | import ( | ||||||
| 	"context" | 	"context" | ||||||
| 	"os" |  | ||||||
| 	"strings" | 	"strings" | ||||||
| 	"sync/atomic" | 	"sync/atomic" | ||||||
| 	"testing" | 	"testing" | ||||||
| 	"time" | 	"time" | ||||||
|  |  | ||||||
| 	"github.com/libp2p/go-libp2p" | 	"github.com/libp2p/go-libp2p" | ||||||
| 	"github.com/libp2p/go-libp2p/core/event" |  | ||||||
| 	"github.com/libp2p/go-libp2p/core/host" | 	"github.com/libp2p/go-libp2p/core/host" | ||||||
| 	"github.com/libp2p/go-libp2p/core/network" | 	"github.com/libp2p/go-libp2p/core/network" | ||||||
| 	"github.com/libp2p/go-libp2p/core/peer" | 	"github.com/libp2p/go-libp2p/core/peer" | ||||||
| @@ -97,7 +95,7 @@ func newRelay(t *testing.T) host.Host { | |||||||
| 			} | 			} | ||||||
| 		} | 		} | ||||||
| 		return false | 		return false | ||||||
| 	}, 500*time.Millisecond, 10*time.Millisecond) | 	}, time.Second, 10*time.Millisecond) | ||||||
| 	return h | 	return h | ||||||
| } | } | ||||||
|  |  | ||||||
| @@ -121,7 +119,7 @@ func TestSingleCandidate(t *testing.T) { | |||||||
| 	) | 	) | ||||||
| 	defer h.Close() | 	defer h.Close() | ||||||
|  |  | ||||||
| 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 3*time.Second, 100*time.Millisecond) | 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 10*time.Second, 100*time.Millisecond) | ||||||
| 	// test that we don't add any more relays | 	// test that we don't add any more relays | ||||||
| 	require.Never(t, func() bool { return numRelays(h) > 1 }, 200*time.Millisecond, 50*time.Millisecond) | 	require.Never(t, func() bool { return numRelays(h) > 1 }, 200*time.Millisecond, 50*time.Millisecond) | ||||||
| 	require.Equal(t, 1, counter, "expected the peer source callback to only have been called once") | 	require.Equal(t, 1, counter, "expected the peer source callback to only have been called once") | ||||||
| @@ -179,7 +177,7 @@ func TestWaitForCandidates(t *testing.T) { | |||||||
| 	r2 := newRelay(t) | 	r2 := newRelay(t) | ||||||
| 	t.Cleanup(func() { r2.Close() }) | 	t.Cleanup(func() { r2.Close() }) | ||||||
| 	peerChan <- peer.AddrInfo{ID: r2.ID(), Addrs: r2.Addrs()} | 	peerChan <- peer.AddrInfo{ID: r2.ID(), Addrs: r2.Addrs()} | ||||||
| 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 3*time.Second, 100*time.Millisecond) | 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 10*time.Second, 100*time.Millisecond) | ||||||
| } | } | ||||||
|  |  | ||||||
| func TestBackoff(t *testing.T) { | func TestBackoff(t *testing.T) { | ||||||
| @@ -225,15 +223,19 @@ func TestBackoff(t *testing.T) { | |||||||
| 	) | 	) | ||||||
| 	defer h.Close() | 	defer h.Close() | ||||||
|  |  | ||||||
| 	require.Eventually(t, func() bool { return reservations.Load() == 1 }, 3*time.Second, 20*time.Millisecond) | 	require.Eventually(t, func() bool { | ||||||
|  | 		return reservations.Load() == 1 | ||||||
|  | 	}, 10*time.Second, 20*time.Millisecond, "reservations load should be 1 was %d", reservations.Load()) | ||||||
| 	// make sure we don't add any relays yet | 	// make sure we don't add any relays yet | ||||||
| 	for i := 0; i < 2; i++ { | 	for i := 0; i < 2; i++ { | ||||||
| 		cl.Add(backoff / 3) | 		cl.Add(backoff / 3) | ||||||
| 		require.Equal(t, 1, int(reservations.Load())) | 		require.Equal(t, 1, int(reservations.Load())) | ||||||
| 	} | 	} | ||||||
| 	cl.Add(backoff / 2) | 	cl.Add(backoff) | ||||||
| 	require.Eventually(t, func() bool { return reservations.Load() == 2 }, 3*time.Second, 20*time.Millisecond) | 	require.Eventually(t, func() bool { | ||||||
| 	require.Less(t, int(counter.Load()), 100) // just make sure we're not busy-looping | 		return reservations.Load() == 2 | ||||||
|  | 	}, 10*time.Second, 100*time.Millisecond, "reservations load should be 2 was %d", reservations.Load()) | ||||||
|  | 	require.Less(t, int(counter.Load()), 300) // just make sure we're not busy-looping | ||||||
| 	require.Equal(t, 2, int(reservations.Load())) | 	require.Equal(t, 2, int(reservations.Load())) | ||||||
| } | } | ||||||
|  |  | ||||||
| @@ -252,7 +254,7 @@ func TestStaticRelays(t *testing.T) { | |||||||
| 	) | 	) | ||||||
| 	defer h.Close() | 	defer h.Close() | ||||||
|  |  | ||||||
| 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 2*time.Second, 50*time.Millisecond) | 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 10*time.Second, 50*time.Millisecond) | ||||||
| } | } | ||||||
|  |  | ||||||
| func TestConnectOnDisconnect(t *testing.T) { | func TestConnectOnDisconnect(t *testing.T) { | ||||||
| @@ -275,7 +277,7 @@ func TestConnectOnDisconnect(t *testing.T) { | |||||||
| 	) | 	) | ||||||
| 	defer h.Close() | 	defer h.Close() | ||||||
|  |  | ||||||
| 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 3*time.Second, 100*time.Millisecond) | 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 10*time.Second, 100*time.Millisecond) | ||||||
| 	relaysInUse := usedRelays(h) | 	relaysInUse := usedRelays(h) | ||||||
| 	require.Len(t, relaysInUse, 1) | 	require.Len(t, relaysInUse, 1) | ||||||
| 	oldRelay := relaysInUse[0] | 	oldRelay := relaysInUse[0] | ||||||
| @@ -286,7 +288,7 @@ func TestConnectOnDisconnect(t *testing.T) { | |||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 3*time.Second, 100*time.Millisecond) | 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 10*time.Second, 100*time.Millisecond) | ||||||
| 	relaysInUse = usedRelays(h) | 	relaysInUse = usedRelays(h) | ||||||
| 	require.Len(t, relaysInUse, 1) | 	require.Len(t, relaysInUse, 1) | ||||||
| 	require.NotEqualf(t, oldRelay, relaysInUse[0], "old relay should not be used again") | 	require.NotEqualf(t, oldRelay, relaysInUse[0], "old relay should not be used again") | ||||||
| @@ -332,28 +334,31 @@ func TestMaxAge(t *testing.T) { | |||||||
| 	) | 	) | ||||||
| 	defer h.Close() | 	defer h.Close() | ||||||
|  |  | ||||||
| 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 3*time.Second, 100*time.Millisecond) | 	require.Eventually(t, func() bool { | ||||||
|  | 		return numRelays(h) > 0 | ||||||
|  | 	}, 10*time.Second, 100*time.Millisecond) | ||||||
| 	relays := usedRelays(h) | 	relays := usedRelays(h) | ||||||
| 	require.Len(t, relays, 1) | 	require.Len(t, relays, 1) | ||||||
|  |  | ||||||
| 	waitFor := 500 * time.Millisecond |  | ||||||
| 	tick := 100 * time.Millisecond |  | ||||||
| 	if os.Getenv("CI") != "" { |  | ||||||
| 		// Only increase the waitFor since we are increasing the mock clock every tick. |  | ||||||
| 		waitFor *= 10 |  | ||||||
| 	} |  | ||||||
| 	require.Eventually(t, func() bool { | 	require.Eventually(t, func() bool { | ||||||
| 		// we don't know exactly when the timer is reset, just advance our timer multiple times if necessary | 		// we don't know exactly when the timer is reset, just advance our timer multiple times if necessary | ||||||
| 		cl.Add(time.Second) | 		cl.Add(30 * time.Second) | ||||||
| 		return len(peerChans) == 0 | 		return len(peerChans) == 0 | ||||||
| 	}, waitFor, tick) | 	}, 10*time.Second, 100*time.Millisecond) | ||||||
|  |  | ||||||
| 	cl.Add(10 * time.Minute) | 	cl.Add(10 * time.Minute) | ||||||
| 	for _, r := range relays2 { | 	for _, r := range relays2 { | ||||||
| 		peerChan2 <- peer.AddrInfo{ID: r.ID(), Addrs: r.Addrs()} | 		peerChan2 <- peer.AddrInfo{ID: r.ID(), Addrs: r.Addrs()} | ||||||
| 	} | 	} | ||||||
| 	cl.Add(11 * time.Minute) | 	cl.Add(11 * time.Minute) | ||||||
|  |  | ||||||
|  | 	require.Eventually(t, func() bool { | ||||||
|  | 		relays = usedRelays(h) | ||||||
|  | 		return len(relays) == 1 | ||||||
|  | 	}, 10*time.Second, 100*time.Millisecond) | ||||||
|  |  | ||||||
| 	// by now the 3 relays should have been garbage collected | 	// by now the 3 relays should have been garbage collected | ||||||
|  | 	// And we should only be using a single relay. Lets close it. | ||||||
| 	var oldRelay peer.ID | 	var oldRelay peer.ID | ||||||
| 	for _, r := range relays1 { | 	for _, r := range relays1 { | ||||||
| 		if r.ID() == relays[0] { | 		if r.ID() == relays[0] { | ||||||
| @@ -369,7 +374,7 @@ func TestMaxAge(t *testing.T) { | |||||||
| 			return false | 			return false | ||||||
| 		} | 		} | ||||||
| 		return relays[0] != oldRelay | 		return relays[0] != oldRelay | ||||||
| 	}, 3*time.Second, 100*time.Millisecond) | 	}, 10*time.Second, 100*time.Millisecond) | ||||||
|  |  | ||||||
| 	require.Len(t, relays, 1) | 	require.Len(t, relays, 1) | ||||||
| 	ids := make([]peer.ID, 0, len(relays2)) | 	ids := make([]peer.ID, 0, len(relays2)) | ||||||
| @@ -379,40 +384,6 @@ func TestMaxAge(t *testing.T) { | |||||||
| 	require.Contains(t, ids, relays[0]) | 	require.Contains(t, ids, relays[0]) | ||||||
| } | } | ||||||
|  |  | ||||||
| func expectDeltaInAddrUpdated(t *testing.T, addrUpdated event.Subscription, expectedDelta int) { |  | ||||||
| 	t.Helper() |  | ||||||
| 	delta := 0 |  | ||||||
| 	for { |  | ||||||
| 		select { |  | ||||||
| 		case evAny := <-addrUpdated.Out(): |  | ||||||
| 			ev := evAny.(event.EvtLocalAddressesUpdated) |  | ||||||
| 			for _, updatedAddr := range ev.Removed { |  | ||||||
| 				if updatedAddr.Action == event.Removed { |  | ||||||
| 					if _, err := updatedAddr.Address.ValueForProtocol(ma.P_CIRCUIT); err == nil { |  | ||||||
| 						delta-- |  | ||||||
| 						if delta == expectedDelta { |  | ||||||
| 							return |  | ||||||
| 						} |  | ||||||
| 					} |  | ||||||
| 				} |  | ||||||
| 			} |  | ||||||
| 			for _, updatedAddr := range ev.Current { |  | ||||||
| 				if updatedAddr.Action == event.Added { |  | ||||||
| 					if _, err := updatedAddr.Address.ValueForProtocol(ma.P_CIRCUIT); err == nil { |  | ||||||
| 						delta++ |  | ||||||
| 						if delta == expectedDelta { |  | ||||||
| 							return |  | ||||||
| 						} |  | ||||||
| 					} |  | ||||||
| 				} |  | ||||||
| 			} |  | ||||||
| 		case <-time.After(10 * time.Second): |  | ||||||
| 			t.Fatal("timeout waiting for address updated event") |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
|  |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func TestReconnectToStaticRelays(t *testing.T) { | func TestReconnectToStaticRelays(t *testing.T) { | ||||||
| 	cl := clock.NewMock() | 	cl := clock.NewMock() | ||||||
| 	var staticRelays []peer.AddrInfo | 	var staticRelays []peer.AddrInfo | ||||||
| @@ -428,16 +399,14 @@ func TestReconnectToStaticRelays(t *testing.T) { | |||||||
| 	h := newPrivateNodeWithStaticRelays(t, | 	h := newPrivateNodeWithStaticRelays(t, | ||||||
| 		staticRelays, | 		staticRelays, | ||||||
| 		autorelay.WithClock(cl), | 		autorelay.WithClock(cl), | ||||||
|  | 		autorelay.WithBackoff(30*time.Minute), | ||||||
| 	) | 	) | ||||||
|  |  | ||||||
| 	defer h.Close() | 	defer h.Close() | ||||||
|  |  | ||||||
| 	addrUpdated, err := h.EventBus().Subscribe(new(event.EvtLocalAddressesUpdated)) |  | ||||||
| 	require.NoError(t, err) |  | ||||||
|  |  | ||||||
| 	expectDeltaInAddrUpdated(t, addrUpdated, 1) |  | ||||||
|  |  | ||||||
| 	cl.Add(time.Minute) | 	cl.Add(time.Minute) | ||||||
|  | 	require.Eventually(t, func() bool { | ||||||
|  | 		return numRelays(h) == 1 | ||||||
|  | 	}, 10*time.Second, 100*time.Millisecond) | ||||||
|  |  | ||||||
| 	relaysInUse := usedRelays(h) | 	relaysInUse := usedRelays(h) | ||||||
| 	oldRelay := relaysInUse[0] | 	oldRelay := relaysInUse[0] | ||||||
| @@ -446,12 +415,18 @@ func TestReconnectToStaticRelays(t *testing.T) { | |||||||
| 			r.Network().ClosePeer(h.ID()) | 			r.Network().ClosePeer(h.ID()) | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  | 	require.Eventually(t, func() bool { | ||||||
|  | 		return numRelays(h) == 0 | ||||||
|  | 	}, 10*time.Second, 100*time.Millisecond) | ||||||
|  |  | ||||||
| 	cl.Add(time.Hour) | 	cl.Add(time.Hour) | ||||||
| 	expectDeltaInAddrUpdated(t, addrUpdated, -1) | 	require.Eventually(t, func() bool { | ||||||
|  | 		return numRelays(h) == 1 | ||||||
|  | 	}, 10*time.Second, 100*time.Millisecond) | ||||||
| } | } | ||||||
|  |  | ||||||
| func TestMinInterval(t *testing.T) { | func TestMinInterval(t *testing.T) { | ||||||
|  | 	cl := clock.NewMock() | ||||||
| 	h := newPrivateNode(t, | 	h := newPrivateNode(t, | ||||||
| 		func(context.Context, int) <-chan peer.AddrInfo { | 		func(context.Context, int) <-chan peer.AddrInfo { | ||||||
| 			peerChan := make(chan peer.AddrInfo, 1) | 			peerChan := make(chan peer.AddrInfo, 1) | ||||||
| @@ -461,6 +436,7 @@ func TestMinInterval(t *testing.T) { | |||||||
| 			peerChan <- peer.AddrInfo{ID: r1.ID(), Addrs: r1.Addrs()} | 			peerChan <- peer.AddrInfo{ID: r1.ID(), Addrs: r1.Addrs()} | ||||||
| 			return peerChan | 			return peerChan | ||||||
| 		}, | 		}, | ||||||
|  | 		autorelay.WithClock(cl), | ||||||
| 		autorelay.WithMinCandidates(2), | 		autorelay.WithMinCandidates(2), | ||||||
| 		autorelay.WithNumRelays(1), | 		autorelay.WithNumRelays(1), | ||||||
| 		autorelay.WithBootDelay(time.Hour), | 		autorelay.WithBootDelay(time.Hour), | ||||||
| @@ -468,7 +444,9 @@ func TestMinInterval(t *testing.T) { | |||||||
| 	) | 	) | ||||||
| 	defer h.Close() | 	defer h.Close() | ||||||
|  |  | ||||||
|  | 	cl.Add(500 * time.Millisecond) | ||||||
| 	// The second call to peerSource should happen after 1 second | 	// The second call to peerSource should happen after 1 second | ||||||
| 	require.Never(t, func() bool { return numRelays(h) > 0 }, 500*time.Millisecond, 100*time.Millisecond) | 	require.Never(t, func() bool { return numRelays(h) > 0 }, 500*time.Millisecond, 100*time.Millisecond) | ||||||
| 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 1*time.Second, 100*time.Millisecond) | 	cl.Add(500 * time.Millisecond) | ||||||
|  | 	require.Eventually(t, func() bool { return numRelays(h) > 0 }, 10*time.Second, 100*time.Millisecond) | ||||||
| } | } | ||||||
|   | |||||||
| @@ -14,6 +14,10 @@ func (h *AutoRelayHost) Close() error { | |||||||
| 	return h.Host.Close() | 	return h.Host.Close() | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func (h *AutoRelayHost) Start() { | ||||||
|  | 	h.ar.Start() | ||||||
|  | } | ||||||
|  |  | ||||||
| func NewAutoRelayHost(h host.Host, ar *AutoRelay) *AutoRelayHost { | func NewAutoRelayHost(h host.Host, ar *AutoRelay) *AutoRelayHost { | ||||||
| 	return &AutoRelayHost{Host: h, ar: ar} | 	return &AutoRelayHost{Host: h, ar: ar} | ||||||
| } | } | ||||||
|   | |||||||
| @@ -99,11 +99,20 @@ func newRelayFinder(host *basic.BasicHost, peerSource PeerSource, conf *config) | |||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
|  | type scheduledWorkTimes struct { | ||||||
|  | 	leastFrequentInterval       time.Duration | ||||||
|  | 	nextRefresh                 time.Time | ||||||
|  | 	nextBackoff                 time.Time | ||||||
|  | 	nextOldCandidateCheck       time.Time | ||||||
|  | 	nextAllowedCallToPeerSource time.Time | ||||||
|  | } | ||||||
|  |  | ||||||
| func (rf *relayFinder) background(ctx context.Context) { | func (rf *relayFinder) background(ctx context.Context) { | ||||||
|  | 	peerSourceRateLimiter := make(chan struct{}, 1) | ||||||
| 	rf.refCount.Add(1) | 	rf.refCount.Add(1) | ||||||
| 	go func() { | 	go func() { | ||||||
| 		defer rf.refCount.Done() | 		defer rf.refCount.Done() | ||||||
| 		rf.findNodes(ctx) | 		rf.findNodes(ctx, peerSourceRateLimiter) | ||||||
| 	}() | 	}() | ||||||
|  |  | ||||||
| 	rf.refCount.Add(1) | 	rf.refCount.Add(1) | ||||||
| @@ -121,17 +130,33 @@ func (rf *relayFinder) background(ctx context.Context) { | |||||||
|  |  | ||||||
| 	bootDelayTimer := rf.conf.clock.Timer(rf.conf.bootDelay) | 	bootDelayTimer := rf.conf.clock.Timer(rf.conf.bootDelay) | ||||||
| 	defer bootDelayTimer.Stop() | 	defer bootDelayTimer.Stop() | ||||||
| 	refreshTicker := rf.conf.clock.Ticker(rsvpRefreshInterval) |  | ||||||
| 	defer refreshTicker.Stop() | 	// This is the least frequent event. It's our fallback timer if we don't have any other work to do. | ||||||
| 	backoffTicker := rf.conf.clock.Ticker(rf.conf.backoff / 5) | 	leastFrequentInterval := rf.conf.minInterval | ||||||
| 	defer backoffTicker.Stop() | 	if rf.conf.backoff > leastFrequentInterval { | ||||||
| 	oldCandidateTicker := rf.conf.clock.Ticker(rf.conf.maxCandidateAge / 5) | 		leastFrequentInterval = rf.conf.backoff | ||||||
| 	defer oldCandidateTicker.Stop() | 	} | ||||||
|  | 	if rf.conf.maxCandidateAge > leastFrequentInterval { | ||||||
|  | 		leastFrequentInterval = rf.conf.maxCandidateAge | ||||||
|  | 	} | ||||||
|  | 	if rsvpRefreshInterval > leastFrequentInterval { | ||||||
|  | 		leastFrequentInterval = rf.conf.maxCandidateAge | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	now := rf.conf.clock.Now() | ||||||
|  |  | ||||||
|  | 	scheduledWork := &scheduledWorkTimes{ | ||||||
|  | 		leastFrequentInterval:       leastFrequentInterval, | ||||||
|  | 		nextRefresh:                 now.Add(rsvpRefreshInterval), | ||||||
|  | 		nextBackoff:                 now.Add(rf.conf.backoff / 5), | ||||||
|  | 		nextOldCandidateCheck:       now.Add(rf.conf.maxCandidateAge / 5), | ||||||
|  | 		nextAllowedCallToPeerSource: now.Add(-time.Second), // allow immediately | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	workTimer := rf.conf.clock.Timer(rf.runScheduledWork(ctx, now, scheduledWork, peerSourceRateLimiter).Sub(now)) | ||||||
|  | 	defer workTimer.Stop() | ||||||
|  |  | ||||||
| 	for { | 	for { | ||||||
| 		// when true, we need to identify push |  | ||||||
| 		var push bool |  | ||||||
|  |  | ||||||
| 		select { | 		select { | ||||||
| 		case ev, ok := <-subConnectedness.Out(): | 		case ev, ok := <-subConnectedness.Out(): | ||||||
| 			if !ok { | 			if !ok { | ||||||
| @@ -141,6 +166,8 @@ func (rf *relayFinder) background(ctx context.Context) { | |||||||
| 			if evt.Connectedness != network.NotConnected { | 			if evt.Connectedness != network.NotConnected { | ||||||
| 				continue | 				continue | ||||||
| 			} | 			} | ||||||
|  | 			push := false | ||||||
|  |  | ||||||
| 			rf.relayMx.Lock() | 			rf.relayMx.Lock() | ||||||
| 			if rf.usingRelay(evt.Peer) { // we were disconnected from a relay | 			if rf.usingRelay(evt.Peer) { // we were disconnected from a relay | ||||||
| 				log.Debugw("disconnected from relay", "id", evt.Peer) | 				log.Debugw("disconnected from relay", "id", evt.Peer) | ||||||
| @@ -150,85 +177,156 @@ func (rf *relayFinder) background(ctx context.Context) { | |||||||
| 				push = true | 				push = true | ||||||
| 			} | 			} | ||||||
| 			rf.relayMx.Unlock() | 			rf.relayMx.Unlock() | ||||||
|  |  | ||||||
|  | 			if push { | ||||||
|  | 				rf.clearCachedAddrsAndSignalAddressChange() | ||||||
|  | 			} | ||||||
| 		case <-rf.candidateFound: | 		case <-rf.candidateFound: | ||||||
| 			rf.notifyMaybeConnectToRelay() | 			rf.notifyMaybeConnectToRelay() | ||||||
| 		case <-bootDelayTimer.C: | 		case <-bootDelayTimer.C: | ||||||
| 			rf.notifyMaybeConnectToRelay() | 			rf.notifyMaybeConnectToRelay() | ||||||
| 		case <-rf.relayUpdated: | 		case <-rf.relayUpdated: | ||||||
| 			push = true | 			rf.clearCachedAddrsAndSignalAddressChange() | ||||||
| 		case now := <-refreshTicker.C: | 		case <-workTimer.C: | ||||||
| 			push = rf.refreshReservations(ctx, now) | 			now := rf.conf.clock.Now() | ||||||
| 		case now := <-backoffTicker.C: | 			nextTime := rf.runScheduledWork(ctx, now, scheduledWork, peerSourceRateLimiter) | ||||||
| 			rf.candidateMx.Lock() | 			workTimer.Reset(nextTime.Sub(now)) | ||||||
| 			for id, t := range rf.backoff { |  | ||||||
| 				if !t.Add(rf.conf.backoff).After(now) { |  | ||||||
| 					log.Debugw("removing backoff for node", "id", id) |  | ||||||
| 					delete(rf.backoff, id) |  | ||||||
| 				} |  | ||||||
| 			} |  | ||||||
| 			rf.candidateMx.Unlock() |  | ||||||
| 		case now := <-oldCandidateTicker.C: |  | ||||||
| 			var deleted bool |  | ||||||
| 			rf.candidateMx.Lock() |  | ||||||
| 			for id, cand := range rf.candidates { |  | ||||||
| 				if !cand.added.Add(rf.conf.maxCandidateAge).After(now) { |  | ||||||
| 					deleted = true |  | ||||||
| 					log.Debugw("deleting candidate due to age", "id", id) |  | ||||||
| 					delete(rf.candidates, id) |  | ||||||
| 				} |  | ||||||
| 			} |  | ||||||
| 			rf.candidateMx.Unlock() |  | ||||||
| 			if deleted { |  | ||||||
| 				rf.notifyMaybeNeedNewCandidates() |  | ||||||
| 			} |  | ||||||
| 		case <-ctx.Done(): | 		case <-ctx.Done(): | ||||||
| 			return | 			return | ||||||
| 		} | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
| 		if push { | func (rf *relayFinder) clearCachedAddrsAndSignalAddressChange() { | ||||||
| 			rf.relayMx.Lock() | 	rf.relayMx.Lock() | ||||||
| 			rf.cachedAddrs = nil | 	rf.cachedAddrs = nil | ||||||
| 			rf.relayMx.Unlock() | 	rf.relayMx.Unlock() | ||||||
| 			rf.host.SignalAddressChange() | 	rf.host.SignalAddressChange() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (rf *relayFinder) runScheduledWork(ctx context.Context, now time.Time, scheduledWork *scheduledWorkTimes, peerSourceRateLimiter chan<- struct{}) time.Time { | ||||||
|  | 	nextTime := now.Add(scheduledWork.leastFrequentInterval) | ||||||
|  |  | ||||||
|  | 	if now.After(scheduledWork.nextRefresh) { | ||||||
|  | 		scheduledWork.nextRefresh = now.Add(rsvpRefreshInterval) | ||||||
|  | 		if rf.refreshReservations(ctx, now) { | ||||||
|  | 			rf.clearCachedAddrsAndSignalAddressChange() | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
|  | 	if now.After(scheduledWork.nextBackoff) { | ||||||
|  | 		scheduledWork.nextBackoff = rf.clearBackoff(now) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	if now.After(scheduledWork.nextOldCandidateCheck) { | ||||||
|  | 		scheduledWork.nextOldCandidateCheck = rf.clearOldCandidates(now) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	if now.After(scheduledWork.nextAllowedCallToPeerSource) { | ||||||
|  | 		scheduledWork.nextAllowedCallToPeerSource = scheduledWork.nextAllowedCallToPeerSource.Add(rf.conf.minInterval) | ||||||
|  | 		select { | ||||||
|  | 		case peerSourceRateLimiter <- struct{}{}: | ||||||
|  | 		default: | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// Find the next time we need to run scheduled work. | ||||||
|  | 	if scheduledWork.nextRefresh.Before(nextTime) { | ||||||
|  | 		nextTime = scheduledWork.nextRefresh | ||||||
|  | 	} | ||||||
|  | 	if scheduledWork.nextBackoff.Before(nextTime) { | ||||||
|  | 		nextTime = scheduledWork.nextBackoff | ||||||
|  | 	} | ||||||
|  | 	if scheduledWork.nextOldCandidateCheck.Before(nextTime) { | ||||||
|  | 		nextTime = scheduledWork.nextOldCandidateCheck | ||||||
|  | 	} | ||||||
|  | 	if scheduledWork.nextAllowedCallToPeerSource.Before(nextTime) { | ||||||
|  | 		nextTime = scheduledWork.nextAllowedCallToPeerSource | ||||||
|  | 	} | ||||||
|  | 	if nextTime == now { | ||||||
|  | 		// Only happens in CI with a mock clock | ||||||
|  | 		nextTime = nextTime.Add(1) // avoids an infinite loop | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	return nextTime | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // clearOldCandidates clears old candidates from the map. Returns the next time | ||||||
|  | // to run this function. | ||||||
|  | func (rf *relayFinder) clearOldCandidates(now time.Time) time.Time { | ||||||
|  | 	// If we don't have any candidates, we should run this again in rf.conf.maxCandidateAge. | ||||||
|  | 	nextTime := now.Add(rf.conf.maxCandidateAge) | ||||||
|  |  | ||||||
|  | 	var deleted bool | ||||||
|  | 	rf.candidateMx.Lock() | ||||||
|  | 	defer rf.candidateMx.Unlock() | ||||||
|  | 	for id, cand := range rf.candidates { | ||||||
|  | 		expiry := cand.added.Add(rf.conf.maxCandidateAge) | ||||||
|  | 		if expiry.After(now) { | ||||||
|  | 			if expiry.Before(nextTime) { | ||||||
|  | 				nextTime = expiry | ||||||
|  | 			} | ||||||
|  | 		} else { | ||||||
|  | 			deleted = true | ||||||
|  | 			log.Debugw("deleting candidate due to age", "id", id) | ||||||
|  | 			delete(rf.candidates, id) | ||||||
|  |  | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	if deleted { | ||||||
|  | 		rf.notifyMaybeNeedNewCandidates() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	return nextTime | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // clearBackoff clears old backoff entries from the map. Returns the next time | ||||||
|  | // to run this function. | ||||||
|  | func (rf *relayFinder) clearBackoff(now time.Time) time.Time { | ||||||
|  | 	nextTime := now.Add(rf.conf.backoff) | ||||||
|  |  | ||||||
|  | 	rf.candidateMx.Lock() | ||||||
|  | 	defer rf.candidateMx.Unlock() | ||||||
|  | 	for id, t := range rf.backoff { | ||||||
|  | 		expiry := t.Add(rf.conf.backoff) | ||||||
|  | 		if expiry.After(now) { | ||||||
|  | 			if expiry.Before(nextTime) { | ||||||
|  | 				nextTime = expiry | ||||||
|  | 			} | ||||||
|  | 		} else { | ||||||
|  | 			log.Debugw("removing backoff for node", "id", id) | ||||||
|  | 			delete(rf.backoff, id) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	return nextTime | ||||||
| } | } | ||||||
|  |  | ||||||
| // findNodes accepts nodes from the channel and tests if they support relaying. | // findNodes accepts nodes from the channel and tests if they support relaying. | ||||||
| // It is run on both public and private nodes. | // It is run on both public and private nodes. | ||||||
| // It garbage collects old entries, so that nodes doesn't overflow. | // It garbage collects old entries, so that nodes doesn't overflow. | ||||||
| // This makes sure that as soon as we need to find relay candidates, we have them available. | // This makes sure that as soon as we need to find relay candidates, we have them available. | ||||||
| func (rf *relayFinder) findNodes(ctx context.Context) { | // peerSourceRateLimiter is used to limit how often we call the peer source. | ||||||
| 	peerChan := rf.peerSource(ctx, rf.conf.maxCandidates) | func (rf *relayFinder) findNodes(ctx context.Context, peerSourceRateLimiter <-chan struct{}) { | ||||||
|  | 	var peerChan <-chan peer.AddrInfo | ||||||
| 	var wg sync.WaitGroup | 	var wg sync.WaitGroup | ||||||
| 	lastCallToPeerSource := rf.conf.clock.Now() |  | ||||||
|  |  | ||||||
| 	timer := newTimer(rf.conf.clock) |  | ||||||
| 	for { | 	for { | ||||||
| 		rf.candidateMx.Lock() | 		rf.candidateMx.Lock() | ||||||
| 		numCandidates := len(rf.candidates) | 		numCandidates := len(rf.candidates) | ||||||
| 		rf.candidateMx.Unlock() | 		rf.candidateMx.Unlock() | ||||||
|  |  | ||||||
| 		if peerChan == nil { | 		if peerChan == nil && numCandidates < rf.conf.minCandidates { | ||||||
| 			now := rf.conf.clock.Now() | 			select { | ||||||
| 			nextAllowedCallToPeerSource := lastCallToPeerSource.Add(rf.conf.minInterval).Sub(now) | 			case <-peerSourceRateLimiter: | ||||||
| 			if numCandidates < rf.conf.minCandidates { | 				peerChan = rf.peerSource(ctx, rf.conf.maxCandidates) | ||||||
| 				log.Debugw("not enough candidates. Resetting timer", "num", numCandidates, "desired", rf.conf.minCandidates) | 			case <-ctx.Done(): | ||||||
| 				timer.Reset(nextAllowedCallToPeerSource) | 				return | ||||||
| 			} | 			} | ||||||
| 		} | 		} | ||||||
|  |  | ||||||
| 		select { | 		select { | ||||||
| 		case <-rf.maybeRequestNewCandidates: | 		case <-rf.maybeRequestNewCandidates: | ||||||
| 			continue | 			continue | ||||||
| 		case now := <-timer.Chan(): |  | ||||||
| 			timer.SetRead() |  | ||||||
| 			if peerChan != nil { |  | ||||||
| 				// We're still reading peers from the peerChan. No need to query for more peers now. |  | ||||||
| 				continue |  | ||||||
| 			} |  | ||||||
| 			lastCallToPeerSource = now |  | ||||||
| 			peerChan = rf.peerSource(ctx, rf.conf.maxCandidates) |  | ||||||
| 		case pi, ok := <-peerChan: | 		case pi, ok := <-peerChan: | ||||||
| 			if !ok { | 			if !ok { | ||||||
| 				wg.Wait() | 				wg.Wait() | ||||||
| @@ -530,9 +628,12 @@ func (rf *relayFinder) usingRelay(p peer.ID) bool { | |||||||
| // selectCandidates returns an ordered slice of relay candidates. | // selectCandidates returns an ordered slice of relay candidates. | ||||||
| // Callers should attempt to obtain reservations with the candidates in this order. | // Callers should attempt to obtain reservations with the candidates in this order. | ||||||
| func (rf *relayFinder) selectCandidates() []*candidate { | func (rf *relayFinder) selectCandidates() []*candidate { | ||||||
|  | 	now := rf.conf.clock.Now() | ||||||
| 	candidates := make([]*candidate, 0, len(rf.candidates)) | 	candidates := make([]*candidate, 0, len(rf.candidates)) | ||||||
| 	for _, cand := range rf.candidates { | 	for _, cand := range rf.candidates { | ||||||
| 		candidates = append(candidates, cand) | 		if cand.added.Add(rf.conf.maxCandidateAge).After(now) { | ||||||
|  | 			candidates = append(candidates, cand) | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// TODO: better relay selection strategy; this just selects random relays, | 	// TODO: better relay selection strategy; this just selects random relays, | ||||||
|   | |||||||
| @@ -1,42 +0,0 @@ | |||||||
| package autorelay |  | ||||||
|  |  | ||||||
| import ( |  | ||||||
| 	"time" |  | ||||||
|  |  | ||||||
| 	"github.com/benbjohnson/clock" |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| type timer struct { |  | ||||||
| 	timer   *clock.Timer |  | ||||||
| 	running bool |  | ||||||
| 	read    bool |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func newTimer(cl clock.Clock) *timer { |  | ||||||
| 	t := cl.Timer(100 * time.Hour) // There's no way to initialize a stopped timer |  | ||||||
| 	t.Stop() |  | ||||||
| 	return &timer{timer: t} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *timer) Chan() <-chan time.Time { |  | ||||||
| 	return t.timer.C |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *timer) Stop() { |  | ||||||
| 	if !t.running { |  | ||||||
| 		return |  | ||||||
| 	} |  | ||||||
| 	if !t.timer.Stop() && !t.read { |  | ||||||
| 		<-t.timer.C |  | ||||||
| 	} |  | ||||||
| 	t.read = false |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *timer) SetRead() { |  | ||||||
| 	t.read = true |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *timer) Reset(d time.Duration) { |  | ||||||
| 	t.Stop() |  | ||||||
| 	t.timer.Reset(d) |  | ||||||
| } |  | ||||||
		Reference in New Issue
	
	Block a user
	 Marco Munizaga
					Marco Munizaga