mirror of
https://github.com/lightningnetwork/lnd.git
synced 2025-08-27 22:21:18 +02:00
discovery: switch to bytes based rate limiting for outbound msgs
In this commit, we revamp the old message based rate limiting. First, we move to meter by bytes/s instead of messages/s. The old logic had an error in that it limited groups of message replies, instead of each message. With this new approach, we'll use the newly added SerializedSize method to implement fine grained bandwidth metering. We need to pick two values, the burst rate, and the msg bytes rate. The burst rate is the max amt that can be sent in a given period of time. We need to set this above 65 KB, or the max msg limit, otherwise no messages can be sent. The bucket starts with this many tokens (bytes). As those are depleted, the amount of tokens is refilled at the msg bytes rate. As conservative values, we've chosen 200 KB as the burst rate, and 100 KB/s as the limit.
This commit is contained in:
@@ -1,6 +1,7 @@
|
||||
package discovery
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
@@ -11,11 +12,11 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/btcsuite/btcd/chaincfg/chainhash"
|
||||
"github.com/lightningnetwork/lnd/fn/v2"
|
||||
"github.com/lightningnetwork/lnd/graph"
|
||||
graphdb "github.com/lightningnetwork/lnd/graph/db"
|
||||
"github.com/lightningnetwork/lnd/lnpeer"
|
||||
"github.com/lightningnetwork/lnd/lnwire"
|
||||
"golang.org/x/time/rate"
|
||||
)
|
||||
|
||||
// SyncerType encapsulates the different types of syncing mechanisms for a
|
||||
@@ -152,15 +153,6 @@ func (s syncerState) String() string {
|
||||
}
|
||||
|
||||
const (
|
||||
// DefaultMaxUndelayedQueryReplies specifies how many gossip queries we
|
||||
// will respond to immediately before starting to delay responses.
|
||||
DefaultMaxUndelayedQueryReplies = 10
|
||||
|
||||
// DefaultDelayedQueryReplyInterval is the length of time we will wait
|
||||
// before responding to gossip queries after replying to
|
||||
// maxUndelayedQueryReplies queries.
|
||||
DefaultDelayedQueryReplyInterval = 5 * time.Second
|
||||
|
||||
// maxQueryChanRangeReplies specifies the default limit of replies to
|
||||
// process for a single QueryChannelRange request.
|
||||
maxQueryChanRangeReplies = 500
|
||||
@@ -250,21 +242,12 @@ type gossipSyncerCfg struct {
|
||||
// sendToPeer sends a variadic number of messages to the remote peer.
|
||||
// This method should not block while waiting for sends to be written
|
||||
// to the wire.
|
||||
sendToPeer func(...lnwire.Message) error
|
||||
sendToPeer func(context.Context, ...lnwire.Message) error
|
||||
|
||||
// sendToPeerSync sends a variadic number of messages to the remote
|
||||
// peer, blocking until all messages have been sent successfully or a
|
||||
// write error is encountered.
|
||||
sendToPeerSync func(...lnwire.Message) error
|
||||
|
||||
// maxUndelayedQueryReplies specifies how many gossip queries we will
|
||||
// respond to immediately before starting to delay responses.
|
||||
maxUndelayedQueryReplies int
|
||||
|
||||
// delayedQueryReplyInterval is the length of time we will wait before
|
||||
// responding to gossip queries after replying to
|
||||
// maxUndelayedQueryReplies queries.
|
||||
delayedQueryReplyInterval time.Duration
|
||||
sendToPeerSync func(context.Context, ...lnwire.Message) error
|
||||
|
||||
// noSyncChannels will prevent the GossipSyncer from spawning a
|
||||
// channelGraphSyncer, meaning we will not try to reconcile unknown
|
||||
@@ -390,12 +373,6 @@ type GossipSyncer struct {
|
||||
|
||||
cfg gossipSyncerCfg
|
||||
|
||||
// rateLimiter dictates the frequency with which we will reply to gossip
|
||||
// queries from a peer. This is used to delay responses to peers to
|
||||
// prevent DOS vulnerabilities if they are spamming with an unreasonable
|
||||
// number of queries.
|
||||
rateLimiter *rate.Limiter
|
||||
|
||||
// syncedSignal is a channel that, if set, will be closed when the
|
||||
// GossipSyncer reaches its terminal chansSynced state.
|
||||
syncedSignal chan struct{}
|
||||
@@ -406,43 +383,23 @@ type GossipSyncer struct {
|
||||
|
||||
sync.Mutex
|
||||
|
||||
quit chan struct{}
|
||||
wg sync.WaitGroup
|
||||
// cg is a helper that encapsulates a wait group and quit channel and
|
||||
// allows contexts that either block or cancel on those depending on
|
||||
// the use case.
|
||||
cg *fn.ContextGuard
|
||||
}
|
||||
|
||||
// newGossipSyncer returns a new instance of the GossipSyncer populated using
|
||||
// the passed config.
|
||||
func newGossipSyncer(cfg gossipSyncerCfg, sema chan struct{}) *GossipSyncer {
|
||||
// If no parameter was specified for max undelayed query replies, set it
|
||||
// to the default of 5 queries.
|
||||
if cfg.maxUndelayedQueryReplies <= 0 {
|
||||
cfg.maxUndelayedQueryReplies = DefaultMaxUndelayedQueryReplies
|
||||
}
|
||||
|
||||
// If no parameter was specified for delayed query reply interval, set
|
||||
// to the default of 5 seconds.
|
||||
if cfg.delayedQueryReplyInterval <= 0 {
|
||||
cfg.delayedQueryReplyInterval = DefaultDelayedQueryReplyInterval
|
||||
}
|
||||
|
||||
// Construct a rate limiter that will govern how frequently we reply to
|
||||
// gossip queries from this peer. The limiter will automatically adjust
|
||||
// during periods of quiescence, and increase the reply interval under
|
||||
// load.
|
||||
interval := rate.Every(cfg.delayedQueryReplyInterval)
|
||||
rateLimiter := rate.NewLimiter(
|
||||
interval, cfg.maxUndelayedQueryReplies,
|
||||
)
|
||||
|
||||
return &GossipSyncer{
|
||||
cfg: cfg,
|
||||
rateLimiter: rateLimiter,
|
||||
syncTransitionReqs: make(chan *syncTransitionReq),
|
||||
historicalSyncReqs: make(chan *historicalSyncReq),
|
||||
gossipMsgs: make(chan lnwire.Message, syncerBufferSize),
|
||||
queryMsgs: make(chan lnwire.Message, syncerBufferSize),
|
||||
syncerSema: sema,
|
||||
quit: make(chan struct{}),
|
||||
cg: fn.NewContextGuard(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -456,11 +413,11 @@ func (g *GossipSyncer) Start() {
|
||||
// supports gossip queries, and only spawn replyHandler if we
|
||||
// advertise support
|
||||
if !g.cfg.noSyncChannels {
|
||||
g.wg.Add(1)
|
||||
g.cg.WgAdd(1)
|
||||
go g.channelGraphSyncer()
|
||||
}
|
||||
if !g.cfg.noReplyQueries {
|
||||
g.wg.Add(1)
|
||||
g.cg.WgAdd(1)
|
||||
go g.replyHandler()
|
||||
}
|
||||
})
|
||||
@@ -473,8 +430,7 @@ func (g *GossipSyncer) Stop() {
|
||||
log.Debugf("Stopping GossipSyncer(%x)", g.cfg.peerPub[:])
|
||||
defer log.Debugf("GossipSyncer(%x) stopped", g.cfg.peerPub[:])
|
||||
|
||||
close(g.quit)
|
||||
g.wg.Wait()
|
||||
g.cg.Quit()
|
||||
})
|
||||
}
|
||||
|
||||
@@ -500,7 +456,8 @@ func (g *GossipSyncer) handleSyncingChans() {
|
||||
|
||||
// Send the msg to the remote peer, which is non-blocking as
|
||||
// `sendToPeer` only queues the msg in Brontide.
|
||||
err = g.cfg.sendToPeer(queryRangeMsg)
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
err = g.cfg.sendToPeer(ctx, queryRangeMsg)
|
||||
if err != nil {
|
||||
log.Errorf("Unable to send chan range query: %v", err)
|
||||
return
|
||||
@@ -515,7 +472,7 @@ func (g *GossipSyncer) handleSyncingChans() {
|
||||
// properly channel graph state with the remote peer, and also that we only
|
||||
// send them messages which actually pass their defined update horizon.
|
||||
func (g *GossipSyncer) channelGraphSyncer() {
|
||||
defer g.wg.Done()
|
||||
defer g.cg.WgDone()
|
||||
|
||||
for {
|
||||
state := g.syncState()
|
||||
@@ -563,7 +520,7 @@ func (g *GossipSyncer) channelGraphSyncer() {
|
||||
log.Warnf("Unexpected message: %T in state=%v",
|
||||
msg, state)
|
||||
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return
|
||||
}
|
||||
|
||||
@@ -613,7 +570,7 @@ func (g *GossipSyncer) channelGraphSyncer() {
|
||||
log.Warnf("Unexpected message: %T in state=%v",
|
||||
msg, state)
|
||||
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return
|
||||
}
|
||||
|
||||
@@ -659,7 +616,7 @@ func (g *GossipSyncer) channelGraphSyncer() {
|
||||
case req := <-g.historicalSyncReqs:
|
||||
g.handleHistoricalSync(req)
|
||||
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -674,7 +631,7 @@ func (g *GossipSyncer) channelGraphSyncer() {
|
||||
//
|
||||
// NOTE: This method MUST be run as a goroutine.
|
||||
func (g *GossipSyncer) replyHandler() {
|
||||
defer g.wg.Done()
|
||||
defer g.cg.WgDone()
|
||||
|
||||
for {
|
||||
select {
|
||||
@@ -692,7 +649,7 @@ func (g *GossipSyncer) replyHandler() {
|
||||
"query: %v", err)
|
||||
}
|
||||
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -716,7 +673,8 @@ func (g *GossipSyncer) sendGossipTimestampRange(firstTimestamp time.Time,
|
||||
TimestampRange: timestampRange,
|
||||
}
|
||||
|
||||
if err := g.cfg.sendToPeer(localUpdateHorizon); err != nil {
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
if err := g.cfg.sendToPeer(ctx, localUpdateHorizon); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
@@ -772,7 +730,8 @@ func (g *GossipSyncer) synchronizeChanIDs() bool {
|
||||
|
||||
// With our chunk obtained, we'll send over our next query, then return
|
||||
// false indicating that we're net yet fully synced.
|
||||
err := g.cfg.sendToPeer(&lnwire.QueryShortChanIDs{
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
err := g.cfg.sendToPeer(ctx, &lnwire.QueryShortChanIDs{
|
||||
ChainHash: g.cfg.chainHash,
|
||||
EncodingType: lnwire.EncodingSortedPlain,
|
||||
ShortChanIDs: queryChunk,
|
||||
@@ -1047,23 +1006,6 @@ func (g *GossipSyncer) genChanRangeQuery(
|
||||
// replyPeerQueries is called in response to any query by the remote peer.
|
||||
// We'll examine our state and send back our best response.
|
||||
func (g *GossipSyncer) replyPeerQueries(msg lnwire.Message) error {
|
||||
reservation := g.rateLimiter.Reserve()
|
||||
delay := reservation.Delay()
|
||||
|
||||
// If we've already replied a handful of times, we will start to delay
|
||||
// responses back to the remote peer. This can help prevent DOS attacks
|
||||
// where the remote peer spams us endlessly.
|
||||
if delay > 0 {
|
||||
log.Infof("GossipSyncer(%x): rate limiting gossip replies, "+
|
||||
"responding in %s", g.cfg.peerPub[:], delay)
|
||||
|
||||
select {
|
||||
case <-time.After(delay):
|
||||
case <-g.quit:
|
||||
return ErrGossipSyncerExiting
|
||||
}
|
||||
}
|
||||
|
||||
switch msg := msg.(type) {
|
||||
|
||||
// In this state, we'll also handle any incoming channel range queries
|
||||
@@ -1096,7 +1038,9 @@ func (g *GossipSyncer) replyChanRangeQuery(query *lnwire.QueryChannelRange) erro
|
||||
"chain=%v, we're on chain=%v", query.ChainHash,
|
||||
g.cfg.chainHash)
|
||||
|
||||
return g.cfg.sendToPeerSync(&lnwire.ReplyChannelRange{
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
|
||||
return g.cfg.sendToPeerSync(ctx, &lnwire.ReplyChannelRange{
|
||||
ChainHash: query.ChainHash,
|
||||
FirstBlockHeight: query.FirstBlockHeight,
|
||||
NumBlocks: query.NumBlocks,
|
||||
@@ -1169,7 +1113,9 @@ func (g *GossipSyncer) replyChanRangeQuery(query *lnwire.QueryChannelRange) erro
|
||||
)
|
||||
}
|
||||
|
||||
return g.cfg.sendToPeerSync(&lnwire.ReplyChannelRange{
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
|
||||
return g.cfg.sendToPeerSync(ctx, &lnwire.ReplyChannelRange{
|
||||
ChainHash: query.ChainHash,
|
||||
NumBlocks: numBlocks,
|
||||
FirstBlockHeight: firstHeight,
|
||||
@@ -1273,7 +1219,9 @@ func (g *GossipSyncer) replyShortChanIDs(query *lnwire.QueryShortChanIDs) error
|
||||
"chain=%v, we're on chain=%v", query.ChainHash,
|
||||
g.cfg.chainHash)
|
||||
|
||||
return g.cfg.sendToPeerSync(&lnwire.ReplyShortChanIDsEnd{
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
|
||||
return g.cfg.sendToPeerSync(ctx, &lnwire.ReplyShortChanIDsEnd{
|
||||
ChainHash: query.ChainHash,
|
||||
Complete: 0,
|
||||
})
|
||||
@@ -1304,7 +1252,8 @@ func (g *GossipSyncer) replyShortChanIDs(query *lnwire.QueryShortChanIDs) error
|
||||
// each one individually and synchronously to throttle the sends and
|
||||
// perform buffering of responses in the syncer as opposed to the peer.
|
||||
for _, msg := range replyMsgs {
|
||||
err := g.cfg.sendToPeerSync(msg)
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
err := g.cfg.sendToPeerSync(ctx, msg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -1312,7 +1261,9 @@ func (g *GossipSyncer) replyShortChanIDs(query *lnwire.QueryShortChanIDs) error
|
||||
|
||||
// Regardless of whether we had any messages to reply with, send over
|
||||
// the sentinel message to signal that the stream has terminated.
|
||||
return g.cfg.sendToPeerSync(&lnwire.ReplyShortChanIDsEnd{
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
|
||||
return g.cfg.sendToPeerSync(ctx, &lnwire.ReplyShortChanIDsEnd{
|
||||
ChainHash: query.ChainHash,
|
||||
Complete: 1,
|
||||
})
|
||||
@@ -1341,7 +1292,7 @@ func (g *GossipSyncer) ApplyGossipFilter(filter *lnwire.GossipTimestampRange) er
|
||||
|
||||
select {
|
||||
case <-g.syncerSema:
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return ErrGossipSyncerExiting
|
||||
}
|
||||
|
||||
@@ -1372,13 +1323,14 @@ func (g *GossipSyncer) ApplyGossipFilter(filter *lnwire.GossipTimestampRange) er
|
||||
}
|
||||
|
||||
// We'll conclude by launching a goroutine to send out any updates.
|
||||
g.wg.Add(1)
|
||||
g.cg.WgAdd(1)
|
||||
go func() {
|
||||
defer g.wg.Done()
|
||||
defer g.cg.WgDone()
|
||||
defer returnSema()
|
||||
|
||||
for _, msg := range newUpdatestoSend {
|
||||
err := g.cfg.sendToPeerSync(msg)
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
err := g.cfg.sendToPeerSync(ctx, msg)
|
||||
switch {
|
||||
case err == ErrGossipSyncerExiting:
|
||||
return
|
||||
@@ -1411,7 +1363,7 @@ func (g *GossipSyncer) FilterGossipMsgs(msgs ...msgWithSenders) {
|
||||
// If we've been signaled to exit, or are exiting, then we'll stop
|
||||
// short.
|
||||
select {
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return
|
||||
default:
|
||||
}
|
||||
@@ -1520,7 +1472,11 @@ func (g *GossipSyncer) FilterGossipMsgs(msgs ...msgWithSenders) {
|
||||
return
|
||||
}
|
||||
|
||||
g.cfg.sendToPeer(msgsToSend...)
|
||||
ctx, _ := g.cg.Create(context.Background())
|
||||
if err = g.cfg.sendToPeer(ctx, msgsToSend...); err != nil {
|
||||
log.Errorf("unable to send gossip msgs: %v", err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// ProcessQueryMsg is used by outside callers to pass new channel time series
|
||||
@@ -1553,7 +1509,7 @@ func (g *GossipSyncer) ProcessQueryMsg(msg lnwire.Message, peerQuit <-chan struc
|
||||
select {
|
||||
case msgChan <- msg:
|
||||
case <-peerQuit:
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
}
|
||||
|
||||
return nil
|
||||
@@ -1601,14 +1557,14 @@ func (g *GossipSyncer) ProcessSyncTransition(newSyncType SyncerType) error {
|
||||
}:
|
||||
case <-time.After(syncTransitionTimeout):
|
||||
return ErrSyncTransitionTimeout
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return ErrGossipSyncerExiting
|
||||
}
|
||||
|
||||
select {
|
||||
case err := <-errChan:
|
||||
return err
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return ErrGossipSyncerExiting
|
||||
}
|
||||
}
|
||||
@@ -1687,14 +1643,14 @@ func (g *GossipSyncer) historicalSync() error {
|
||||
}:
|
||||
case <-time.After(syncTransitionTimeout):
|
||||
return ErrSyncTransitionTimeout
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return ErrGossiperShuttingDown
|
||||
}
|
||||
|
||||
select {
|
||||
case <-done:
|
||||
return nil
|
||||
case <-g.quit:
|
||||
case <-g.cg.Done():
|
||||
return ErrGossiperShuttingDown
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user