diff --git a/config.go b/config.go index d5fbe30b8..3bc244468 100644 --- a/config.go +++ b/config.go @@ -720,6 +720,7 @@ func DefaultConfig() Config { MsgRateBytes: discovery.DefaultMsgBytesPerSecond, MsgBurstBytes: discovery.DefaultMsgBytesBurst, FilterConcurrency: discovery.DefaultFilterConcurrency, + BanThreshold: discovery.DefaultBanThreshold, }, Invoices: &lncfg.Invoices{ HoldExpiryDelta: lncfg.DefaultHoldInvoiceExpiryDelta, diff --git a/discovery/ban.go b/discovery/ban.go index cd70d7c38..5229c706c 100644 --- a/discovery/ban.go +++ b/discovery/ban.go @@ -2,6 +2,7 @@ package discovery import ( "errors" + "math" "sync" "time" @@ -13,15 +14,14 @@ import ( ) const ( + // DefaultBanThreshold is the default value to be used for banThreshold. + DefaultBanThreshold = 100 + // maxBannedPeers limits the maximum number of banned pubkeys that // we'll store. // TODO(eugene): tune. maxBannedPeers = 10_000 - // banThreshold is the point at which non-channel peers will be banned. - // TODO(eugene): tune. - banThreshold = 100 - // banTime is the amount of time that the non-channel peer will be // banned for. Channel announcements from channel peers will be dropped // if it's not one of our channels. @@ -126,7 +126,7 @@ func (c *cachedBanInfo) Size() (uint64, error) { } // isBanned returns true if the ban score is greater than the ban threshold. -func (c *cachedBanInfo) isBanned() bool { +func (c *cachedBanInfo) isBanned(banThreshold uint64) bool { return c.score >= banThreshold } @@ -144,15 +144,26 @@ type banman struct { wg sync.WaitGroup quit chan struct{} + + // banThreshold is the point at which non-channel peers will be banned. + banThreshold uint64 } // newBanman creates a new banman with the default maxBannedPeers. -func newBanman() *banman { +func newBanman(banThreshold uint64) *banman { + // If the ban threshold is set to 0, we'll use the max value to + // effectively disable banning. + if banThreshold == 0 { + log.Warn("Banning is disabled due to zero banThreshold") + banThreshold = math.MaxUint64 + } + return &banman{ peerBanIndex: lru.NewCache[[33]byte, *cachedBanInfo]( maxBannedPeers, ), - quit: make(chan struct{}), + quit: make(chan struct{}), + banThreshold: banThreshold, } } @@ -193,7 +204,7 @@ func (b *banman) purgeBanEntries() { keysToRemove := make([][33]byte, 0) sweepEntries := func(pubkey [33]byte, banInfo *cachedBanInfo) bool { - if banInfo.isBanned() { + if banInfo.isBanned(b.banThreshold) { // If the peer is banned, check if the ban timer has // expired. if banInfo.lastUpdate.Add(banTime).Before(time.Now()) { @@ -227,7 +238,7 @@ func (b *banman) isBanned(pubkey [33]byte) bool { return false default: - return banInfo.isBanned() + return banInfo.isBanned(b.banThreshold) } } diff --git a/discovery/ban_test.go b/discovery/ban_test.go index e4149028b..72320ecb0 100644 --- a/discovery/ban_test.go +++ b/discovery/ban_test.go @@ -12,12 +12,14 @@ import ( func TestPurgeBanEntries(t *testing.T) { t.Parallel() - b := newBanman() + testBanThreshold := uint64(10) + + b := newBanman(testBanThreshold) // Ban a peer by repeatedly incrementing its ban score. peer1 := [33]byte{0x00} - for i := 0; i < banThreshold; i++ { + for range testBanThreshold { b.incrementBanScore(peer1) } diff --git a/discovery/chan_series.go b/discovery/chan_series.go index a6787edf9..ed7140d7d 100644 --- a/discovery/chan_series.go +++ b/discovery/chan_series.go @@ -120,6 +120,10 @@ func (c *ChanSeries) UpdatesInHorizon(chain chainhash.Hash, if err != nil { return nil, err } + + // nodesFromChan records the nodes seen from the channels. + nodesFromChan := make(map[[33]byte]struct{}, len(chansInHorizon)*2) + for _, channel := range chansInHorizon { // If the channel hasn't been fully advertised yet, or is a // private channel, then we'll skip it as we can't construct a @@ -136,7 +140,21 @@ func (c *ChanSeries) UpdatesInHorizon(chain chainhash.Hash, return nil, err } - updates = append(updates, chanAnn) + // Create a slice to hold the `channel_announcement` and + // potentially two `channel_update` msgs. + // + // NOTE: Based on BOLT7, if a channel_announcement has no + // corresponding channel_updates, we must not send the + // channel_announcement. Thus we use this slice to decide we + // want to send this `channel_announcement` or not. By the end + // of the operation, if the len of the slice is 1, we will not + // send the `channel_announcement`. Otherwise, when sending the + // msgs, the `channel_announcement` must be sent prior to any + // corresponding `channel_update` or `node_annoucement`, that's + // why we create a slice here to maintain the order. + chanUpdates := make([]lnwire.Message, 0, 3) + chanUpdates = append(chanUpdates, chanAnn) + if edge1 != nil { // We don't want to send channel updates that don't // conform to the spec (anymore). @@ -145,18 +163,32 @@ func (c *ChanSeries) UpdatesInHorizon(chain chainhash.Hash, log.Errorf("not sending invalid channel "+ "update %v: %v", edge1, err) } else { - updates = append(updates, edge1) + chanUpdates = append(chanUpdates, edge1) } } + if edge2 != nil { err := netann.ValidateChannelUpdateFields(0, edge2) if err != nil { log.Errorf("not sending invalid channel "+ "update %v: %v", edge2, err) } else { - updates = append(updates, edge2) + chanUpdates = append(chanUpdates, edge2) } } + + // If there's no corresponding `channel_update` to send, skip + // sending this `channel_announcement`. + if len(chanUpdates) < 2 { + continue + } + + // Append the all the msgs to the slice. + updates = append(updates, chanUpdates...) + + // Record the nodes seen. + nodesFromChan[channel.Info.NodeKey1Bytes] = struct{}{} + nodesFromChan[channel.Info.NodeKey2Bytes] = struct{}{} } // Next, we'll send out all the node announcements that have an update @@ -168,8 +200,15 @@ func (c *ChanSeries) UpdatesInHorizon(chain chainhash.Hash, if err != nil { return nil, err } + for _, nodeAnn := range nodeAnnsInHorizon { - nodeAnn := nodeAnn + // If this node has not been seen in the above channels, we can + // skip sending its NodeAnnouncement. + if _, seen := nodesFromChan[nodeAnn.PubKeyBytes]; !seen { + log.Debugf("Skipping forwarding as node %x not found "+ + "in channel announcement", nodeAnn.PubKeyBytes) + continue + } // Ensure we only forward nodes that are publicly advertised to // prevent leaking information about nodes. diff --git a/discovery/gossiper.go b/discovery/gossiper.go index 2473eda24..967f9ee94 100644 --- a/discovery/gossiper.go +++ b/discovery/gossiper.go @@ -403,6 +403,10 @@ type Config struct { // FilterConcurrency is the maximum number of concurrent gossip filter // applications that can be processed. FilterConcurrency int + + // BanThreshold is the score used to decide whether a given peer is + // banned or not. + BanThreshold uint64 } // processedNetworkMsg is a wrapper around networkMsg and a boolean. It is @@ -586,7 +590,7 @@ func New(cfg Config, selfKeyDesc *keychain.KeyDescriptor) *AuthenticatedGossiper maxRejectedUpdates, ), chanUpdateRateLimiter: make(map[uint64][2]*rate.Limiter), - banman: newBanman(), + banman: newBanman(cfg.BanThreshold), } gossiper.vb = NewValidationBarrier(1000, gossiper.quit) @@ -2598,7 +2602,6 @@ func (d *AuthenticatedGossiper) handleChanAnnouncement(ctx context.Context, if closed { err = fmt.Errorf("ignoring closed channel %v", scid) - log.Error(err) // If this is an announcement from us, we'll just ignore it. if !nMsg.isRemote { @@ -2606,23 +2609,14 @@ func (d *AuthenticatedGossiper) handleChanAnnouncement(ctx context.Context, return nil, false } + log.Warnf("Increasing ban score for peer=%v due to outdated "+ + "channel announcement for channel %v", nMsg.peer, scid) + // Increment the peer's ban score if they are sending closed // channel announcements. - d.banman.incrementBanScore(nMsg.peer.PubKey()) - - // If the peer is banned and not a channel peer, we'll - // disconnect them. - shouldDc, dcErr := d.ShouldDisconnect(nMsg.peer.IdentityKey()) + dcErr := d.handleBadPeer(nMsg.peer) if dcErr != nil { - log.Errorf("failed to check if we should disconnect "+ - "peer: %v", dcErr) - nMsg.err <- dcErr - - return nil, false - } - - if shouldDc { - nMsg.peer.Disconnect(ErrPeerBanned) + err = dcErr } nMsg.err <- err @@ -2707,7 +2701,7 @@ func (d *AuthenticatedGossiper) handleChanAnnouncement(ctx context.Context, // ShortChannelID is an alias, then we'll skip validation as it will // not map to a legitimate tx. This is not a DoS vector as only we can // add an alias ChannelAnnouncement from the gossiper. - if !(d.cfg.AssumeChannelValid || d.cfg.IsAlias(scid)) { //nolint:nestif + if !(d.cfg.AssumeChannelValid || d.cfg.IsAlias(scid)) { op, capacity, script, err := d.validateFundingTransaction( ctx, ann, tapscriptRoot, ) @@ -2726,15 +2720,6 @@ func (d *AuthenticatedGossiper) handleChanAnnouncement(ctx context.Context, key, &cachedReject{}, ) - // Increment the peer's ban score. We check - // isRemote so we don't actually ban the peer in - // case of a local bug. - if nMsg.isRemote { - d.banman.incrementBanScore( - nMsg.peer.PubKey(), - ) - } - case errors.Is(err, ErrChannelSpent): key := newRejectCacheKey( scid.ToUint64(), @@ -2758,23 +2743,19 @@ func (d *AuthenticatedGossiper) handleChanAnnouncement(ctx context.Context, return nil, false } - // Increment the peer's ban score. We check - // isRemote so we don't accidentally ban - // ourselves in case of a bug. - if nMsg.isRemote { - d.banman.incrementBanScore( - nMsg.peer.PubKey(), - ) - } - default: // Otherwise, this is just a regular rejected - // edge. + // edge. We won't increase the ban score for the + // remote peer. key := newRejectCacheKey( scid.ToUint64(), sourceToPub(nMsg.source), ) _, _ = d.recentRejects.Put(key, &cachedReject{}) + + nMsg.err <- err + + return nil, false } if !nMsg.isRemote { @@ -2785,19 +2766,15 @@ func (d *AuthenticatedGossiper) handleChanAnnouncement(ctx context.Context, return nil, false } - shouldDc, dcErr := d.ShouldDisconnect( - nMsg.peer.IdentityKey(), - ) + log.Warnf("Increasing ban score for peer=%v due to "+ + "invalid channel announcement for channel %v", + nMsg.peer, scid) + + // Increment the peer's ban score if they are sending + // us invalid channel announcements. + dcErr := d.handleBadPeer(nMsg.peer) if dcErr != nil { - log.Errorf("failed to check if we should "+ - "disconnect peer: %v", dcErr) - nMsg.err <- dcErr - - return nil, false - } - - if shouldDc { - nMsg.peer.Disconnect(ErrPeerBanned) + err = dcErr } nMsg.err <- err @@ -3060,16 +3037,29 @@ func (d *AuthenticatedGossiper) handleChanUpdate(ctx context.Context, // Check that the ChanUpdate is not too far into the future, this could // reveal some faulty implementation therefore we log an error. if time.Until(timestamp) > graph.DefaultChannelPruneExpiry { - log.Errorf("Skewed timestamp (%v) for edge policy of "+ - "short_chan_id(%v), timestamp too far in the future: "+ - "peer=%v, msg=%s, is_remote=%v", timestamp.Unix(), - shortChanID, nMsg.peer, nMsg.msg.MsgType(), - nMsg.isRemote, - ) - - nMsg.err <- fmt.Errorf("skewed timestamp of edge policy, "+ + err := fmt.Errorf("skewed timestamp of edge policy, "+ "timestamp too far in the future: %v", timestamp.Unix()) + // If this is a channel_update from us, we'll just ignore it. + if !nMsg.isRemote { + nMsg.err <- err + return nil, false + } + + log.Errorf("Increasing ban score for peer=%v due to bad "+ + "channel_update with short_chan_id(%v): timestamp(%v) "+ + "too far in the future", nMsg.peer, shortChanID, + timestamp.Unix()) + + // Increment the peer's ban score if they are skewed channel + // updates. + dcErr := d.handleBadPeer(nMsg.peer) + if dcErr != nil { + err = dcErr + } + + nMsg.err <- err + return nil, false } @@ -3841,6 +3831,29 @@ func (d *AuthenticatedGossiper) validateFundingTransaction(_ context.Context, nil } +// handleBadPeer takes a misbehaving peer and increases its ban score. Once +// increased, it will disconnect the peer if its ban score has reached +// `banThreshold` and it doesn't have a channel with us. +func (d *AuthenticatedGossiper) handleBadPeer(peer lnpeer.Peer) error { + // Increment the peer's ban score for misbehavior. + d.banman.incrementBanScore(peer.PubKey()) + + // If the peer is banned and not a channel peer, we'll disconnect them. + shouldDc, dcErr := d.ShouldDisconnect(peer.IdentityKey()) + if dcErr != nil { + log.Errorf("failed to check if we should disconnect peer: %v", + dcErr) + + return dcErr + } + + if shouldDc { + peer.Disconnect(ErrPeerBanned) + } + + return nil +} + // makeFundingScript is used to make the funding script for both segwit v0 and // segwit v1 (taproot) channels. func makeFundingScript(bitcoinKey1, bitcoinKey2 []byte, diff --git a/discovery/gossiper_test.go b/discovery/gossiper_test.go index 6b9a1d58f..58c975bd4 100644 --- a/discovery/gossiper_test.go +++ b/discovery/gossiper_test.go @@ -995,6 +995,7 @@ func createTestCtx(t *testing.T, startHeight uint32, isChanPeer bool) ( GetAlias: getAlias, FindChannel: mockFindChannel, ScidCloser: newMockScidCloser(isChanPeer), + BanThreshold: DefaultBanThreshold, }, selfKeyDesc) if err := gossiper.Start(); err != nil { @@ -4656,7 +4657,7 @@ func TestChanAnnBanningNonChanPeer(t *testing.T) { } // Loop 100 times to get nodePeer banned. - for i := 0; i < 100; i++ { + for i := range DefaultBanThreshold { // Craft a valid channel announcement for a channel we don't // have. We will ensure that it fails validation by modifying // the tx script. @@ -4746,7 +4747,7 @@ func TestChanAnnBanningChanPeer(t *testing.T) { nodePeer := &mockPeer{remoteKeyPriv1.PubKey(), nil, nil, atomic.Bool{}} // Loop 100 times to get nodePeer banned. - for i := 0; i < 100; i++ { + for i := range DefaultBanThreshold { // Craft a valid channel announcement for a channel we don't // have. We will ensure that it fails validation by modifying // the router. diff --git a/discovery/sync_manager.go b/discovery/sync_manager.go index c52fec8a2..378085e49 100644 --- a/discovery/sync_manager.go +++ b/discovery/sync_manager.go @@ -211,7 +211,6 @@ type SyncManager struct { // newSyncManager constructs a new SyncManager backed by the given config. func newSyncManager(cfg *SyncManagerCfg) *SyncManager { - filterConcurrency := cfg.FilterConcurrency if filterConcurrency == 0 { filterConcurrency = DefaultFilterConcurrency diff --git a/docs/gossip_rate_limiting.md b/docs/gossip_rate_limiting.md index 68f30b95d..2ac4ecf5d 100644 --- a/docs/gossip_rate_limiting.md +++ b/docs/gossip_rate_limiting.md @@ -62,6 +62,25 @@ Large routing nodes handling many simultaneous peer connections might benefit from increasing this value to 10 or 15, while resource-constrained nodes should keep it at the default or even reduce it slightly. +### Preventing Spam: gossip.ban-threshold + +To protect your node from spam and misbehaving peers, LND uses a ban score +system controlled by `gossip.ban-threshold`. Each time a peer sends a gossip +message that is considered invalid, its ban score is incremented. Once the score +reaches this threshold, the peer is banned for a default of 48 hours, and your +node will no longer process gossip messages from them. + +A gossip message can be considered invalid for several reasons, including: +- Invalid signature on the announcement. +- Stale timestamp, older than what we already have. +- Too many channel updates for the same channel in a short period. +- Announcing a channel that is not found on-chain. +- Announcing a channel that has already been closed. +- Announcing a channel with an invalid proof. + +The default value is 100. Setting this value to 0 disables banning completely, +which is not recommended for most operators. + ### Understanding Connection Limits: num-restricted-slots The `num-restricted-slots` configuration deserves special attention because it diff --git a/docs/release-notes/release-notes-0.20.0.md b/docs/release-notes/release-notes-0.20.0.md index 4b7b9b91d..1b106d1a1 100644 --- a/docs/release-notes/release-notes-0.20.0.md +++ b/docs/release-notes/release-notes-0.20.0.md @@ -38,6 +38,10 @@ messages in the dynamic commitment set. +- [Fixed](https://github.com/lightningnetwork/lnd/pull/10102) a case that we may + send unnecessary `channel_announcement` and `node_announcement` messages when + replying to a `gossip_timestamp_filter` query. + # New Features * Use persisted [nodeannouncement](https://github.com/lightningnetwork/lnd/pull/8825) @@ -95,6 +99,12 @@ circuit. The indices are only available for forwarding events saved after v0.20. a canceled invoice. Supports deleting a canceled invoice by providing its payment hash. +* A [new config](https://github.com/lightningnetwork/lnd/pull/10102) + `gossip.ban-threshold` is added to allow users to configure the ban score + threshold for peers. When a peer's ban score exceeds this value, they will be + disconnected and banned. Setting the value to 0 effectively disables banning + by setting the threshold to the maximum possible value. + ## lncli Additions * [`lncli sendpayment` and `lncli queryroutes` now support the diff --git a/lncfg/gossip.go b/lncfg/gossip.go index 0c297e324..635496c0b 100644 --- a/lncfg/gossip.go +++ b/lncfg/gossip.go @@ -39,10 +39,11 @@ type Gossip struct { MsgBurstBytes uint64 `long:"msg-burst-bytes" description:"The maximum burst of outbound gossip data, in bytes, that can be sent at once. This works in conjunction with gossip.msg-rate-bytes as part of a token bucket rate-limiting scheme. This value represents the size of the token bucket. It allows for short, high-speed bursts of traffic, with the long-term rate controlled by gossip.msg-rate-bytes. This value must be larger than the maximum lightning message size (~65KB) to allow sending large gossip messages."` FilterConcurrency int `long:"filter-concurrency" description:"The maximum number of concurrent gossip filter applications that can be processed. If not set, defaults to 5."` + + BanThreshold uint64 `long:"ban-threshold" description:"The score at which a peer is banned. A peer's ban score is incremented for each invalid gossip message. Invalid messages include those with bad signatures, stale timestamps, excessive updates, or invalid chain data. Once the score reaches this threshold, the peer is banned. Set to 0 to disable banning."` } // Parse the pubkeys for the pinned syncers. - func (g *Gossip) Parse() error { pinnedSyncers := make(discovery.PinnedSyncers) for _, pubkeyStr := range g.PinnedSyncersRaw { diff --git a/sample-lnd.conf b/sample-lnd.conf index 22ffc3916..bc6f07fa2 100644 --- a/sample-lnd.conf +++ b/sample-lnd.conf @@ -1797,6 +1797,23 @@ ; See docs/gossip_rate_limiting.md for mor information. ; gossip.filter-concurrency=5 +; The score at which a peer is banned. Each time a peer sends a gossip message +; that is considered invalid, its ban score is incremented. Once the score +; reaches this threshold, the peer is banned for a default of 48 hours, and we +; will no longer process gossip messages from them. This is a measure to +; protect the node from spam and misbehaving peers. Setting this value to 0 +; disables banning completely. +; +; A gossip message can be considered invalid for several reasons, including: +; - Invalid signature on the announcement. +; - Stale timestamp, older than what we already have. +; - Too many channel updates for the same channel in a short period. +; - Announcing a channel that is not found on-chain. +; - Announcing a channel that has already been closed. +; - Announcing a channel with an invalid proof. +; +; gossip.ban-threshold=100 + [invoices] ; If a hold invoice has accepted htlcs that reach their expiry height and are diff --git a/server.go b/server.go index cfbc55da7..8770f7ae5 100644 --- a/server.go +++ b/server.go @@ -1102,6 +1102,7 @@ func newServer(ctx context.Context, cfg *Config, listenAddrs []net.Addr, MsgRateBytes: cfg.Gossip.MsgRateBytes, MsgBurstBytes: cfg.Gossip.MsgBurstBytes, FilterConcurrency: cfg.Gossip.FilterConcurrency, + BanThreshold: cfg.Gossip.BanThreshold, }, nodeKeyDesc) accessCfg := &accessManConfig{