lnwallet: delegate all channel funding logic to the new chanfunding package

In this commit, we begin to integrate the new channel funding package
into the existing codebase. With this set of changes, we'll no longer
construct and sign the funding transaction within this package, instead
delegating it to the new chanfunding package. We use the new
chanfunding.WalletAssembler to carry out all channel funding, providing
it with an implementation of all its interfaces backed by the wallet.
This commit is contained in:
Olaoluwa Osuntokun 2019-10-31 21:39:17 -07:00
parent d422ebbc66
commit 7a64a7d3a4
No known key found for this signature in database
GPG Key ID: BC13F65E2DC84465
5 changed files with 292 additions and 309 deletions

View File

@ -2864,7 +2864,7 @@ func TestFundingManagerFundAll(t *testing.T) {
Value: btcutil.Amount( Value: btcutil.Amount(
0.05 * btcutil.SatoshiPerBitcoin, 0.05 * btcutil.SatoshiPerBitcoin,
), ),
PkScript: make([]byte, 22), PkScript: coinPkScript,
OutPoint: wire.OutPoint{ OutPoint: wire.OutPoint{
Hash: chainhash.Hash{}, Hash: chainhash.Hash{},
Index: 0, Index: 0,
@ -2875,7 +2875,7 @@ func TestFundingManagerFundAll(t *testing.T) {
Value: btcutil.Amount( Value: btcutil.Amount(
0.06 * btcutil.SatoshiPerBitcoin, 0.06 * btcutil.SatoshiPerBitcoin,
), ),
PkScript: make([]byte, 22), PkScript: coinPkScript,
OutPoint: wire.OutPoint{ OutPoint: wire.OutPoint{
Hash: chainhash.Hash{}, Hash: chainhash.Hash{},
Index: 1, Index: 1,

View File

@ -41,6 +41,7 @@ import (
"github.com/lightningnetwork/lnd/lnwallet" "github.com/lightningnetwork/lnd/lnwallet"
"github.com/lightningnetwork/lnd/lnwallet/btcwallet" "github.com/lightningnetwork/lnd/lnwallet/btcwallet"
"github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/lnwallet/chainfee"
"github.com/lightningnetwork/lnd/lnwallet/chanfunding"
"github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/lnwire"
) )
@ -616,7 +617,7 @@ func testFundingTransactionLockedOutputs(miner *rpctest.Harness,
if err == nil { if err == nil {
t.Fatalf("not error returned, should fail on coin selection") t.Fatalf("not error returned, should fail on coin selection")
} }
if _, ok := err.(*lnwallet.ErrInsufficientFunds); !ok { if _, ok := err.(*chanfunding.ErrInsufficientFunds); !ok {
t.Fatalf("error not coinselect error: %v", err) t.Fatalf("error not coinselect error: %v", err)
} }
if failedReservation != nil { if failedReservation != nil {
@ -655,7 +656,7 @@ func testFundingCancellationNotEnoughFunds(miner *rpctest.Harness,
// Attempt to create another channel with 44 BTC, this should fail. // Attempt to create another channel with 44 BTC, this should fail.
_, err = alice.InitChannelReservation(req) _, err = alice.InitChannelReservation(req)
if _, ok := err.(*lnwallet.ErrInsufficientFunds); !ok { if _, ok := err.(*chanfunding.ErrInsufficientFunds); !ok {
t.Fatalf("coin selection succeeded should have insufficient funds: %v", t.Fatalf("coin selection succeeded should have insufficient funds: %v",
err) err)
} }
@ -699,7 +700,7 @@ func testCancelNonExistentReservation(miner *rpctest.Harness,
// Create our own reservation, give it some ID. // Create our own reservation, give it some ID.
res, err := lnwallet.NewChannelReservation( res, err := lnwallet.NewChannelReservation(
10000, 10000, feePerKw, alice, 22, 10, &testHdSeed, 10000, 10000, feePerKw, alice, 22, 10, &testHdSeed,
lnwire.FFAnnounceChannel, true, lnwire.FFAnnounceChannel, true, nil, [32]byte{},
) )
if err != nil { if err != nil {
t.Fatalf("unable to create res: %v", err) t.Fatalf("unable to create res: %v", err)

View File

@ -11,6 +11,7 @@ import (
"github.com/lightningnetwork/lnd/channeldb" "github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/input" "github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/lnwallet/chainfee"
"github.com/lightningnetwork/lnd/lnwallet/chanfunding"
"github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/lnwire"
) )
@ -114,7 +115,10 @@ type ChannelReservation struct {
// commitment state. // commitment state.
pushMSat lnwire.MilliSatoshi pushMSat lnwire.MilliSatoshi
wallet *LightningWallet wallet *LightningWallet
chanFunder chanfunding.Assembler
fundingIntent chanfunding.Intent
} }
// NewChannelReservation creates a new channel reservation. This function is // NewChannelReservation creates a new channel reservation. This function is
@ -124,8 +128,8 @@ type ChannelReservation struct {
func NewChannelReservation(capacity, localFundingAmt btcutil.Amount, func NewChannelReservation(capacity, localFundingAmt btcutil.Amount,
commitFeePerKw chainfee.SatPerKWeight, wallet *LightningWallet, commitFeePerKw chainfee.SatPerKWeight, wallet *LightningWallet,
id uint64, pushMSat lnwire.MilliSatoshi, chainHash *chainhash.Hash, id uint64, pushMSat lnwire.MilliSatoshi, chainHash *chainhash.Hash,
flags lnwire.FundingFlag, flags lnwire.FundingFlag, tweaklessCommit bool,
tweaklessCommit bool) (*ChannelReservation, error) { fundingAssembler chanfunding.Assembler) (*ChannelReservation, error) {
var ( var (
ourBalance lnwire.MilliSatoshi ourBalance lnwire.MilliSatoshi
@ -213,6 +217,14 @@ func NewChannelReservation(capacity, localFundingAmt btcutil.Amount,
} else { } else {
chanType |= channeldb.SingleFunderBit chanType |= channeldb.SingleFunderBit
} }
// If this intent isn't one that's able to provide us with a
// funding transaction, then we'll set the chanType bit to
// signal that we don't have access to one.
if _, ok := fundingAssembler.(chanfunding.FundingTxAssembler); !ok {
chanType |= channeldb.NoFundingTxBit
}
} else { } else {
// Otherwise, this is a dual funder channel, and no side is // Otherwise, this is a dual funder channel, and no side is
// technically the "initiator" // technically the "initiator"
@ -253,6 +265,7 @@ func NewChannelReservation(capacity, localFundingAmt btcutil.Amount,
pushMSat: pushMSat, pushMSat: pushMSat,
reservationID: id, reservationID: id,
wallet: wallet, wallet: wallet,
chanFunder: fundingAssembler,
}, nil }, nil
} }

View File

@ -5,7 +5,6 @@ import (
"crypto/sha256" "crypto/sha256"
"errors" "errors"
"fmt" "fmt"
"math"
"net" "net"
"sync" "sync"
"sync/atomic" "sync/atomic"
@ -99,6 +98,12 @@ type InitFundingReserveMsg struct {
// commitment format or not. // commitment format or not.
Tweakless bool Tweakless bool
// ChanFunder is an optional channel funder that allows the caller to
// control exactly how the channel funding is carried out. If not
// specified, then the default chanfunding.WalletAssembler will be
// used.
ChanFunder chanfunding.Assembler
// err is a channel in which all errors will be sent across. Will be // err is a channel in which all errors will be sent across. Will be
// nil if this initial set is successful. // nil if this initial set is successful.
// //
@ -448,11 +453,25 @@ func (l *LightningWallet) handleFundingReserveRequest(req *InitFundingReserveMsg
return return
} }
// If no chanFunder was provided, then we'll assume the default
// assembler, which is backed by the wallet's internal coin selection.
if req.ChanFunder == nil {
cfg := chanfunding.WalletConfig{
CoinSource: &CoinSource{l},
CoinSelectLocker: l,
CoinLocker: l,
Signer: l.Cfg.Signer,
DustLimit: DefaultDustLimit(),
}
req.ChanFunder = chanfunding.NewWalletAssembler(cfg)
}
localFundingAmt := req.LocalFundingAmt localFundingAmt := req.LocalFundingAmt
remoteFundingAmt := req.RemoteFundingAmt
var ( var (
selected *coinSelection fundingIntent chanfunding.Intent
err error err error
) )
// If we're on the receiving end of a single funder channel then we // If we're on the receiving end of a single funder channel then we
@ -463,9 +482,18 @@ func (l *LightningWallet) handleFundingReserveRequest(req *InitFundingReserveMsg
// Coin selection is done on the basis of sat/kw, so we'll use // Coin selection is done on the basis of sat/kw, so we'll use
// the fee rate passed in to perform coin selection. // the fee rate passed in to perform coin selection.
var err error var err error
selected, err = l.selectCoinsAndChange( fundingReq := &chanfunding.Request{
req.FundingFeePerKw, req.LocalFundingAmt, req.MinConfs, RemoteAmt: req.RemoteFundingAmt,
req.SubtractFees, LocalAmt: req.LocalFundingAmt,
MinConfs: req.MinConfs,
SubtractFees: req.SubtractFees,
FeeRate: req.FundingFeePerKw,
ChangeAddr: func() (btcutil.Address, error) {
return l.NewAddress(WitnessPubKey, true)
},
}
fundingIntent, err = req.ChanFunder.ProvisionChannel(
fundingReq,
) )
if err != nil { if err != nil {
req.err <- err req.err <- err
@ -473,31 +501,38 @@ func (l *LightningWallet) handleFundingReserveRequest(req *InitFundingReserveMsg
return return
} }
localFundingAmt = selected.fundingAmt localFundingAmt = fundingIntent.LocalFundingAmt()
remoteFundingAmt = fundingIntent.RemoteFundingAmt()
} }
// The total channel capacity will be the size of the funding output we // The total channel capacity will be the size of the funding output we
// created plus the remote contribution. // created plus the remote contribution.
capacity := localFundingAmt + req.RemoteFundingAmt capacity := localFundingAmt + remoteFundingAmt
id := atomic.AddUint64(&l.nextFundingID, 1) id := atomic.AddUint64(&l.nextFundingID, 1)
reservation, err := NewChannelReservation( reservation, err := NewChannelReservation(
capacity, localFundingAmt, req.CommitFeePerKw, l, id, capacity, localFundingAmt, req.CommitFeePerKw, l, id,
req.PushMSat, l.Cfg.NetParams.GenesisHash, req.Flags, req.PushMSat, l.Cfg.NetParams.GenesisHash, req.Flags,
req.Tweakless, req.Tweakless, req.ChanFunder,
) )
if err != nil { if err != nil {
selected.unlockCoins() if fundingIntent != nil {
fundingIntent.Cancel()
}
req.err <- err req.err <- err
req.resp <- nil req.resp <- nil
return return
} }
err = l.initOurContribution( err = l.initOurContribution(
reservation, selected, req.NodeAddr, req.NodeID, reservation, fundingIntent, req.NodeAddr, req.NodeID,
) )
if err != nil { if err != nil {
selected.unlockCoins() if fundingIntent != nil {
fundingIntent.Cancel()
}
req.err <- err req.err <- err
req.resp <- nil req.resp <- nil
return return
@ -520,26 +555,34 @@ func (l *LightningWallet) handleFundingReserveRequest(req *InitFundingReserveMsg
// and change reserved for the channel, and derives the keys to use for this // and change reserved for the channel, and derives the keys to use for this
// channel. // channel.
func (l *LightningWallet) initOurContribution(reservation *ChannelReservation, func (l *LightningWallet) initOurContribution(reservation *ChannelReservation,
selected *coinSelection, nodeAddr net.Addr, nodeID *btcec.PublicKey) error { fundingIntent chanfunding.Intent, nodeAddr net.Addr,
nodeID *btcec.PublicKey) error {
// Grab the mutex on the ChannelReservation to ensure thread-safety // Grab the mutex on the ChannelReservation to ensure thread-safety
reservation.Lock() reservation.Lock()
defer reservation.Unlock() defer reservation.Unlock()
if selected != nil { // At this point, if we have a funding intent, we'll use it to populate
reservation.ourContribution.Inputs = selected.coins // the existing reservation state entries for our coin selection.
reservation.ourContribution.ChangeOutputs = selected.change if fundingIntent != nil {
if intent, ok := fundingIntent.(*chanfunding.FullIntent); ok {
for _, coin := range intent.InputCoins {
reservation.ourContribution.Inputs = append(
reservation.ourContribution.Inputs,
&wire.TxIn{
PreviousOutPoint: coin.OutPoint,
},
)
}
reservation.ourContribution.ChangeOutputs = intent.ChangeOutputs
}
reservation.fundingIntent = fundingIntent
} }
reservation.nodeAddr = nodeAddr reservation.nodeAddr = nodeAddr
reservation.partialState.IdentityPub = nodeID reservation.partialState.IdentityPub = nodeID
// Next, we'll grab a series of keys from the wallet which will be used
// for the duration of the channel. The keys include: our multi-sig
// key, the base revocation key, the base htlc key,the base payment
// key, and the delayed payment key.
//
// TODO(roasbeef): "salt" each key as well?
var err error var err error
reservation.ourContribution.MultiSigKey, err = l.DeriveNextKey( reservation.ourContribution.MultiSigKey, err = l.DeriveNextKey(
keychain.KeyFamilyMultiSig, keychain.KeyFamilyMultiSig,
@ -709,101 +752,74 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
pendingReservation.Lock() pendingReservation.Lock()
defer pendingReservation.Unlock() defer pendingReservation.Unlock()
// Create a blank, fresh transaction. Soon to be a complete funding
// transaction which will allow opening a lightning channel.
pendingReservation.fundingTx = wire.NewMsgTx(1)
fundingTx := pendingReservation.fundingTx
// Some temporary variables to cut down on the resolution verbosity. // Some temporary variables to cut down on the resolution verbosity.
pendingReservation.theirContribution = req.contribution pendingReservation.theirContribution = req.contribution
theirContribution := req.contribution theirContribution := req.contribution
ourContribution := pendingReservation.ourContribution ourContribution := pendingReservation.ourContribution
// Add all multi-party inputs and outputs to the transaction. var (
for _, ourInput := range ourContribution.Inputs { chanPoint *wire.OutPoint
fundingTx.AddTxIn(ourInput) err error
}
for _, theirInput := range theirContribution.Inputs {
fundingTx.AddTxIn(theirInput)
}
for _, ourChangeOutput := range ourContribution.ChangeOutputs {
fundingTx.AddTxOut(ourChangeOutput)
}
for _, theirChangeOutput := range theirContribution.ChangeOutputs {
fundingTx.AddTxOut(theirChangeOutput)
}
ourKey := pendingReservation.ourContribution.MultiSigKey
theirKey := theirContribution.MultiSigKey
// Finally, add the 2-of-2 multi-sig output which will set up the lightning
// channel.
channelCapacity := int64(pendingReservation.partialState.Capacity)
witnessScript, multiSigOut, err := input.GenFundingPkScript(
ourKey.PubKey.SerializeCompressed(),
theirKey.PubKey.SerializeCompressed(), channelCapacity,
) )
if err != nil {
req.err <- err
return
}
// Sort the transaction. Since both side agree to a canonical ordering, // At this point, we can now construct our channel point. Depending on
// by sorting we no longer need to send the entire transaction. Only // which type of intent we obtained from our chanfunding.Assembler,
// signatures will be exchanged. // we'll carry out a distinct set of steps.
fundingTx.AddTxOut(multiSigOut) switch fundingIntent := pendingReservation.fundingIntent.(type) {
txsort.InPlaceSort(pendingReservation.fundingTx) case *chanfunding.FullIntent:
// Now that we know their public key, we can bind theirs as
// well as ours to the funding intent.
fundingIntent.BindKeys(
&pendingReservation.ourContribution.MultiSigKey,
theirContribution.MultiSigKey.PubKey,
)
// Next, sign all inputs that are ours, collecting the signatures in // With our keys bound, we can now construct+sign the final
// order of the inputs. // funding transaction and also obtain the chanPoint that
pendingReservation.ourFundingInputScripts = make([]*input.Script, 0, // creates the channel.
len(ourContribution.Inputs)) fundingTx, err := fundingIntent.CompileFundingTx(
signDesc := input.SignDescriptor{ theirContribution.Inputs,
HashType: txscript.SigHashAll, theirContribution.ChangeOutputs,
SigHashes: txscript.NewTxSigHashes(fundingTx),
}
for i, txIn := range fundingTx.TxIn {
info, err := l.FetchInputInfo(&txIn.PreviousOutPoint)
if err == ErrNotMine {
continue
} else if err != nil {
req.err <- err
return
}
signDesc.Output = &wire.TxOut{
PkScript: info.PkScript,
Value: int64(info.Value),
}
signDesc.InputIndex = i
inputScript, err := l.Cfg.Signer.ComputeInputScript(
fundingTx, &signDesc,
) )
if err != nil { if err != nil {
req.err <- err req.err <- fmt.Errorf("unable to construct funding "+
"tx: %v", err)
return
}
chanPoint, err = fundingIntent.ChanPoint()
if err != nil {
req.err <- fmt.Errorf("unable to obtain chan "+
"point: %v", err)
return return
} }
txIn.SignatureScript = inputScript.SigScript // Finally, we'll populate the relevant information in our
txIn.Witness = inputScript.Witness // pendingReservation so the rest of the funding flow can
pendingReservation.ourFundingInputScripts = append( // continue as normal.
pendingReservation.ourFundingInputScripts, pendingReservation.fundingTx = fundingTx
inputScript, pendingReservation.partialState.FundingOutpoint = *chanPoint
pendingReservation.ourFundingInputScripts = make(
[]*input.Script, 0, len(ourContribution.Inputs),
) )
for _, txIn := range fundingTx.TxIn {
_, err := l.FetchInputInfo(&txIn.PreviousOutPoint)
if err != nil {
continue
}
pendingReservation.ourFundingInputScripts = append(
pendingReservation.ourFundingInputScripts,
&input.Script{
Witness: txIn.Witness,
SigScript: txIn.SignatureScript,
},
)
}
walletLog.Debugf("Funding tx for ChannelPoint(%v) "+
"generated: %v", chanPoint, spew.Sdump(fundingTx))
} }
// Locate the index of the multi-sig outpoint in order to record it
// since the outputs are canonically sorted. If this is a single funder
// workflow, then we'll also need to send this to the remote node.
fundingTxID := fundingTx.TxHash()
_, multiSigIndex := input.FindScriptOutputIndex(fundingTx, multiSigOut.PkScript)
fundingOutpoint := wire.NewOutPoint(&fundingTxID, multiSigIndex)
pendingReservation.partialState.FundingOutpoint = *fundingOutpoint
walletLog.Debugf("Funding tx for ChannelPoint(%v) generated: %v",
fundingOutpoint, spew.Sdump(fundingTx))
// Initialize an empty sha-chain for them, tracking the current pending // Initialize an empty sha-chain for them, tracking the current pending
// revocation hash (we don't yet know the preimage so we can't add it // revocation hash (we don't yet know the preimage so we can't add it
// to the chain). // to the chain).
@ -819,10 +835,7 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
// Create the txin to our commitment transaction; required to construct // Create the txin to our commitment transaction; required to construct
// the commitment transactions. // the commitment transactions.
fundingTxIn := wire.TxIn{ fundingTxIn := wire.TxIn{
PreviousOutPoint: wire.OutPoint{ PreviousOutPoint: *chanPoint,
Hash: fundingTxID,
Index: multiSigIndex,
},
} }
// With the funding tx complete, create both commitment transactions. // With the funding tx complete, create both commitment transactions.
@ -879,21 +892,32 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
txsort.InPlaceSort(theirCommitTx) txsort.InPlaceSort(theirCommitTx)
walletLog.Debugf("Local commit tx for ChannelPoint(%v): %v", walletLog.Debugf("Local commit tx for ChannelPoint(%v): %v",
fundingOutpoint, spew.Sdump(ourCommitTx)) chanPoint, spew.Sdump(ourCommitTx))
walletLog.Debugf("Remote commit tx for ChannelPoint(%v): %v", walletLog.Debugf("Remote commit tx for ChannelPoint(%v): %v",
fundingOutpoint, spew.Sdump(theirCommitTx)) chanPoint, spew.Sdump(theirCommitTx))
// Record newly available information within the open channel state. // Record newly available information within the open channel state.
chanState.FundingOutpoint = *fundingOutpoint chanState.FundingOutpoint = *chanPoint
chanState.LocalCommitment.CommitTx = ourCommitTx chanState.LocalCommitment.CommitTx = ourCommitTx
chanState.RemoteCommitment.CommitTx = theirCommitTx chanState.RemoteCommitment.CommitTx = theirCommitTx
// Next, we'll obtain the funding witness script, and the funding
// output itself so we can generate a valid signature for the remote
// party.
fundingIntent := pendingReservation.fundingIntent
fundingWitnessScript, fundingOutput, err := fundingIntent.FundingOutput()
if err != nil {
req.err <- fmt.Errorf("unable to obtain funding output")
return
}
// Generate a signature for their version of the initial commitment // Generate a signature for their version of the initial commitment
// transaction. // transaction.
signDesc = input.SignDescriptor{ ourKey := ourContribution.MultiSigKey
WitnessScript: witnessScript, signDesc := input.SignDescriptor{
WitnessScript: fundingWitnessScript,
KeyDesc: ourKey, KeyDesc: ourKey,
Output: multiSigOut, Output: fundingOutput,
HashType: txscript.SigHashAll, HashType: txscript.SigHashAll,
SigHashes: txscript.NewTxSigHashes(theirCommitTx), SigHashes: txscript.NewTxSigHashes(theirCommitTx),
InputIndex: 0, InputIndex: 0,
@ -949,6 +973,62 @@ func (l *LightningWallet) handleSingleContribution(req *addSingleContributionMsg
return return
} }
// verifyFundingInputs attempts to verify all remote inputs to the funding
// transaction.
func (l *LightningWallet) verifyFundingInputs(fundingTx *wire.MsgTx,
remoteInputScripts []*input.Script) error {
sigIndex := 0
fundingHashCache := txscript.NewTxSigHashes(fundingTx)
inputScripts := remoteInputScripts
for i, txin := range fundingTx.TxIn {
if len(inputScripts) != 0 && len(txin.Witness) == 0 {
// Attach the input scripts so we can verify it below.
txin.Witness = inputScripts[sigIndex].Witness
txin.SignatureScript = inputScripts[sigIndex].SigScript
// Fetch the alleged previous output along with the
// pkscript referenced by this input.
//
// TODO(roasbeef): when dual funder pass actual
// height-hint
pkScript, err := input.WitnessScriptHash(
txin.Witness[len(txin.Witness)-1],
)
if err != nil {
return fmt.Errorf("cannot create script: %v", err)
}
output, err := l.Cfg.ChainIO.GetUtxo(
&txin.PreviousOutPoint,
pkScript, 0, l.quit,
)
if output == nil {
return fmt.Errorf("input to funding tx does "+
"not exist: %v", err)
}
// Ensure that the witness+sigScript combo is valid.
vm, err := txscript.NewEngine(
output.PkScript, fundingTx, i,
txscript.StandardVerifyFlags, nil,
fundingHashCache, output.Value,
)
if err != nil {
return fmt.Errorf("cannot create script "+
"engine: %s", err)
}
if err = vm.Execute(); err != nil {
return fmt.Errorf("cannot validate "+
"transaction: %s", err)
}
sigIndex++
}
}
return nil
}
// handleFundingCounterPartySigs is the final step in the channel reservation // handleFundingCounterPartySigs is the final step in the channel reservation
// workflow. During this step, we validate *all* the received signatures for // workflow. During this step, we validate *all* the received signatures for
@ -974,59 +1054,17 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
// signatures to their inputs. // signatures to their inputs.
res.theirFundingInputScripts = msg.theirFundingInputScripts res.theirFundingInputScripts = msg.theirFundingInputScripts
inputScripts := msg.theirFundingInputScripts inputScripts := msg.theirFundingInputScripts
// Only if we have the final funding transaction do we need to verify
// the final set of inputs. Otherwise, it may be the case that the
// channel was funded via an external wallet.
fundingTx := res.fundingTx fundingTx := res.fundingTx
sigIndex := 0 if res.partialState.ChanType.HasFundingTx() {
fundingHashCache := txscript.NewTxSigHashes(fundingTx) err := l.verifyFundingInputs(fundingTx, inputScripts)
for i, txin := range fundingTx.TxIn { if err != nil {
if len(inputScripts) != 0 && len(txin.Witness) == 0 { msg.err <- err
// Attach the input scripts so we can verify it below. msg.completeChan <- nil
txin.Witness = inputScripts[sigIndex].Witness return
txin.SignatureScript = inputScripts[sigIndex].SigScript
// Fetch the alleged previous output along with the
// pkscript referenced by this input.
//
// TODO(roasbeef): when dual funder pass actual
// height-hint
pkScript, err := input.WitnessScriptHash(
txin.Witness[len(txin.Witness)-1],
)
if err != nil {
msg.err <- fmt.Errorf("cannot create script: "+
"%v", err)
msg.completeChan <- nil
return
}
output, err := l.Cfg.ChainIO.GetUtxo(
&txin.PreviousOutPoint,
pkScript, 0, l.quit,
)
if output == nil {
msg.err <- fmt.Errorf("input to funding tx "+
"does not exist: %v", err)
msg.completeChan <- nil
return
}
// Ensure that the witness+sigScript combo is valid.
vm, err := txscript.NewEngine(output.PkScript,
fundingTx, i, txscript.StandardVerifyFlags, nil,
fundingHashCache, output.Value)
if err != nil {
msg.err <- fmt.Errorf("cannot create script "+
"engine: %s", err)
msg.completeChan <- nil
return
}
if err = vm.Execute(); err != nil {
msg.err <- fmt.Errorf("cannot validate "+
"transaction: %s", err)
msg.completeChan <- nil
return
}
sigIndex++
} }
} }
@ -1055,8 +1093,10 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
// is complete, allowing us to spend from the funding transaction. // is complete, allowing us to spend from the funding transaction.
channelValue := int64(res.partialState.Capacity) channelValue := int64(res.partialState.Capacity)
hashCache := txscript.NewTxSigHashes(commitTx) hashCache := txscript.NewTxSigHashes(commitTx)
sigHash, err := txscript.CalcWitnessSigHash(witnessScript, hashCache, sigHash, err := txscript.CalcWitnessSigHash(
txscript.SigHashAll, commitTx, 0, channelValue) witnessScript, hashCache, txscript.SigHashAll, commitTx,
0, channelValue,
)
if err != nil { if err != nil {
msg.err <- err msg.err <- err
msg.completeChan <- nil msg.completeChan <- nil
@ -1203,8 +1243,10 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
return return
} }
sigHash, err := txscript.CalcWitnessSigHash(witnessScript, hashCache, sigHash, err := txscript.CalcWitnessSigHash(
txscript.SigHashAll, ourCommitTx, 0, channelValue) witnessScript, hashCache, txscript.SigHashAll, ourCommitTx, 0,
channelValue,
)
if err != nil { if err != nil {
req.err <- err req.err <- err
req.completeChan <- nil req.completeChan <- nil
@ -1218,7 +1260,8 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
req.err <- err req.err <- err
req.completeChan <- nil req.completeChan <- nil
return return
} else if !sig.Verify(sigHash, theirKey.PubKey) { }
if !sig.Verify(sigHash, theirKey.PubKey) {
req.err <- fmt.Errorf("counterparty's commitment signature " + req.err <- fmt.Errorf("counterparty's commitment signature " +
"is invalid") "is invalid")
req.completeChan <- nil req.completeChan <- nil
@ -1291,138 +1334,6 @@ func (l *LightningWallet) WithCoinSelectLock(f func() error) error {
return f() return f()
} }
// coinSelection holds the result from selectCoinsAndChange.
type coinSelection struct {
coins []*wire.TxIn
change []*wire.TxOut
fundingAmt btcutil.Amount
unlockCoins func()
}
// selectCoinsAndChange performs coin selection in order to obtain witness
// outputs which sum to at least 'amt' amount of satoshis. If necessary,
// a change address will also be generated. If coin selection is
// successful/possible, then the selected coins and change outputs are
// returned, and the value of the resulting funding output. This method locks
// the selected outputs, and a function closure to unlock them in case of an
// error is returned.
func (l *LightningWallet) selectCoinsAndChange(feeRate chainfee.SatPerKWeight,
amt btcutil.Amount, minConfs int32, subtractFees bool) (
*coinSelection, error) {
// We hold the coin select mutex while querying for outputs, and
// performing coin selection in order to avoid inadvertent double
// spends across funding transactions.
l.coinSelectMtx.Lock()
defer l.coinSelectMtx.Unlock()
walletLog.Infof("Performing funding tx coin selection using %v "+
"sat/kw as fee rate", int64(feeRate))
// Find all unlocked unspent witness outputs that satisfy the minimum
// number of confirmations required.
utxos, err := l.ListUnspentWitness(minConfs, math.MaxInt32)
if err != nil {
return nil, err
}
coins := make([]chanfunding.Coin, len(utxos), 0)
for _, utxo := range utxos {
coins = append(coins, chanfunding.Coin{
TxOut: wire.TxOut{
Value: int64(utxo.Value),
PkScript: utxo.PkScript,
},
OutPoint: utxo.OutPoint,
})
}
var (
selectedCoins []chanfunding.Coin
fundingAmt btcutil.Amount
changeAmt btcutil.Amount
)
// Perform coin selection over our available, unlocked unspent outputs
// in order to find enough coins to meet the funding amount
// requirements.
switch {
// In case this request want the fees subtracted from the local amount,
// we'll call the specialized method for that. This ensures that we
// won't deduct more that the specified balance from our wallet.
case subtractFees:
dustLimit := l.Cfg.DefaultConstraints.DustLimit
selectedCoins, fundingAmt, changeAmt, err = chanfunding.CoinSelectSubtractFees(
feeRate, amt, dustLimit, coins,
)
if err != nil {
return nil, err
}
// Ótherwise do a normal coin selection where we target a given funding
// amount.
default:
fundingAmt = amt
selectedCoins, changeAmt, err = chanfunding.CoinSelect(feeRate, amt, coins)
if err != nil {
return nil, err
}
}
// Record any change output(s) generated as a result of the coin
// selection, but only if the addition of the output won't lead to the
// creation of dust.
var changeOutputs []*wire.TxOut
if changeAmt != 0 && changeAmt > DefaultDustLimit() {
changeAddr, err := l.NewAddress(WitnessPubKey, true)
if err != nil {
return nil, err
}
changeScript, err := txscript.PayToAddrScript(changeAddr)
if err != nil {
return nil, err
}
changeOutputs = make([]*wire.TxOut, 1)
changeOutputs[0] = &wire.TxOut{
Value: int64(changeAmt),
PkScript: changeScript,
}
}
// Lock the selected coins. These coins are now "reserved", this
// prevents concurrent funding requests from referring to and this
// double-spending the same set of coins.
inputs := make([]*wire.TxIn, len(selectedCoins))
for i, coin := range selectedCoins {
outpoint := &coin.OutPoint
l.lockedOutPoints[*outpoint] = struct{}{}
l.LockOutpoint(*outpoint)
// Empty sig script, we'll actually sign if this reservation is
// queued up to be completed (the other side accepts).
inputs[i] = wire.NewTxIn(outpoint, nil, nil)
}
unlock := func() {
l.coinSelectMtx.Lock()
defer l.coinSelectMtx.Unlock()
for _, coin := range selectedCoins {
outpoint := &coin.OutPoint
delete(l.lockedOutPoints, *outpoint)
l.UnlockOutpoint(*outpoint)
}
}
return &coinSelection{
coins: inputs,
change: changeOutputs,
fundingAmt: fundingAmt,
unlockCoins: unlock,
}, nil
}
// DeriveStateHintObfuscator derives the bytes to be used for obfuscating the // DeriveStateHintObfuscator derives the bytes to be used for obfuscating the
// state hints from the root to be used for a new channel. The obfuscator is // state hints from the root to be used for a new channel. The obfuscator is
// generated via the following computation: // generated via the following computation:
@ -1513,3 +1424,56 @@ func (l *LightningWallet) ValidateChannel(channelState *channeldb.OpenChannel,
return nil return nil
} }
// CoinSource is a wrapper around the wallet that implements the
// chanfunding.CoinSource interface.
type CoinSource struct {
wallet *LightningWallet
}
// NewCoinSource creates a new instance of the CoinSource wrapper struct.
func NewCoinSource(w *LightningWallet) *CoinSource {
return &CoinSource{wallet: w}
}
// ListCoins returns all UTXOs from the source that have between
// minConfs and maxConfs number of confirmations.
func (c *CoinSource) ListCoins(minConfs int32,
maxConfs int32) ([]chanfunding.Coin, error) {
utxos, err := c.wallet.ListUnspentWitness(minConfs, maxConfs)
if err != nil {
return nil, err
}
var coins []chanfunding.Coin
for _, utxo := range utxos {
coins = append(coins, chanfunding.Coin{
TxOut: wire.TxOut{
Value: int64(utxo.Value),
PkScript: utxo.PkScript,
},
OutPoint: utxo.OutPoint,
})
}
return coins, nil
}
// CoinFromOutPoint attempts to locate details pertaining to a coin based on
// its outpoint. If the coin isn't under the control of the backing CoinSource,
// then an error should be returned.
func (c *CoinSource) CoinFromOutPoint(op wire.OutPoint) (*chanfunding.Coin, error) {
inputInfo, err := c.wallet.FetchInputInfo(&op)
if err != nil {
return nil, err
}
return &chanfunding.Coin{
TxOut: wire.TxOut{
Value: int64(inputInfo.Value),
PkScript: inputInfo.PkScript,
},
OutPoint: inputInfo.OutPoint,
}, nil
}

View File

@ -1,6 +1,7 @@
package lnd package lnd
import ( import (
"encoding/hex"
"fmt" "fmt"
"sync" "sync"
"sync/atomic" "sync/atomic"
@ -20,6 +21,10 @@ import (
"github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/lnwallet/chainfee"
) )
var (
coinPkScript, _ = hex.DecodeString("001431df1bde03c074d0cf21ea2529427e1499b8f1de")
)
// The block height returned by the mock BlockChainIO's GetBestBlock. // The block height returned by the mock BlockChainIO's GetBestBlock.
const fundingBroadcastHeight = 123 const fundingBroadcastHeight = 123
@ -297,7 +302,7 @@ func (m *mockWalletController) ListUnspentWitness(minconfirms,
utxo := &lnwallet.Utxo{ utxo := &lnwallet.Utxo{
AddressType: lnwallet.WitnessPubKey, AddressType: lnwallet.WitnessPubKey,
Value: btcutil.Amount(10 * btcutil.SatoshiPerBitcoin), Value: btcutil.Amount(10 * btcutil.SatoshiPerBitcoin),
PkScript: make([]byte, 22), PkScript: coinPkScript,
OutPoint: wire.OutPoint{ OutPoint: wire.OutPoint{
Hash: chainhash.Hash{}, Hash: chainhash.Hash{},
Index: m.index, Index: m.index,