Merge pull request #8183 from starius/close-tx-in-static-backup

chanbackup, server, rpcserver: put close unsigned tx, remote signature and commit height to SCB
This commit is contained in:
Oliver Gugger 2024-10-14 16:44:19 +02:00 committed by GitHub
commit 10802305e5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 737 additions and 49 deletions

View File

@ -7,6 +7,7 @@ import (
"github.com/btcsuite/btcd/btcec/v2"
"github.com/btcsuite/btcd/wire"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/kvdb"
)
@ -53,6 +54,48 @@ func assembleChanBackup(addrSource AddressSource,
return &single, nil
}
// buildCloseTxInputs generates inputs needed to force close a channel from
// an open channel. Anyone having these inputs and the signer, can sign the
// force closure transaction. Warning! If the channel state updates, an attempt
// to close the channel using this method with outdated CloseTxInputs can result
// in loss of funds! This may happen if an outdated channel backup is attempted
// to be used to force close the channel.
func buildCloseTxInputs(
targetChan *channeldb.OpenChannel) fn.Option[CloseTxInputs] {
log.Debugf("Crafting CloseTxInputs for ChannelPoint(%v)",
targetChan.FundingOutpoint)
localCommit := targetChan.LocalCommitment
if localCommit.CommitTx == nil {
log.Infof("CommitTx is nil for ChannelPoint(%v), "+
"skipping CloseTxInputs. This is possible when "+
"DLP is active.", targetChan.FundingOutpoint)
return fn.None[CloseTxInputs]()
}
// We need unsigned force close tx and the counterparty's signature.
inputs := CloseTxInputs{
CommitTx: localCommit.CommitTx,
CommitSig: localCommit.CommitSig,
}
// In case of a taproot channel, commit height is needed as well to
// produce verification nonce for the taproot channel using shachain.
if targetChan.ChanType.IsTaproot() {
inputs.CommitHeight = localCommit.CommitHeight
}
// In case of a custom taproot channel, TapscriptRoot is needed as well.
if targetChan.ChanType.HasTapscriptRoot() {
inputs.TapscriptRoot = targetChan.TapscriptRoot
}
return fn.Some(inputs)
}
// FetchBackupForChan attempts to create a plaintext static channel backup for
// the target channel identified by its channel point. If we're unable to find
// the target channel, then an error will be returned.

View File

@ -48,6 +48,19 @@ type ChannelEvent struct {
NewChans []ChannelWithAddrs
}
// manualUpdate holds a group of channel state updates and an error channel
// to send back an error happened upon update processing or file updating.
type manualUpdate struct {
// singles hold channels backups. They can be either new or known
// channels in the Swapper.
singles []Single
// errChan is the channel to send an error back. If the update handling
// and the subsequent file updating succeeds, nil is sent.
// The channel must have capacity of 1 to prevent Swapper blocking.
errChan chan error
}
// ChannelSubscription represents an intent to be notified of any updates to
// the primary channel state.
type ChannelSubscription struct {
@ -90,6 +103,8 @@ type SubSwapper struct {
// over.
chanEvents *ChannelSubscription
manualUpdates chan manualUpdate
// keyRing is the main key ring that will allow us to pack the new
// multi backup.
keyRing keychain.KeyRing
@ -126,11 +141,12 @@ func NewSubSwapper(startingChans []Single, chanNotifier ChannelNotifier,
}
return &SubSwapper{
backupState: backupState,
chanEvents: chanEvents,
keyRing: keyRing,
Swapper: backupSwapper,
quit: make(chan struct{}),
backupState: backupState,
chanEvents: chanEvents,
keyRing: keyRing,
Swapper: backupSwapper,
quit: make(chan struct{}),
manualUpdates: make(chan manualUpdate),
}, nil
}
@ -168,6 +184,43 @@ func (s *SubSwapper) Stop() error {
return nil
}
// ManualUpdate inserts/updates channel states into the swapper. The updates
// are processed in another goroutine. The method waits for the updates to be
// fully processed and the file to be updated on-disk before returning.
func (s *SubSwapper) ManualUpdate(singles []Single) error {
// Create the channel to send an error back. If the update handling
// and the subsequent file updating succeeds, nil is sent.
// The channel must have capacity of 1 to prevent Swapper blocking.
errChan := make(chan error, 1)
// Create the update object to insert into the processing loop.
update := manualUpdate{
singles: singles,
errChan: errChan,
}
select {
case s.manualUpdates <- update:
case <-s.quit:
return fmt.Errorf("swapper stopped when sending manual update")
}
// Wait for processing, block on errChan.
select {
case err := <-errChan:
if err != nil {
return fmt.Errorf("processing of manual update "+
"failed: %w", err)
}
case <-s.quit:
return fmt.Errorf("swapper stopped when waiting for outcome")
}
// Success.
return nil
}
// updateBackupFile updates the backup file in place given the current state of
// the SubSwapper. We accept the set of channels that were closed between this
// update and the last to make sure we leave them out of our backup set union.
@ -267,9 +320,10 @@ func (s *SubSwapper) backupUpdater() {
log.Debugf("Adding channel %v to backup state",
newChan.FundingOutpoint)
s.backupState[newChan.FundingOutpoint] = NewSingle(
single := NewSingle(
newChan.OpenChannel, newChan.Addrs,
)
s.backupState[newChan.FundingOutpoint] = single
}
// For all closed channels, we'll remove the prior
@ -293,13 +347,45 @@ func (s *SubSwapper) backupUpdater() {
"num_old_chans=%v, num_new_chans=%v",
oldStateSize, newStateSize)
// With out new state constructed, we'll, atomically
// Without new state constructed, we'll, atomically
// update the on-disk backup state.
if err := s.updateBackupFile(closedChans...); err != nil {
log.Errorf("unable to update backup file: %v",
err)
}
// We received a manual update. Handle it and update the file.
case manualUpdate := <-s.manualUpdates:
oldStateSize := len(s.backupState)
// For all open channels, we'll create a new SCB given
// the required information.
for _, single := range manualUpdate.singles {
log.Debugf("Manual update of channel %v",
single.FundingOutpoint)
s.backupState[single.FundingOutpoint] = single
}
newStateSize := len(s.backupState)
log.Infof("Updating on-disk multi SCB backup: "+
"num_old_chans=%v, num_new_chans=%v",
oldStateSize, newStateSize)
// Without new state constructed, we'll, atomically
// update the on-disk backup state.
err := s.updateBackupFile()
if err != nil {
log.Errorf("unable to update backup file: %v",
err)
}
// Send the error (or nil) to the caller of
// ManualUpdate. The error channel must have capacity of
// 1 not to block here.
manualUpdate.errChan <- err
// TODO(roasbeef): refresh periodically on a time basis due to
// possible addr changes from node

View File

@ -277,4 +277,18 @@ func TestSubSwapperUpdater(t *testing.T) {
// Verify that the new set of backups, now has one less after the
// sub-swapper switches the new set with the old.
assertExpectedBackupSwap(t, swapper, subSwapper, keyRing, backupSet)
// Check ManualUpdate method.
channel, err := genRandomOpenChannelShell()
require.NoError(t, err)
single := NewSingle(channel, nil)
backupSet[channel.FundingOutpoint] = single
require.NoError(t, subSwapper.ManualUpdate([]Single{single}))
// Verify that the state of the backup is as expected.
assertExpectedBackupSwap(t, swapper, subSwapper, keyRing, backupSet)
// Check the case ManualUpdate returns an error.
swapper.fail = true
require.Error(t, subSwapper.ManualUpdate([]Single{single}))
}

View File

@ -2,6 +2,7 @@ package chanbackup
import (
"bytes"
"errors"
"fmt"
"io"
"net"
@ -11,6 +12,7 @@ import (
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/wire"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/keychain"
"github.com/lightningnetwork/lnd/lnencrypt"
"github.com/lightningnetwork/lnd/lnwire"
@ -52,8 +54,55 @@ const (
// SimpleTaprootVersion is a version that denotes this channel is using
// the musig2 based taproot commitment format.
SimpleTaprootVersion = 5
// TapscriptRootVersion is a version that denotes this is a MuSig2
// channel with a top level tapscript commitment.
TapscriptRootVersion = 6
// closeTxVersionMask is the byte mask used that is ORed to version byte
// on wire indicating that the backup has CloseTxInputs.
closeTxVersionMask = 1 << 7
)
// Encode returns encoding of the version to put into channel backup.
// Argument "closeTx" specifies if the backup includes force close transaction.
func (v SingleBackupVersion) Encode(closeTx bool) byte {
encoded := byte(v)
// If the backup includes closing transaction, set this bit in the
// encoded version.
if closeTx {
encoded |= closeTxVersionMask
}
return encoded
}
// DecodeVersion decodes the encoding of the version from a channel backup.
// It returns the version and if the backup includes the force close tx.
func DecodeVersion(encoded byte) (SingleBackupVersion, bool) {
// Find if it has a closing transaction by inspecting the bit.
closeTx := (encoded & closeTxVersionMask) != 0
// The version byte also encodes the closeTxVersion feature, so we
// extract it here and return it separately to the backup version.
version := SingleBackupVersion(encoded &^ closeTxVersionMask)
return version, closeTx
}
// IsTaproot returns if this is a backup of a taproot channel. This will also be
// true for simple taproot overlay channels when a version is added.
func (v SingleBackupVersion) IsTaproot() bool {
return v == SimpleTaprootVersion || v == TapscriptRootVersion
}
// HasTapscriptRoot returns true if the channel is using a top level tapscript
// root commitment.
func (v SingleBackupVersion) HasTapscriptRoot() bool {
return v == TapscriptRootVersion
}
// Single is a static description of an existing channel that can be used for
// the purposes of backing up. The fields in this struct allow a node to
// recover the settled funds within a channel in the case of partial or
@ -138,11 +187,47 @@ type Single struct {
//
// - ScriptEnforcedLeaseVersion
LeaseExpiry uint32
// CloseTxInputs contains data needed to produce a force close tx
// using for example the "chantools scbforceclose" command.
//
// The field is optional.
CloseTxInputs fn.Option[CloseTxInputs]
}
// CloseTxInputs contains data needed to produce a force close transaction
// using for example the "chantools scbforceclose" command.
type CloseTxInputs struct {
// CommitTx is the latest version of the commitment state, broadcast
// able by us, but not signed. It can be signed by for example the
// "chantools scbforceclose" command.
CommitTx *wire.MsgTx
// CommitSig is one half of the signature required to fully complete
// the script for the commitment transaction above. This is the
// signature signed by the remote party for our version of the
// commitment transactions.
CommitSig []byte
// CommitHeight is the update number that this ChannelDelta represents
// the total number of commitment updates to this point. This can be
// viewed as sort of a "commitment height" as this number is
// monotonically increasing.
//
// This field is filled only for taproot channels.
CommitHeight uint64
// TapscriptRoot is the root of the tapscript tree that will be used to
// create the funding output. This is an optional field that should
// only be set for overlay taproot channels (HasTapscriptRoot).
TapscriptRoot fn.Option[chainhash.Hash]
}
// NewSingle creates a new static channel backup based on an existing open
// channel. We also pass in the set of addresses that we used in the past to
// connect to the channel peer.
// connect to the channel peer. If possible, we include the data needed to
// produce a force close transaction from the most recent state using externally
// provided private key.
func NewSingle(channel *channeldb.OpenChannel,
nodeAddrs []net.Addr) Single {
@ -218,7 +303,11 @@ func NewSingle(channel *channeldb.OpenChannel,
switch {
case channel.ChanType.IsTaproot():
single.Version = SimpleTaprootVersion
if channel.ChanType.HasTapscriptRoot() {
single.Version = TapscriptRootVersion
} else {
single.Version = SimpleTaprootVersion
}
case channel.ChanType.HasLeaseExpiration():
single.Version = ScriptEnforcedLeaseVersion
@ -237,9 +326,18 @@ func NewSingle(channel *channeldb.OpenChannel,
single.Version = DefaultSingleVersion
}
// Include unsigned force-close transaction for the most recent channel
// state as well as the data needed to produce the signature, given the
// private key is provided separately.
single.CloseTxInputs = buildCloseTxInputs(channel)
return single
}
// errEmptyTapscriptRoot is returned by Serialize if field TapscriptRoot is
// empty, when it should be filled according to the channel version.
var errEmptyTapscriptRoot = errors.New("field TapscriptRoot is not filled")
// Serialize attempts to write out the serialized version of the target
// StaticChannelBackup into the passed io.Writer.
func (s *Single) Serialize(w io.Writer) error {
@ -252,6 +350,7 @@ func (s *Single) Serialize(w io.Writer) error {
case AnchorsZeroFeeHtlcTxCommitVersion:
case ScriptEnforcedLeaseVersion:
case SimpleTaprootVersion:
case TapscriptRootVersion:
default:
return fmt.Errorf("unable to serialize w/ unknown "+
"version: %v", s.Version)
@ -320,6 +419,60 @@ func (s *Single) Serialize(w io.Writer) error {
}
}
// Encode version enum and hasCloseTx flag to version byte.
version := s.Version.Encode(s.CloseTxInputs.IsSome())
// Serialize CloseTxInputs if it is provided. Fill err if it fails.
err := fn.MapOptionZ(s.CloseTxInputs, func(inputs CloseTxInputs) error {
err := inputs.CommitTx.Serialize(&singleBytes)
if err != nil {
return err
}
err = lnwire.WriteElements(
&singleBytes,
uint16(len(inputs.CommitSig)), inputs.CommitSig,
)
if err != nil {
return err
}
if !s.Version.IsTaproot() {
return nil
}
// Write fields needed for taproot channels.
err = lnwire.WriteElements(
&singleBytes, inputs.CommitHeight,
)
if err != nil {
return err
}
if s.Version.HasTapscriptRoot() {
opt := inputs.TapscriptRoot
var tapscriptRoot chainhash.Hash
tapscriptRoot, err = opt.UnwrapOrErr(
errEmptyTapscriptRoot,
)
if err != nil {
return err
}
err = lnwire.WriteElements(
&singleBytes, tapscriptRoot[:],
)
if err != nil {
return err
}
}
return nil
})
if err != nil {
return fmt.Errorf("failed to encode CloseTxInputs: %w", err)
}
// TODO(yy): remove the type assertion when we finished refactoring db
// into using write buffer.
buf, ok := w.(*bytes.Buffer)
@ -329,7 +482,7 @@ func (s *Single) Serialize(w io.Writer) error {
return lnwire.WriteElements(
buf,
byte(s.Version),
version,
uint16(len(singleBytes.Bytes())),
singleBytes.Bytes(),
)
@ -420,7 +573,9 @@ func (s *Single) Deserialize(r io.Reader) error {
return err
}
s.Version = SingleBackupVersion(version)
// Decode version byte to version enum and hasCloseTx flag.
var hasCloseTx bool
s.Version, hasCloseTx = DecodeVersion(version)
switch s.Version {
case DefaultSingleVersion:
@ -429,6 +584,7 @@ func (s *Single) Deserialize(r io.Reader) error {
case AnchorsZeroFeeHtlcTxCommitVersion:
case ScriptEnforcedLeaseVersion:
case SimpleTaprootVersion:
case TapscriptRootVersion:
default:
return fmt.Errorf("unable to de-serialize w/ unknown "+
"version: %v", s.Version)
@ -533,6 +689,50 @@ func (s *Single) Deserialize(r io.Reader) error {
}
}
if !hasCloseTx {
return nil
}
// Deserialize CloseTxInputs if it is present in serialized data.
commitTx := &wire.MsgTx{}
if err := commitTx.Deserialize(r); err != nil {
return err
}
var commitSigLen uint16
if err := lnwire.ReadElement(r, &commitSigLen); err != nil {
return err
}
commitSig := make([]byte, commitSigLen)
if err := lnwire.ReadElement(r, commitSig); err != nil {
return err
}
var commitHeight uint64
if s.Version.IsTaproot() {
err := lnwire.ReadElement(r, &commitHeight)
if err != nil {
return err
}
}
tapscriptRootOpt := fn.None[chainhash.Hash]()
if s.Version.HasTapscriptRoot() {
var tapscriptRoot chainhash.Hash
err := lnwire.ReadElement(r, tapscriptRoot[:])
if err != nil {
return err
}
tapscriptRootOpt = fn.Some(tapscriptRoot)
}
s.CloseTxInputs = fn.Some(CloseTxInputs{
CommitTx: commitTx,
CommitSig: commitSig,
CommitHeight: commitHeight,
TapscriptRoot: tapscriptRootOpt,
})
return nil
}

View File

@ -13,6 +13,7 @@ import (
"github.com/btcsuite/btcd/wire"
"github.com/davecgh/go-spew/spew"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/keychain"
"github.com/lightningnetwork/lnd/lnencrypt"
"github.com/lightningnetwork/lnd/lnwire"
@ -95,6 +96,29 @@ func assertSingleEqual(t *testing.T, a, b Single) {
a.Addresses[i], b.Addresses[i])
}
}
// Make sure that CloseTxInputs are present either in both backups,
// or in none of them.
require.Equal(t, a.CloseTxInputs.IsSome(), b.CloseTxInputs.IsSome())
if a.CloseTxInputs.IsSome() {
// Cache CloseTxInputs into short variables.
ai := a.CloseTxInputs.UnwrapOrFail(t)
bi := b.CloseTxInputs.UnwrapOrFail(t)
// Compare serialized unsigned transactions.
var abuf, bbuf bytes.Buffer
require.NoError(t, ai.CommitTx.Serialize(&abuf))
require.NoError(t, bi.CommitTx.Serialize(&bbuf))
aBytes := abuf.Bytes()
bBytes := bbuf.Bytes()
require.Equal(t, aBytes, bBytes)
// Compare counterparty's signature and commit height.
require.Equal(t, ai.CommitSig, bi.CommitSig)
require.Equal(t, ai.CommitHeight, bi.CommitHeight)
require.Equal(t, ai.TapscriptRoot, bi.TapscriptRoot)
}
}
func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) {
@ -124,7 +148,7 @@ func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) {
isInitiator = true
}
chanType := channeldb.ChannelType(rand.Intn(8))
chanType := channeldb.ChannelType(rand.Intn(1 << 12))
localCfg := channeldb.ChannelConfig{
ChannelStateBounds: channeldb.ChannelStateBounds{},
@ -184,6 +208,29 @@ func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) {
},
}
var localCommit channeldb.ChannelCommitment
if chanType.IsTaproot() {
var commitSig [64]byte
if _, err := rand.Read(commitSig[:]); err != nil {
return nil, err
}
localCommit = channeldb.ChannelCommitment{
CommitTx: sampleCommitTx,
CommitSig: commitSig[:],
CommitHeight: rand.Uint64(),
}
}
var tapscriptRootOption fn.Option[chainhash.Hash]
if chanType.HasTapscriptRoot() {
var tapscriptRoot chainhash.Hash
if _, err := rand.Read(tapscriptRoot[:]); err != nil {
return nil, err
}
tapscriptRootOption = fn.Some(tapscriptRoot)
}
return &channeldb.OpenChannel{
ChainHash: chainHash,
ChanType: chanType,
@ -196,10 +243,61 @@ func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) {
IdentityPub: pub,
LocalChanCfg: localCfg,
RemoteChanCfg: remoteCfg,
LocalCommitment: localCommit,
RevocationProducer: shaChainProducer,
TapscriptRoot: tapscriptRootOption,
}, nil
}
// TestVersionEncoding tests encoding and decoding of version byte.
func TestVersionEncoding(t *testing.T) {
cases := []struct {
version SingleBackupVersion
hasCloseTx bool
versionByte byte
}{
{
version: DefaultSingleVersion,
hasCloseTx: false,
versionByte: DefaultSingleVersion,
},
{
version: DefaultSingleVersion,
hasCloseTx: true,
versionByte: DefaultSingleVersion | closeTxVersionMask,
},
{
version: AnchorsCommitVersion,
hasCloseTx: false,
versionByte: AnchorsCommitVersion,
},
{
version: AnchorsCommitVersion,
hasCloseTx: true,
versionByte: AnchorsCommitVersion | closeTxVersionMask,
},
}
for _, tc := range cases {
gotVersionByte := tc.version.Encode(tc.hasCloseTx)
require.Equal(t, tc.versionByte, gotVersionByte)
gotVersion, gotHasCloseTx := DecodeVersion(tc.versionByte)
require.Equal(t, tc.version, gotVersion)
require.Equal(t, tc.hasCloseTx, gotHasCloseTx)
}
}
var sampleCommitTx = &wire.MsgTx{
TxIn: []*wire.TxIn{
{PreviousOutPoint: wire.OutPoint{Hash: [32]byte{1}}},
},
TxOut: []*wire.TxOut{
{Value: 1e8, PkScript: []byte("1")},
{Value: 2e8, PkScript: []byte("2")},
},
}
// TestSinglePackUnpack tests that we're able to unpack a previously packed
// channel backup.
func TestSinglePackUnpack(t *testing.T) {
@ -220,6 +318,9 @@ func TestSinglePackUnpack(t *testing.T) {
// decode/encode the final SCB.
version SingleBackupVersion
// closeTxInputs is the data needed to produce a force close tx.
closeTxInputs fn.Option[CloseTxInputs]
// valid tests us if this test case should pass or not.
valid bool
}{
@ -250,16 +351,111 @@ func TestSinglePackUnpack(t *testing.T) {
valid: true,
},
// The new taproot channel version should
// pack/unpack with no problem.
{
version: SimpleTaprootVersion,
valid: true,
},
// The new tapscript root channel version should pack/unpack
// with no problem.
{
version: TapscriptRootVersion,
valid: true,
},
// A non-default version, atm this should result in a failure.
{
version: 99,
valid: false,
},
// Versions with CloseTxInputs.
{
version: DefaultSingleVersion,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
}),
valid: true,
},
{
version: TweaklessCommitVersion,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
}),
valid: true,
},
{
version: AnchorsCommitVersion,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
}),
valid: true,
},
{
version: ScriptEnforcedLeaseVersion,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
}),
valid: true,
},
{
version: SimpleTaprootVersion,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
CommitHeight: 42,
}),
valid: true,
},
{
version: TapscriptRootVersion,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
CommitHeight: 42,
TapscriptRoot: fn.Some(chainhash.Hash{1}),
}),
valid: true,
},
{
version: 99,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
}),
valid: false,
},
{
version: 99,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
CommitHeight: 42,
}),
valid: false,
},
{
version: TapscriptRootVersion,
closeTxInputs: fn.Some(CloseTxInputs{
CommitTx: sampleCommitTx,
CommitSig: []byte("signature"),
CommitHeight: 42,
// TapscriptRoot is not filled.
}),
valid: false,
},
}
for i, versionCase := range versionTestCases {
// First, we'll re-assign SCB version to what was indicated in
// the test case.
singleChanBackup.Version = versionCase.version
singleChanBackup.CloseTxInputs = versionCase.closeTxInputs
var b bytes.Buffer

View File

@ -162,6 +162,13 @@ func (c *chanDBRestorer) openChannelShell(backup chanbackup.Single) (
chanType |= channeldb.SingleFunderTweaklessBit
chanType |= channeldb.SimpleTaprootFeatureBit
case chanbackup.TapscriptRootVersion:
chanType = channeldb.ZeroHtlcTxFeeBit
chanType |= channeldb.AnchorOutputsBit
chanType |= channeldb.SingleFunderTweaklessBit
chanType |= channeldb.SimpleTaprootFeatureBit
chanType |= channeldb.TapscriptRootBit
default:
return nil, fmt.Errorf("unknown Single version: %w", err)
}

View File

@ -2679,10 +2679,10 @@ func exportChanBackup(ctx *cli.Context) error {
printJSON(struct {
ChanPoint string `json:"chan_point"`
ChanBackup []byte `json:"chan_backup"`
ChanBackup string `json:"chan_backup"`
}{
ChanPoint: chanPoint.String(),
ChanBackup: chanBackup.ChanBackup,
ChanBackup: hex.EncodeToString(chanBackup.ChanBackup),
})
return nil
}

View File

@ -20,9 +20,15 @@ import (
)
// commitSweepResolver is a resolver that will attempt to sweep the commitment
// output paying to us, in the case that the remote party broadcasts their
// version of the commitment transaction. We can sweep this output immediately,
// as it doesn't have a time-lock delay.
// output paying to us (local channel balance). In the case that the local
// party (we) broadcasts their version of the commitment transaction, we have
// to wait before sweeping it, as it has a CSV delay. For anchor channel
// type, even if the remote party broadcasts the commitment transaction,
// we have to wait one block after commitment transaction is confirmed,
// because CSV 1 is put into the script of UTXO representing local balance.
// Additionally, if the channel is a channel lease, we have to wait for
// CLTV to expire.
// https://docs.lightning.engineering/lightning-network-tools/pool/overview
type commitSweepResolver struct {
// localChanCfg is used to provide the resolver with the keys required
// to identify whether the commitment transaction was broadcast by the

View File

@ -258,6 +258,22 @@ An example of using file system level notification to [copy the backup to a
distinct volume/partition/drive can be found
here](https://gist.github.com/alexbosworth/2c5e185aedbdac45a03655b709e255a3).
##### Last resort manual force close
Reserve this option as a last resort when the peer is offline and all other
avenues to retrieve funds from the channel have been exhausted. The primary
motivation for introducing this option is to provide a means of recovery,
albeit with some risk, rather than losing the funds indefinitely. This is a very
dangerous option, so it should only be used after consulting with a recovery
specialist or after opening an issue to make sure!!!
Starting with release 0.19.0 LND includes unsigned force close transaction
for a channel into channel.backup file and RPCs returning channel backups.
To generate a force close transaction from the backup file, utilize the
`chantools scbforceclose` command. However, exercise caution as this action is
perilous. If the channel has been updated since the backup creation, another
node or a watchtower may issue a penalty transaction, seizing all funds!
#### Using the `ExportChanBackup` RPC
Another way to obtain SCBS for all or a target channel is via the new

View File

@ -51,12 +51,18 @@
* [Allow](https://github.com/lightningnetwork/lnd/pull/9017) the compression of logs during rotation with ZSTD via the `logcompressor` startup argument.
* The SCB file now [contains more data][https://github.com/lightningnetwork/lnd/pull/8183]
that enable a last resort rescue for certain cases where the peer is no longer
around.
* LND updates channel.backup file at shutdown time.
## RPC Updates
## lncli Updates
## Code Health
## Breaking Changes
## Performance Improvements
@ -89,6 +95,7 @@
# Contributors (Alphabetical Order)
* Boris Nagaev
* CharlieZKSmith
* Elle Mouton
* Pins

View File

@ -429,7 +429,7 @@ func testChannelBackupRestoreBasic(ht *lntest.HarnessTest) {
func runChanRestoreScenarioBasic(ht *lntest.HarnessTest,
restoreMethod restoreMethodType) {
// Create a new retore scenario.
// Create a new restore scenario.
crs := newChanRestoreScenario(
ht, lnrpc.CommitmentType_UNKNOWN_COMMITMENT_TYPE, false,
)
@ -470,7 +470,7 @@ func testChannelBackupRestoreUnconfirmed(ht *lntest.HarnessTest) {
// runChanRestoreScenarioUnConfirmed checks that Dave is able to restore for an
// unconfirmed channel.
func runChanRestoreScenarioUnConfirmed(ht *lntest.HarnessTest, useFile bool) {
// Create a new retore scenario.
// Create a new restore scenario.
crs := newChanRestoreScenario(
ht, lnrpc.CommitmentType_UNKNOWN_COMMITMENT_TYPE, false,
)
@ -608,7 +608,7 @@ func testChannelBackupRestoreCommitTypes(ht *lntest.HarnessTest) {
func runChanRestoreScenarioCommitTypes(ht *lntest.HarnessTest,
ct lnrpc.CommitmentType, zeroConf bool) {
// Create a new retore scenario.
// Create a new restore scenario.
crs := newChanRestoreScenario(ht, ct, zeroConf)
carol, dave := crs.carol, crs.dave
@ -668,7 +668,7 @@ func runChanRestoreScenarioCommitTypes(ht *lntest.HarnessTest,
// testChannelBackupRestoreLegacy checks a channel with the legacy revocation
// producer format and makes sure old SCBs can still be recovered.
func testChannelBackupRestoreLegacy(ht *lntest.HarnessTest) {
// Create a new retore scenario.
// Create a new restore scenario.
crs := newChanRestoreScenario(
ht, lnrpc.CommitmentType_UNKNOWN_COMMITMENT_TYPE, false,
)
@ -786,8 +786,22 @@ func runChanRestoreScenarioForceClose(ht *lntest.HarnessTest, zeroConf bool) {
ht.AssertNumTxsInMempool(1)
// Now that we're able to make our restored now, we'll shutdown the old
// Dave node as we'll be storing it shortly below.
ht.Shutdown(dave)
// Dave node as we'll be storing it shortly below. Use SuspendNode, not
// Shutdown to keep its directory including channel.backup file.
ht.SuspendNode(dave)
// Read Dave's channel.backup file again to make sure it was updated
// upon Dave's shutdown. In case LND state is lost and DLP protocol
// fails, the channel.backup file and the commit tx in it are the
// measure of last resort to recover funds from the channel. The file
// is updated upon LND server shutdown to update the commit tx just in
// case it is used this way. If an outdated commit tx is broadcasted,
// the funds may be lost in a justice transaction. The file is encrypted
// and we can't decrypt it here, so we just check that the content of
// the file has changed.
multi2, err := os.ReadFile(backupFilePath)
require.NoError(ht, err)
require.NotEqual(ht, multi, multi2)
// Mine a block to confirm the closing tx from Dave.
ht.MineBlocksAndAssertNumTxes(1, 1)

View File

@ -217,7 +217,7 @@ func (h *HarnessTest) AssertTxNotInMempool(txid chainhash.Hash) *wire.MsgTx {
}
// AssertNumTxsInMempool polls until finding the desired number of transactions
// in the provided miner's mempool. It will asserrt if this number is not met
// in the provided miner's mempool. It will assert if this number is not met
// after the given timeout.
func (h *HarnessTest) AssertNumTxsInMempool(n int) []*chainhash.Hash {
return h.miner.AssertNumTxsInMempool(n)

View File

@ -196,7 +196,7 @@ func (h *HarnessMiner) MineBlocks(num uint32) []*wire.MsgBlock {
}
// AssertNumTxsInMempool polls until finding the desired number of transactions
// in the provided miner's mempool. It will asserrt if this number is not met
// in the provided miner's mempool. It will assert if this number is not met
// after the given timeout.
func (h *HarnessMiner) AssertNumTxsInMempool(n int) []*chainhash.Hash {
var (

View File

@ -6433,29 +6433,84 @@ func (lc *LightningChannel) AbsoluteThawHeight() (uint32, error) {
return lc.channelState.AbsoluteThawHeight()
}
// getSignedCommitTx function take the latest commitment transaction and
// populate it with witness data.
func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) {
// Fetch the current commitment transaction, along with their signature
// for the transaction.
localCommit := lc.channelState.LocalCommitment
commitTx := localCommit.CommitTx.Copy()
// SignedCommitTxInputs contains data needed to create a signed commit
// transaction using a signer. See GetSignedCommitTx.
type SignedCommitTxInputs struct {
// CommitTx is the latest version of the commitment state, broadcast
// able by us.
CommitTx *wire.MsgTx
ourKey := lc.channelState.LocalChanCfg.MultiSigKey
theirKey := lc.channelState.RemoteChanCfg.MultiSigKey
// CommitSig is one half of the signature required to fully complete
// the script for the commitment transaction above. This is the
// signature signed by the remote party for our version of the
// commitment transactions.
CommitSig []byte
// OurKey is our key to be used within the 2-of-2 output script
// for the owner of this channel.
OurKey keychain.KeyDescriptor
// TheirKey is their key to be used within the 2-of-2 output script
// for the owner of this channel.
TheirKey keychain.KeyDescriptor
// SignDesc is the primary sign descriptor that is capable of signing
// the commitment transaction that spends the multi-sig output.
SignDesc *input.SignDescriptor
// Taproot holds fields needed in case of a taproot channel.
// Iff the channel is of taproot type, this field is filled.
Taproot fn.Option[TaprootSignedCommitTxInputs]
}
// TaprootSignedCommitTxInputs contains additional data needed to create a
// signed commit transaction using a signer, used in case of a taproot channel.
// See GetSignedCommitTx.
type TaprootSignedCommitTxInputs struct {
// CommitHeight is the update number that this channel state represents.
// It is the total number of commitment updates up to this point. This
// can be viewed as sort of a "commitment height" as this number is
// monotonically increasing. This number is used to make a signature
// for a taproot channel, since it is used by shachain nonce producer
// (TaprootNonceProducer).
CommitHeight uint64
// TaprootNonceProducer is used to generate a shachain tree for the
// purpose of generating verification nonces for taproot channels.
TaprootNonceProducer shachain.Producer
// TapscriptRoot is the root of the tapscript tree that will be used to
// create the funding output. This is an optional field that should
// only be set for taproot channels.
TapscriptRoot fn.Option[chainhash.Hash]
}
// GetSignedCommitTx creates the witness stack of a channel commitment
// transaction. It can handle all commitment types (taproot, legacy). It is
// exported to give outside tooling the possibility to recreate the witness.
// A key use case is generating the witness data for a commitment transaction
// from a Static Channel Backup (SCB).
func GetSignedCommitTx(inputs SignedCommitTxInputs,
signer input.Signer) (*wire.MsgTx, error) {
commitTx := inputs.CommitTx.Copy()
var witness wire.TxWitness
switch {
// If this is a taproot channel, then we'll need to re-derive the nonce
// we need to generate a new signature
case lc.channelState.ChanType.IsTaproot():
case inputs.Taproot.IsSome():
// Extract Taproot from fn.Option. It is safe to call
// UnsafeFromSome because we just checked that it is some.
taproot := inputs.Taproot.UnsafeFromSome()
// First, we'll need to re-derive the local nonce we sent to
// the remote party to create this musig session. We pass in
// the same height here as we're generating the nonce needed
// for the _current_ state.
localNonce, err := channeldb.NewMusigVerificationNonce(
ourKey.PubKey, lc.currentHeight,
lc.taprootNonceProducer,
inputs.OurKey.PubKey, taproot.CommitHeight,
taproot.TaprootNonceProducer,
)
if err != nil {
return nil, fmt.Errorf("unable to re-derive "+
@ -6463,19 +6518,20 @@ func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) {
}
tapscriptTweak := fn.MapOption(TapscriptRootToTweak)(
lc.channelState.TapscriptRoot,
taproot.TapscriptRoot,
)
// Now that we have the local nonce, we'll re-create the musig
// session we had for this height.
musigSession := NewPartialMusigSession(
*localNonce, ourKey, theirKey, lc.Signer,
&lc.fundingOutput, LocalMusigCommit, tapscriptTweak,
*localNonce, inputs.OurKey, inputs.TheirKey, signer,
inputs.SignDesc.Output, LocalMusigCommit,
tapscriptTweak,
)
var remoteSig lnwire.PartialSigWithNonce
err = remoteSig.Decode(
bytes.NewReader(localCommit.CommitSig),
bytes.NewReader(inputs.CommitSig),
)
if err != nil {
return nil, fmt.Errorf("unable to decode remote "+
@ -6521,15 +6577,15 @@ func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) {
// Otherwise, the final witness we generate will be a normal p2wsh
// multi-sig spend.
default:
theirSig, err := ecdsa.ParseDERSignature(localCommit.CommitSig)
theirSig, err := ecdsa.ParseDERSignature(inputs.CommitSig)
if err != nil {
return nil, err
}
// With this, we then generate the full witness so the caller
// can broadcast a fully signed transaction.
lc.signDesc.SigHashes = input.NewTxSigHashesV0Only(commitTx)
ourSig, err := lc.Signer.SignOutputRaw(commitTx, lc.signDesc)
inputs.SignDesc.SigHashes = input.NewTxSigHashesV0Only(commitTx)
ourSig, err := signer.SignOutputRaw(commitTx, inputs.SignDesc)
if err != nil {
return nil, err
}
@ -6537,9 +6593,9 @@ func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) {
// With the final signature generated, create the witness stack
// required to spend from the multi-sig output.
witness = input.SpendMultiSig(
lc.signDesc.WitnessScript,
ourKey.PubKey.SerializeCompressed(), ourSig,
theirKey.PubKey.SerializeCompressed(), theirSig,
inputs.SignDesc.WitnessScript,
inputs.OurKey.PubKey.SerializeCompressed(), ourSig,
inputs.TheirKey.PubKey.SerializeCompressed(), theirSig,
)
}
@ -6548,6 +6604,32 @@ func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) {
return commitTx, nil
}
// getSignedCommitTx method takes the latest commitment transaction and
// populates it with witness data.
func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) {
// Fetch the current commitment transaction, along with their signature
// for the transaction.
localCommit := lc.channelState.LocalCommitment
inputs := SignedCommitTxInputs{
CommitTx: localCommit.CommitTx,
CommitSig: localCommit.CommitSig,
OurKey: lc.channelState.LocalChanCfg.MultiSigKey,
TheirKey: lc.channelState.RemoteChanCfg.MultiSigKey,
SignDesc: lc.signDesc,
}
if lc.channelState.ChanType.IsTaproot() {
inputs.Taproot = fn.Some(TaprootSignedCommitTxInputs{
CommitHeight: lc.currentHeight,
TaprootNonceProducer: lc.taprootNonceProducer,
TapscriptRoot: lc.channelState.TapscriptRoot,
})
}
return GetSignedCommitTx(inputs, lc.Signer)
}
// CommitOutputResolution carries the necessary information required to allow
// us to sweep our commitment output in the case that either party goes to
// chain.

View File

@ -584,7 +584,7 @@ type MusigSessionCfg struct {
// funding input.
InputTxOut *wire.TxOut
// TapscriptRoot is an optional tweak that can be used to modify the
// TapscriptTweak is an optional tweak that can be used to modify the
// MuSig2 public key used in the session.
TapscriptTweak fn.Option[chainhash.Hash]
}

View File

@ -2504,6 +2504,23 @@ func (s *server) Stop() error {
if err := s.htlcNotifier.Stop(); err != nil {
srvrLog.Warnf("failed to stop htlcNotifier: %v", err)
}
// Update channel.backup file. Make sure to do it before
// stopping chanSubSwapper.
singles, err := chanbackup.FetchStaticChanBackups(
s.chanStateDB, s.addrSource,
)
if err != nil {
srvrLog.Warnf("failed to fetch channel states: %v",
err)
} else {
err := s.chanSubSwapper.ManualUpdate(singles)
if err != nil {
srvrLog.Warnf("Manual update of channel "+
"backup failed: %v", err)
}
}
if err := s.chanSubSwapper.Stop(); err != nil {
srvrLog.Warnf("failed to stop chanSubSwapper: %v", err)
}