htlcswitch+lnwallet: calculate fee exposure as commit fees + dust

This commit expands the definition of the dust limit to take into
account commitment fees as well as dust HTLCs. The dust limit is now
known as a fee exposure threshold. Dust HTLCs are fees anyways so it
makes sense to account for commitment fees as well. The link has
been modified slightly to calculate dust. In the future, the switch
dust calculations can be removed.
This commit is contained in:
Eugene Siegel 2024-06-03 12:43:33 -04:00
parent b7c59b36a7
commit d6001d033b
No known key found for this signature in database
GPG Key ID: 118759E83439A9B1
11 changed files with 456 additions and 88 deletions

View File

@ -441,7 +441,7 @@ type Config struct {
GcCanceledInvoicesOnTheFly bool `long:"gc-canceled-invoices-on-the-fly" description:"If true, we'll delete newly canceled invoices on the fly."`
DustThreshold uint64 `long:"dust-threshold" description:"Sets the dust sum threshold in satoshis for a channel after which dust HTLC's will be failed."`
MaxFeeExposure uint64 `long:"dust-threshold" description:"Sets the max fee exposure in satoshis for a channel after which HTLC's will be failed."`
Fee *lncfg.Fee `group:"fee" namespace:"fee"`
@ -684,7 +684,7 @@ func DefaultConfig() Config {
MaxOutgoingCltvExpiry: htlcswitch.DefaultMaxOutgoingCltvExpiry,
MaxChannelFeeAllocation: htlcswitch.DefaultMaxLinkFeeAllocation,
MaxCommitFeeRateAnchors: lnwallet.DefaultAnchorsCommitMaxFeeRateSatPerVByte,
DustThreshold: uint64(htlcswitch.DefaultDustThreshold.ToSatoshis()),
MaxFeeExposure: uint64(htlcswitch.DefaultMaxFeeExposure.ToSatoshis()),
LogWriter: build.NewRotatingLogWriter(),
DB: lncfg.DefaultDB(),
Cluster: lncfg.DefaultCluster(),

View File

@ -3,9 +3,11 @@ package htlcswitch
import (
"context"
"github.com/btcsuite/btcd/btcutil"
"github.com/btcsuite/btcd/wire"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/invoices"
"github.com/lightningnetwork/lnd/lntypes"
"github.com/lightningnetwork/lnd/lnwallet/chainfee"
@ -59,8 +61,10 @@ type packetHandler interface {
// whether a link has too much dust exposure.
type dustHandler interface {
// getDustSum returns the dust sum on either the local or remote
// commitment.
getDustSum(remote bool) lnwire.MilliSatoshi
// commitment. An optional fee parameter can be passed in which is used
// to calculate the dust sum.
getDustSum(remote bool,
fee fn.Option[chainfee.SatPerKWeight]) lnwire.MilliSatoshi
// getFeeRate returns the current channel feerate.
getFeeRate() chainfee.SatPerKWeight
@ -68,6 +72,10 @@ type dustHandler interface {
// getDustClosure returns a closure that can evaluate whether a passed
// HTLC is dust.
getDustClosure() dustClosure
// getCommitFee returns the commitment fee in satoshis from either the
// local or remote commitment. This does not include dust.
getCommitFee(remote bool) btcutil.Amount
}
// scidAliasHandler is an interface that the ChannelLink implements so it can

View File

@ -21,6 +21,7 @@ import (
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/htlcswitch/hodl"
"github.com/lightningnetwork/lnd/htlcswitch/hop"
"github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/invoices"
"github.com/lightningnetwork/lnd/lnpeer"
"github.com/lightningnetwork/lnd/lntypes"
@ -278,6 +279,10 @@ type ChannelLinkConfig struct {
// by failing back any blinding-related payloads as if they were
// invalid.
DisallowRouteBlinding bool
// MaxFeeExposure is the threshold in milli-satoshis after which we'll
// restrict the flow of HTLCs and fee updates.
MaxFeeExposure lnwire.MilliSatoshi
}
// channelLink is the service which drives a channel's commitment update
@ -447,6 +452,11 @@ func NewChannelLink(cfg ChannelLinkConfig,
logPrefix := fmt.Sprintf("ChannelLink(%v):", channel.ChannelPoint())
// If the max fee exposure isn't set, use the default.
if cfg.MaxFeeExposure == 0 {
cfg.MaxFeeExposure = DefaultMaxFeeExposure
}
return &channelLink{
cfg: cfg,
channel: channel,
@ -1591,6 +1601,20 @@ func (l *channelLink) handleDownstreamUpdateAdd(pkt *htlcPacket) error {
return nil
}
// Check if we can add the HTLC here without exceededing the max fee
// exposure threshold.
if l.isOverexposedWithHtlc(htlc, false) {
l.log.Debugf("Unable to handle downstream HTLC - max fee " +
"exposure exceeded")
l.mailBox.FailAdd(pkt)
return NewDetailedLinkError(
lnwire.NewTemporaryChannelFailure(nil),
OutgoingFailureDownstreamHtlcAdd,
)
}
// A new payment has been initiated via the downstream channel,
// so we add the new HTLC to our local log, then update the
// commitment chains.
@ -1958,6 +1982,18 @@ func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) {
return
}
// We have to check the limit here rather than later in the
// switch because the counterparty can keep sending HTLC's
// without sending a revoke. This would mean that the switch
// check would only occur later.
if l.isOverexposedWithHtlc(msg, true) {
l.fail(LinkFailureError{code: ErrInternalError},
"peer sent us an HTLC that exceeded our max "+
"fee exposure")
return
}
// We just received an add request from an upstream peer, so we
// add it to our state machine, then add the HTLC to our
// "settle" list in the event that we know the preimage.
@ -2375,9 +2411,32 @@ func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) {
l.RWMutex.Unlock()
case *lnwire.UpdateFee:
// Check and see if their proposed fee-rate would make us
// exceed the fee threshold.
fee := chainfee.SatPerKWeight(msg.FeePerKw)
isDust, err := l.exceedsFeeExposureLimit(fee)
if err != nil {
// This shouldn't typically happen. If it does, it
// indicates something is wrong with our channel state.
l.log.Errorf("Unable to determine if fee threshold " +
"exceeded")
l.fail(LinkFailureError{code: ErrInternalError},
"error calculating fee exposure: %v", err)
return
}
if isDust {
// The proposed fee-rate makes us exceed the fee
// threshold.
l.fail(LinkFailureError{code: ErrInternalError},
"fee threshold exceeded: %v", err)
return
}
// We received fee update from peer. If we are the initiator we
// will fail the channel, if not we will apply the update.
fee := chainfee.SatPerKWeight(msg.FeePerKw)
if err := l.channel.ReceiveUpdateFee(fee); err != nil {
l.fail(LinkFailureError{code: ErrInvalidUpdate},
"error receiving fee update: %v", err)
@ -2668,8 +2727,10 @@ func (l *channelLink) MayAddOutgoingHtlc(amt lnwire.MilliSatoshi) error {
// method.
//
// NOTE: Part of the dustHandler interface.
func (l *channelLink) getDustSum(remote bool) lnwire.MilliSatoshi {
return l.channel.GetDustSum(remote)
func (l *channelLink) getDustSum(remote bool,
dryRunFee fn.Option[chainfee.SatPerKWeight]) lnwire.MilliSatoshi {
return l.channel.GetDustSum(remote, dryRunFee)
}
// getFeeRate is a wrapper method that retrieves the underlying channel's
@ -2692,6 +2753,130 @@ func (l *channelLink) getDustClosure() dustClosure {
return dustHelper(chanType, localDustLimit, remoteDustLimit)
}
// getCommitFee returns either the local or remote CommitFee in satoshis. This
// is used so that the Switch can have access to the commitment fee without
// needing to have a *LightningChannel. This doesn't include dust.
//
// NOTE: Part of the dustHandler interface.
func (l *channelLink) getCommitFee(remote bool) btcutil.Amount {
if remote {
return l.channel.State().RemoteCommitment.CommitFee
}
return l.channel.State().LocalCommitment.CommitFee
}
// exceedsFeeExposureLimit returns whether or not the new proposed fee-rate
// increases the total dust and fees within the channel past the configured
// fee threshold. It first calculates the dust sum over every update in the
// update log with the proposed fee-rate and taking into account both the local
// and remote dust limits. It uses every update in the update log instead of
// what is actually on the local and remote commitments because it is assumed
// that in a worst-case scenario, every update in the update log could
// theoretically be on either commitment transaction and this needs to be
// accounted for with this fee-rate. It then calculates the local and remote
// commitment fees given the proposed fee-rate. Finally, it tallies the results
// and determines if the fee threshold has been exceeded.
func (l *channelLink) exceedsFeeExposureLimit(
feePerKw chainfee.SatPerKWeight) (bool, error) {
dryRunFee := fn.Some[chainfee.SatPerKWeight](feePerKw)
// Get the sum of dust for both the local and remote commitments using
// this "dry-run" fee.
localDustSum := l.getDustSum(false, dryRunFee)
remoteDustSum := l.getDustSum(true, dryRunFee)
// Calculate the local and remote commitment fees using this dry-run
// fee.
localFee, remoteFee, err := l.channel.CommitFeeTotalAt(feePerKw)
if err != nil {
return false, err
}
// Finally, check whether the max fee exposure was exceeded on either
// future commitment transaction with the fee-rate.
totalLocalDust := localDustSum + lnwire.NewMSatFromSatoshis(localFee)
if totalLocalDust > l.cfg.MaxFeeExposure {
return true, nil
}
totalRemoteDust := remoteDustSum + lnwire.NewMSatFromSatoshis(
remoteFee,
)
return totalRemoteDust > l.cfg.MaxFeeExposure, nil
}
// isOverexposedWithHtlc calculates whether the proposed HTLC will make the
// channel exceed the fee threshold. It first fetches the largest fee-rate that
// may be on any unrevoked commitment transaction. Then, using this fee-rate,
// determines if the to-be-added HTLC is dust. If the HTLC is dust, it adds to
// the overall dust sum. If it is not dust, it contributes to weight, which
// also adds to the overall dust sum by an increase in fees. If the dust sum on
// either commitment exceeds the configured fee threshold, this function
// returns true.
func (l *channelLink) isOverexposedWithHtlc(htlc *lnwire.UpdateAddHTLC,
incoming bool) bool {
dustClosure := l.getDustClosure()
feeRate := l.channel.WorstCaseFeeRate()
amount := htlc.Amount.ToSatoshis()
// See if this HTLC is dust on both the local and remote commitments.
isLocalDust := dustClosure(feeRate, incoming, true, amount)
isRemoteDust := dustClosure(feeRate, incoming, false, amount)
// Calculate the dust sum for the local and remote commitments.
localDustSum := l.getDustSum(false, fn.None[chainfee.SatPerKWeight]())
remoteDustSum := l.getDustSum(true, fn.None[chainfee.SatPerKWeight]())
// Grab the larger of the local and remote commitment fees w/o dust.
commitFee := l.getCommitFee(false)
if l.getCommitFee(true) > commitFee {
commitFee = l.getCommitFee(true)
}
localDustSum += lnwire.NewMSatFromSatoshis(commitFee)
remoteDustSum += lnwire.NewMSatFromSatoshis(commitFee)
// Calculate the additional fee increase if this is a non-dust HTLC.
weight := lntypes.WeightUnit(input.HTLCWeight)
additional := lnwire.NewMSatFromSatoshis(
feeRate.FeeForWeight(weight),
)
if isLocalDust {
// If this is dust, it doesn't contribute to weight but does
// contribute to the overall dust sum.
localDustSum += lnwire.NewMSatFromSatoshis(amount)
} else {
// Account for the fee increase that comes with an increase in
// weight.
localDustSum += additional
}
if localDustSum > l.cfg.MaxFeeExposure {
// The max fee exposure was exceeded.
return true
}
if isRemoteDust {
// If this is dust, it doesn't contribute to weight but does
// contribute to the overall dust sum.
remoteDustSum += lnwire.NewMSatFromSatoshis(amount)
} else {
// Account for the fee increase that comes with an increase in
// weight.
remoteDustSum += additional
}
return remoteDustSum > l.cfg.MaxFeeExposure
}
// dustClosure is a function that evaluates whether an HTLC is dust. It returns
// true if the HTLC is dust. It takes in a feerate, a boolean denoting whether
// the HTLC is incoming (i.e. one that the remote sent), a boolean denoting
@ -3060,6 +3245,19 @@ func (l *channelLink) updateChannelFee(feePerKw chainfee.SatPerKWeight) error {
return nil
}
// Check and see if our proposed fee-rate would make us exceed the fee
// threshold.
thresholdExceeded, err := l.exceedsFeeExposureLimit(feePerKw)
if err != nil {
// This shouldn't typically happen. If it does, it indicates
// something is wrong with our channel state.
return err
}
if thresholdExceeded {
return fmt.Errorf("link fee threshold exceeded")
}
// First, we'll update the local fee on our commitment.
if err := l.channel.UpdateFee(feePerKw); err != nil {
return err

View File

@ -4471,10 +4471,20 @@ func TestChannelLinkUpdateCommitFee(t *testing.T) {
// Triggering the link to update the fee of the channel with a fee rate
// that exceeds its maximum fee allocation should result in a fee rate
// corresponding to the maximum fee allocation.
// corresponding to the maximum fee allocation. Increase the dust
// threshold so that we don't trigger that logic.
highFeeExposure := lnwire.NewMSatFromSatoshis(
2 * btcutil.SatoshiPerBitcoin,
)
const maxFeeRate chainfee.SatPerKWeight = 207180182
n.aliceChannelLink.cfg.MaxFeeExposure = highFeeExposure
n.firstBobChannelLink.cfg.MaxFeeExposure = highFeeExposure
triggerFeeUpdate(maxFeeRate+1, minRelayFee, maxFeeRate, true)
// Decrease the max fee exposure back to normal.
n.aliceChannelLink.cfg.MaxFeeExposure = DefaultMaxFeeExposure
n.firstBobChannelLink.cfg.MaxFeeExposure = DefaultMaxFeeExposure
// Triggering the link to update the fee of the channel with a fee rate
// that is below the current min relay fee rate should result in a fee
// rate corresponding to the minimum relay fee.

View File

@ -26,6 +26,7 @@ import (
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/clock"
"github.com/lightningnetwork/lnd/contractcourt"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/htlcswitch/hop"
"github.com/lightningnetwork/lnd/invoices"
"github.com/lightningnetwork/lnd/lnpeer"
@ -200,7 +201,7 @@ func initSwitchWithDB(startingHeight uint32, db *channeldb.DB) (*Switch, error)
HtlcNotifier: &mockHTLCNotifier{},
Clock: clock.NewDefaultClock(),
MailboxDeliveryTimeout: time.Hour,
DustThreshold: DefaultDustThreshold,
MaxFeeExposure: DefaultMaxFeeExposure,
SignAliasUpdate: signAliasUpdate,
IsAlias: isAlias,
}
@ -813,7 +814,9 @@ func (f *mockChannelLink) handleSwitchPacket(pkt *htlcPacket) error {
return nil
}
func (f *mockChannelLink) getDustSum(remote bool) lnwire.MilliSatoshi {
func (f *mockChannelLink) getDustSum(remote bool,
dryRunFee fn.Option[chainfee.SatPerKWeight]) lnwire.MilliSatoshi {
return 0
}
@ -828,6 +831,10 @@ func (f *mockChannelLink) getDustClosure() dustClosure {
)
}
func (f *mockChannelLink) getCommitFee(remote bool) btcutil.Amount {
return 0
}
func (f *mockChannelLink) HandleChannelUpdate(lnwire.Message) {
}

View File

@ -18,6 +18,7 @@ import (
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/clock"
"github.com/lightningnetwork/lnd/contractcourt"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/htlcswitch/hop"
"github.com/lightningnetwork/lnd/kvdb"
"github.com/lightningnetwork/lnd/lntypes"
@ -75,14 +76,15 @@ var (
// failed to be processed.
ErrLocalAddFailed = errors.New("local add HTLC failed")
// errDustThresholdExceeded is only surfaced to callers of SendHTLC and
// signals that sending the HTLC would exceed the outgoing link's dust
// threshold.
errDustThresholdExceeded = errors.New("dust threshold exceeded")
// errFeeExposureExceeded is only surfaced to callers of SendHTLC and
// signals that sending the HTLC would exceed the outgoing link's fee
// exposure threshold.
errFeeExposureExceeded = errors.New("fee exposure exceeded")
// DefaultDustThreshold is the default threshold after which we'll fail
// payments if they are dust. This is currently set to 500m msats.
DefaultDustThreshold = lnwire.MilliSatoshi(500_000_000)
// DefaultMaxFeeExposure is the default threshold after which we'll
// fail payments if they increase our fee exposure. This is currently
// set to 500m msats.
DefaultMaxFeeExposure = lnwire.MilliSatoshi(500_000_000)
)
// plexPacket encapsulates switch packet and adds error channel to receive
@ -210,9 +212,9 @@ type Config struct {
// a mailbox via AddPacket.
MailboxDeliveryTimeout time.Duration
// DustThreshold is the threshold in milli-satoshis after which we'll
// fail incoming or outgoing dust payments for a particular channel.
DustThreshold lnwire.MilliSatoshi
// MaxFeeExposure is the threshold in milli-satoshis after which we'll
// fail incoming or outgoing payments for a particular channel.
MaxFeeExposure lnwire.MilliSatoshi
// SignAliasUpdate is used when sending FailureMessages backwards for
// option_scid_alias channels. This avoids a potential privacy leak by
@ -559,9 +561,9 @@ func (s *Switch) SendHTLC(firstHop lnwire.ShortChannelID, attemptID uint64,
return linkErr
}
// Evaluate whether this HTLC would increase our exposure to dust. If
// it does, don't send it out and instead return an error.
if s.evaluateDustThreshold(link, htlc.Amount, false) {
// Evaluate whether this HTLC would bypass our fee exposure. If it
// does, don't send it out and instead return an error.
if s.dustExceedsFeeThreshold(link, htlc.Amount, false) {
// Notify the htlc notifier of a link failure on our outgoing
// link. We use the FailTemporaryChannelFailure in place of a
// more descriptive error message.
@ -579,7 +581,7 @@ func (s *Switch) SendHTLC(firstHop lnwire.ShortChannelID, attemptID uint64,
false,
)
return errDustThresholdExceeded
return errFeeExposureExceeded
}
circuit := newPaymentCircuit(&htlc.PaymentHash, packet)
@ -1247,9 +1249,10 @@ func (s *Switch) handlePacketForward(packet *htlcPacket) error {
return s.failAddPacket(packet, linkErr)
}
// Evaluate whether this HTLC would increase our exposure to
// dust on the incoming link. If it does, fail it backwards.
if s.evaluateDustThreshold(
// Evaluate whether this HTLC would increase our fee exposure
// over the threshold on the incoming link. If it does, fail it
// backwards.
if s.dustExceedsFeeThreshold(
incomingLink, packet.incomingAmount, true,
) {
// The incoming dust exceeds the threshold, so we fail
@ -1261,9 +1264,10 @@ func (s *Switch) handlePacketForward(packet *htlcPacket) error {
return s.failAddPacket(packet, linkErr)
}
// Also evaluate whether this HTLC would increase our exposure
// to dust on the destination link. If it does, fail it back.
if s.evaluateDustThreshold(
// Also evaluate whether this HTLC would increase our fee
// exposure over the threshold on the destination link. If it
// does, fail it back.
if s.dustExceedsFeeThreshold(
destination, packet.amount, false,
) {
// The outgoing dust exceeds the threshold, so we fail
@ -2768,14 +2772,15 @@ func (s *Switch) BestHeight() uint32 {
return atomic.LoadUint32(&s.bestHeight)
}
// evaluateDustThreshold takes in a ChannelLink, HTLC amount, and a boolean to
// determine whether the default dust threshold has been exceeded. This
// dustExceedsFeeThreshold takes in a ChannelLink, HTLC amount, and a boolean
// to determine whether the default fee threshold has been exceeded. This
// heuristic takes into account the trimmed-to-dust mechanism. The sum of the
// commitment's dust with the mailbox's dust with the amount is checked against
// the default threshold. If incoming is true, then the amount is not included
// in the sum as it was already included in the commitment's dust. A boolean is
// returned telling the caller whether the HTLC should be failed back.
func (s *Switch) evaluateDustThreshold(link ChannelLink,
// the fee exposure threshold. If incoming is true, then the amount is not
// included in the sum as it was already included in the commitment's dust. A
// boolean is returned telling the caller whether the HTLC should be failed
// back.
func (s *Switch) dustExceedsFeeThreshold(link ChannelLink,
amount lnwire.MilliSatoshi, incoming bool) bool {
// Retrieve the link's current commitment feerate and dustClosure.
@ -2801,7 +2806,9 @@ func (s *Switch) evaluateDustThreshold(link ChannelLink,
// If the htlc is dust on the local commitment, we'll obtain the dust
// sum for it.
if isLocalDust {
localSum := link.getDustSum(false)
localSum := link.getDustSum(
false, fn.None[chainfee.SatPerKWeight](),
)
localSum += localMailDust
// Optionally include the HTLC amount only for outgoing
@ -2810,8 +2817,8 @@ func (s *Switch) evaluateDustThreshold(link ChannelLink,
localSum += amount
}
// Finally check against the defined dust threshold.
if localSum > s.cfg.DustThreshold {
// Finally check against the defined fee threshold.
if localSum > s.cfg.MaxFeeExposure {
return true
}
}
@ -2819,7 +2826,9 @@ func (s *Switch) evaluateDustThreshold(link ChannelLink,
// Also check if the htlc is dust on the remote commitment, if we've
// reached this point.
if isRemoteDust {
remoteSum := link.getDustSum(true)
remoteSum := link.getDustSum(
true, fn.None[chainfee.SatPerKWeight](),
)
remoteSum += remoteMailDust
// Optionally include the HTLC amount only for outgoing
@ -2828,8 +2837,8 @@ func (s *Switch) evaluateDustThreshold(link ChannelLink,
remoteSum += amount
}
// Finally check against the defined dust threshold.
if remoteSum > s.cfg.DustThreshold {
// Finally check against the defined fee threshold.
if remoteSum > s.cfg.MaxFeeExposure {
return true
}
}

View File

@ -18,10 +18,12 @@ import (
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/contractcourt"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/htlcswitch/hodl"
"github.com/lightningnetwork/lnd/htlcswitch/hop"
"github.com/lightningnetwork/lnd/lntest/mock"
"github.com/lightningnetwork/lnd/lntypes"
"github.com/lightningnetwork/lnd/lnwallet/chainfee"
"github.com/lightningnetwork/lnd/lnwire"
"github.com/lightningnetwork/lnd/ticker"
"github.com/stretchr/testify/require"
@ -4259,7 +4261,7 @@ func TestInterceptableSwitchWatchDog(t *testing.T) {
}
// TestSwitchDustForwarding tests that the switch properly fails HTLC's which
// have incoming or outgoing links that breach their dust thresholds.
// have incoming or outgoing links that breach their fee thresholds.
func TestSwitchDustForwarding(t *testing.T) {
t.Parallel()
@ -4288,14 +4290,15 @@ func TestSwitchDustForwarding(t *testing.T) {
// We'll test that once the default threshold is exceeded on the
// Alice -> Bob channel, either side's calls to SendHTLC will fail.
//
// Alice will send 357 HTLC's of 700sats. Bob will also send 357 HTLC's
// of 700sats. If either side attempts to send a dust HTLC, it will
// fail so amounts below 800sats will breach the dust threshold.
// Alice will send 354 HTLC's of 700sats. Bob will also send 354 HTLC's
// of 700sats.
numHTLCs := 354
aliceAttemptID, bobAttemptID := numHTLCs, numHTLCs
amt := lnwire.NewMSatFromSatoshis(700)
aliceBobFirstHop := n.aliceChannelLink.ShortChanID()
sendDustHtlcs(t, n, true, amt, aliceBobFirstHop)
sendDustHtlcs(t, n, false, amt, aliceBobFirstHop)
sendDustHtlcs(t, n, true, amt, aliceBobFirstHop, numHTLCs)
sendDustHtlcs(t, n, false, amt, aliceBobFirstHop, numHTLCs)
// Generate the parameters needed for Bob to send another dust HTLC.
_, timelock, hops := generateHops(
@ -4320,7 +4323,7 @@ func TestSwitchDustForwarding(t *testing.T) {
timeout := time.After(15 * time.Second)
pollInterval := 300 * time.Millisecond
expectedDust := 357 * 2 * amt
expectedDust := 354 * 2 * amt
for {
<-time.After(pollInterval)
@ -4331,7 +4334,9 @@ func TestSwitchDustForwarding(t *testing.T) {
default:
}
linkDust := link.getDustSum(remote)
linkDust := link.getDustSum(
remote, fn.None[chainfee.SatPerKWeight](),
)
localMailDust, remoteMailDust := mbox.DustPackets()
totalDust := linkDust
@ -4349,18 +4354,34 @@ func TestSwitchDustForwarding(t *testing.T) {
return true
}
// Wait until Bob is almost at the dust threshold.
// Wait until Bob is almost at the fee threshold.
bobMbox := n.bobServer.htlcSwitch.mailOrchestrator.GetOrCreateMailBox(
n.firstBobChannelLink.ChanID(),
n.firstBobChannelLink.ShortChanID(),
)
require.True(t, checkAlmostDust(n.firstBobChannelLink, bobMbox, false))
// Assert that the HTLC is failed due to the dust threshold.
// Sending one more HTLC should fail. SendHTLC won't error, but the
// HTLC should be failed backwards.
err = n.bobServer.htlcSwitch.SendHTLC(
aliceBobFirstHop, uint64(357), failingHtlc,
aliceBobFirstHop, uint64(bobAttemptID), failingHtlc,
)
require.ErrorIs(t, err, errDustThresholdExceeded)
require.Nil(t, err)
// Use the network result store to ensure the HTLC was failed
// backwards.
bobResultChan, err := n.bobServer.htlcSwitch.GetAttemptResult(
uint64(bobAttemptID), failingHash, newMockDeobfuscator(),
)
require.NoError(t, err)
result, ok := <-bobResultChan
require.True(t, ok)
assertFailureCode(
t, result.Error, lnwire.CodeTemporaryChannelFailure,
)
bobAttemptID++
// Generate the parameters needed for bob to send a non-dust HTLC.
nondustAmt := lnwire.NewMSatFromSatoshis(10_000)
@ -4371,8 +4392,9 @@ func TestSwitchDustForwarding(t *testing.T) {
blob, err = generateRoute(hops...)
require.NoError(t, err)
// Now attempt to send an HTLC above Bob's dust limit. It should
// succeed.
// Now attempt to send an HTLC above Bob's dust limit. Even though this
// is not a dust HTLC, it should fail because the increase in weight
// pushes us over the threshold.
nondustPreimage := lntypes.Preimage{0, 0, 4}
nondustHash := nondustPreimage.Hash()
nondustHtlc := &lnwire.UpdateAddHTLC{
@ -4382,12 +4404,23 @@ func TestSwitchDustForwarding(t *testing.T) {
OnionBlob: blob,
}
// Assert that SendHTLC succeeds and evaluateDustThreshold returns
// false.
err = n.bobServer.htlcSwitch.SendHTLC(
aliceBobFirstHop, uint64(358), nondustHtlc,
aliceBobFirstHop, uint64(bobAttemptID), nondustHtlc,
)
require.NoError(t, err)
require.True(t, checkAlmostDust(n.firstBobChannelLink, bobMbox, false))
// Check that the HTLC failed.
bobResultChan, err = n.bobServer.htlcSwitch.GetAttemptResult(
uint64(bobAttemptID), nondustHash, newMockDeobfuscator(),
)
require.NoError(t, err)
result, ok = <-bobResultChan
require.True(t, ok)
assertFailureCode(
t, result.Error, lnwire.CodeTemporaryChannelFailure,
)
// Introduce Carol into the mix and assert that sending a multi-hop
// dust HTLC to Alice will fail. Bob should fail back the HTLC with a
@ -4417,21 +4450,19 @@ func TestSwitchDustForwarding(t *testing.T) {
carolHtlc,
)
require.NoError(t, err)
carolAttemptID++
carolResultChan, err := n.carolServer.htlcSwitch.GetAttemptResult(
uint64(carolAttemptID-1), carolHash, newMockDeobfuscator(),
uint64(carolAttemptID), carolHash, newMockDeobfuscator(),
)
require.NoError(t, err)
result, ok := <-carolResultChan
result, ok = <-carolResultChan
require.True(t, ok)
assertFailureCode(
t, result.Error, lnwire.CodeTemporaryChannelFailure,
)
// Send an HTLC from Alice to Carol and assert that it is failed at the
// call to SendHTLC.
// Send an HTLC from Alice to Carol and assert that it gets failed.
htlcAmt, totalTimelock, aliceHops := generateHops(
amt, testStartingHeight, n.firstBobChannelLink,
n.carolChannelLink,
@ -4450,7 +4481,7 @@ func TestSwitchDustForwarding(t *testing.T) {
}
// Wait until Alice's expected dust for the remote commitment is just
// under the dust threshold.
// under the fee threshold.
aliceOrch := n.aliceServer.htlcSwitch.mailOrchestrator
aliceMbox := aliceOrch.GetOrCreateMailBox(
n.aliceChannelLink.ChanID(), n.aliceChannelLink.ShortChanID(),
@ -4458,23 +4489,36 @@ func TestSwitchDustForwarding(t *testing.T) {
require.True(t, checkAlmostDust(n.aliceChannelLink, aliceMbox, true))
err = n.aliceServer.htlcSwitch.SendHTLC(
n.aliceChannelLink.ShortChanID(), uint64(357),
n.aliceChannelLink.ShortChanID(), uint64(aliceAttemptID),
aliceMultihopHtlc,
)
require.ErrorIs(t, err, errDustThresholdExceeded)
require.Nil(t, err)
aliceResultChan, err := n.aliceServer.htlcSwitch.GetAttemptResult(
uint64(aliceAttemptID), aliceMultihopHash,
newMockDeobfuscator(),
)
require.NoError(t, err)
result, ok = <-aliceResultChan
require.True(t, ok)
assertFailureCode(
t, result.Error, lnwire.CodeTemporaryChannelFailure,
)
// Check that there are numHTLCs circuits open for both Alice and Bob.
require.Equal(t, numHTLCs, n.aliceServer.htlcSwitch.circuits.NumOpen())
require.Equal(t, numHTLCs, n.bobServer.htlcSwitch.circuits.NumOpen())
}
// sendDustHtlcs is a helper function used to send many dust HTLC's to test the
// Switch's dust-threshold logic. It takes a boolean denoting whether or not
// Alice is the sender.
func sendDustHtlcs(t *testing.T, n *threeHopNetwork, alice bool,
amt lnwire.MilliSatoshi, sid lnwire.ShortChannelID) {
amt lnwire.MilliSatoshi, sid lnwire.ShortChannelID, numHTLCs int) {
t.Helper()
// The number of dust HTLC's we'll send for both Alice and Bob.
numHTLCs := 357
// Extract the destination into a variable. If alice is the sender, the
// destination is Bob.
destLink := n.aliceChannelLink
@ -4527,8 +4571,8 @@ func sendDustHtlcs(t *testing.T, n *threeHopNetwork, alice bool,
}
for {
// It may be the case that the dust threshold is hit
// before all 357*2 HTLC's are sent due to double
// It may be the case that the fee threshold is hit
// before all numHTLCs*2 HTLC's are sent due to double
// counting. Get around this by continuing to send
// until successful.
err = sendingSwitch.SendHTLC(sid, attemptID, htlc)
@ -4542,7 +4586,7 @@ func sendDustHtlcs(t *testing.T, n *threeHopNetwork, alice bool,
}
// TestSwitchMailboxDust tests that the switch takes into account the mailbox
// dust when evaluating the dust threshold. The mockChannelLink does not have
// dust when evaluating the fee threshold. The mockChannelLink does not have
// channel state, so this only tests the switch-mailbox interaction.
func TestSwitchMailboxDust(t *testing.T) {
t.Parallel()
@ -4612,7 +4656,7 @@ func TestSwitchMailboxDust(t *testing.T) {
var carolHTLCID uint64
// It will take aliceCount HTLC's of 350sats to fill up Alice's mailbox
// to the point where another would put Alice over the dust threshold.
// to the point where another would put Alice over the fee threshold.
aliceCount := 1428
mailbox := s.mailOrchestrator.GetOrCreateMailBox(chanID1, aliceChanID)
@ -4634,10 +4678,10 @@ func TestSwitchMailboxDust(t *testing.T) {
carolHTLCID++
}
// Sending one more HTLC to Alice should result in the dust threshold
// Sending one more HTLC to Alice should result in the fee threshold
// being breached.
err = s.SendHTLC(aliceChanID, 0, addMsg)
require.ErrorIs(t, err, errDustThresholdExceeded)
require.ErrorIs(t, err, errFeeExposureExceeded)
// We'll now call ForwardPackets from Bob to ensure that the mailbox
// sum is also accounted for in the forwarding case.

View File

@ -26,6 +26,7 @@ import (
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/keychain"
"github.com/lightningnetwork/lnd/lntypes"
@ -2989,7 +2990,7 @@ func (lc *LightningChannel) fetchCommitmentView(remoteChain bool,
// initiator.
htlcView := lc.fetchHTLCView(theirLogIndex, ourLogIndex)
ourBalance, theirBalance, _, filteredHTLCView, err := lc.computeView(
htlcView, remoteChain, true,
htlcView, remoteChain, true, fn.None[chainfee.SatPerKWeight](),
)
if err != nil {
return nil, err
@ -3949,7 +3950,7 @@ func (lc *LightningChannel) validateCommitmentSanity(theirLogCounter,
}
ourBalance, theirBalance, commitWeight, filteredView, err := lc.computeView(
view, remoteChain, false,
view, remoteChain, false, fn.None[chainfee.SatPerKWeight](),
)
if err != nil {
return err
@ -4711,13 +4712,15 @@ func (lc *LightningChannel) ProcessChanSyncMsg(
// view (settling unsettled HTLCs), commitment weight and feePerKw, after
// applying the HTLCs to the latest commitment. The returned balances are the
// balances *before* subtracting the commitment fee from the initiator's
// balance.
// balance. It accepts a "dry run" feerate argument to calculate a potential
// commitment transaction fee.
//
// If the updateState boolean is set true, the add and remove heights of the
// HTLCs will be set to the next commitment height.
func (lc *LightningChannel) computeView(view *htlcView, remoteChain bool,
updateState bool) (lnwire.MilliSatoshi, lnwire.MilliSatoshi,
lntypes.WeightUnit, *htlcView, error) {
updateState bool, dryRunFee fn.Option[chainfee.SatPerKWeight]) (
lnwire.MilliSatoshi, lnwire.MilliSatoshi, lntypes.WeightUnit,
*htlcView, error) {
commitChain := lc.localCommitChain
dustLimit := lc.channelState.LocalChanCfg.DustLimit
@ -4763,6 +4766,12 @@ func (lc *LightningChannel) computeView(view *htlcView, remoteChain bool,
}
feePerKw := filteredHTLCView.feePerKw
// Here we override the view's fee-rate if a dry-run fee-rate was
// passed in.
if !updateState {
feePerKw = dryRunFee.UnwrapOr(feePerKw)
}
// We need to first check ourBalance and theirBalance to be negative
// because MilliSathoshi is a unsigned type and can underflow in
// `evaluateHTLCView`. This should never happen for views which do not
@ -5954,7 +5963,9 @@ func (lc *LightningChannel) addHTLC(htlc *lnwire.UpdateAddHTLC,
// commitment tx.
//
// NOTE: This over-estimates the dust exposure.
func (lc *LightningChannel) GetDustSum(remote bool) lnwire.MilliSatoshi {
func (lc *LightningChannel) GetDustSum(remote bool,
dryRunFee fn.Option[chainfee.SatPerKWeight]) lnwire.MilliSatoshi {
lc.RLock()
defer lc.RUnlock()
@ -5971,6 +5982,9 @@ func (lc *LightningChannel) GetDustSum(remote bool) lnwire.MilliSatoshi {
chanType := lc.channelState.ChanType
feeRate := chainfee.SatPerKWeight(commit.FeePerKw)
// Optionally use the dry-run fee-rate.
feeRate = dryRunFee.UnwrapOr(feeRate)
// Grab all of our HTLCs and evaluate against the dust limit.
for e := lc.localUpdateLog.Front(); e != nil; e = e.Next() {
pd := e.Value.(*PaymentDescriptor)
@ -8256,7 +8270,7 @@ func (lc *LightningChannel) availableCommitmentBalance(
// into account HTLCs to determine the commit weight, which the
// initiator must pay the fee for.
ourBalance, theirBalance, commitWeight, filteredView, err := lc.computeView(
view, remoteChain, false,
view, remoteChain, false, fn.None[chainfee.SatPerKWeight](),
)
if err != nil {
lc.log.Errorf("Unable to fetch available balance: %v", err)
@ -8456,6 +8470,54 @@ func (lc *LightningChannel) UpdateFee(feePerKw chainfee.SatPerKWeight) error {
return nil
}
// CommitFeeTotalAt applies a proposed feerate to the channel and returns the
// commitment fee with this new feerate. It does not modify the underlying
// LightningChannel.
func (lc *LightningChannel) CommitFeeTotalAt(
feePerKw chainfee.SatPerKWeight) (btcutil.Amount, btcutil.Amount,
error) {
lc.RLock()
defer lc.RUnlock()
dryRunFee := fn.Some[chainfee.SatPerKWeight](feePerKw)
// We want to grab every update in both update logs to calculate the
// commitment fees in the worst-case with this fee-rate.
localIdx := lc.localUpdateLog.logIndex
remoteIdx := lc.remoteUpdateLog.logIndex
localHtlcView := lc.fetchHTLCView(remoteIdx, localIdx)
var localCommitFee, remoteCommitFee btcutil.Amount
// Compute the local commitment's weight.
_, _, localWeight, _, err := lc.computeView(
localHtlcView, false, false, dryRunFee,
)
if err != nil {
return 0, 0, err
}
localCommitFee = feePerKw.FeeForWeight(localWeight)
// Create another view in case for some reason the prior one was
// mutated.
remoteHtlcView := lc.fetchHTLCView(remoteIdx, localIdx)
// Compute the remote commitment's weight.
_, _, remoteWeight, _, err := lc.computeView(
remoteHtlcView, true, false, dryRunFee,
)
if err != nil {
return 0, 0, err
}
remoteCommitFee = feePerKw.FeeForWeight(remoteWeight)
return localCommitFee, remoteCommitFee, err
}
// ReceiveUpdateFee handles an updated fee sent from remote. This method will
// return an error if called as channel initiator.
func (lc *LightningChannel) ReceiveUpdateFee(feePerKw chainfee.SatPerKWeight) error {
@ -8810,6 +8872,22 @@ func (lc *LightningChannel) CommitFeeRate() chainfee.SatPerKWeight {
return chainfee.SatPerKWeight(lc.channelState.LocalCommitment.FeePerKw)
}
// WorstCaseFeeRate returns the higher feerate from either the local commitment
// or the remote commitment.
func (lc *LightningChannel) WorstCaseFeeRate() chainfee.SatPerKWeight {
lc.RLock()
defer lc.RUnlock()
localFeeRate := lc.channelState.LocalCommitment.FeePerKw
remoteFeeRate := lc.channelState.RemoteCommitment.FeePerKw
if localFeeRate > remoteFeeRate {
return chainfee.SatPerKWeight(localFeeRate)
}
return chainfee.SatPerKWeight(remoteFeeRate)
}
// IsPending returns true if the channel's funding transaction has been fully
// confirmed, and false otherwise.
func (lc *LightningChannel) IsPending() bool {

View File

@ -21,6 +21,7 @@ import (
"github.com/davecgh/go-spew/spew"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/lntypes"
"github.com/lightningnetwork/lnd/lnwallet/chainfee"
@ -9750,9 +9751,13 @@ func testGetDustSum(t *testing.T, chantype channeldb.ChannelType) {
checkDust := func(c *LightningChannel, expLocal,
expRemote lnwire.MilliSatoshi) {
localDustSum := c.GetDustSum(false)
localDustSum := c.GetDustSum(
false, fn.None[chainfee.SatPerKWeight](),
)
require.Equal(t, expLocal, localDustSum)
remoteDustSum := c.GetDustSum(true)
remoteDustSum := c.GetDustSum(
true, fn.None[chainfee.SatPerKWeight](),
)
require.Equal(t, expRemote, remoteDustSum)
}

View File

@ -381,6 +381,10 @@ type Config struct {
// invalid.
DisallowRouteBlinding bool
// MaxFeeExposure limits the number of outstanding fees in a channel.
// This value will be passed to created links.
MaxFeeExposure lnwire.MilliSatoshi
// Quit is the server's quit channel. If this is closed, we halt operation.
Quit chan struct{}
}
@ -1194,6 +1198,7 @@ func (p *Brontide) addLink(chanPoint *wire.OutPoint,
GetAliases: p.cfg.GetAliases,
PreviouslySentShutdown: shutdownMsg,
DisallowRouteBlinding: p.cfg.DisallowRouteBlinding,
MaxFeeExposure: p.cfg.MaxFeeExposure,
}
// Before adding our new link, purge the switch of any pending or live

View File

@ -638,7 +638,7 @@ func newServer(cfg *Config, listenAddrs []net.Addr,
s.htlcNotifier = htlcswitch.NewHtlcNotifier(time.Now)
thresholdSats := btcutil.Amount(cfg.DustThreshold)
thresholdSats := btcutil.Amount(cfg.MaxFeeExposure)
thresholdMSats := lnwire.NewMSatFromSatoshis(thresholdSats)
s.aliasMgr, err = aliasmgr.NewManager(dbs.ChanStateDB)
@ -678,7 +678,7 @@ func newServer(cfg *Config, listenAddrs []net.Addr,
RejectHTLC: cfg.RejectHTLC,
Clock: clock.NewDefaultClock(),
MailboxDeliveryTimeout: cfg.Htlcswitch.MailboxDeliveryTimeout,
DustThreshold: thresholdMSats,
MaxFeeExposure: thresholdMSats,
SignAliasUpdate: s.signAliasUpdate,
IsAlias: aliasmgr.IsAlias,
}, uint32(currentHeight))
@ -3863,6 +3863,9 @@ func (s *server) peerConnected(conn net.Conn, connReq *connmgr.ConnReq,
towerClient = s.towerClientMgr
}
thresholdSats := btcutil.Amount(s.cfg.MaxFeeExposure)
thresholdMSats := lnwire.NewMSatFromSatoshis(thresholdSats)
// Now that we've established a connection, create a peer, and it to the
// set of currently active peers. Configure the peer with the incoming
// and outgoing broadcast deltas to prevent htlcs from being accepted or
@ -3932,6 +3935,7 @@ func (s *server) peerConnected(conn net.Conn, connReq *connmgr.ConnReq,
RequestAlias: s.aliasMgr.RequestAlias,
AddLocalAlias: s.aliasMgr.AddLocalAlias,
DisallowRouteBlinding: s.cfg.ProtocolOptions.NoRouteBlinding(),
MaxFeeExposure: thresholdMSats,
Quit: s.quit,
}