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 package lnwallet
import ( import (
"bytes"
"crypto/sha256" "crypto/sha256"
"fmt" "fmt"
"net" "net"
"strings"
"sync" "sync"
"sync/atomic" "sync/atomic"
"github.com/davecgh/go-spew/spew" "github.com/davecgh/go-spew/spew"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb" "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/roasbeef/btcutil/hdkeychain"
"github.com/lightningnetwork/lnd/shachain" "github.com/lightningnetwork/lnd/shachain"
@ -65,15 +67,19 @@ func (e *ErrInsufficientFunds) Error() string {
// initFundingReserveReq is the first message sent to initiate the workflow // initFundingReserveReq is the first message sent to initiate the workflow
// required to open a payment channel with a remote peer. The initial required // required to open a payment channel with a remote peer. The initial required
// parameters are configurable across channels. These parameters are to be // 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 // chosen depending on the fee climate within the network, and time value of
// be locked up within the channel. Upon success a ChannelReservation will be // funds to be locked up within the channel. Upon success a ChannelReservation
// created in order to track the lifetime of this pending channel. Outputs // will be created in order to track the lifetime of this pending channel.
// selected will be 'locked', making them unavailable, for any other pending // Outputs selected will be 'locked', making them unavailable, for any other
// reservations. Therefore, all channels in reservation limbo will be periodically // pending reservations. Therefore, all channels in reservation limbo will be
// after a timeout period in order to avoid "exhaustion" attacks. // periodically after a timeout period in order to avoid "exhaustion" attacks.
// //
// TODO(roasbeef): zombie reservation sweeper goroutine. // TODO(roasbeef): zombie reservation sweeper goroutine.
type initFundingReserveMsg struct { 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 // nodeId is the ID of the remote node we would like to open a channel
// with. // with.
nodeID *btcec.PublicKey nodeID *btcec.PublicKey
@ -83,10 +89,6 @@ type initFundingReserveMsg struct {
// this funding workflow. // this funding workflow.
nodeAddr *net.TCPAddr 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 is the amount of funds requested for this channel.
fundingAmount btcutil.Amount fundingAmount btcutil.Amount
@ -100,19 +102,10 @@ type initFundingReserveMsg struct {
// of the accepted base fee rate of the network. // of the accepted base fee rate of the network.
feePerKw btcutil.Amount feePerKw btcutil.Amount
// ourDustLimit is the threshold below which no HTLC output should be // pushSat is the number of satoshis that should be pushed over the
// 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
// responder as part of the initial channel creation. // responder as part of the initial channel creation.
pushSat btcutil.Amount 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 // 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.
// //
@ -210,19 +203,10 @@ type addSingleFunderSigsMsg struct {
// transaction as assembled by the workflow initiator. // transaction as assembled by the workflow initiator.
fundingOutpoint *wire.OutPoint 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 // theirCommitmentSig are the 1/2 of the signatures needed to
// succesfully spend our version of the commitment transaction. // succesfully spend our version of the commitment transaction.
theirCommitmentSig []byte 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 // This channel is used to return the completed channel after the wallet
// has completed all of its stages in the funding process. // has completed all of its stages in the funding process.
completeChan chan *channeldb.OpenChannel completeChan chan *channeldb.OpenChannel
@ -459,28 +443,22 @@ out:
// and final step verifies all signatures for the inputs of the funding // and final step verifies all signatures for the inputs of the funding
// transaction, and that the signature we records for our version of the // transaction, and that the signature we records for our version of the
// commitment transaction is valid. // commitment transaction is valid.
func (l *LightningWallet) InitChannelReservation(capacity, func (l *LightningWallet) InitChannelReservation(
ourFundAmt btcutil.Amount, theirID *btcec.PublicKey, capacity, ourFundAmt, pushSat, feePerKw btcutil.Amount,
theirAddr *net.TCPAddr, numConfs uint16, csvDelay uint32, theirID *btcec.PublicKey, theirAddr *net.TCPAddr,
ourDustLimit btcutil.Amount, pushSat btcutil.Amount, chainHash *chainhash.Hash) (*ChannelReservation, error) {
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
errChan := make(chan error, 1) errChan := make(chan error, 1)
respChan := make(chan *ChannelReservation, 1) respChan := make(chan *ChannelReservation, 1)
l.msgChan <- &initFundingReserveMsg{ l.msgChan <- &initFundingReserveMsg{
capacity: capacity, chainHash: chainHash,
numConfs: numConfs,
fundingAmount: ourFundAmt,
csvDelay: csvDelay,
ourDustLimit: ourDustLimit,
feePerKw: feePerKw,
pushSat: pushSat,
nodeID: theirID, nodeID: theirID,
nodeAddr: theirAddr, nodeAddr: theirAddr,
fundingAmount: ourFundAmt,
capacity: capacity,
feePerKw: feePerKw,
pushSat: pushSat,
err: errChan, err: errChan,
resp: respChan, resp: respChan,
} }
@ -499,24 +477,26 @@ func (l *LightningWallet) handleFundingReserveRequest(req *initFundingReserveMsg
return 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) id := atomic.AddUint64(&l.nextFundingID, 1)
reservation := NewChannelReservation(req.capacity, req.fundingAmount, 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 // Grab the mutex on the ChannelReservation to ensure thread-safety
reservation.Lock() reservation.Lock()
defer reservation.Unlock() defer reservation.Unlock()
reservation.nodeAddr = req.nodeAddr 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.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 // 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 // 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. // funding transaction gets into the _next_ block.
// //
// TODO(roasbeef): shouldn't be targeting 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, err := l.selectCoinsAndChange(satPerByte, req.fundingAmount,
ourContribution) reservation.ourContribution)
if err != nil { if err != nil {
req.err <- err req.err <- err
req.resp <- nil 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 // Next, we'll grab a series of keys from the wallet which will be used
// multi-sig funding transaction, and the other for the commitment // for the duration of the channel. The keys include: our multi-sig
// transaction. // key, the base revocation key, the base payment key, and the delayed
multiSigKey, err := l.NewRawKey() // payment key.
var err error
reservation.ourContribution.MultiSigKey, err = l.NewRawKey()
if err != nil { if err != nil {
req.err <- err req.err <- err
req.resp <- nil req.resp <- nil
return 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 { if err != nil {
req.err <- err req.err <- err
req.resp <- nil req.resp <- nil
return 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 // With the above keys created, we'll also need to initialization our
// channel close. // initial revocation tree state. In order to do so in a deterministic
deliveryAddress, err := l.NewAddress(WitnessPubKey, false) // 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 { if err != nil {
req.err <- err req.err <- err
req.resp <- nil req.resp <- nil
return return
} }
deliveryScript, err := txscript.PayToAddrScript(deliveryAddress) bestHash, _, err := l.Cfg.ChainIO.GetBestBlock()
if err != nil { if err != nil {
req.err <- err req.err <- err
req.resp <- nil req.resp <- nil
return return
} }
reservation.partialState.OurDeliveryScript = deliveryScript revocationRoot := DeriveRevocationRoot(masterElkremRoot, *bestHash,
ourContribution.DeliveryAddress = deliveryAddress 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 // Create a limbo and record entry for this newly pending funding
// request. // request.
@ -692,6 +705,7 @@ func CreateCommitmentTxns(localBalance, remoteBalance btcutil.Amount,
// both versions of the commitment transaction, and our signature for their // both versions of the commitment transaction, and our signature for their
// version of the commitment transaction. // version of the commitment transaction.
func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) { func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
l.limboMtx.Lock() l.limboMtx.Lock()
pendingReservation, ok := l.fundingLimbo[req.pendingFundingID] pendingReservation, ok := l.fundingLimbo[req.pendingFundingID]
l.limboMtx.Unlock() l.limboMtx.Unlock()
@ -728,7 +742,7 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
fundingTx.AddTxOut(theirChangeOutput) fundingTx.AddTxOut(theirChangeOutput)
} }
ourKey := pendingReservation.partialState.OurMultiSigKey ourKey := pendingReservation.ourContribution.MultiSigKey
theirKey := theirContribution.MultiSigKey theirKey := theirContribution.MultiSigKey
// Finally, add the 2-of-2 multi-sig output which will set up the lightning // 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 req.err <- err
return return
} }
pendingReservation.partialState.FundingWitnessScript = witnessScript
// Sort the transaction. Since both side agree to a canonical // Sort the transaction. Since both side agree to a canonical ordering,
// ordering, by sorting we no longer need to send the entire // by sorting we no longer need to send the entire transaction. Only
// transaction. Only signatures will be exchanged. // signatures will be exchanged.
fundingTx.AddTxOut(multiSigOut) fundingTx.AddTxOut(multiSigOut)
txsort.InPlaceSort(pendingReservation.fundingTx) txsort.InPlaceSort(pendingReservation.fundingTx)
// Next, sign all inputs that are ours, collecting the signatures in // Next, sign all inputs that are ours, collecting the signatures in
// order of the inputs. // order of the inputs.
pendingReservation.ourFundingInputScripts = make([]*InputScript, 0, len(ourContribution.Inputs)) pendingReservation.ourFundingInputScripts = make([]*InputScript, 0,
len(ourContribution.Inputs))
signDesc := SignDescriptor{ signDesc := SignDescriptor{
HashType: txscript.SigHashAll, HashType: txscript.SigHashAll,
SigHashes: txscript.NewTxSigHashes(fundingTx), SigHashes: txscript.NewTxSigHashes(fundingTx),
@ -788,71 +802,66 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
fundingTxID := fundingTx.TxHash() fundingTxID := fundingTx.TxHash()
_, multiSigIndex := FindScriptOutputIndex(fundingTx, multiSigOut.PkScript) _, multiSigIndex := FindScriptOutputIndex(fundingTx, multiSigOut.PkScript)
fundingOutpoint := wire.NewOutPoint(&fundingTxID, multiSigIndex) fundingOutpoint := wire.NewOutPoint(&fundingTxID, multiSigIndex)
pendingReservation.partialState.FundingOutpoint = fundingOutpoint pendingReservation.partialState.FundingOutpoint = *fundingOutpoint
// 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).
s := shachain.NewRevocationStore() s := shachain.NewRevocationStore()
pendingReservation.partialState.RevocationStore = s pendingReservation.partialState.RevocationStore = s
pendingReservation.partialState.TheirCurrentRevocation = theirContribution.RevocationKey
masterElkremRoot, err := l.deriveMasterRevocationRoot() // Store their current commitment point. We'll need this after the
if err != nil { // first state transition in order to verify the authenticity of the
req.err <- err // revocation.
return chanState := pendingReservation.partialState
} chanState.RemoteCurrentRevocation = theirContribution.FirstCommitmentPoint
// Now that we have their commitment key, we can create the revocation // Create the txin to our commitment transaction; required to construct
// 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
// the commitment transactions. // 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. // With the funding tx complete, create both commitment transactions.
// TODO(roasbeef): much cleanup + de-duplication localBalance := pendingReservation.partialState.LocalBalance
ourBalance := pendingReservation.partialState.OurBalance remoteBalance := pendingReservation.partialState.RemoteBalance
theirBalance := pendingReservation.partialState.TheirBalance ourCommitTx, theirCommitTx, err := CreateCommitmentTxns(
ourCommitKey := ourContribution.CommitKey localBalance, remoteBalance, ourContribution.ChannelConfig,
ourCommitTx, err := CreateCommitTx(fundingTxIn, ourCommitKey, theirContribution.ChannelConfig,
theirCommitKey, ourRevokeKey, ourContribution.CsvDelay, ourContribution.FirstCommitmentPoint,
ourBalance, theirBalance, theirContribution.FirstCommitmentPoint, fundingTxIn,
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)
if err != nil { if err != nil {
req.err <- err req.err <- err
return return
} }
// With both commitment transactions constructed, generate the state // With both commitment transactions constructed, generate the state
// obsfucator then use it to encode the current state number withi both // obsfucator then use it to encode the current state number within
// commitment transactions. // both commitment transactions.
// TODO(roasbeef): define obsfucator scheme for dual funder
var stateObsfucator [StateHintSize]byte var stateObsfucator [StateHintSize]byte
if pendingReservation.partialState.IsInitiator { if chanState.ChanType == channeldb.SingleFunder {
stateObsfucator, err = deriveStateHintObfuscator(producer) stateObsfucator = deriveStateHintObfuscator(
if err != nil { ourContribution.PaymentBasePoint,
req.err <- err theirContribution.PaymentBasePoint,
return )
} 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) err = initStateHints(ourCommitTx, theirCommitTx, stateObsfucator)
@ -861,27 +870,15 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
return 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, // ordering. This lets us skip sending the entire transaction over,
// instead we'll just send signatures. // instead we'll just send signatures.
txsort.InPlaceSort(ourCommitTx) txsort.InPlaceSort(ourCommitTx)
txsort.InPlaceSort(theirCommitTx) txsort.InPlaceSort(theirCommitTx)
deliveryScript, err := txscript.PayToAddrScript(theirContribution.DeliveryAddress) // Record newly available information within the open channel state.
if err != nil { chanState.FundingOutpoint = *fundingOutpoint
req.err <- err chanState.CommitTx = *ourCommitTx
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
// Generate a signature for their version of the initial commitment // Generate a signature for their version of the initial commitment
// transaction. // transaction.
@ -912,70 +909,33 @@ func (l *LightningWallet) handleSingleContribution(req *addSingleContributionMsg
pendingReservation, ok := l.fundingLimbo[req.pendingFundingID] pendingReservation, ok := l.fundingLimbo[req.pendingFundingID]
l.limboMtx.Unlock() l.limboMtx.Unlock()
if !ok { 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 return
} }
// Grab the mutex on the ChannelReservation to ensure thead-safety // Grab the mutex on the channelReservation to ensure thread-safety.
pendingReservation.Lock() pendingReservation.Lock()
defer pendingReservation.Unlock() defer pendingReservation.Unlock()
// TODO(roasbeef): verify sanity of remote party's parameters, fail if
// disagree
// Simply record the counterparty's contribution into the pending // Simply record the counterparty's contribution into the pending
// reservation data as they'll be solely funding the channel entirely. // reservation data as they'll be solely funding the channel entirely.
pendingReservation.theirContribution = req.contribution pendingReservation.theirContribution = req.contribution
theirContribution := pendingReservation.theirContribution theirContribution := pendingReservation.theirContribution
chanState := pendingReservation.partialState
// 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[:])
// 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).
remotePreimageStore := shachain.NewRevocationStore() remotePreimageStore := shachain.NewRevocationStore()
pendingReservation.partialState.RevocationStore = remotePreimageStore chanState.RevocationStore = remotePreimageStore
// Record the counterpaty's remaining contributions to the channel, // Now that we've received their first commitment point, we'll store it
// converting their delivery address into a public key script. // within the channel state so we can sync it to disk once the funding
deliveryScript, err := txscript.PayToAddrScript(theirContribution.DeliveryAddress) // process is complete.
if err != nil { chanState.RemoteCurrentRevocation = theirContribution.FirstCommitmentPoint
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
req.err <- nil req.err <- nil
return return
@ -1004,7 +964,7 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
res, ok := l.fundingLimbo[msg.pendingFundingID] res, ok := l.fundingLimbo[msg.pendingFundingID]
l.limboMtx.RUnlock() l.limboMtx.RUnlock()
if !ok { 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 return
} }
@ -1060,21 +1020,28 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
// At this point, we can also record and verify their signature for our // At this point, we can also record and verify their signature for our
// commitment transaction. // commitment transaction.
res.theirCommitmentSig = msg.theirCommitmentSig res.theirCommitmentSig = msg.theirCommitmentSig
commitTx := res.partialState.OurCommitTx commitTx := res.partialState.CommitTx
ourKey := res.ourContribution.MultiSigKey
theirKey := res.theirContribution.MultiSigKey theirKey := res.theirContribution.MultiSigKey
// Re-generate both the witnessScript and p2sh output. We sign the // Re-generate both the witnessScript and p2sh output. We sign the
// witnessScript script, but include the p2sh output as the subscript // witnessScript script, but include the p2sh output as the subscript
// for verification. // 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 // Next, create the spending scriptSig, and then verify that the script
// is complete, allowing us to spend from the funding transaction. // is complete, allowing us to spend from the funding transaction.
theirCommitSig := msg.theirCommitmentSig theirCommitSig := msg.theirCommitmentSig
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(witnessScript, hashCache,
txscript.SigHashAll, commitTx, 0, channelValue) txscript.SigHashAll, &commitTx, 0, channelValue)
if err != nil { if err != nil {
msg.err <- fmt.Errorf("counterparty's commitment signature is "+ msg.err <- fmt.Errorf("counterparty's commitment signature is "+
"invalid: %v", err) "invalid: %v", err)
@ -1094,7 +1061,7 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
msg.completeChan <- nil msg.completeChan <- nil
return return
} }
res.partialState.OurCommitSig = theirCommitSig res.partialState.CommitSig = theirCommitSig
// Funding complete, this entry can be removed from limbo. // Funding complete, this entry can be removed from limbo.
l.limboMtx.Lock() l.limboMtx.Lock()
@ -1113,19 +1080,16 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
return return
} }
walletLog.Infof("Broadcasting funding tx for ChannelPoint(%v): %v", // As we've completed the funding process, we'll no convert the
res.partialState.FundingOutpoint, spew.Sdump(fundingTx)) // contribution structs into their underlying channel config objects to
// he stored within the database.
// Broacast the finalized funding transaction to the network. res.partialState.LocalChanCfg = res.ourContribution.toChanConfig()
if err := l.PublishTransaction(fundingTx); err != nil { res.partialState.RemoteChanCfg = res.theirContribution.toChanConfig()
msg.err <- err
msg.completeChan <- nil
return
}
// Add the complete funding transaction to the DB, in it's open bucket // Add the complete funding transaction to the DB, in it's open bucket
// which will be used for the lifetime of this channel. // 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 nodeAddr := res.nodeAddr
err = res.partialState.SyncPending(nodeAddr, uint32(bestHeight)) err = res.partialState.SyncPending(nodeAddr, uint32(bestHeight))
if err != nil { if err != nil {
@ -1134,6 +1098,19 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs
return 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.completeChan <- res.partialState
msg.err <- nil msg.err <- nil
} }
@ -1157,60 +1134,55 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
pendingReservation.Lock() pendingReservation.Lock()
defer pendingReservation.Unlock() defer pendingReservation.Unlock()
pendingReservation.partialState.FundingOutpoint = req.fundingOutpoint chanState := pendingReservation.partialState
pendingReservation.partialState.TheirCurrentRevocation = req.revokeKey chanState.FundingOutpoint = *req.fundingOutpoint
pendingReservation.partialState.ChanID = req.fundingOutpoint
pendingReservation.partialState.StateHintObsfucator = req.obsfucator
fundingTxIn := wire.NewTxIn(req.fundingOutpoint, nil, nil) fundingTxIn := wire.NewTxIn(req.fundingOutpoint, nil, nil)
// Now that we have the funding outpoint, we can generate both versions // Now that we have the funding outpoint, we can generate both versions
// of the commitment transaction, and generate a signature for the // of the commitment transaction, and generate a signature for the
// remote node's commitment transactions. // remote node's commitment transactions.
ourCommitKey := pendingReservation.ourContribution.CommitKey localBalance := pendingReservation.partialState.LocalBalance
theirCommitKey := pendingReservation.theirContribution.CommitKey remoteBalance := pendingReservation.partialState.RemoteBalance
ourBalance := pendingReservation.partialState.OurBalance ourCommitTx, theirCommitTx, err := CreateCommitmentTxns(
theirBalance := pendingReservation.partialState.TheirBalance localBalance, remoteBalance,
ourCommitTx, err := CreateCommitTx(fundingTxIn, ourCommitKey, pendingReservation.ourContribution.ChannelConfig,
theirCommitKey, pendingReservation.ourContribution.RevocationKey, pendingReservation.theirContribution.ChannelConfig,
pendingReservation.ourContribution.CsvDelay, ourBalance, pendingReservation.ourContribution.FirstCommitmentPoint,
theirBalance, pendingReservation.ourContribution.DustLimit) pendingReservation.theirContribution.FirstCommitmentPoint,
if err != nil { fundingTxIn,
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
}
// With both commitment transactions constructed, generate the state // With both commitment transactions constructed, we can now use the
// obsfucator then use it to encode the current state number within // generator state obfuscator to encode the current state number within
// both commitment transactions. // 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 { if err != nil {
req.err <- err req.err <- err
req.completeChan <- nil req.completeChan <- nil
return 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 // ordering. This ensures that both parties sign the same sighash
// without further synchronization. // without further synchronization.
txsort.InPlaceSort(ourCommitTx) txsort.InPlaceSort(ourCommitTx)
pendingReservation.partialState.OurCommitTx = ourCommitTx
txsort.InPlaceSort(theirCommitTx) txsort.InPlaceSort(theirCommitTx)
chanState.CommitTx = *ourCommitTx
witnessScript := pendingReservation.partialState.FundingWitnessScript
channelValue := int64(pendingReservation.partialState.Capacity) channelValue := int64(pendingReservation.partialState.Capacity)
hashCache := txscript.NewTxSigHashes(ourCommitTx) hashCache := txscript.NewTxSigHashes(ourCommitTx)
theirKey := pendingReservation.theirContribution.MultiSigKey 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, sigHash, err := txscript.CalcWitnessSigHash(witnessScript, hashCache,
txscript.SigHashAll, ourCommitTx, 0, channelValue) txscript.SigHashAll, ourCommitTx, 0, channelValue)
@ -1232,7 +1204,7 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
req.completeChan <- nil req.completeChan <- nil
return return
} }
pendingReservation.partialState.OurCommitSig = req.theirCommitmentSig chanState.CommitSig = req.theirCommitmentSig
// With their signature for our version of the commitment transactions // With their signature for our version of the commitment transactions
// verified, we can now generate a signature for their version, // verified, we can now generate a signature for their version,
@ -1254,7 +1226,7 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
SigHashes: txscript.NewTxSigHashes(theirCommitTx), SigHashes: txscript.NewTxSigHashes(theirCommitTx),
InputIndex: 0, InputIndex: 0,
} }
sigTheirCommit, err := l.Signer.SignOutputRaw(theirCommitTx, &signDesc) sigTheirCommit, err := l.Cfg.Signer.SignOutputRaw(theirCommitTx, &signDesc)
if err != nil { if err != nil {
req.err <- err req.err <- err
req.completeChan <- nil req.completeChan <- nil
@ -1262,7 +1234,7 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) {
} }
pendingReservation.ourCommitmentSig = sigTheirCommit pendingReservation.ourCommitmentSig = sigTheirCommit
_, bestHeight, err := l.ChainIO.GetBestBlock() _, bestHeight, err := l.Cfg.ChainIO.GetBestBlock()
if err != nil { if err != nil {
req.err <- err req.err <- err
req.completeChan <- nil 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 // Add the complete funding transaction to the DB, in it's open bucket
// which will be used for the lifetime of this channel. // which will be used for the lifetime of this channel.
err = pendingReservation.partialState.SyncPending( chanState.LocalChanCfg = pendingReservation.ourContribution.toChanConfig()
pendingReservation.nodeAddr, chanState.RemoteChanCfg = pendingReservation.theirContribution.toChanConfig()
uint32(bestHeight), err = chanState.SyncPending(pendingReservation.nodeAddr, uint32(bestHeight))
)
if err != nil { if err != nil {
req.err <- err req.err <- err
req.completeChan <- nil req.completeChan <- nil
return return
} }
req.completeChan <- pendingReservation.partialState req.completeChan <- chanState
req.err <- nil req.err <- nil
l.limboMtx.Lock() l.limboMtx.Lock()