peer: handling for channel shutdown messages

This commit changes the cooperative channel close workflow to comply
with the latest spec. This adds steps to handle and send shutdown
messages as well as moving responsibility for sending the channel close
message from the initiator to the responder.
This commit is contained in:
bryanvu 2017-03-24 18:26:09 -07:00 committed by Olaoluwa Osuntokun
parent d7bb600c23
commit 514760f529
5 changed files with 336 additions and 215 deletions

View File

@ -901,7 +901,7 @@ const (
CloseBreach
)
// closeChanReq represents a request to close a particular channel specified by
// closeLinkReq represents a request to close a particular channel specified by
// its outpoint.
type closeLinkReq struct {
CloseType LinkCloseType

View File

@ -2522,81 +2522,17 @@ func (lc *LightningChannel) ForceClose() (*ForceCloseSummary, error) {
}, nil
}
// InitCooperativeClose initiates a cooperative closure of an active lightning
// channel. This method should only be executed once all pending HTLCs (if any)
// on the channel have been cleared/removed. Upon completion, the source
// channel will shift into the "closing" state, which indicates that all
// incoming/outgoing HTLC requests should be rejected. A signature for the
// closing transaction, and the txid of the closing transaction are returned.
// The initiator of the channel closure should then watch the blockchain for a
// confirmation of the closing transaction before considering the channel
// terminated. In the case of an unresponsive remote party, the initiator can
// either choose to execute a force closure, or backoff for a period of time,
// and retry the cooperative closure.
// CreateCloseProposal is used by both parties in a cooperative channel close
// workflow to generate proposed close transactions and signatures. This method
// should only be executed once all pending HTLCs (if any) on the channel have
// been cleared/removed. Upon completion, the source channel will shift into
// the "closing" state, which indicates that all incoming/outgoing HTLC
// requests should be rejected. A signature for the closing transaction is
// returned.
//
// TODO(roasbeef): caller should initiate signal to reject all incoming HTLCs,
// settle any inflight.
func (lc *LightningChannel) InitCooperativeClose() ([]byte, *chainhash.Hash, error) {
lc.Lock()
defer lc.Unlock()
// If we're already closing the channel, then ignore this request.
if lc.status == channelClosing || lc.status == channelClosed {
// TODO(roasbeef): check to ensure no pending payments
return nil, nil, ErrChanClosing
}
// Calculate the fee for the commitment transaction based on its size.
// For a cooperative close, there should be no HTLCs.
commitFee := (lc.channelState.FeePerKw * commitWeight) / 1000
if lc.channelState.IsInitiator {
lc.channelState.OurBalance = lc.channelState.OurBalance - commitFee
} else {
lc.channelState.TheirBalance = lc.channelState.TheirBalance - commitFee
}
closeTx := CreateCooperativeCloseTx(lc.fundingTxIn,
lc.channelState.OurDustLimit, lc.channelState.TheirDustLimit,
lc.channelState.OurBalance, lc.channelState.TheirBalance,
lc.channelState.OurDeliveryScript, lc.channelState.TheirDeliveryScript,
lc.channelState.IsInitiator)
// Ensure that the transaction doesn't explicitly violate any
// consensus rules such as being too big, or having any value with a
// negative output.
tx := btcutil.NewTx(closeTx)
if err := blockchain.CheckTransactionSanity(tx); err != nil {
return nil, nil, err
}
// Finally, sign the completed cooperative closure transaction. As the
// initiator we'll simply send our signature over to the remote party,
// using the generated txid to be notified once the closure transaction
// has been confirmed.
lc.signDesc.SigHashes = txscript.NewTxSigHashes(closeTx)
closeSig, err := lc.signer.SignOutputRaw(closeTx, lc.signDesc)
if err != nil {
return nil, nil, err
}
// As everything checks out, indicate in the channel status that a
// channel closure has been initiated.
lc.status = channelClosing
closeTxSha := closeTx.TxHash()
return closeSig, &closeTxSha, nil
}
// CompleteCooperativeClose completes the cooperative closure of the target
// active lightning channel. This method should be called in response to the
// remote node initiating a cooperative channel closure. A fully signed closure
// transaction is returned. It is the duty of the responding node to broadcast
// a signed+valid closure transaction to the network.
//
// NOTE: The passed remote sig is expected to be a fully complete signature
// including the proper sighash byte.
func (lc *LightningChannel) CompleteCooperativeClose(remoteSig []byte) (*wire.MsgTx, error) {
func (lc *LightningChannel) CreateCloseProposal(feeRate uint64) ([]byte, error) {
lc.Lock()
defer lc.Unlock()
@ -2608,7 +2544,6 @@ func (lc *LightningChannel) CompleteCooperativeClose(remoteSig []byte) (*wire.Ms
// Calculate the fee for the commitment transaction based on its size.
// For a cooperative close, there should be no HTLCs.
// TODO(roasbeef): should be using fee passed in
commitFee := (lc.channelState.FeePerKw * commitWeight) / 1000
if lc.channelState.IsInitiator {
@ -2617,13 +2552,63 @@ func (lc *LightningChannel) CompleteCooperativeClose(remoteSig []byte) (*wire.Ms
lc.channelState.TheirBalance = lc.channelState.TheirBalance - commitFee
}
closeTx := CreateCooperativeCloseTx(lc.fundingTxIn,
lc.channelState.OurDustLimit, lc.channelState.TheirDustLimit,
lc.channelState.OurBalance, lc.channelState.TheirBalance,
lc.channelState.OurDeliveryScript,
lc.channelState.TheirDeliveryScript,
lc.channelState.IsInitiator)
// Ensure that the transaction doesn't explicitly violate any
// consensus rules such as being too big, or having any value with a
// negative output.
tx := btcutil.NewTx(closeTx)
if err := blockchain.CheckTransactionSanity(tx); err != nil {
return nil, err
}
// Finally, sign the completed cooperative closure transaction. As the
// initiator we'll simply send our signature over to the remote party,
// using the generated txid to be notified once the closure transaction
// has been confirmed.
lc.signDesc.SigHashes = txscript.NewTxSigHashes(closeTx)
sig, err := lc.signer.SignOutputRaw(closeTx, lc.signDesc)
if err != nil {
return nil, err
}
// As everything checks out, indicate in the channel status that a
// channel closure has been initiated.
lc.status = channelClosing
return sig, nil
}
// CompleteCooperativeClose completes the cooperative closure of the target
// active lightning channel. A fully signed closure transaction as well as the
// signature itself are returned.
//
// NOTE: The passed local and remote sigs are expected to be fully complete
// signatures including the proper sighash byte.
func (lc *LightningChannel) CompleteCooperativeClose(localSig,
remoteSig []byte, proposedFee uint64) (*wire.MsgTx, error) {
lc.Lock()
defer lc.Unlock()
// If the channel is already closed, then ignore this request.
if lc.status == channelClosed {
// TODO(roasbeef): check to ensure no pending payments
return nil, ErrChanClosing
}
// Create the transaction used to return the current settled balance
// on this active channel back to both parties. In this current model,
// the initiator pays full fees for the cooperative close transaction.
closeTx := CreateCooperativeCloseTx(lc.fundingTxIn,
lc.channelState.OurDustLimit, lc.channelState.TheirDustLimit,
lc.channelState.OurBalance, lc.channelState.TheirBalance,
lc.channelState.OurDeliveryScript, lc.channelState.TheirDeliveryScript,
lc.channelState.OurDeliveryScript,
lc.channelState.TheirDeliveryScript,
lc.channelState.IsInitiator)
// Ensure that the transaction doesn't explicitly validate any
@ -2633,23 +2618,14 @@ func (lc *LightningChannel) CompleteCooperativeClose(remoteSig []byte) (*wire.Ms
if err := blockchain.CheckTransactionSanity(tx); err != nil {
return nil, err
}
// With the transaction created, we can finally generate our half of
// the 2-of-2 multi-sig needed to redeem the funding output.
hashCache := txscript.NewTxSigHashes(closeTx)
lc.signDesc.SigHashes = hashCache
closeSig, err := lc.signer.SignOutputRaw(closeTx, lc.signDesc)
if err != nil {
return nil, err
}
// Finally, construct the witness stack minding the order of the
// pubkeys+sigs on the stack.
ourKey := lc.channelState.OurMultiSigKey.SerializeCompressed()
theirKey := lc.channelState.TheirMultiSigKey.SerializeCompressed()
ourSig := append(closeSig, byte(txscript.SigHashAll))
witness := SpendMultiSig(lc.signDesc.WitnessScript, ourKey, ourSig,
theirKey, remoteSig)
witness := SpendMultiSig(lc.signDesc.WitnessScript, ourKey,
localSig, theirKey, remoteSig)
closeTx.TxIn[0].Witness = witness
// Validate the finalized transaction to ensure the output script is

View File

@ -53,6 +53,12 @@ var (
numReqConfs = uint16(1)
)
const (
// proposedFee is a hard-coded fee value, intended to be replaced by a
// more robust fee estimation implemention.
proposedFee = 5000
)
type mockSigner struct {
key *btcec.PrivateKey
}
@ -705,40 +711,35 @@ func TestCooperativeChannelClosure(t *testing.T) {
}
defer cleanUp()
// First we test the channel initiator requesting a cooperative close.
sig, txid, err := aliceChannel.InitCooperativeClose()
// First we test creating of cooperative close proposals.
aliceSig, err := aliceChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to initiate alice cooperative close: %v", err)
t.Fatalf("unable to create alice coop close proposal: %v", err)
}
finalSig := append(sig, byte(txscript.SigHashAll))
closeTx, err := bobChannel.CompleteCooperativeClose(finalSig)
aliceCloseSig := append(aliceSig, byte(txscript.SigHashAll))
bobSig, err := bobChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to create bob coop close proposal: %v", err)
}
bobCloseSig := append(bobSig, byte(txscript.SigHashAll))
aliceCloseTx, err := bobChannel.CompleteCooperativeClose(bobCloseSig,
aliceCloseSig, proposedFee)
if err != nil {
t.Fatalf("unable to complete alice cooperative close: %v", err)
}
bobCloseSha := closeTx.TxHash()
if !bobCloseSha.IsEqual(txid) {
t.Fatalf("alice's transactions doesn't match: %x vs %x",
bobCloseSha[:], txid[:])
}
bobCloseSha := aliceCloseTx.TxHash()
aliceChannel.status = channelOpen
bobChannel.status = channelOpen
// Next we test the channel recipient requesting a cooperative closure.
// First we test the channel initiator requesting a cooperative close.
sig, txid, err = bobChannel.InitCooperativeClose()
if err != nil {
t.Fatalf("unable to initiate bob cooperative close: %v", err)
}
finalSig = append(sig, byte(txscript.SigHashAll))
closeTx, err = aliceChannel.CompleteCooperativeClose(finalSig)
bobCloseTx, err := aliceChannel.CompleteCooperativeClose(aliceCloseSig,
bobCloseSig, proposedFee)
if err != nil {
t.Fatalf("unable to complete bob cooperative close: %v", err)
}
aliceCloseSha := closeTx.TxHash()
if !aliceCloseSha.IsEqual(txid) {
t.Fatalf("bob's closure transactions don't match: %x vs %x",
aliceCloseSha[:], txid[:])
aliceCloseSha := bobCloseTx.TxHash()
if bobCloseSha != aliceCloseSha {
t.Fatalf("alice and bob close transactions don't match: %v", err)
}
}
@ -1581,12 +1582,20 @@ func TestCooperativeCloseDustAdherence(t *testing.T) {
// Both sides currently have over 1 BTC settled as part of their
// balances. As a result, performing a cooperative closure now result
// in both sides having an output within the closure transaction.
closeSig, _, err := aliceChannel.InitCooperativeClose()
aliceSig, err := aliceChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to close channel: %v", err)
}
closeSig = append(closeSig, byte(txscript.SigHashAll))
closeTx, err := bobChannel.CompleteCooperativeClose(closeSig)
aliceCloseSig := append(aliceSig, byte(txscript.SigHashAll))
bobSig, err := bobChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to close channel: %v", err)
}
bobCloseSig := append(bobSig, byte(txscript.SigHashAll))
closeTx, err := bobChannel.CompleteCooperativeClose(bobCloseSig,
aliceCloseSig, proposedFee)
if err != nil {
t.Fatalf("unable to accept channel close: %v", err)
}
@ -1608,12 +1617,20 @@ func TestCooperativeCloseDustAdherence(t *testing.T) {
// Attempt another cooperative channel closure. It should succeed
// without any issues.
closeSig, _, err = aliceChannel.InitCooperativeClose()
aliceSig, err = aliceChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to close channel: %v", err)
}
closeSig = append(closeSig, byte(txscript.SigHashAll))
closeTx, err = bobChannel.CompleteCooperativeClose(closeSig)
aliceCloseSig = append(aliceSig, byte(txscript.SigHashAll))
bobSig, err = bobChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to close channel: %v", err)
}
bobCloseSig = append(bobSig, byte(txscript.SigHashAll))
closeTx, err = bobChannel.CompleteCooperativeClose(bobCloseSig,
aliceCloseSig, proposedFee)
if err != nil {
t.Fatalf("unable to accept channel close: %v", err)
}
@ -1636,12 +1653,20 @@ func TestCooperativeCloseDustAdherence(t *testing.T) {
// Our final attempt at another cooperative channel closure. It should
// succeed without any issues.
closeSig, _, err = aliceChannel.InitCooperativeClose()
aliceSig, err = aliceChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to close channel: %v", err)
}
closeSig = append(closeSig, byte(txscript.SigHashAll))
closeTx, err = bobChannel.CompleteCooperativeClose(closeSig)
aliceCloseSig = append(aliceSig, byte(txscript.SigHashAll))
bobSig, err = bobChannel.CreateCloseProposal(proposedFee)
if err != nil {
t.Fatalf("unable to close channel: %v", err)
}
bobCloseSig = append(bobSig, byte(txscript.SigHashAll))
closeTx, err = bobChannel.CompleteCooperativeClose(bobCloseSig,
aliceCloseSig, proposedFee)
if err != nil {
t.Fatalf("unable to accept channel close: %v", err)
}

View File

@ -13,7 +13,7 @@ type Shutdown struct {
// ChannelID serves to identify which channel is to be closed.
ChannelID ChannelID
// Address is the script to which
// Address is the script to which the channel funds will be paid.
Address DeliveryAddress
}

308
peer.go
View File

@ -6,12 +6,10 @@ import (
"crypto/sha256"
"fmt"
"net"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/btcsuite/btcd/txscript"
"github.com/davecgh/go-spew/spew"
"github.com/go-errors/errors"
"github.com/lightningnetwork/lightning-onion"
@ -24,6 +22,7 @@ import (
"github.com/roasbeef/btcd/btcec"
"github.com/roasbeef/btcd/chaincfg/chainhash"
"github.com/roasbeef/btcd/connmgr"
"github.com/roasbeef/btcd/txscript"
"github.com/roasbeef/btcd/wire"
"github.com/roasbeef/btcutil"
)
@ -40,6 +39,10 @@ const (
// messages to be sent across the wire, requested by objects outside
// this struct.
outgoingQueueLen = 50
// proposedFee is a hard-coded fee value, intended to be replaced by a
// more robust fee estimation implemention.
proposedFee = 5000
)
// outgoinMsg packages an lnwire.Message to be sent out on the wire, along with
@ -133,10 +136,13 @@ type peer struct {
// a particular channel are sent over.
localCloseChanReqs chan *closeLinkReq
// remoteCloseChanReqs is a channel in which any remote requests
// (initiated by the remote peer) close a particular channel are sent
// over.
remoteCloseChanReqs chan *lnwire.ClosingSigned
// shutdownChanReqs is used to send the Shutdown messages that initiate
// the cooperative close workflow.
shutdownChanReqs chan *lnwire.Shutdown
// closingSignedChanReqs is used to send signatures for proposed
// channel close transactions during the cooperative close workflow.
closingSignedChanReqs chan *lnwire.ClosingSigned
server *server
@ -182,8 +188,9 @@ func newPeer(conn net.Conn, connReq *connmgr.ConnReq, server *server,
chanSnapshotReqs: make(chan *chanSnapshotReq),
newChannels: make(chan *newChannelMsg, 1),
localCloseChanReqs: make(chan *closeLinkReq),
remoteCloseChanReqs: make(chan *lnwire.ClosingSigned),
localCloseChanReqs: make(chan *closeLinkReq),
shutdownChanReqs: make(chan *lnwire.Shutdown),
closingSignedChanReqs: make(chan *lnwire.ClosingSigned),
localSharedFeatures: nil,
globalSharedFeatures: nil,
@ -445,8 +452,11 @@ out:
p.server.fundingMgr.processFundingSignComplete(msg, p.addr)
case *lnwire.FundingLocked:
p.server.fundingMgr.processFundingLocked(msg, p.addr)
case *lnwire.Shutdown:
p.shutdownChanReqs <- msg
case *lnwire.ClosingSigned:
p.remoteCloseChanReqs <- msg
p.closingSignedChanReqs <- msg
case *lnwire.Error:
p.server.fundingMgr.processFundingError(msg, p.addr)
@ -746,6 +756,17 @@ func (p *peer) ChannelSnapshots() []*channeldb.ChannelSnapshot {
//
// NOTE: This method MUST be run as a goroutine.
func (p *peer) channelManager() {
// chanShutdowns is a map of channels for which our node has initiated
// a cooperative channel close. When an lnwire.Shutdown is received,
// this allows the node to determine the next step to be taken in the
// workflow.
chanShutdowns := make(map[lnwire.ChannelID]*closeLinkReq)
// shutdownSigs is a map of signatures maintained by the responder in
// a cooperative channel close. This map enables us to respond to
// subsequent steps in the workflow without having to recalculate our
// signature for the channel close transaction.
shutdownSigs := make(map[lnwire.ChannelID][]byte)
out:
for {
select {
@ -795,10 +816,34 @@ out:
close(newChanReq.done)
case req := <-p.localCloseChanReqs:
chanID := lnwire.NewChanIDFromOutPoint(req.chanPoint)
chanShutdowns[chanID] = req
p.handleLocalClose(req)
case req := <-p.remoteCloseChanReqs:
p.handleRemoteClose(req)
case req := <-p.shutdownChanReqs:
_, ok := chanShutdowns[req.ChannelID]
if !ok {
// we're the responder.
closeSig := p.handleShutdownResponse(req)
shutdownSigs[req.ChannelID] = closeSig
}
case req := <-p.closingSignedChanReqs:
localCloseReq, ok := chanShutdowns[req.ChannelID]
if ok {
// we're the initiator
p.handleInitClosingSigned(localCloseReq, req)
// this won't work if there's fee disagreement
delete(chanShutdowns, req.ChannelID)
} else {
// we're the responder
responderSig := append(shutdownSigs[req.ChannelID],
byte(txscript.SigHashAll))
p.handleResponseClosingSigned(req, responderSig)
delete(shutdownSigs, req.ChannelID)
}
case <-p.quit:
break out
@ -808,51 +853,11 @@ out:
p.wg.Done()
}
// executeCooperativeClose executes the initial phase of a user-executed
// cooperative channel close. The channel state machine is transitioned to the
// closing phase, then our half of the closing witness is sent over to the
// remote peer.
func (p *peer) executeCooperativeClose(channel *lnwallet.LightningChannel) (*chainhash.Hash, error) {
// Shift the channel state machine into a 'closing' state. This
// generates a signature for the closing tx, as well as a txid of the
// closing tx itself, allowing us to watch the network to determine
// when the remote node broadcasts the fully signed closing
// transaction.
sig, txid, err := channel.InitCooperativeClose()
if err != nil {
return nil, err
}
chanPoint := channel.ChannelPoint()
peerLog.Infof("Executing cooperative closure of "+
"ChanPoint(%v) with peerID(%v), txid=%v", chanPoint, p.id, txid)
// With our signature for the close tx generated, send the signature to
// the remote peer instructing it to close this particular channel
// point.
// TODO(roasbeef): remove encoding redundancy
closeSig, err := btcec.ParseSignature(sig, btcec.S256())
if err != nil {
return nil, err
}
chanID := lnwire.NewChanIDFromOutPoint(chanPoint)
closeReq := lnwire.NewClosingSigned(chanID, 5000, closeSig)
p.queueMsg(closeReq, nil)
return txid, nil
}
// handleLocalClose kicks-off the workflow to execute a cooperative or forced
// unilateral closure of the channel initiated by a local subsystem.
// TODO(roasbeef): if no more active channels with peer call Remove on connMgr
// with peerID
func (p *peer) handleLocalClose(req *closeLinkReq) {
var (
err error
closingTxid *chainhash.Hash
)
chanID := lnwire.NewChanIDFromOutPoint(req.chanPoint)
p.activeChanMtx.RLock()
@ -864,10 +869,11 @@ func (p *peer) handleLocalClose(req *closeLinkReq) {
// out this channel on-chain, so we execute the cooperative channel
// closure workflow.
case CloseRegular:
closingTxid, err = p.executeCooperativeClose(channel)
peerLog.Infof("Attempting cooperative close of "+
"ChannelPoint(%v) with txid: %v", req.chanPoint,
closingTxid)
err := p.sendShutdown(channel)
if err != nil {
req.err <- err
return
}
// A type of CloseBreach indicates that the counterparty has breached
// the channel therefore we need to clean up our local state.
@ -882,10 +888,93 @@ func (p *peer) handleLocalClose(req *closeLinkReq) {
}
return
}
}
// handleShutdownResponse is called when a responder in a cooperative channel
// close workflow receives a Shutdown message. This is the second step in the
// cooperative close workflow. This function generates a close transaction with
// a proposed fee amount and sends the signed transaction to the initiator.
func (p *peer) handleShutdownResponse(msg *lnwire.Shutdown) []byte {
p.activeChanMtx.RLock()
channel, ok := p.activeChannels[msg.ChannelID]
p.activeChanMtx.RUnlock()
if !ok {
peerLog.Errorf("unable to close channel, ChannelPoint(%v) is "+
"unknown", msg.ChannelID)
return nil
}
if err := p.sendShutdown(channel); err != nil {
peerLog.Errorf("error while sending shutdown message: %v", err)
return nil
}
// TODO(bvu): add logic to wait for HTLCs to close here.
closeSig, err := p.sendClosingSigned(channel)
if err != nil {
peerLog.Errorf("error generating/sending ClosingSigned: %v", err)
return nil
}
return closeSig
}
// handleInitClosingSigned is called when the initiator in a cooperative
// channel close workflow receives a ClosingSigned message from the responder.
// This method completes the channel close transaction, sends back a
// corresponding ClosingSigned message, then broadcasts the channel close
// transaction. It also performs channel cleanup and reports status back to the
// caller. This is the initiator's final step in the channel close workflow.
//
// Following the broadcast, both the initiator and responder in the channel
// closure workflow should watch the blockchain for a confirmation of the
// closing transaction before considering the channel terminated. In the case
// of an unresponsive remote party, the initiator can either choose to execute
// a force closure, or backoff for a period of time, and retry the cooperative
// closure.
func (p *peer) handleInitClosingSigned(req *closeLinkReq, msg *lnwire.ClosingSigned) {
chanID := lnwire.NewChanIDFromOutPoint(req.chanPoint)
p.activeChanMtx.RLock()
channel := p.activeChannels[chanID]
p.activeChanMtx.RUnlock()
initiatorSig, err := channel.CreateCloseProposal(proposedFee)
if err != nil {
req.err <- err
return
}
initSig := append(initiatorSig, byte(txscript.SigHashAll))
// Complete coop close transaction.
responderSig := msg.Signature
respSig := append(responderSig.Serialize(), byte(txscript.SigHashAll))
closeTx, err := channel.CompleteCooperativeClose(initSig, respSig,
proposedFee)
if err != nil {
req.err <- err
// TODO(roasbeef): send ErrorGeneric to other side
return
}
closingTxid := closeTx.TxHash()
// Shutdown initiator sends ClosingSigned back to the responder.
parsedSig, err := btcec.ParseSignature(initSig, btcec.S256())
closingSigned := lnwire.NewClosingSigned(chanID, proposedFee, parsedSig)
p.queueMsg(closingSigned, nil)
// Finally, broadcast the closure transaction to the network.
peerLog.Infof("Broadcasting cooperative close tx: %v",
newLogClosure(func() string {
return spew.Sdump(closeTx)
}))
if err := p.server.lnwallet.PublishTransaction(closeTx); err != nil {
peerLog.Errorf("channel close tx from "+
"ChannelPoint(%v) rejected: %v",
req.chanPoint, err)
// TODO(roasbeef): send ErrorGeneric to other side
return
}
// Once we've completed the cooperative channel closure, we'll wipe the
// channel so we reject any incoming forward or payment requests via
@ -901,7 +990,7 @@ func (p *peer) handleLocalClose(req *closeLinkReq) {
chanInfo := channel.StateSnapshot()
closeSummary := &channeldb.ChannelCloseSummary{
ChanPoint: *req.chanPoint,
ClosingTXID: *closingTxid,
ClosingTXID: closingTxid,
RemotePub: &chanInfo.RemoteIdentity,
Capacity: chanInfo.Capacity,
SettledBalance: chanInfo.LocalBalance,
@ -934,7 +1023,7 @@ func (p *peer) handleLocalClose(req *closeLinkReq) {
// confirmation.
notifier := p.server.chainNotifier
go waitForChanToClose(uint32(bestHeight), notifier, req.err,
req.chanPoint, closingTxid, func() {
req.chanPoint, &closingTxid, func() {
// First, we'll mark the database as being fully closed
// so we'll no longer watch for its ultimate closure
@ -958,27 +1047,24 @@ func (p *peer) handleLocalClose(req *closeLinkReq) {
})
}
// handleRemoteClose completes a request for cooperative channel closure
// initiated by the remote node.
func (p *peer) handleRemoteClose(req *lnwire.ClosingSigned) {
// handleResponseClosingSigned is called when the responder in a cooperative
// close workflow receives a ClosingSigned message. This function handles the
// finalization of the cooperative close from the perspective of the responder.
func (p *peer) handleResponseClosingSigned(msg *lnwire.ClosingSigned,
respSig []byte) {
p.activeChanMtx.RLock()
channel, ok := p.activeChannels[req.ChannelID]
channel := p.activeChannels[msg.ChannelID]
p.activeChanMtx.RUnlock()
if !ok {
peerLog.Errorf("unable to close channel, ChannelID(%v) is "+
"unknown", req.ChannelID)
return
}
// Now that we have the initiator's signature for the closure
// transaction, we can assemble the final closure transaction, complete
// with our signature.
initiatorSig := msg.Signature
initSig := append(initiatorSig.Serialize(), byte(txscript.SigHashAll))
chanPoint := channel.ChannelPoint()
// Now that we have their signature for the closure transaction, we
// can assemble the final closure transaction, complete with our
// signature.
sig := req.Signature
closeSig := append(sig.Serialize(), byte(txscript.SigHashAll))
closeTx, err := channel.CompleteCooperativeClose(closeSig)
closeTx, err := channel.CompleteCooperativeClose(respSig, initSig,
proposedFee)
if err != nil {
peerLog.Errorf("unable to complete cooperative "+
"close for ChannelPoint(%v): %v",
@ -986,32 +1072,16 @@ func (p *peer) handleRemoteClose(req *lnwire.ClosingSigned) {
// TODO(roasbeef): send ErrorGeneric to other side
return
}
peerLog.Infof("Broadcasting cooperative close tx: %v",
newLogClosure(func() string {
return spew.Sdump(closeTx)
}))
if err != nil {
peerLog.Errorf("unable to get current height: %v", err)
return
}
closeTxid := closeTx.TxHash()
_, bestHeight, err := p.server.bio.GetBestBlock()
if err != nil {
peerLog.Errorf("unable to get best height: %v", err)
}
// Finally, broadcast the closure transaction, to the network.
err = p.server.lnwallet.PublishTransaction(closeTx)
if err != nil && !strings.Contains(err.Error(), "already have") {
peerLog.Errorf("channel close tx from "+
"ChannelPoint(%v) rejected: %v",
chanPoint, err)
// TODO(roasbeef): send ErrorGeneric to other side
// * remove check above to error
return
}
// Once we've completed the cooperative channel closure, we'll wipe the
// channel so we reject any incoming forward or payment requests via
// this channel.
p.server.breachArbiter.settledContracts <- chanPoint
// We've just broadcast the transaction which closes the channel, so
@ -1024,7 +1094,6 @@ func (p *peer) handleRemoteClose(req *lnwire.ClosingSigned) {
// Clear out the current channel state, marking the channel as being
// closed within the database.
closeTxid := closeTx.TxHash()
chanInfo := channel.StateSnapshot()
closeSummary := &channeldb.ChannelCloseSummary{
ChanPoint: *chanPoint,
@ -1068,6 +1137,10 @@ func waitForChanToClose(bestHeight uint32, notifier chainntnfs.ChainNotifier,
errChan chan error, chanPoint *wire.OutPoint,
closingTxID *chainhash.Hash, cb func()) {
peerLog.Infof("Waiting for confirmation of cooperative close of "+
"ChannelPoint(%v) with txid: %v", chanPoint,
closingTxID)
// TODO(roasbeef): add param for num needed confs
confNtfn, err := notifier.RegisterConfirmationsNtfn(closingTxID, 1,
bestHeight)
@ -1095,6 +1168,53 @@ func waitForChanToClose(bestHeight uint32, notifier chainntnfs.ChainNotifier,
cb()
}
// sendShutdown handles the creation and sending of the Shutdown messages
// sent between peers to initiate the cooperative channel close workflow. In
// addition, sendShutdown also signals to the HTLC switch to stop accepting
// HTLCs for the specified channel.
func (p *peer) sendShutdown(channel *lnwallet.LightningChannel) error {
_, addrs, _, err := txscript.ExtractPkScriptAddrs(
channel.LocalDeliveryScript, activeNetParams.Params,
)
if err != nil {
return err
}
address := lnwire.DeliveryAddress(addrs[0].ScriptAddress())
chanID := lnwire.NewChanIDFromOutPoint(channel.ChannelPoint())
shutdown := lnwire.NewShutdown(chanID, address)
p.queueMsg(shutdown, nil)
// Prevent the HTLC switch from receiving additional HTLCs for
// this channel.
p.server.htlcSwitch.UnregisterLink(p.addr.IdentityKey, &chanID)
return nil
}
// sendClosingSigned handles the creation and sending of proposed channel
// close transactions.
func (p *peer) sendClosingSigned(channel *lnwallet.LightningChannel) ([]byte, error) {
// We agree with the proposed fee, so we send back our signature
// for the proposed transaction.
closeSig, err := channel.CreateCloseProposal(proposedFee)
if err != nil {
return nil, err
}
// TODO(roasbeef): remove encoding redundancy
parsedSig, err := btcec.ParseSignature(closeSig, btcec.S256())
if err != nil {
return nil, err
}
chanID := lnwire.NewChanIDFromOutPoint(channel.ChannelPoint())
closingSigned := lnwire.NewClosingSigned(chanID, proposedFee, parsedSig)
p.queueMsg(closingSigned, nil)
return closeSig, nil
}
// wipeChannel removes the passed channel from all indexes associated with the
// peer, and deletes the channel from the database.
func wipeChannel(p *peer, channel *lnwallet.LightningChannel) error {