lnwallet: update internal funding workflow to account for BOLT-0002

This commit updates the engine that drives the reservation workflow to
utilize the new ChannelConfig and ChannelConstraint structs added in
prior commits.  With this change, the internal reservation engine has
been modified to retain the prior dual funder workflow, but also be
able to properly manage and execute the new single funder workflow
defined in BOLT-0002.
This commit is contained in:
Olaoluwa Osuntokun 2017-07-29 19:20:08 -07:00
parent 441e5b2d7c
commit 9edc335049
No known key found for this signature in database
GPG Key ID: 9CC5B105D03521A2
1 changed files with 217 additions and 246 deletions

View File

@ -1,16 +1,18 @@
package lnwallet
import (
"bytes"
"crypto/sha256"
"fmt"
"net"
"strings"
"sync"
"sync/atomic"
"github.com/davecgh/go-spew/spew"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/roasbeef/btcd/chaincfg"
"github.com/roasbeef/btcd/blockchain"
"github.com/roasbeef/btcd/chaincfg/chainhash"
"github.com/roasbeef/btcutil/hdkeychain"
"github.com/lightningnetwork/lnd/shachain"
@ -65,15 +67,19 @@ func (e *ErrInsufficientFunds) Error() string {
// initFundingReserveReq is the first message sent to initiate the workflow
// required to open a payment channel with a remote peer. The initial required
// parameters are configurable across channels. These parameters are to be
// chosen depending on the fee climate within the network, and time value of funds to
// be locked up within the channel. Upon success a ChannelReservation will be
// created in order to track the lifetime of this pending channel. Outputs
// selected will be 'locked', making them unavailable, for any other pending
// reservations. Therefore, all channels in reservation limbo will be periodically
// after a timeout period in order to avoid "exhaustion" attacks.
// chosen depending on the fee climate within the network, and time value of
// funds to be locked up within the channel. Upon success a ChannelReservation
// will be created in order to track the lifetime of this pending channel.
// Outputs selected will be 'locked', making them unavailable, for any other
// pending reservations. Therefore, all channels in reservation limbo will be
// periodically after a timeout period in order to avoid "exhaustion" attacks.
//
// TODO(roasbeef): zombie reservation sweeper goroutine.
type initFundingReserveMsg struct {
// chainHash denotes that chain to be used to ultimately open the
// target channel.
chainHash *chainhash.Hash
// nodeId is the ID of the remote node we would like to open a channel
// with.
nodeID *btcec.PublicKey
@ -83,10 +89,6 @@ type initFundingReserveMsg struct {
// this funding workflow.
nodeAddr *net.TCPAddr
// numConfs is the number of confirmations required before the channel
// is considered open.
numConfs uint16
// fundingAmount is the amount of funds requested for this channel.
fundingAmount btcutil.Amount
@ -100,19 +102,10 @@ type initFundingReserveMsg struct {
// of the accepted base fee rate of the network.
feePerKw btcutil.Amount
// ourDustLimit is the threshold below which no HTLC output should be
// generated for our commitment transaction; ie. HTLCs below
// this amount are not enforceable onchain from our point of view.
ourDustLimit btcutil.Amount
// pushSat is the number of satoshis that should be pushed over the the
// pushSat is the number of satoshis that should be pushed over the
// responder as part of the initial channel creation.
pushSat btcutil.Amount
// csvDelay is the delay on the "pay-to-self" output(s) of the
// commitment transaction.
csvDelay uint32
// err is a channel in which all errors will be sent across. Will be
// nil if this initial set is successful.
//
@ -210,19 +203,10 @@ type addSingleFunderSigsMsg struct {
// transaction as assembled by the workflow initiator.
fundingOutpoint *wire.OutPoint
// revokeKey is the revocation public key derived by the remote node to
// be used within the initial version of the commitment transaction we
// construct for them.
revokeKey *btcec.PublicKey
// theirCommitmentSig are the 1/2 of the signatures needed to
// succesfully spend our version of the commitment transaction.
theirCommitmentSig []byte
// obsfucator is the bytes to be used to obsfucate the state hints on
// the commitment transaction.
obsfucator [StateHintSize]byte
// This channel is used to return the completed channel after the wallet
// has completed all of its stages in the funding process.
completeChan chan *channeldb.OpenChannel
@ -459,28 +443,22 @@ out:
// and final step verifies all signatures for the inputs of the funding
// transaction, and that the signature we records for our version of the
// commitment transaction is valid.
func (l *LightningWallet) InitChannelReservation(capacity,
ourFundAmt btcutil.Amount, theirID *btcec.PublicKey,
theirAddr *net.TCPAddr, numConfs uint16, csvDelay uint32,
ourDustLimit btcutil.Amount, pushSat btcutil.Amount,
feePerKw btcutil.Amount) (*ChannelReservation, error) {
// TODO(roasbeef): make the above into an initial config as part of the
// refactor to implement spec compliant funding flow
func (l *LightningWallet) InitChannelReservation(
capacity, ourFundAmt, pushSat, feePerKw btcutil.Amount,
theirID *btcec.PublicKey, theirAddr *net.TCPAddr,
chainHash *chainhash.Hash) (*ChannelReservation, error) {
errChan := make(chan error, 1)
respChan := make(chan *ChannelReservation, 1)
l.msgChan <- &initFundingReserveMsg{
capacity: capacity,
numConfs: numConfs,
fundingAmount: ourFundAmt,
csvDelay: csvDelay,
ourDustLimit: ourDustLimit,
feePerKw: feePerKw,
pushSat: pushSat,
chainHash: chainHash,
nodeID: theirID,
nodeAddr: theirAddr,
fundingAmount: ourFundAmt,
capacity: capacity,
feePerKw: feePerKw,
pushSat: pushSat,
err: errChan,
resp: respChan,
}
@ -499,24 +477,26 @@ func (l *LightningWallet) handleFundingReserveRequest(req *initFundingReserveMsg
return
}
// If the funding request is for a different chain than the one the
// wallet is aware of, then we'll reject the request.
if *l.Cfg.NetParams.GenesisHash != *req.chainHash {
req.err <- fmt.Errorf("unable to create channel reservation "+
"for chain=%v, wallet is on chain=%v",
req.chainHash, l.Cfg.NetParams.GenesisHash)
req.resp <- nil
return
}
id := atomic.AddUint64(&l.nextFundingID, 1)
reservation := NewChannelReservation(req.capacity, req.fundingAmount,
req.feePerKw, l, id, req.numConfs, req.pushSat)
req.feePerKw, l, id, req.pushSat, l.Cfg.NetParams.GenesisHash)
// Grab the mutex on the ChannelReservation to ensure thread-safety
reservation.Lock()
defer reservation.Unlock()
reservation.nodeAddr = req.nodeAddr
reservation.ourContribution.CsvDelay = req.csvDelay
reservation.ourContribution.DustLimit = req.ourDustLimit
reservation.partialState.NumConfsRequired = req.numConfs
reservation.partialState.IdentityPub = req.nodeID
reservation.partialState.LocalCsvDelay = req.csvDelay
reservation.partialState.OurDustLimit = req.ourDustLimit
ourContribution := reservation.ourContribution
// If we're on the receiving end of a single funder channel then we
// don't need to perform any coin selection. Otherwise, attempt to
@ -527,9 +507,9 @@ func (l *LightningWallet) handleFundingReserveRequest(req *initFundingReserveMsg
// funding transaction gets into the _next_ block.
//
// TODO(roasbeef): shouldn't be targeting next block
satPerByte := l.FeeEstimator.EstimateFeePerByte(1)
satPerByte := l.Cfg.FeeEstimator.EstimateFeePerByte(1)
err := l.selectCoinsAndChange(satPerByte, req.fundingAmount,
ourContribution)
reservation.ourContribution)
if err != nil {
req.err <- err
req.resp <- nil
@ -537,42 +517,75 @@ func (l *LightningWallet) handleFundingReserveRequest(req *initFundingReserveMsg
}
}
// Grab two fresh keys from our HD chain, one will be used for the
// multi-sig funding transaction, and the other for the commitment
// transaction.
multiSigKey, err := l.NewRawKey()
// 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 payment key, and the delayed
// payment key.
var err error
reservation.ourContribution.MultiSigKey, err = l.NewRawKey()
if err != nil {
req.err <- err
req.resp <- nil
return
}
commitKey, err := l.NewRawKey()
reservation.ourContribution.RevocationBasePoint, err = l.NewRawKey()
if err != nil {
req.err <- err
req.resp <- nil
return
}
reservation.ourContribution.PaymentBasePoint, err = l.NewRawKey()
if err != nil {
req.err <- err
req.resp <- nil
return
}
reservation.ourContribution.DelayBasePoint, err = l.NewRawKey()
if err != nil {
req.err <- err
req.resp <- nil
return
}
reservation.partialState.OurMultiSigKey = multiSigKey
ourContribution.MultiSigKey = multiSigKey
reservation.partialState.OurCommitKey = commitKey
ourContribution.CommitKey = commitKey
// Generate a fresh address to be used in the case of a cooperative
// channel close.
deliveryAddress, err := l.NewAddress(WitnessPubKey, false)
// With the above keys created, we'll also need to initialization our
// initial revocation tree state. In order to do so in a deterministic
// manner (for recovery purposes), we'll use the current block hash
// along with the identity public key of the node we're creating the
// channel with. In the event of a recovery, given these two items and
// the initialize wallet HD seed, we can derive all of our revocation
// secrets.
masterElkremRoot, err := l.deriveMasterRevocationRoot()
if err != nil {
req.err <- err
req.resp <- nil
return
}
deliveryScript, err := txscript.PayToAddrScript(deliveryAddress)
bestHash, _, err := l.Cfg.ChainIO.GetBestBlock()
if err != nil {
req.err <- err
req.resp <- nil
return
}
reservation.partialState.OurDeliveryScript = deliveryScript
ourContribution.DeliveryAddress = deliveryAddress
revocationRoot := DeriveRevocationRoot(masterElkremRoot, *bestHash,
req.nodeID)
// Once we have the root, we can then generate our shachain producer
// and from that generate the per-commitment point.
producer := shachain.NewRevocationProducer(revocationRoot)
firstPreimage, err := producer.AtIndex(0)
if err != nil {
req.err <- err
req.resp <- nil
return
}
reservation.ourContribution.FirstCommitmentPoint = ComputeCommitmentPoint(
firstPreimage[:],
)
reservation.partialState.RevocationProducer = producer
reservation.ourContribution.ChannelConstraints = l.Cfg.DefaultConstraints
// TODO(roasbeef): turn above into: initContributio()
// Create a limbo and record entry for this newly pending funding
// request.
@ -692,6 +705,7 @@ func CreateCommitmentTxns(localBalance, remoteBalance btcutil.Amount,
// both versions of the commitment transaction, and our signature for their
// version of the commitment transaction.
func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
l.limboMtx.Lock()
pendingReservation, ok := l.fundingLimbo[req.pendingFundingID]
l.limboMtx.Unlock()
@ -728,7 +742,7 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
fundingTx.AddTxOut(theirChangeOutput)
}
ourKey := pendingReservation.partialState.OurMultiSigKey
ourKey := pendingReservation.ourContribution.MultiSigKey
theirKey := theirContribution.MultiSigKey
// Finally, add the 2-of-2 multi-sig output which will set up the lightning
@ -740,17 +754,17 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
req.err <- err
return
}
pendingReservation.partialState.FundingWitnessScript = witnessScript
// Sort the transaction. Since both side agree to a canonical
// ordering, by sorting we no longer need to send the entire
// transaction. Only signatures will be exchanged.
// Sort the transaction. Since both side agree to a canonical ordering,
// by sorting we no longer need to send the entire transaction. Only
// signatures will be exchanged.
fundingTx.AddTxOut(multiSigOut)
txsort.InPlaceSort(pendingReservation.fundingTx)
// Next, sign all inputs that are ours, collecting the signatures in
// order of the inputs.
pendingReservation.ourFundingInputScripts = make([]*InputScript, 0, len(ourContribution.Inputs))
pendingReservation.ourFundingInputScripts = make([]*InputScript, 0,
len(ourContribution.Inputs))
signDesc := SignDescriptor{
HashType: txscript.SigHashAll,
SigHashes: txscript.NewTxSigHashes(fundingTx),
@ -788,71 +802,66 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
fundingTxID := fundingTx.TxHash()
_, multiSigIndex := FindScriptOutputIndex(fundingTx, multiSigOut.PkScript)
fundingOutpoint := wire.NewOutPoint(&fundingTxID, multiSigIndex)
pendingReservation.partialState.FundingOutpoint = fundingOutpoint
pendingReservation.partialState.FundingOutpoint = *fundingOutpoint
// 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
// to the chain).
s := shachain.NewRevocationStore()
pendingReservation.partialState.RevocationStore = s
pendingReservation.partialState.TheirCurrentRevocation = theirContribution.RevocationKey
masterElkremRoot, err := l.deriveMasterRevocationRoot()
if err != nil {
req.err <- err
return
}
// Store their current commitment point. We'll need this after the
// first state transition in order to verify the authenticity of the
// revocation.
chanState := pendingReservation.partialState
chanState.RemoteCurrentRevocation = theirContribution.FirstCommitmentPoint
// Now that we have their commitment key, we can create the revocation
// key for the first version of our commitment transaction. To do so,
// we'll first create our root, then produce the first pre-image.
root := DeriveRevocationRoot(masterElkremRoot, ourKey, theirKey)
producer := shachain.NewRevocationProducer(*root)
pendingReservation.partialState.RevocationProducer = producer
firstPreimage, err := producer.AtIndex(0)
if err != nil {
req.err <- err
return
}
theirCommitKey := theirContribution.CommitKey
ourRevokeKey := DeriveRevocationPubkey(theirCommitKey, firstPreimage[:])
// Create the txIn to our commitment transaction; required to construct
// Create the txin to our commitment transaction; required to construct
// the commitment transactions.
fundingTxIn := wire.NewTxIn(wire.NewOutPoint(&fundingTxID, multiSigIndex), nil, nil)
fundingTxIn := &wire.TxIn{
PreviousOutPoint: wire.OutPoint{
Hash: fundingTxID,
Index: multiSigIndex,
},
}
// With the funding tx complete, create both commitment transactions.
// TODO(roasbeef): much cleanup + de-duplication
ourBalance := pendingReservation.partialState.OurBalance
theirBalance := pendingReservation.partialState.TheirBalance
ourCommitKey := ourContribution.CommitKey
ourCommitTx, err := CreateCommitTx(fundingTxIn, ourCommitKey,
theirCommitKey, ourRevokeKey, ourContribution.CsvDelay,
ourBalance, theirBalance,
pendingReservation.ourContribution.DustLimit)
if err != nil {
req.err <- err
return
}
theirCommitTx, err := CreateCommitTx(fundingTxIn, theirCommitKey,
ourCommitKey, theirContribution.RevocationKey,
theirContribution.CsvDelay, theirBalance, ourBalance,
pendingReservation.theirContribution.DustLimit)
localBalance := pendingReservation.partialState.LocalBalance
remoteBalance := pendingReservation.partialState.RemoteBalance
ourCommitTx, theirCommitTx, err := CreateCommitmentTxns(
localBalance, remoteBalance, ourContribution.ChannelConfig,
theirContribution.ChannelConfig,
ourContribution.FirstCommitmentPoint,
theirContribution.FirstCommitmentPoint, fundingTxIn,
)
if err != nil {
req.err <- err
return
}
// With both commitment transactions constructed, generate the state
// obsfucator then use it to encode the current state number withi both
// commitment transactions.
// TODO(roasbeef): define obsfucator scheme for dual funder
// obsfucator then use it to encode the current state number within
// both commitment transactions.
var stateObsfucator [StateHintSize]byte
if pendingReservation.partialState.IsInitiator {
stateObsfucator, err = deriveStateHintObfuscator(producer)
if err != nil {
req.err <- err
return
if chanState.ChanType == channeldb.SingleFunder {
stateObsfucator = deriveStateHintObfuscator(
ourContribution.PaymentBasePoint,
theirContribution.PaymentBasePoint,
)
} else {
ourSer := ourContribution.PaymentBasePoint.SerializeCompressed()
theirSer := theirContribution.PaymentBasePoint.SerializeCompressed()
switch bytes.Compare(ourSer, theirSer) {
case -1:
stateObsfucator = deriveStateHintObfuscator(
ourContribution.PaymentBasePoint,
theirContribution.PaymentBasePoint,
)
default:
stateObsfucator = deriveStateHintObfuscator(
theirContribution.PaymentBasePoint,
ourContribution.PaymentBasePoint,
)
}
}
err = initStateHints(ourCommitTx, theirCommitTx, stateObsfucator)
@ -861,27 +870,15 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
return
}
// Sort both transactions according to the agreed upon cannonical
// Sort both transactions according to the agreed upon canonical
// ordering. This lets us skip sending the entire transaction over,
// instead we'll just send signatures.
txsort.InPlaceSort(ourCommitTx)
txsort.InPlaceSort(theirCommitTx)
deliveryScript, err := txscript.PayToAddrScript(theirContribution.DeliveryAddress)
if err != nil {
req.err <- err
return
}
// Record newly available information witin the open channel state.
pendingReservation.partialState.RemoteCsvDelay = theirContribution.CsvDelay
pendingReservation.partialState.TheirDeliveryScript = deliveryScript
pendingReservation.partialState.ChanID = fundingOutpoint
pendingReservation.partialState.TheirCommitKey = theirCommitKey
pendingReservation.partialState.TheirMultiSigKey = theirContribution.MultiSigKey
pendingReservation.partialState.OurCommitTx = ourCommitTx
pendingReservation.partialState.StateHintObsfucator = stateObsfucator
pendingReservation.ourContribution.RevocationKey = ourRevokeKey
// Record newly available information within the open channel state.
chanState.FundingOutpoint = *fundingOutpoint
chanState.CommitTx = *ourCommitTx
// Generate a signature for their version of the initial commitment
// transaction.
@ -912,70 +909,33 @@ func (l *LightningWallet) handleSingleContribution(req *addSingleContributionMsg
pendingReservation, ok := l.fundingLimbo[req.pendingFundingID]
l.limboMtx.Unlock()
if !ok {
req.err <- fmt.Errorf("attempted to update non-existant funding state")
req.err <- fmt.Errorf("attempted to update non-existent funding state")
return
}
// Grab the mutex on the ChannelReservation to ensure thead-safety
// Grab the mutex on the channelReservation to ensure thread-safety.
pendingReservation.Lock()
defer pendingReservation.Unlock()
// TODO(roasbeef): verify sanity of remote party's parameters, fail if
// disagree
// Simply record the counterparty's contribution into the pending
// reservation data as they'll be solely funding the channel entirely.
pendingReservation.theirContribution = req.contribution
theirContribution := pendingReservation.theirContribution
// Additionally, we can now also record the redeem script of the
// funding transaction.
// TODO(roasbeef): switch to proper pubkey derivation
ourKey := pendingReservation.partialState.OurMultiSigKey
theirKey := theirContribution.MultiSigKey
channelCapacity := int64(pendingReservation.partialState.Capacity)
witnessScript, _, err := GenFundingPkScript(ourKey.SerializeCompressed(),
theirKey.SerializeCompressed(), channelCapacity)
if err != nil {
req.err <- err
return
}
pendingReservation.partialState.FundingWitnessScript = witnessScript
masterElkremRoot, err := l.deriveMasterRevocationRoot()
if err != nil {
req.err <- err
return
}
// Now that we know their commitment key, we can create the revocation
// key for our version of the initial commitment transaction.
root := DeriveRevocationRoot(masterElkremRoot, ourKey, theirKey)
producer := shachain.NewRevocationProducer(*root)
firstPreimage, err := producer.AtIndex(0)
if err != nil {
req.err <- err
return
}
pendingReservation.partialState.RevocationProducer = producer
theirCommitKey := theirContribution.CommitKey
ourRevokeKey := DeriveRevocationPubkey(theirCommitKey, firstPreimage[:])
chanState := pendingReservation.partialState
// 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
// to the chain).
remotePreimageStore := shachain.NewRevocationStore()
pendingReservation.partialState.RevocationStore = remotePreimageStore
chanState.RevocationStore = remotePreimageStore
// Record the counterpaty's remaining contributions to the channel,
// converting their delivery address into a public key script.
deliveryScript, err := txscript.PayToAddrScript(theirContribution.DeliveryAddress)
if err != nil {
req.err <- err
return
}
pendingReservation.partialState.RemoteCsvDelay = theirContribution.CsvDelay
pendingReservation.partialState.TheirDeliveryScript = deliveryScript
pendingReservation.partialState.TheirCommitKey = theirContribution.CommitKey
pendingReservation.partialState.TheirMultiSigKey = theirContribution.MultiSigKey
pendingReservation.ourContribution.RevocationKey = ourRevokeKey
// Now that we've received their first commitment point, we'll store it
// within the channel state so we can sync it to disk once the funding
// process is complete.
chanState.RemoteCurrentRevocation = theirContribution.FirstCommitmentPoint
req.err <- nil
return
@ -1004,7 +964,7 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
res, ok := l.fundingLimbo[msg.pendingFundingID]
l.limboMtx.RUnlock()
if !ok {
msg.err <- fmt.Errorf("attempted to update non-existant funding state")
msg.err <- fmt.Errorf("attempted to update non-existent funding state")
return
}
@ -1060,21 +1020,28 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
// At this point, we can also record and verify their signature for our
// commitment transaction.
res.theirCommitmentSig = msg.theirCommitmentSig
commitTx := res.partialState.OurCommitTx
commitTx := res.partialState.CommitTx
ourKey := res.ourContribution.MultiSigKey
theirKey := res.theirContribution.MultiSigKey
// Re-generate both the witnessScript and p2sh output. We sign the
// witnessScript script, but include the p2sh output as the subscript
// for verification.
witnessScript := res.partialState.FundingWitnessScript
witnessScript, _, err := GenFundingPkScript(ourKey.SerializeCompressed(),
theirKey.SerializeCompressed(), int64(res.partialState.Capacity))
if err != nil {
msg.err <- err
msg.completeChan <- nil
return
}
// Next, create the spending scriptSig, and then verify that the script
// is complete, allowing us to spend from the funding transaction.
theirCommitSig := msg.theirCommitmentSig
channelValue := int64(res.partialState.Capacity)
hashCache := txscript.NewTxSigHashes(commitTx)
hashCache := txscript.NewTxSigHashes(&commitTx)
sigHash, err := txscript.CalcWitnessSigHash(witnessScript, hashCache,
txscript.SigHashAll, commitTx, 0, channelValue)
txscript.SigHashAll, &commitTx, 0, channelValue)
if err != nil {
msg.err <- fmt.Errorf("counterparty's commitment signature is "+
"invalid: %v", err)
@ -1094,7 +1061,7 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
msg.completeChan <- nil
return
}
res.partialState.OurCommitSig = theirCommitSig
res.partialState.CommitSig = theirCommitSig
// Funding complete, this entry can be removed from limbo.
l.limboMtx.Lock()
@ -1113,19 +1080,16 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
return
}
walletLog.Infof("Broadcasting funding tx for ChannelPoint(%v): %v",
res.partialState.FundingOutpoint, spew.Sdump(fundingTx))
// Broacast the finalized funding transaction to the network.
if err := l.PublishTransaction(fundingTx); err != nil {
msg.err <- err
msg.completeChan <- nil
return
}
// As we've completed the funding process, we'll no convert the
// contribution structs into their underlying channel config objects to
// he stored within the database.
res.partialState.LocalChanCfg = res.ourContribution.toChanConfig()
res.partialState.RemoteChanCfg = res.theirContribution.toChanConfig()
// Add the complete funding transaction to the DB, in it's open bucket
// which will be used for the lifetime of this channel.
// TODO(roasbeef): revisit faul-tolerance of this flow
// TODO(roasbeef):
// * attempt to retransmit funding transactions on re-start
nodeAddr := res.nodeAddr
err = res.partialState.SyncPending(nodeAddr, uint32(bestHeight))
if err != nil {
@ -1134,6 +1098,19 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
return
}
walletLog.Infof("Broadcasting funding tx for ChannelPoint(%v): %v",
res.partialState.FundingOutpoint, spew.Sdump(fundingTx))
// Broadcast the finalized funding transaction to the network.
if err := l.PublishTransaction(fundingTx); err != nil {
// TODO(roasbeef): need to make this into a concrete error
if !strings.Contains(err.Error(), "already have") {
msg.err <- err
msg.completeChan <- nil
return
}
}
msg.completeChan <- res.partialState
msg.err <- nil
}
@ -1157,60 +1134,55 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
pendingReservation.Lock()
defer pendingReservation.Unlock()
pendingReservation.partialState.FundingOutpoint = req.fundingOutpoint
pendingReservation.partialState.TheirCurrentRevocation = req.revokeKey
pendingReservation.partialState.ChanID = req.fundingOutpoint
pendingReservation.partialState.StateHintObsfucator = req.obsfucator
chanState := pendingReservation.partialState
chanState.FundingOutpoint = *req.fundingOutpoint
fundingTxIn := wire.NewTxIn(req.fundingOutpoint, nil, nil)
// Now that we have the funding outpoint, we can generate both versions
// of the commitment transaction, and generate a signature for the
// remote node's commitment transactions.
ourCommitKey := pendingReservation.ourContribution.CommitKey
theirCommitKey := pendingReservation.theirContribution.CommitKey
ourBalance := pendingReservation.partialState.OurBalance
theirBalance := pendingReservation.partialState.TheirBalance
ourCommitTx, err := CreateCommitTx(fundingTxIn, ourCommitKey,
theirCommitKey, pendingReservation.ourContribution.RevocationKey,
pendingReservation.ourContribution.CsvDelay, ourBalance,
theirBalance, pendingReservation.ourContribution.DustLimit)
if err != nil {
req.err <- err
req.completeChan <- nil
return
}
theirCommitTx, err := CreateCommitTx(fundingTxIn, theirCommitKey,
ourCommitKey, req.revokeKey,
pendingReservation.theirContribution.CsvDelay, theirBalance,
ourBalance, pendingReservation.theirContribution.DustLimit)
if err != nil {
req.err <- err
req.completeChan <- nil
return
}
localBalance := pendingReservation.partialState.LocalBalance
remoteBalance := pendingReservation.partialState.RemoteBalance
ourCommitTx, theirCommitTx, err := CreateCommitmentTxns(
localBalance, remoteBalance,
pendingReservation.ourContribution.ChannelConfig,
pendingReservation.theirContribution.ChannelConfig,
pendingReservation.ourContribution.FirstCommitmentPoint,
pendingReservation.theirContribution.FirstCommitmentPoint,
fundingTxIn,
)
// With both commitment transactions constructed, generate the state
// obsfucator then use it to encode the current state number within
// With both commitment transactions constructed, we can now use the
// generator state obfuscator to encode the current state number within
// both commitment transactions.
err = initStateHints(ourCommitTx, theirCommitTx, req.obsfucator)
stateObsfucator := deriveStateHintObfuscator(
pendingReservation.theirContribution.PaymentBasePoint,
pendingReservation.ourContribution.PaymentBasePoint)
err = initStateHints(ourCommitTx, theirCommitTx, stateObsfucator)
if err != nil {
req.err <- err
req.completeChan <- nil
return
}
// Sort both transactions according to the agreed upon cannonical
// Sort both transactions according to the agreed upon canonical
// ordering. This ensures that both parties sign the same sighash
// without further synchronization.
txsort.InPlaceSort(ourCommitTx)
pendingReservation.partialState.OurCommitTx = ourCommitTx
txsort.InPlaceSort(theirCommitTx)
chanState.CommitTx = *ourCommitTx
witnessScript := pendingReservation.partialState.FundingWitnessScript
channelValue := int64(pendingReservation.partialState.Capacity)
hashCache := txscript.NewTxSigHashes(ourCommitTx)
theirKey := pendingReservation.theirContribution.MultiSigKey
ourKey := pendingReservation.partialState.OurMultiSigKey
ourKey := pendingReservation.ourContribution.MultiSigKey
witnessScript, _, err := GenFundingPkScript(ourKey.SerializeCompressed(),
theirKey.SerializeCompressed(), channelValue)
if err != nil {
req.err <- err
req.completeChan <- nil
return
}
sigHash, err := txscript.CalcWitnessSigHash(witnessScript, hashCache,
txscript.SigHashAll, ourCommitTx, 0, channelValue)
@ -1232,7 +1204,7 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
req.completeChan <- nil
return
}
pendingReservation.partialState.OurCommitSig = req.theirCommitmentSig
chanState.CommitSig = req.theirCommitmentSig
// With their signature for our version of the commitment transactions
// verified, we can now generate a signature for their version,
@ -1254,7 +1226,7 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
SigHashes: txscript.NewTxSigHashes(theirCommitTx),
InputIndex: 0,
}
sigTheirCommit, err := l.Signer.SignOutputRaw(theirCommitTx, &signDesc)
sigTheirCommit, err := l.Cfg.Signer.SignOutputRaw(theirCommitTx, &signDesc)
if err != nil {
req.err <- err
req.completeChan <- nil
@ -1262,7 +1234,7 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
}
pendingReservation.ourCommitmentSig = sigTheirCommit
_, bestHeight, err := l.ChainIO.GetBestBlock()
_, bestHeight, err := l.Cfg.ChainIO.GetBestBlock()
if err != nil {
req.err <- err
req.completeChan <- nil
@ -1271,17 +1243,16 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
// Add the complete funding transaction to the DB, in it's open bucket
// which will be used for the lifetime of this channel.
err = pendingReservation.partialState.SyncPending(
pendingReservation.nodeAddr,
uint32(bestHeight),
)
chanState.LocalChanCfg = pendingReservation.ourContribution.toChanConfig()
chanState.RemoteChanCfg = pendingReservation.theirContribution.toChanConfig()
err = chanState.SyncPending(pendingReservation.nodeAddr, uint32(bestHeight))
if err != nil {
req.err <- err
req.completeChan <- nil
return
}
req.completeChan <- pendingReservation.partialState
req.completeChan <- chanState
req.err <- nil
l.limboMtx.Lock()