Merge pull request #9454 from ziggie1984/add_custom_error_msg

Add Custom Error msg and Prioritise replayed HTLCs
This commit is contained in:
Olaoluwa Osuntokun 2025-01-29 22:48:22 -06:00 committed by GitHub
commit 32cdbb43f6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 581 additions and 340 deletions

View File

@ -0,0 +1,8 @@
## Bug Fixes
* [Improved user experience](https://github.com/lightningnetwork/lnd/pull/9454)
by returning a custom error code when HTLC carries incorrect custom records.
# Contributors (Alphabetical Order)
* Ziggie

View File

@ -508,4 +508,9 @@ type AuxTrafficShaper interface {
PaymentBandwidth(htlcBlob, commitmentBlob fn.Option[tlv.Blob],
linkBandwidth,
htlcAmt lnwire.MilliSatoshi) (lnwire.MilliSatoshi, error)
// IsCustomHTLC returns true if the HTLC carries the set of relevant
// custom records to put it under the purview of the traffic shaper,
// meaning that it's from a custom channel.
IsCustomHTLC(htlcRecords lnwire.CustomRecords) bool
}

View File

@ -4164,21 +4164,20 @@ func (l *channelLink) processExitHop(add lnwire.UpdateAddHTLC,
return nil
}
// In case the traffic shaper is active, we'll check if the HTLC has
// custom records and skip the amount check in the onion payload below.
isCustomHTLC := fn.MapOptionZ(
l.cfg.AuxTrafficShaper,
func(ts AuxTrafficShaper) bool {
return ts.IsCustomHTLC(add.CustomRecords)
},
)
// As we're the exit hop, we'll double check the hop-payload included in
// the HTLC to ensure that it was crafted correctly by the sender and
// is compatible with the HTLC we were extended.
//
// For a special case, if the fwdInfo doesn't have any blinded path
// information, and the incoming HTLC had special extra data, then
// we'll skip this amount check. The invoice acceptor will make sure we
// reject the HTLC if it's not containing the correct amount after
// examining the custom data.
hasBlindedPath := fwdInfo.NextBlinding.IsSome()
customHTLC := len(add.CustomRecords) > 0 && !hasBlindedPath
log.Tracef("Exit hop has_blinded_path=%v custom_htlc_bypass=%v",
hasBlindedPath, customHTLC)
if !customHTLC && add.Amount < fwdInfo.AmountToForward {
// is compatible with the HTLC we were extended. If an external
// validator is active we might bypass the amount check.
if !isCustomHTLC && add.Amount < fwdInfo.AmountToForward {
l.log.Errorf("onion payload of incoming htlc(%x) has "+
"incompatible value: expected <=%v, got %v",
add.PaymentHash, add.Amount, fwdInfo.AmountToForward)

View File

@ -9,6 +9,7 @@ import (
"time"
"github.com/lightningnetwork/lnd/clock"
"github.com/lightningnetwork/lnd/fn/v2"
"github.com/lightningnetwork/lnd/lntypes"
"github.com/lightningnetwork/lnd/lnwire"
"github.com/lightningnetwork/lnd/queue"
@ -653,56 +654,38 @@ func (i *InvoiceRegistry) startHtlcTimer(invoiceRef InvoiceRef,
func (i *InvoiceRegistry) cancelSingleHtlc(invoiceRef InvoiceRef,
key CircuitKey, result FailResolutionResult) error {
updateInvoice := func(invoice *Invoice) (*InvoiceUpdateDesc, error) {
updateInvoice := func(invoice *Invoice, setID *SetID) (
*InvoiceUpdateDesc, error) {
// Only allow individual htlc cancellation on open invoices.
if invoice.State != ContractOpen {
log.Debugf("cancelSingleHtlc: invoice %v no longer "+
"open", invoiceRef)
log.Debugf("CancelSingleHtlc: cannot cancel htlc %v "+
"on invoice %v, invoice is no longer open", key,
invoiceRef)
return nil, nil
}
// Lookup the current status of the htlc in the database.
var (
htlcState HtlcState
setID *SetID
)
// Also for AMP invoices we fetch the relevant HTLCs, so
// the HTLC should be found, otherwise we return an error.
htlc, ok := invoice.Htlcs[key]
if !ok {
// If this is an AMP invoice, then all the HTLCs won't
// be read out, so we'll consult the other mapping to
// try to find the HTLC state in question here.
var found bool
for ampSetID, htlcSet := range invoice.AMPState {
ampSetID := ampSetID
for htlcKey := range htlcSet.InvoiceKeys {
if htlcKey == key {
htlcState = htlcSet.State
setID = &ampSetID
found = true
break
}
}
}
if !found {
return nil, fmt.Errorf("htlc %v not found", key)
}
} else {
htlcState = htlc.State
return nil, fmt.Errorf("htlc %v not found on "+
"invoice %v", key, invoiceRef)
}
htlcState := htlc.State
// Cancellation is only possible if the htlc wasn't already
// resolved.
if htlcState != HtlcStateAccepted {
log.Debugf("cancelSingleHtlc: htlc %v on invoice %v "+
log.Debugf("CancelSingleHtlc: htlc %v on invoice %v "+
"is already resolved", key, invoiceRef)
return nil, nil
}
log.Debugf("cancelSingleHtlc: cancelling htlc %v on invoice %v",
log.Debugf("CancelSingleHtlc: cancelling htlc %v on invoice %v",
key, invoiceRef)
// Return an update descriptor that cancels htlc and keeps
@ -728,7 +711,7 @@ func (i *InvoiceRegistry) cancelSingleHtlc(invoiceRef InvoiceRef,
func(invoice *Invoice) (
*InvoiceUpdateDesc, error) {
updateDesc, err := updateInvoice(invoice)
updateDesc, err := updateInvoice(invoice, setID)
if err != nil {
return nil, err
}
@ -755,8 +738,13 @@ func (i *InvoiceRegistry) cancelSingleHtlc(invoiceRef InvoiceRef,
key, int32(htlc.AcceptHeight), result,
)
log.Debugf("Signaling htlc(%v) cancellation of invoice(%v) "+
"with resolution(%v) to the link subsystem", key,
invoiceRef, result)
i.notifyHodlSubscribers(resolution)
}
return nil
}
@ -1086,29 +1074,83 @@ func (i *InvoiceRegistry) notifyExitHopHtlcLocked(
updateSubscribers bool
)
callback := func(inv *Invoice) (*InvoiceUpdateDesc, error) {
// First check if this is a replayed htlc and resolve it
// according to its current state. We cannot decide differently
// once the HTLC has already been processed before.
isReplayed, res, err := resolveReplayedHtlc(ctx, inv)
if err != nil {
return nil, err
}
if isReplayed {
resolution = res
return nil, nil
}
// In case the HTLC interceptor cancels the HTLC set, we do NOT
// cancel the invoice however we cancel the complete HTLC set.
if cancelSet {
// If the invoice is not open, something is wrong, we
// fail just the HTLC with the specific error.
if inv.State != ContractOpen {
log.Errorf("Invoice state (%v) is not OPEN, "+
"cancelling HTLC set not allowed by "+
"external source", inv.State)
resolution = NewFailResolution(
ctx.circuitKey, ctx.currentHeight,
ResultInvoiceNotOpen,
)
return nil, nil
}
// The error `ExternalValidationFailed` error
// information will be packed in the
// `FailIncorrectDetails` msg when sending the msg to
// the peer. Error codes are defined by the BOLT 04
// specification. The error text can be arbitrary
// therefore we return a custom error msg.
resolution = NewFailResolution(
ctx.circuitKey, ctx.currentHeight,
ExternalValidationFailed,
)
// We cancel all HTLCs which are in the accepted state.
//
// NOTE: The current HTLC is not included because it
// was never accepted in the first place.
htlcs := inv.HTLCSet(ctx.setID(), HtlcStateAccepted)
htlcKeys := fn.KeySet[CircuitKey](htlcs)
// The external source did cancel the htlc set, so we
// cancel all HTLCs in the set. We however keep the
// invoice in the open state.
//
// NOTE: The invoice event loop will still call the
// `cancelSingleHTLC` method for MPP payments, however
// because the HTLCs are already cancled back it will be
// a NOOP.
update := &InvoiceUpdateDesc{
UpdateType: CancelHTLCsUpdate,
CancelHtlcs: htlcKeys,
SetID: setID,
}
return update, nil
}
updateDesc, res, err := updateInvoice(ctx, inv)
if err != nil {
return nil, err
}
// Set resolution in outer scope only after successful update.
resolution = res
// Only send an update if the invoice state was changed.
updateSubscribers = updateDesc != nil &&
updateDesc.State != nil
// Assign resolution to outer scope variable.
if cancelSet {
// If a cancel signal was set for the htlc set, we set
// the resolution as a failure with an underpayment
// indication. Something was wrong with this htlc, so
// we probably can't settle the invoice at all.
resolution = NewFailResolution(
ctx.circuitKey, ctx.currentHeight,
ResultAmountTooLow,
)
} else {
resolution = res
}
return updateDesc, nil
}
@ -1417,6 +1459,8 @@ func (i *InvoiceRegistry) cancelInvoiceImpl(ctx context.Context,
}
invoiceRef := InvoiceRefByHash(payHash)
// We pass a nil setID which means no HTLCs will be read out.
invoice, err := i.idb.UpdateInvoice(ctx, invoiceRef, nil, updateInvoice)
// Implement idempotency by returning success if the invoice was already
@ -1443,6 +1487,8 @@ func (i *InvoiceRegistry) cancelInvoiceImpl(ctx context.Context,
// that are waiting for resolution. Any htlcs that were already canceled
// before, will be notified again. This isn't necessary but doesn't hurt
// either.
//
// TODO(ziggie): Also consider AMP HTLCs here.
for key, htlc := range invoice.Htlcs {
if htlc.State != HtlcStateCanceled {
continue

View File

@ -20,6 +20,7 @@ import (
"github.com/lightningnetwork/lnd/lnwire"
"github.com/lightningnetwork/lnd/record"
"github.com/lightningnetwork/lnd/sqldb"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
)
@ -108,12 +109,20 @@ func TestInvoiceRegistry(t *testing.T) {
name: "SpontaneousAmpPayment",
test: testSpontaneousAmpPayment,
},
{
name: "FailPartialMPPPaymentExternal",
test: testFailPartialMPPPaymentExternal,
},
{
name: "FailPartialAMPPayment",
test: testFailPartialAMPPayment,
},
}
makeKeyValueDB := func(t *testing.T) (invpkg.InvoiceDB,
*clock.TestClock) {
testClock := clock.NewTestClock(testNow)
testClock := clock.NewTestClock(testTime)
db, err := channeldb.MakeTestInvoiceDB(
t, channeldb.OptionClock(testClock),
)
@ -147,7 +156,7 @@ func TestInvoiceRegistry(t *testing.T) {
},
)
testClock := clock.NewTestClock(testNow)
testClock := clock.NewTestClock(testTime)
return invpkg.NewSQLStore(executor, testClock), testClock
}
@ -204,7 +213,7 @@ func testSettleInvoice(t *testing.T,
require.Equal(t, subscription.PayHash(), &testInvoicePaymentHash)
// Add the invoice.
testInvoice := newInvoice(t, false)
testInvoice := newInvoice(t, false, false)
addIdx, err := ctx.registry.AddInvoice(
ctxb, testInvoice, testInvoicePaymentHash,
)
@ -395,7 +404,7 @@ func testCancelInvoiceImpl(t *testing.T, gc bool,
require.Equal(t, subscription.PayHash(), &testInvoicePaymentHash)
// Add the invoice.
testInvoice := newInvoice(t, false)
testInvoice := newInvoice(t, false, false)
_, err = ctx.registry.AddInvoice(
ctxb, testInvoice, testInvoicePaymentHash,
)
@ -555,7 +564,7 @@ func testSettleHoldInvoice(t *testing.T,
require.Equal(t, subscription.PayHash(), &testInvoicePaymentHash)
// Add the invoice.
invoice := newInvoice(t, true)
invoice := newInvoice(t, true, false)
_, err = registry.AddInvoice(ctxb, invoice, testInvoicePaymentHash)
require.NoError(t, err)
@ -716,7 +725,7 @@ func testCancelHoldInvoice(t *testing.T,
ctxb := context.Background()
// Add the invoice.
invoice := newInvoice(t, true)
invoice := newInvoice(t, true, false)
_, err = registry.AddInvoice(ctxb, invoice, testInvoicePaymentHash)
require.NoError(t, err)
@ -1043,7 +1052,7 @@ func testMppPayment(t *testing.T,
ctxb := context.Background()
// Add the invoice.
testInvoice := newInvoice(t, false)
testInvoice := newInvoice(t, false, false)
_, err := ctx.registry.AddInvoice(
ctxb, testInvoice, testInvoicePaymentHash,
)
@ -1141,7 +1150,7 @@ func testMppPaymentWithOverpayment(t *testing.T,
ctx := newTestContext(t, nil, makeDB)
// Add the invoice.
testInvoice := newInvoice(t, false)
testInvoice := newInvoice(t, false, false)
_, err := ctx.registry.AddInvoice(
ctxb, testInvoice, testInvoicePaymentHash,
)
@ -1432,7 +1441,7 @@ func testHeightExpiryWithRegistryImpl(t *testing.T, numParts int, settle bool,
// Add a hold invoice, we set a non-nil payment request so that this
// invoice is not considered a keysend by the expiry watcher.
testInvoice := newInvoice(t, false)
testInvoice := newInvoice(t, false, false)
testInvoice.HodlInvoice = true
testInvoice.PaymentRequest = []byte{1, 2, 3}
@ -1545,7 +1554,7 @@ func testMultipleSetHeightExpiry(t *testing.T,
ctx := newTestContext(t, nil, makeDB)
// Add a hold invoice.
testInvoice := newInvoice(t, true)
testInvoice := newInvoice(t, true, false)
ctxb := context.Background()
_, err := ctx.registry.AddInvoice(
@ -2109,3 +2118,326 @@ func testSpontaneousAmpPaymentImpl(
}
}
}
// testFailPartialMPPPaymentExternal tests that the HTLC set is cancelled back
// as soon as the HTLC interceptor denies one of the HTLCs.
func testFailPartialMPPPaymentExternal(t *testing.T,
makeDB func(t *testing.T) (invpkg.InvoiceDB, *clock.TestClock)) {
t.Parallel()
mockHtlcInterceptor := &invpkg.MockHtlcModifier{}
cfg := defaultRegistryConfig()
cfg.HtlcInterceptor = mockHtlcInterceptor
ctx := newTestContext(t, &cfg, makeDB)
// Add an invoice which we are going to pay via a MPP set.
testInvoice := newInvoice(t, false, false)
ctxb := context.Background()
_, err := ctx.registry.AddInvoice(
ctxb, testInvoice, testInvoicePaymentHash,
)
require.NoError(t, err)
mppPayload := &mockPayload{
mpp: record.NewMPP(testInvoiceAmount, [32]byte{}),
}
// Send first HTLC which pays part of the invoice but keeps the invoice
// in an open state because the amount is less than the invoice amount.
hodlChan1 := make(chan interface{}, 1)
resolution, err := ctx.registry.NotifyExitHopHtlc(
testInvoicePaymentHash, testInvoice.Terms.Value/3,
testHtlcExpiry, testCurrentHeight, getCircuitKey(1),
hodlChan1, nil, mppPayload,
)
require.NoError(t, err)
require.Nil(t, resolution, "did not expect direct resolution")
// Register the expected response from the interceptor so that the
// whole HTLC set is cancelled.
expectedResponse := invpkg.HtlcModifyResponse{
CancelSet: true,
}
mockHtlcInterceptor.On("Intercept", mock.Anything, mock.Anything).
Return(nil, expectedResponse)
// Send htlc 2. We expect the HTLC to be cancelled because the
// interceptor will deny it.
resolution, err = ctx.registry.NotifyExitHopHtlc(
testInvoicePaymentHash, testInvoice.Terms.Value/2,
testHtlcExpiry, testCurrentHeight, getCircuitKey(2), nil,
nil, mppPayload,
)
require.NoError(t, err)
failResolution, ok := resolution.(*invpkg.HtlcFailResolution)
require.True(t, ok, "expected fail resolution, got: %T", resolution)
// Make sure the resolution includes the custom error msg.
require.Equal(t, invpkg.ExternalValidationFailed,
failResolution.Outcome, "expected ExternalValidationFailed, "+
"got: %v", failResolution.Outcome)
// Expect HLTC 1 also to be cancelled because it is part of the cancel
// set and the interceptor cancelled the whole set after receiving the
// second HTLC.
select {
case resolution := <-hodlChan1:
htlcResolution, _ := resolution.(invpkg.HtlcResolution)
failResolution, ok = htlcResolution.(*invpkg.HtlcFailResolution)
require.True(
t, ok, "expected fail resolution, got: %T",
htlcResolution,
)
require.Equal(
t, invpkg.ExternalValidationFailed,
failResolution.Outcome, "expected "+
"ExternalValidationFailed, got: %v",
failResolution.Outcome,
)
case <-time.After(testTimeout):
t.Fatal("timeout waiting for HTLC resolution")
}
// Assert that the invoice is still open.
inv, err := ctx.registry.LookupInvoice(ctxb, testInvoicePaymentHash)
require.NoError(t, err)
require.Equal(t, invpkg.ContractOpen, inv.State, "expected "+
"OPEN invoice")
// Now let the invoice expire.
currentTime := ctx.clock.Now()
ctx.clock.SetTime(currentTime.Add(61 * time.Minute))
// Make sure the invoices changes to the canceled state.
require.Eventuallyf(t, func() bool {
inv, err := ctx.registry.LookupInvoice(
ctxb, testInvoicePaymentHash,
)
require.NoError(t, err)
return inv.State == invpkg.ContractCanceled
}, testTimeout, time.Millisecond*100, "invoice not canceled")
// Fetch the invoice again and compare the number of cancelled HTLCs.
inv, err = ctx.registry.LookupInvoice(
ctxb, testInvoicePaymentHash,
)
require.NoError(t, err)
// Make sure all HTLCs are in the canceled state which in our case is
// only the first one because the second HTLC was never added to the
// invoice registry in the first place.
require.Len(t, inv.Htlcs, 1)
require.Equal(
t, invpkg.HtlcStateCanceled, inv.Htlcs[getCircuitKey(1)].State,
)
}
// testFailPartialAMPPayment tests the MPP timeout logic for AMP invoices. It
// makes sure that all HTLCs are cancelled if the full invoice amount is not
// received. Moreover it points out some TODOs to make AMP invoices more robust.
func testFailPartialAMPPayment(t *testing.T,
makeDB func(t *testing.T) (invpkg.InvoiceDB, *clock.TestClock)) {
t.Parallel()
ctx := newTestContext(t, nil, makeDB)
ctxb := context.Background()
const (
expiry = uint32(testCurrentHeight + 20)
numShards = 4
)
var (
shardAmt = testInvoiceAmount / lnwire.MilliSatoshi(numShards)
setID [32]byte
payAddr [32]byte
)
_, err := rand.Read(payAddr[:])
require.NoError(t, err)
// Create an AMP invoice we are going to pay via a multi-part payment.
ampInvoice := newInvoice(t, false, true)
// An AMP invoice is referenced by the payment address.
ampInvoice.Terms.PaymentAddr = payAddr
_, err = ctx.registry.AddInvoice(
ctxb, ampInvoice, testInvoicePaymentHash,
)
require.NoError(t, err)
// Generate a random setID for the HTLCs.
_, err = rand.Read(setID[:])
require.NoError(t, err)
htlcPayload1 := &mockPayload{
mpp: record.NewMPP(testInvoiceAmount, payAddr),
// We are not interested in settling the AMP HTLC so we don't
// use valid shares.
amp: record.NewAMP([32]byte{1}, setID, 1),
}
// Send first HTLC which pays part of the invoice.
hodlChan1 := make(chan interface{}, 1)
resolution, err := ctx.registry.NotifyExitHopHtlc(
lntypes.Hash{1}, shardAmt, expiry, testCurrentHeight,
getCircuitKey(1), hodlChan1, nil, htlcPayload1,
)
require.NoError(t, err)
require.Nil(t, resolution, "did not expect direct resolution")
htlcPayload2 := &mockPayload{
mpp: record.NewMPP(testInvoiceAmount, payAddr),
// We are not interested in settling the AMP HTLC so we don't
// use valid shares.
amp: record.NewAMP([32]byte{2}, setID, 2),
}
// Send htlc 2 which should be added to the invoice as expected.
hodlChan2 := make(chan interface{}, 1)
resolution, err = ctx.registry.NotifyExitHopHtlc(
lntypes.Hash{2}, shardAmt, expiry, testCurrentHeight,
getCircuitKey(2), hodlChan2, nil, htlcPayload2,
)
require.NoError(t, err)
require.Nil(t, resolution, "did not expect direct resolution")
// Now time-out the HTLCs. The HoldDuration is 30 seconds after the
// HTLC will be cancelled.
currentTime := ctx.clock.Now()
ctx.clock.SetTime(currentTime.Add(35 * time.Second))
// Expect HLTC 1 to be canceled via the MPPTimeout fail resolution.
select {
case resolution := <-hodlChan1:
htlcResolution, _ := resolution.(invpkg.HtlcResolution)
failRes, ok := htlcResolution.(*invpkg.HtlcFailResolution)
require.True(
t, ok, "expected fail resolution, got: %T", resolution,
)
require.Equal(
t, invpkg.ResultMppTimeout, failRes.Outcome,
"expected MPPTimeout, got: %v", failRes.Outcome,
)
case <-time.After(testTimeout):
t.Fatal("timeout waiting for HTLC resolution")
}
// Expect HLTC 2 to be canceled via the MPPTimeout fail resolution.
select {
case resolution := <-hodlChan2:
htlcResolution, _ := resolution.(invpkg.HtlcResolution)
failRes, ok := htlcResolution.(*invpkg.HtlcFailResolution)
require.True(
t, ok, "expected fail resolution, got: %T", resolution,
)
require.Equal(
t, invpkg.ResultMppTimeout, failRes.Outcome,
"expected MPPTimeout, got: %v", failRes.Outcome,
)
case <-time.After(testTimeout):
t.Fatal("timeout waiting for HTLC resolution")
}
// The AMP invoice should still be open.
inv, err := ctx.registry.LookupInvoice(ctxb, testInvoicePaymentHash)
require.NoError(t, err)
require.Equal(t, invpkg.ContractOpen, inv.State, "expected "+
"OPEN invoice")
// Because one HTLC of the set was cancelled we expect the AMPState to
// be set to canceled.
ampState, ok := inv.AMPState[setID]
require.True(t, ok, "expected AMPState to be set")
require.Equal(t, invpkg.HtlcStateCanceled, ampState.State, "expected "+
"AMPState CANCELED")
// The following is a bug and should not be allowed because the sub
// AMP invoice is already marked as canceled. However LND will accept
// other HTLCs to the AMP sub-invoice.
//
// TODO(ziggie): Fix this bug.
htlcPayload3 := &mockPayload{
mpp: record.NewMPP(testInvoiceAmount, payAddr),
// We are not interested in settling the AMP HTLC so we don't
// use valid shares.
amp: record.NewAMP([32]byte{3}, setID, 3),
}
// Send htlc 3 which should be added to the invoice as expected.
hodlChan3 := make(chan interface{}, 1)
resolution, err = ctx.registry.NotifyExitHopHtlc(
lntypes.Hash{3}, shardAmt, expiry, testCurrentHeight,
getCircuitKey(3), hodlChan3, nil, htlcPayload3,
)
require.NoError(t, err)
require.Nil(t, resolution, "did not expect direct resolution")
// TODO(ziggie): This is a race condition between the invoice being
// cancelled and the htlc being added to the invoice. If we do not wait
// here until the HTLC is added to the invoice, the test might fail
// because the HTLC will not be resolved.
require.Eventuallyf(t, func() bool {
inv, err := ctx.registry.LookupInvoice(
ctxb, testInvoicePaymentHash,
)
require.NoError(t, err)
return len(inv.Htlcs) == 3
}, testTimeout, time.Millisecond*100, "HTLC 3 not added to invoice")
// Now also let the invoice expire the invoice expiry is 1 hour.
currentTime = ctx.clock.Now()
ctx.clock.SetTime(currentTime.Add(1 * time.Minute))
// Expect HLTC 3 to be canceled either via the cancelation of the
// invoice or because the MPP timeout kicks in.
select {
case resolution := <-hodlChan3:
htlcResolution, _ := resolution.(invpkg.HtlcResolution)
failRes, ok := htlcResolution.(*invpkg.HtlcFailResolution)
require.True(
t, ok, "expected fail resolution, got: %T", resolution,
)
require.Equal(
t, invpkg.ResultMppTimeout, failRes.Outcome,
"expected MPPTimeout, got: %v", failRes.Outcome,
)
case <-time.After(testTimeout):
t.Fatal("timeout waiting for HTLC resolution")
}
// expire the invoice here.
currentTime = ctx.clock.Now()
ctx.clock.SetTime(currentTime.Add(61 * time.Minute))
require.Eventuallyf(t, func() bool {
inv, err := ctx.registry.LookupInvoice(
ctxb, testInvoicePaymentHash,
)
require.NoError(t, err)
return inv.State == invpkg.ContractCanceled
}, testTimeout, time.Millisecond*100, "invoice not canceled")
// Fetch the invoice again and compare the number of cancelled HTLCs.
inv, err = ctx.registry.LookupInvoice(
ctxb, testInvoicePaymentHash,
)
require.NoError(t, err)
// Make sure all HTLCs are in the cancelled state.
require.Len(t, inv.Htlcs, 3)
for _, htlc := range inv.Htlcs {
require.Equal(t, invpkg.HtlcStateCanceled, htlc.State,
"expected HTLC to be canceled")
}
}

View File

@ -765,6 +765,9 @@ type InvoiceStateUpdateDesc struct {
// InvoiceUpdateCallback is a callback used in the db transaction to update the
// invoice.
// TODO(ziggie): Add the option of additional return values to the callback
// for example the resolution which is currently assigned via an outer scope
// variable.
type InvoiceUpdateCallback = func(invoice *Invoice) (*InvoiceUpdateDesc, error)
// ValidateInvoice assures the invoice passes the checks for all the relevant

View File

@ -86,6 +86,7 @@ func (m *MockInvoiceDB) DeleteCanceledInvoices(ctx context.Context) error {
// MockHtlcModifier is a mock implementation of the HtlcModifier interface.
type MockHtlcModifier struct {
mock.Mock
}
// Intercept generates a new intercept session for the given invoice.
@ -94,9 +95,23 @@ type MockHtlcModifier struct {
// created in the first place, which is only the case if a client is
// registered.
func (m *MockHtlcModifier) Intercept(
_ HtlcModifyRequest, _ func(HtlcModifyResponse)) error {
req HtlcModifyRequest, callback func(HtlcModifyResponse)) error {
return nil
// If no expectations are set, return nil by default.
if len(m.ExpectedCalls) == 0 {
return nil
}
args := m.Called(req, callback)
// If a response was provided to the mock, execute the callback with it.
if response, ok := args.Get(1).(HtlcModifyResponse); ok &&
callback != nil {
callback(response)
}
return args.Error(0)
}
// RegisterInterceptor sets the client callback function that will be

View File

@ -137,9 +137,6 @@ func (s *HtlcModificationInterceptor) Intercept(clientRequest HtlcModifyRequest,
// Wait for the client to respond or an error to occur.
select {
case response := <-responseChan:
log.Debugf("Received invoice HTLC interceptor response: %v",
response)
responseCallback(*response)
return nil

View File

@ -120,6 +120,10 @@ const (
// ResultAmpReconstruction is returned when the derived child
// hash/preimage pairs were invalid for at least one HTLC in the set.
ResultAmpReconstruction
// ExternalValidationFailed is returned when the external validation
// failed.
ExternalValidationFailed
)
// String returns a string representation of the result.
@ -189,6 +193,9 @@ func (f FailResolutionResult) FailureString() string {
case ResultAmpReconstruction:
return "amp reconstruction failed"
case ExternalValidationFailed:
return "external validation failed"
default:
return "unknown failure resolution result"
}
@ -202,7 +209,8 @@ func (f FailResolutionResult) IsSetFailure() bool {
ResultAmpReconstruction,
ResultHtlcSetTotalTooLow,
ResultHtlcSetTotalMismatch,
ResultHtlcSetOverpayment:
ResultHtlcSetOverpayment,
ExternalValidationFailed:
return true

View File

@ -207,7 +207,7 @@ func getCircuitKey(htlcID uint64) invpkg.CircuitKey {
// Note that this invoice *does not* have a payment address set. It will
// create a regular invoice with a preimage is hodl is false, and a hodl
// invoice with no preimage otherwise.
func newInvoice(t *testing.T, hodl bool) *invpkg.Invoice {
func newInvoice(t *testing.T, hodl bool, ampInvoice bool) *invpkg.Invoice {
invoice := &invpkg.Invoice{
Terms: invpkg.ContractTerm{
Value: testInvoiceAmount,
@ -217,6 +217,23 @@ func newInvoice(t *testing.T, hodl bool) *invpkg.Invoice {
CreationDate: testInvoiceCreationDate,
}
// This makes the invoice an AMP invoice. We do not support AMP hodl
// invoices.
if ampInvoice {
ampFeature := lnwire.NewRawFeatureVector(
lnwire.TLVOnionPayloadOptional,
lnwire.PaymentAddrOptional,
lnwire.AMPRequired,
)
ampFeatures := lnwire.NewFeatureVector(
ampFeature, lnwire.Features,
)
invoice.Terms.Features = ampFeatures
return invoice
}
// If creating a hodl invoice, we don't include a preimage.
if hodl {
invoice.HodlInvoice = true

View File

@ -109,41 +109,52 @@ func (i invoiceUpdateCtx) acceptRes(
return newAcceptResolution(i.circuitKey, outcome)
}
// resolveReplayedHtlc returns the HTLC resolution for a replayed HTLC. The
// returned boolean indicates whether the HTLC was replayed or not.
func resolveReplayedHtlc(ctx *invoiceUpdateCtx, inv *Invoice) (bool,
HtlcResolution, error) {
// Don't update the invoice when this is a replayed htlc.
htlc, replayedHTLC := inv.Htlcs[ctx.circuitKey]
if !replayedHTLC {
return false, nil, nil
}
switch htlc.State {
case HtlcStateCanceled:
return true, ctx.failRes(ResultReplayToCanceled), nil
case HtlcStateAccepted:
return true, ctx.acceptRes(resultReplayToAccepted), nil
case HtlcStateSettled:
pre := inv.Terms.PaymentPreimage
// Terms.PaymentPreimage will be nil for AMP invoices.
// Set it to the HTLCs AMP Preimage instead.
if pre == nil {
pre = htlc.AMP.Preimage
}
return true, ctx.settleRes(
*pre,
ResultReplayToSettled,
), nil
default:
return true, nil, errors.New("unknown htlc state")
}
}
// updateInvoice is a callback for DB.UpdateInvoice that contains the invoice
// settlement logic. It returns a HTLC resolution that indicates what the
// outcome of the update was.
//
// NOTE: Make sure replayed HTLCs are always considered before calling this
// function.
func updateInvoice(ctx *invoiceUpdateCtx, inv *Invoice) (
*InvoiceUpdateDesc, HtlcResolution, error) {
// Don't update the invoice when this is a replayed htlc.
htlc, ok := inv.Htlcs[ctx.circuitKey]
if ok {
switch htlc.State {
case HtlcStateCanceled:
return nil, ctx.failRes(ResultReplayToCanceled), nil
case HtlcStateAccepted:
return nil, ctx.acceptRes(resultReplayToAccepted), nil
case HtlcStateSettled:
pre := inv.Terms.PaymentPreimage
// Terms.PaymentPreimage will be nil for AMP invoices.
// Set it to the HTLCs AMP Preimage instead.
if pre == nil {
pre = htlc.AMP.Preimage
}
return nil, ctx.settleRes(
*pre,
ResultReplayToSettled,
), nil
default:
return nil, nil, errors.New("unknown htlc state")
}
}
// If no MPP payload was provided, then we expect this to be a keysend,
// or a payment to an invoice created before we started to require the
// MPP payload.

View File

@ -406,14 +406,6 @@ var allTestCases = []*lntest.TestCase{
Name: "forward interceptor",
TestFunc: testForwardInterceptorBasic,
},
{
Name: "forward interceptor modified htlc",
TestFunc: testForwardInterceptorModifiedHtlc,
},
{
Name: "forward interceptor wire records",
TestFunc: testForwardInterceptorWireRecords,
},
{
Name: "forward interceptor restart",
TestFunc: testForwardInterceptorRestart,

View File

@ -10,7 +10,6 @@ import (
"github.com/btcsuite/btcd/btcutil"
"github.com/lightningnetwork/lnd/chainreg"
"github.com/lightningnetwork/lnd/lnrpc"
"github.com/lightningnetwork/lnd/lnrpc/invoicesrpc"
"github.com/lightningnetwork/lnd/lnrpc/routerrpc"
"github.com/lightningnetwork/lnd/lntest"
"github.com/lightningnetwork/lnd/lntest/node"
@ -346,223 +345,6 @@ func testForwardInterceptorBasic(ht *lntest.HarnessTest) {
}
}
// testForwardInterceptorModifiedHtlc tests that the interceptor can modify the
// amount and custom records of an intercepted HTLC and resume it.
func testForwardInterceptorModifiedHtlc(ht *lntest.HarnessTest) {
const chanAmt = btcutil.Amount(300000)
p := lntest.OpenChannelParams{Amt: chanAmt}
// Initialize the test context with 3 connected nodes.
cfgs := [][]string{nil, nil, nil}
// Open and wait for channels.
_, nodes := ht.CreateSimpleNetwork(cfgs, p)
alice, bob, carol := nodes[0], nodes[1], nodes[2]
// Init the scenario.
ts := &interceptorTestScenario{
ht: ht,
alice: alice,
bob: bob,
carol: carol,
}
// Connect an interceptor to Bob's node.
bobInterceptor, cancelBobInterceptor := bob.RPC.HtlcInterceptor()
// We're going to modify the payment amount and want Carol to accept the
// payment, so we set up an invoice acceptor on Dave.
carolAcceptor, carolCancel := carol.RPC.InvoiceHtlcModifier()
defer carolCancel()
// Prepare the test cases.
invoiceValueAmtMsat := int64(20_000_000)
req := &lnrpc.Invoice{ValueMsat: invoiceValueAmtMsat}
addResponse := carol.RPC.AddInvoice(req)
invoice := carol.RPC.LookupInvoice(addResponse.RHash)
tc := &interceptorTestCase{
amountMsat: invoiceValueAmtMsat,
invoice: invoice,
payAddr: invoice.PaymentAddr,
}
// We initiate a payment from Alice.
done := make(chan struct{})
go func() {
// Signal that all the payments have been sent.
defer close(done)
ts.sendPaymentAndAssertAction(tc)
}()
// We start the htlc interceptor with a simple implementation that saves
// all intercepted packets. These packets are held to simulate a
// pending payment.
packet := ht.ReceiveHtlcInterceptor(bobInterceptor)
// Resume the intercepted HTLC with a modified amount and custom
// records.
customRecords := make(map[uint64][]byte)
// Add custom records entry.
crKey := uint64(65537)
crValue := []byte("custom-records-test-value")
customRecords[crKey] = crValue
// Modify the amount of the HTLC, so we send out less than the original
// amount.
const modifyAmount = 5_000_000
newOutAmountMsat := packet.OutgoingAmountMsat - modifyAmount
err := bobInterceptor.Send(&routerrpc.ForwardHtlcInterceptResponse{
IncomingCircuitKey: packet.IncomingCircuitKey,
OutAmountMsat: newOutAmountMsat,
OutWireCustomRecords: customRecords,
Action: actionResumeModify,
})
require.NoError(ht, err, "failed to send request")
invoicePacket := ht.ReceiveInvoiceHtlcModification(carolAcceptor)
require.EqualValues(
ht, newOutAmountMsat, invoicePacket.ExitHtlcAmt,
)
amtPaid := newOutAmountMsat + modifyAmount
err = carolAcceptor.Send(&invoicesrpc.HtlcModifyResponse{
CircuitKey: invoicePacket.ExitHtlcCircuitKey,
AmtPaid: &amtPaid,
})
require.NoError(ht, err, "carol acceptor response")
// Cancel the context, which will disconnect Bob's interceptor.
cancelBobInterceptor()
// Make sure all goroutines are finished.
select {
case <-done:
case <-time.After(defaultTimeout):
require.Fail(ht, "timeout waiting for sending payment")
}
// Assert that the payment was successful.
var preimage lntypes.Preimage
copy(preimage[:], invoice.RPreimage)
ht.AssertPaymentStatus(alice, preimage, lnrpc.Payment_SUCCEEDED)
}
// testForwardInterceptorWireRecords tests that the interceptor can read any
// wire custom records provided by the sender of a payment as part of the
// update_add_htlc message.
func testForwardInterceptorWireRecords(ht *lntest.HarnessTest) {
const chanAmt = btcutil.Amount(300000)
p := lntest.OpenChannelParams{Amt: chanAmt}
// Initialize the test context with 4 connected nodes.
cfgs := [][]string{nil, nil, nil, nil}
// Open and wait for channels.
_, nodes := ht.CreateSimpleNetwork(cfgs, p)
alice, bob, carol, dave := nodes[0], nodes[1], nodes[2], nodes[3]
// Connect an interceptor to Bob's node.
bobInterceptor, cancelBobInterceptor := bob.RPC.HtlcInterceptor()
defer cancelBobInterceptor()
// Also connect an interceptor on Carol's node to check whether we're
// relaying the TLVs send in update_add_htlc over Alice -> Bob on the
// Bob -> Carol link.
carolInterceptor, cancelCarolInterceptor := carol.RPC.HtlcInterceptor()
defer cancelCarolInterceptor()
// We're going to modify the payment amount and want Dave to accept the
// payment, so we set up an invoice acceptor on Dave.
daveAcceptor, daveCancel := dave.RPC.InvoiceHtlcModifier()
defer daveCancel()
req := &lnrpc.Invoice{ValueMsat: 20_000_000}
addResponse := dave.RPC.AddInvoice(req)
invoice := dave.RPC.LookupInvoice(addResponse.RHash)
customRecords := map[uint64][]byte{
65537: []byte("test"),
}
sendReq := &routerrpc.SendPaymentRequest{
PaymentRequest: invoice.PaymentRequest,
TimeoutSeconds: int32(wait.PaymentTimeout.Seconds()),
FeeLimitMsat: noFeeLimitMsat,
FirstHopCustomRecords: customRecords,
}
ht.SendPaymentAssertInflight(alice, sendReq)
// We start the htlc interceptor with a simple implementation that saves
// all intercepted packets. These packets are held to simulate a
// pending payment.
packet := ht.ReceiveHtlcInterceptor(bobInterceptor)
require.Equal(ht, lntest.CustomRecordsWithUnendorsed(
customRecords,
), packet.InWireCustomRecords)
// Just resume the payment on Bob.
err := bobInterceptor.Send(&routerrpc.ForwardHtlcInterceptResponse{
IncomingCircuitKey: packet.IncomingCircuitKey,
Action: actionResume,
})
require.NoError(ht, err, "failed to send request")
// Assert that the Alice -> Bob custom records in update_add_htlc are
// not propagated on the Bob -> Carol link, just an endorsement signal.
packet = ht.ReceiveHtlcInterceptor(carolInterceptor)
require.Equal(ht, lntest.CustomRecordsWithUnendorsed(nil),
packet.InWireCustomRecords)
// We're going to tell Carol to forward 5k sats less to Dave. We need to
// set custom records on the HTLC as well, to make sure the HTLC isn't
// rejected outright and actually gets to the invoice acceptor.
const modifyAmount = 5_000_000
newOutAmountMsat := packet.OutgoingAmountMsat - modifyAmount
err = carolInterceptor.Send(&routerrpc.ForwardHtlcInterceptResponse{
IncomingCircuitKey: packet.IncomingCircuitKey,
OutAmountMsat: newOutAmountMsat,
OutWireCustomRecords: customRecords,
Action: actionResumeModify,
})
require.NoError(ht, err, "carol interceptor response")
// The payment should get to Dave, and we should be able to intercept
// and modify it, telling Dave to accept it.
invoicePacket := ht.ReceiveInvoiceHtlcModification(daveAcceptor)
require.EqualValues(
ht, newOutAmountMsat, invoicePacket.ExitHtlcAmt,
)
amtPaid := newOutAmountMsat + modifyAmount
err = daveAcceptor.Send(&invoicesrpc.HtlcModifyResponse{
CircuitKey: invoicePacket.ExitHtlcCircuitKey,
AmtPaid: &amtPaid,
})
require.NoError(ht, err, "dave acceptor response")
// Assert that the payment was successful.
var preimage lntypes.Preimage
copy(preimage[:], invoice.RPreimage)
ht.AssertPaymentStatus(
alice, preimage, lnrpc.Payment_SUCCEEDED,
func(p *lnrpc.Payment) error {
recordsEqual := reflect.DeepEqual(
p.FirstHopCustomRecords,
lntest.CustomRecordsWithUnendorsed(
customRecords,
),
)
if !recordsEqual {
return fmt.Errorf("expected custom records to "+
"be equal, got %v expected %v",
p.FirstHopCustomRecords,
sendReq.FirstHopCustomRecords)
}
return nil
},
)
}
// testForwardInterceptorRestart tests that the interceptor can read any wire
// custom records provided by the sender of a payment as part of the
// update_add_htlc message and that those records are persisted correctly and

View File

@ -116,6 +116,7 @@ func testInvoiceHtlcModifierBasic(ht *lntest.HarnessTest) {
&invoicesrpc.HtlcModifyResponse{
CircuitKey: modifierRequest.ExitHtlcCircuitKey,
AmtPaid: &amtPaid,
CancelSet: tc.cancelSet,
},
)
require.NoError(ht, err, "failed to send request")
@ -128,7 +129,9 @@ func testInvoiceHtlcModifierBasic(ht *lntest.HarnessTest) {
require.Fail(ht, "timeout waiting for payment send")
}
ht.Log("Ensure invoice status is settled")
ht.Logf("Ensure invoice status is expected state %v",
tc.finalInvoiceState)
require.Eventually(ht, func() bool {
updatedInvoice := carol.RPC.LookupInvoice(
tc.invoice.RHash,
@ -141,6 +144,13 @@ func testInvoiceHtlcModifierBasic(ht *lntest.HarnessTest) {
tc.invoice.RHash,
)
// If the HTLC modifier canceled the incoming HTLC set, we don't
// expect any HTLCs in the invoice.
if tc.cancelSet {
require.Len(ht, updatedInvoice.Htlcs, 0)
return
}
require.Len(ht, updatedInvoice.Htlcs, 1)
require.Equal(
ht, lntest.CustomRecordsWithUnendorsed(
@ -231,6 +241,10 @@ type acceptorTestCase struct {
// invoice is the invoice that will be paid.
invoice *lnrpc.Invoice
// cancelSet is a boolean which indicates whether the HTLC modifier
// canceled the incoming HTLC set.
cancelSet bool
}
// acceptorTestScenario is a helper struct to hold the test context and provides
@ -282,6 +296,12 @@ func (c *acceptorTestScenario) prepareTestCases() []*acceptorTestCase {
lnwire.MinCustomRecordsTlvType: {1, 2, 3},
},
},
{
invoiceAmountMsat: 9000,
sendAmountMsat: 1000,
finalInvoiceState: lnrpc.Invoice_OPEN,
cancelSet: true,
},
}
for _, t := range cases {

View File

@ -187,13 +187,15 @@ func CreateRPCInvoice(invoice *invoices.Invoice,
AmtPaidSat: int64(satAmtPaid),
AmtPaidMsat: int64(invoice.AmtPaid),
AmtPaid: int64(invoice.AmtPaid),
State: state,
Htlcs: rpcHtlcs,
Features: CreateRPCFeatures(invoice.Terms.Features),
IsKeysend: invoice.IsKeysend(),
PaymentAddr: invoice.Terms.PaymentAddr[:],
IsAmp: invoice.IsAMP(),
IsBlinded: invoice.IsBlinded(),
// This will be set to SETTLED if at least one of the AMP Sets
// is settled (see below).
State: state,
Htlcs: rpcHtlcs,
Features: CreateRPCFeatures(invoice.Terms.Features),
IsKeysend: invoice.IsKeysend(),
PaymentAddr: invoice.Terms.PaymentAddr[:],
IsAmp: invoice.IsAMP(),
IsBlinded: invoice.IsBlinded(),
}
rpcInvoice.AmpInvoiceState = make(map[string]*lnrpc.AMPInvoiceState)

View File

@ -164,3 +164,7 @@ func (*mockTrafficShaper) ProduceHtlcExtraData(totalAmount lnwire.MilliSatoshi,
return totalAmount, nil, nil
}
func (*mockTrafficShaper) IsCustomHTLC(_ lnwire.CustomRecords) bool {
return false
}