mirror of
https://github.com/lightningnetwork/lnd.git
synced 2025-04-13 06:29:07 +02:00
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:
commit
10802305e5
@ -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.
|
||||
|
@ -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
|
||||
|
||||
|
@ -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}))
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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 (
|
||||
|
@ -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.
|
||||
|
@ -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]
|
||||
}
|
||||
|
17
server.go
17
server.go
@ -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)
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user