chanbackup: add Single.CloseTxInputs field

The field is optional. It stores inputs needed to produce signed commit tx using
chantools scbforceclose, which calls function GetSignedCommitTx. New backups
have this field filled if commit tx is available (for all cases except when DLP
is active). If a backup has this data, the field is filled from it, otherwise it
is kept empty.

Modified test function genRandomOpenChannelShell to cover new types of channels
(simple taproot channel and custom channel) and to cover combinations of bits.
Make sure that TapscriptRoot field is properly packed and unpacked.
This commit is contained in:
Boris Nagaev
2023-11-05 19:39:35 -03:00
parent f485e079b7
commit df84148ed2
4 changed files with 421 additions and 5 deletions

View File

@@ -7,6 +7,7 @@ import (
"github.com/btcsuite/btcd/btcec/v2" "github.com/btcsuite/btcd/btcec/v2"
"github.com/btcsuite/btcd/wire" "github.com/btcsuite/btcd/wire"
"github.com/lightningnetwork/lnd/channeldb" "github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/kvdb" "github.com/lightningnetwork/lnd/kvdb"
) )
@@ -53,6 +54,48 @@ func assembleChanBackup(addrSource AddressSource,
return &single, nil 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 // 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 identified by its channel point. If we're unable to find
// the target channel, then an error will be returned. // the target channel, then an error will be returned.

View File

@@ -267,9 +267,10 @@ func (s *SubSwapper) backupUpdater() {
log.Debugf("Adding channel %v to backup state", log.Debugf("Adding channel %v to backup state",
newChan.FundingOutpoint) newChan.FundingOutpoint)
s.backupState[newChan.FundingOutpoint] = NewSingle( single := NewSingle(
newChan.OpenChannel, newChan.Addrs, newChan.OpenChannel, newChan.Addrs,
) )
s.backupState[newChan.FundingOutpoint] = single
} }
// For all closed channels, we'll remove the prior // For all closed channels, we'll remove the prior

View File

@@ -2,6 +2,7 @@ package chanbackup
import ( import (
"bytes" "bytes"
"errors"
"fmt" "fmt"
"io" "io"
"net" "net"
@@ -11,6 +12,7 @@ import (
"github.com/btcsuite/btcd/chaincfg/chainhash" "github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/wire" "github.com/btcsuite/btcd/wire"
"github.com/lightningnetwork/lnd/channeldb" "github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/keychain" "github.com/lightningnetwork/lnd/keychain"
"github.com/lightningnetwork/lnd/lnencrypt" "github.com/lightningnetwork/lnd/lnencrypt"
"github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/lnwire"
@@ -56,8 +58,51 @@ const (
// TapscriptRootVersion is a version that denotes this is a MuSig2 // TapscriptRootVersion is a version that denotes this is a MuSig2
// channel with a top level tapscript commitment. // channel with a top level tapscript commitment.
TapscriptRootVersion = 6 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 // 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 // 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 // recover the settled funds within a channel in the case of partial or
@@ -142,11 +187,47 @@ type Single struct {
// //
// - ScriptEnforcedLeaseVersion // - ScriptEnforcedLeaseVersion
LeaseExpiry uint32 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 // 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 // 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, func NewSingle(channel *channeldb.OpenChannel,
nodeAddrs []net.Addr) Single { nodeAddrs []net.Addr) Single {
@@ -245,9 +326,18 @@ func NewSingle(channel *channeldb.OpenChannel,
single.Version = DefaultSingleVersion 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 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 // Serialize attempts to write out the serialized version of the target
// StaticChannelBackup into the passed io.Writer. // StaticChannelBackup into the passed io.Writer.
func (s *Single) Serialize(w io.Writer) error { func (s *Single) Serialize(w io.Writer) error {
@@ -329,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 // TODO(yy): remove the type assertion when we finished refactoring db
// into using write buffer. // into using write buffer.
buf, ok := w.(*bytes.Buffer) buf, ok := w.(*bytes.Buffer)
@@ -338,7 +482,7 @@ func (s *Single) Serialize(w io.Writer) error {
return lnwire.WriteElements( return lnwire.WriteElements(
buf, buf,
byte(s.Version), version,
uint16(len(singleBytes.Bytes())), uint16(len(singleBytes.Bytes())),
singleBytes.Bytes(), singleBytes.Bytes(),
) )
@@ -429,7 +573,9 @@ func (s *Single) Deserialize(r io.Reader) error {
return err 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 { switch s.Version {
case DefaultSingleVersion: case DefaultSingleVersion:
@@ -543,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 return nil
} }

View File

@@ -13,6 +13,7 @@ import (
"github.com/btcsuite/btcd/wire" "github.com/btcsuite/btcd/wire"
"github.com/davecgh/go-spew/spew" "github.com/davecgh/go-spew/spew"
"github.com/lightningnetwork/lnd/channeldb" "github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/keychain" "github.com/lightningnetwork/lnd/keychain"
"github.com/lightningnetwork/lnd/lnencrypt" "github.com/lightningnetwork/lnd/lnencrypt"
"github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/lnwire"
@@ -95,6 +96,29 @@ func assertSingleEqual(t *testing.T, a, b Single) {
a.Addresses[i], b.Addresses[i]) 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) { func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) {
@@ -124,7 +148,7 @@ func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) {
isInitiator = true isInitiator = true
} }
chanType := channeldb.ChannelType(rand.Intn(8)) chanType := channeldb.ChannelType(rand.Intn(1 << 12))
localCfg := channeldb.ChannelConfig{ localCfg := channeldb.ChannelConfig{
ChannelStateBounds: channeldb.ChannelStateBounds{}, 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{ return &channeldb.OpenChannel{
ChainHash: chainHash, ChainHash: chainHash,
ChanType: chanType, ChanType: chanType,
@@ -196,10 +243,61 @@ func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) {
IdentityPub: pub, IdentityPub: pub,
LocalChanCfg: localCfg, LocalChanCfg: localCfg,
RemoteChanCfg: remoteCfg, RemoteChanCfg: remoteCfg,
LocalCommitment: localCommit,
RevocationProducer: shaChainProducer, RevocationProducer: shaChainProducer,
TapscriptRoot: tapscriptRootOption,
}, nil }, 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 // TestSinglePackUnpack tests that we're able to unpack a previously packed
// channel backup. // channel backup.
func TestSinglePackUnpack(t *testing.T) { func TestSinglePackUnpack(t *testing.T) {
@@ -220,6 +318,9 @@ func TestSinglePackUnpack(t *testing.T) {
// decode/encode the final SCB. // decode/encode the final SCB.
version SingleBackupVersion 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 tests us if this test case should pass or not.
valid bool valid bool
}{ }{
@@ -269,11 +370,92 @@ func TestSinglePackUnpack(t *testing.T) {
version: 99, version: 99,
valid: false, 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 { for i, versionCase := range versionTestCases {
// First, we'll re-assign SCB version to what was indicated in // First, we'll re-assign SCB version to what was indicated in
// the test case. // the test case.
singleChanBackup.Version = versionCase.version singleChanBackup.Version = versionCase.version
singleChanBackup.CloseTxInputs = versionCase.closeTxInputs
var b bytes.Buffer var b bytes.Buffer