mirror of
https://github.com/lightningnetwork/lnd.git
synced 2025-12-04 18:01:57 +01:00
htlcswitch/link: remove overflow queue
This commit removes the overflowQueue from the link. We do so in order to promote better UX for senders, so that HTLCs are failed faster when the commitment is full. This gives the sender the opportunity to try another, more open path, rather than perceive the HTLC as being stuck. At the same time, we remove the total number of active goroutines in lnd by a factor of N where N is the number of active channels.
This commit is contained in:
@@ -7,7 +7,6 @@ import (
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"net"
|
||||
"reflect"
|
||||
"runtime"
|
||||
@@ -263,141 +262,6 @@ func TestChannelLinkSingleHopPayment(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestChannelLinkBidirectionalOneHopPayments tests the ability of channel
|
||||
// link to cope with bigger number of payment updates that commitment
|
||||
// transaction may consist.
|
||||
func TestChannelLinkBidirectionalOneHopPayments(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
channels, cleanUp, _, err := createClusterChannels(
|
||||
btcutil.SatoshiPerBitcoin*3,
|
||||
btcutil.SatoshiPerBitcoin*5)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create channel: %v", err)
|
||||
}
|
||||
defer cleanUp()
|
||||
|
||||
n := newThreeHopNetwork(t, channels.aliceToBob, channels.bobToAlice,
|
||||
channels.bobToCarol, channels.carolToBob, testStartingHeight)
|
||||
if err := n.start(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer n.stop()
|
||||
bobBandwidthBefore := n.firstBobChannelLink.Bandwidth()
|
||||
aliceBandwidthBefore := n.aliceChannelLink.Bandwidth()
|
||||
|
||||
debug := false
|
||||
if debug {
|
||||
// Log message that alice receives.
|
||||
n.aliceServer.intersect(createLogFunc("alice",
|
||||
n.aliceChannelLink.ChanID()))
|
||||
|
||||
// Log message that bob receives.
|
||||
n.bobServer.intersect(createLogFunc("bob",
|
||||
n.firstBobChannelLink.ChanID()))
|
||||
}
|
||||
|
||||
amt := lnwire.NewMSatFromSatoshis(20000)
|
||||
|
||||
htlcAmt, totalTimelock, hopsForwards := generateHops(amt,
|
||||
testStartingHeight, n.firstBobChannelLink)
|
||||
_, _, hopsBackwards := generateHops(amt,
|
||||
testStartingHeight, n.aliceChannelLink)
|
||||
|
||||
type result struct {
|
||||
err error
|
||||
start time.Time
|
||||
number int
|
||||
sender string
|
||||
}
|
||||
|
||||
// Send max available payment number in both sides, thereby testing
|
||||
// the property of channel link to cope with overflowing.
|
||||
count := 2 * input.MaxHTLCNumber
|
||||
resultChan := make(chan *result, count)
|
||||
for i := 0; i < count/2; i++ {
|
||||
go func(i int) {
|
||||
r := &result{
|
||||
start: time.Now(),
|
||||
number: i,
|
||||
sender: "alice",
|
||||
}
|
||||
|
||||
firstHop := n.firstBobChannelLink.ShortChanID()
|
||||
_, r.err = makePayment(
|
||||
n.aliceServer, n.bobServer, firstHop,
|
||||
hopsForwards, amt, htlcAmt, totalTimelock,
|
||||
).Wait(5 * time.Minute)
|
||||
resultChan <- r
|
||||
}(i)
|
||||
}
|
||||
|
||||
for i := 0; i < count/2; i++ {
|
||||
go func(i int) {
|
||||
r := &result{
|
||||
start: time.Now(),
|
||||
number: i,
|
||||
sender: "bob",
|
||||
}
|
||||
|
||||
firstHop := n.aliceChannelLink.ShortChanID()
|
||||
_, r.err = makePayment(
|
||||
n.bobServer, n.aliceServer, firstHop,
|
||||
hopsBackwards, amt, htlcAmt, totalTimelock,
|
||||
).Wait(5 * time.Minute)
|
||||
resultChan <- r
|
||||
}(i)
|
||||
}
|
||||
|
||||
maxDelay := time.Duration(0)
|
||||
minDelay := time.Duration(math.MaxInt64)
|
||||
averageDelay := time.Duration(0)
|
||||
|
||||
// Check that alice invoice was settled and bandwidth of HTLC
|
||||
// links was changed.
|
||||
for i := 0; i < count; i++ {
|
||||
select {
|
||||
case r := <-resultChan:
|
||||
if r.err != nil {
|
||||
t.Fatalf("unable to make payment: %v", r.err)
|
||||
}
|
||||
|
||||
delay := time.Since(r.start)
|
||||
if delay > maxDelay {
|
||||
maxDelay = delay
|
||||
}
|
||||
|
||||
if delay < minDelay {
|
||||
minDelay = delay
|
||||
}
|
||||
averageDelay += delay
|
||||
|
||||
case <-time.After(5 * time.Minute):
|
||||
t.Fatalf("timeout: (%v/%v)", i+1, count)
|
||||
}
|
||||
}
|
||||
|
||||
// TODO(roasbeef): should instead consume async notifications from both
|
||||
// links
|
||||
time.Sleep(time.Second * 2)
|
||||
|
||||
// At the end Bob and Alice balances should be the same as previous,
|
||||
// because they sent the equal amount of money to each other.
|
||||
if aliceBandwidthBefore != n.aliceChannelLink.Bandwidth() {
|
||||
t.Fatalf("alice bandwidth shouldn't have changed: expected %v, got %x",
|
||||
aliceBandwidthBefore, n.aliceChannelLink.Bandwidth())
|
||||
}
|
||||
|
||||
if bobBandwidthBefore != n.firstBobChannelLink.Bandwidth() {
|
||||
t.Fatalf("bob bandwidth shouldn't have changed: expected %v, got %v",
|
||||
bobBandwidthBefore, n.firstBobChannelLink.Bandwidth())
|
||||
}
|
||||
|
||||
t.Logf("Max waiting: %v", maxDelay)
|
||||
t.Logf("Min waiting: %v", minDelay)
|
||||
t.Logf("Average waiting: %v", time.Duration(int(averageDelay)/count))
|
||||
}
|
||||
|
||||
// TestChannelLinkMultiHopPayment checks the ability to send payment over two
|
||||
// hops. In this test we send the payment from Carol to Alice over Bob peer.
|
||||
// (Carol -> Bob -> Alice) and checking that HTLC was settled properly and
|
||||
@@ -536,6 +400,105 @@ func testChannelLinkMultiHopPayment(t *testing.T,
|
||||
}
|
||||
}
|
||||
|
||||
// TestChannelLinkCancelFullCommitment tests the ability for links to cancel
|
||||
// forwarded HTLCs once all of their commitment slots are full.
|
||||
func TestChannelLinkCancelFullCommitment(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
channels, cleanUp, _, err := createClusterChannels(
|
||||
btcutil.SatoshiPerBitcoin*3,
|
||||
btcutil.SatoshiPerBitcoin*5)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create channel: %v", err)
|
||||
}
|
||||
defer cleanUp()
|
||||
|
||||
n := newTwoHopNetwork(
|
||||
t, channels.aliceToBob, channels.bobToAlice, testStartingHeight,
|
||||
)
|
||||
if err := n.start(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer n.stop()
|
||||
|
||||
// Fill up the commitment from Alice's side with 20 sat payments.
|
||||
count := (input.MaxHTLCNumber / 2)
|
||||
amt := lnwire.NewMSatFromSatoshis(20000)
|
||||
|
||||
htlcAmt, totalTimelock, hopsForwards := generateHops(amt,
|
||||
testStartingHeight, n.bobChannelLink)
|
||||
|
||||
firstHop := n.aliceChannelLink.ShortChanID()
|
||||
|
||||
// Create channels to buffer the preimage and error channels used in
|
||||
// making the preliminary payments.
|
||||
preimages := make([]lntypes.Preimage, count)
|
||||
aliceErrChan := make(chan chan error, count)
|
||||
|
||||
var wg sync.WaitGroup
|
||||
for i := 0; i < count; i++ {
|
||||
preimages[i] = lntypes.Preimage{byte(i >> 8), byte(i)}
|
||||
|
||||
wg.Add(1)
|
||||
go func(i int) {
|
||||
defer wg.Done()
|
||||
|
||||
errChan := n.makeHoldPayment(
|
||||
n.aliceServer, n.bobServer, firstHop,
|
||||
hopsForwards, amt, htlcAmt, totalTimelock,
|
||||
preimages[i],
|
||||
)
|
||||
aliceErrChan <- errChan
|
||||
}(i)
|
||||
}
|
||||
|
||||
// Wait for Alice to finish filling her commitment.
|
||||
wg.Wait()
|
||||
close(aliceErrChan)
|
||||
|
||||
// Now make an additional payment from Alice to Bob, this should be
|
||||
// canceled because the commitment in this direction is full.
|
||||
err = <-makePayment(
|
||||
n.aliceServer, n.bobServer, firstHop, hopsForwards, amt,
|
||||
htlcAmt, totalTimelock,
|
||||
).err
|
||||
if err == nil {
|
||||
t.Fatalf("overflow payment should have failed")
|
||||
}
|
||||
lerr, ok := err.(*LinkError)
|
||||
if !ok {
|
||||
t.Fatalf("expected LinkError, got: %T", err)
|
||||
}
|
||||
|
||||
msg := lerr.WireMessage()
|
||||
if _, ok := msg.(*lnwire.FailTemporaryChannelFailure); !ok {
|
||||
t.Fatalf("expected TemporaryChannelFailure, got: %T", msg)
|
||||
}
|
||||
|
||||
// Now, settle all htlcs held by bob and clear the commitment of htlcs.
|
||||
for _, preimage := range preimages {
|
||||
preimage := preimage
|
||||
|
||||
// It's possible that the HTLCs have not been delivered to the
|
||||
// invoice registry at this point, so we poll until we are able
|
||||
// to settle.
|
||||
err = wait.NoError(func() error {
|
||||
return n.bobServer.registry.SettleHodlInvoice(preimage)
|
||||
}, time.Minute)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure that all of the payments sent by alice eventually succeed.
|
||||
for errChan := range aliceErrChan {
|
||||
err := <-errChan
|
||||
if err != nil {
|
||||
t.Fatalf("alice payment failed: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestExitNodeTimelockPayloadMismatch tests that when an exit node receives an
|
||||
// incoming HTLC, if the time lock encoded in the payload of the forwarded HTLC
|
||||
// doesn't match the expected payment value, then the HTLC will be rejected
|
||||
@@ -2369,227 +2332,6 @@ func TestChannelLinkBandwidthConsistency(t *testing.T) {
|
||||
assertLinkBandwidth(t, aliceLink, aliceStartingBandwidth)
|
||||
}
|
||||
|
||||
// TestChannelLinkBandwidthConsistencyOverflow tests that in the case of a
|
||||
// commitment overflow (no more space for new HTLC's), the bandwidth is updated
|
||||
// properly as items are being added and removed from the overflow queue.
|
||||
func TestChannelLinkBandwidthConsistencyOverflow(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
var mockBlob [lnwire.OnionPacketSize]byte
|
||||
|
||||
const chanAmt = btcutil.SatoshiPerBitcoin * 5
|
||||
aliceLink, bobChannel, batchTick, start, cleanUp, _, err :=
|
||||
newSingleLinkTestHarness(chanAmt, 0)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create link: %v", err)
|
||||
}
|
||||
defer cleanUp()
|
||||
|
||||
if err := start(); err != nil {
|
||||
t.Fatalf("unable to start test harness: %v", err)
|
||||
}
|
||||
|
||||
var (
|
||||
coreLink = aliceLink.(*channelLink)
|
||||
defaultCommitFee = coreLink.channel.StateSnapshot().CommitFee
|
||||
aliceStartingBandwidth = aliceLink.Bandwidth()
|
||||
aliceMsgs = coreLink.cfg.Peer.(*mockPeer).sentMsgs
|
||||
)
|
||||
|
||||
estimator := chainfee.NewStaticEstimator(6000, 0)
|
||||
feePerKw, err := estimator.EstimateFeePerKW(1)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to query fee estimator: %v", err)
|
||||
}
|
||||
|
||||
var htlcID uint64
|
||||
addLinkHTLC := func(id uint64, amt lnwire.MilliSatoshi) [32]byte {
|
||||
invoice, htlc, _, err := generatePayment(
|
||||
amt, amt, 5, mockBlob,
|
||||
)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create payment: %v", err)
|
||||
}
|
||||
|
||||
addPkt := &htlcPacket{
|
||||
htlc: htlc,
|
||||
incomingHTLCID: id,
|
||||
amount: amt,
|
||||
obfuscator: NewMockObfuscator(),
|
||||
}
|
||||
circuit := makePaymentCircuit(&htlc.PaymentHash, addPkt)
|
||||
_, err = coreLink.cfg.Switch.commitCircuits(&circuit)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to commit circuit: %v", err)
|
||||
}
|
||||
|
||||
addPkt.circuit = &circuit
|
||||
aliceLink.HandleSwitchPacket(addPkt)
|
||||
return invoice.Terms.PaymentPreimage
|
||||
}
|
||||
|
||||
// We'll first start by adding enough HTLC's to overflow the commitment
|
||||
// transaction, checking the reported link bandwidth for proper
|
||||
// consistency along the way
|
||||
htlcAmt := lnwire.NewMSatFromSatoshis(100000)
|
||||
totalHtlcAmt := lnwire.MilliSatoshi(0)
|
||||
const numHTLCs = input.MaxHTLCNumber / 2
|
||||
var preImages [][32]byte
|
||||
for i := 0; i < numHTLCs; i++ {
|
||||
preImage := addLinkHTLC(htlcID, htlcAmt)
|
||||
preImages = append(preImages, preImage)
|
||||
|
||||
totalHtlcAmt += htlcAmt
|
||||
htlcID++
|
||||
}
|
||||
|
||||
// The HTLCs should all be sent to the remote.
|
||||
var msg lnwire.Message
|
||||
for i := 0; i < numHTLCs; i++ {
|
||||
select {
|
||||
case msg = <-aliceMsgs:
|
||||
case <-time.After(15 * time.Second):
|
||||
t.Fatalf("did not receive message %d", i)
|
||||
}
|
||||
|
||||
addHtlc, ok := msg.(*lnwire.UpdateAddHTLC)
|
||||
if !ok {
|
||||
t.Fatalf("expected UpdateAddHTLC, got %T", msg)
|
||||
}
|
||||
|
||||
_, err := bobChannel.ReceiveHTLC(addHtlc)
|
||||
if err != nil {
|
||||
t.Fatalf("bob failed receiving htlc: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
select {
|
||||
case msg = <-aliceMsgs:
|
||||
t.Fatalf("unexpected message: %T", msg)
|
||||
case <-time.After(20 * time.Millisecond):
|
||||
}
|
||||
|
||||
// TODO(roasbeef): increase sleep
|
||||
time.Sleep(time.Second * 1)
|
||||
commitWeight := int64(input.CommitWeight + input.HTLCWeight*numHTLCs)
|
||||
htlcFee := lnwire.NewMSatFromSatoshis(
|
||||
feePerKw.FeeForWeight(commitWeight),
|
||||
)
|
||||
expectedBandwidth := aliceStartingBandwidth - totalHtlcAmt - htlcFee
|
||||
expectedBandwidth += lnwire.NewMSatFromSatoshis(defaultCommitFee)
|
||||
assertLinkBandwidth(t, aliceLink, expectedBandwidth)
|
||||
|
||||
// The overflow queue should be empty at this point, as the commitment
|
||||
// transaction should be full, but not yet overflown.
|
||||
if coreLink.overflowQueue.Length() != 0 {
|
||||
t.Fatalf("wrong overflow queue length: expected %v, got %v", 0,
|
||||
coreLink.overflowQueue.Length())
|
||||
}
|
||||
|
||||
// At this point, the commitment transaction should now be fully
|
||||
// saturated. We'll continue adding HTLC's, and asserting that the
|
||||
// bandwidth accounting is done properly.
|
||||
const numOverFlowHTLCs = 20
|
||||
for i := 0; i < numOverFlowHTLCs; i++ {
|
||||
preImage := addLinkHTLC(htlcID, htlcAmt)
|
||||
preImages = append(preImages, preImage)
|
||||
|
||||
totalHtlcAmt += htlcAmt
|
||||
htlcID++
|
||||
}
|
||||
|
||||
// No messages should be sent to the remote at this point.
|
||||
select {
|
||||
case msg = <-aliceMsgs:
|
||||
t.Fatalf("unexpected message: %T", msg)
|
||||
case <-time.After(20 * time.Millisecond):
|
||||
}
|
||||
|
||||
time.Sleep(time.Second * 2)
|
||||
expectedBandwidth -= (numOverFlowHTLCs * htlcAmt)
|
||||
assertLinkBandwidth(t, aliceLink, expectedBandwidth)
|
||||
|
||||
// With the extra HTLC's added, the overflow queue should now be
|
||||
// populated with our 20 additional HTLC's.
|
||||
if coreLink.overflowQueue.Length() != numOverFlowHTLCs {
|
||||
t.Fatalf("wrong overflow queue length: expected %v, got %v",
|
||||
numOverFlowHTLCs,
|
||||
coreLink.overflowQueue.Length())
|
||||
}
|
||||
|
||||
// We trigger a state update to lock in the HTLCs. This should
|
||||
// not change Alice's bandwidth.
|
||||
if err := updateState(batchTick, coreLink, bobChannel, true); err != nil {
|
||||
t.Fatalf("unable to update state: %v", err)
|
||||
}
|
||||
time.Sleep(time.Millisecond * 500)
|
||||
assertLinkBandwidth(t, aliceLink, expectedBandwidth)
|
||||
|
||||
// At this point, we'll now settle enough HTLCs to empty the overflow
|
||||
// queue. The resulting bandwidth change should be non-existent as this
|
||||
// will simply transfer over funds to the remote party. However, the
|
||||
// size of the overflow queue should be decreasing
|
||||
for i := 0; i < numOverFlowHTLCs; i++ {
|
||||
err = bobChannel.SettleHTLC(preImages[i], uint64(i), nil, nil, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to settle htlc: %v", err)
|
||||
}
|
||||
|
||||
htlcSettle := &lnwire.UpdateFulfillHTLC{
|
||||
ID: uint64(i),
|
||||
PaymentPreimage: preImages[i],
|
||||
}
|
||||
|
||||
aliceLink.HandleChannelUpdate(htlcSettle)
|
||||
time.Sleep(time.Millisecond * 50)
|
||||
}
|
||||
time.Sleep(time.Millisecond * 500)
|
||||
assertLinkBandwidth(t, aliceLink, expectedBandwidth)
|
||||
|
||||
// We trigger a state update to lock in the Settles.
|
||||
if err := updateState(batchTick, coreLink, bobChannel, false); err != nil {
|
||||
t.Fatalf("unable to update state: %v", err)
|
||||
}
|
||||
|
||||
// After the state update is done, Alice should start sending
|
||||
// HTLCs from the overflow queue.
|
||||
for i := 0; i < numOverFlowHTLCs; i++ {
|
||||
var msg lnwire.Message
|
||||
select {
|
||||
case msg = <-aliceMsgs:
|
||||
case <-time.After(15 * time.Second):
|
||||
t.Fatalf("did not receive message")
|
||||
}
|
||||
|
||||
addHtlc, ok := msg.(*lnwire.UpdateAddHTLC)
|
||||
if !ok {
|
||||
t.Fatalf("expected UpdateAddHTLC, got %T", msg)
|
||||
}
|
||||
|
||||
_, err := bobChannel.ReceiveHTLC(addHtlc)
|
||||
if err != nil {
|
||||
t.Fatalf("bob failed receiving htlc: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
select {
|
||||
case msg = <-aliceMsgs:
|
||||
t.Fatalf("unexpected message: %T", msg)
|
||||
case <-time.After(20 * time.Millisecond):
|
||||
}
|
||||
|
||||
assertLinkBandwidth(t, aliceLink, expectedBandwidth)
|
||||
|
||||
// Finally, at this point, the queue itself should be fully empty. As
|
||||
// enough slots have been drained from the commitment transaction to
|
||||
// allocate the queue items to.
|
||||
time.Sleep(time.Millisecond * 500)
|
||||
if coreLink.overflowQueue.Length() != 0 {
|
||||
t.Fatalf("wrong overflow queue length: expected %v, got %v", 0,
|
||||
coreLink.overflowQueue.Length())
|
||||
}
|
||||
}
|
||||
|
||||
// genAddsAndCircuits creates `numHtlcs` sequential ADD packets and there
|
||||
// corresponding circuits. The provided `htlc` is used in all test packets.
|
||||
func genAddsAndCircuits(numHtlcs int, htlc *lnwire.UpdateAddHTLC) (
|
||||
@@ -5699,9 +5441,8 @@ func TestCheckHtlcForward(t *testing.T) {
|
||||
MaxOutgoingCltvExpiry: DefaultMaxOutgoingCltvExpiry,
|
||||
HtlcNotifier: &mockHTLCNotifier{},
|
||||
},
|
||||
log: log,
|
||||
channel: testChannel.channel,
|
||||
overflowQueue: newPacketQueue(input.MaxHTLCNumber / 2),
|
||||
log: log,
|
||||
channel: testChannel.channel,
|
||||
}
|
||||
|
||||
var hash [32]byte
|
||||
|
||||
Reference in New Issue
Block a user