lnwallet+fundingmgr: interrupt funding flow for PSBT

In case the funding manager detects that a funding flow is requested
to be executed with the help of a PsbtIntent, the normal channel
negotiation with the remote peer is interrupted, as soon as the
accept_channel message was received. With the remote peer's funding
multisig key and our local key, we can derive the funding output
script and its address. This is enough to start the PSBT funding
and signing process which the user will do externally to the daemon.
This commit is contained in:
Oliver Gugger 2020-03-31 09:13:16 +02:00
parent 126f79dbb1
commit 5a52420ab6
No known key found for this signature in database
GPG Key ID: 8E4256593F177720
4 changed files with 419 additions and 19 deletions

View File

@ -26,6 +26,7 @@ import (
"github.com/lightningnetwork/lnd/lnrpc"
"github.com/lightningnetwork/lnd/lnwallet"
"github.com/lightningnetwork/lnd/lnwallet/chainfee"
"github.com/lightningnetwork/lnd/lnwallet/chanfunding"
"github.com/lightningnetwork/lnd/lnwire"
"github.com/lightningnetwork/lnd/routing"
"golang.org/x/crypto/salsa20"
@ -698,7 +699,7 @@ func (f *fundingManager) failFundingFlow(peer lnpeer.Peer, tempChanID [32]byte,
fndgLog.Debugf("Failing funding flow for pending_id=%x: %v",
tempChanID, fundingErr)
ctx, err := f.cancelReservationCtx(peer.IdentityKey(), tempChanID)
ctx, err := f.cancelReservationCtx(peer.IdentityKey(), tempChanID, false)
if err != nil {
fndgLog.Errorf("unable to cancel reservation: %v", err)
}
@ -1547,7 +1548,42 @@ func (f *fundingManager) handleFundingAccept(fmsg *fundingAcceptMsg) {
UpfrontShutdown: msg.UpfrontShutdownScript,
}
err = resCtx.reservation.ProcessContribution(remoteContribution)
if err != nil {
// The wallet has detected that a PSBT funding process was requested by
// the user and has halted the funding process after negotiating the
// multisig keys. We now have everything that is needed for the user to
// start constructing a PSBT that sends to the multisig funding address.
var psbtIntent *chanfunding.PsbtIntent
if psbtErr, ok := err.(*lnwallet.PsbtFundingRequired); ok {
// Return the information that is needed by the user to
// construct the PSBT back to the caller.
addr, amt, packet, err := psbtErr.Intent.FundingParams()
if err != nil {
fndgLog.Errorf("Unable to process PSBT funding params "+
"for contribution from %v: %v", peerKey, err)
f.failFundingFlow(fmsg.peer, msg.PendingChannelID, err)
return
}
var buf bytes.Buffer
err = packet.Serialize(&buf)
if err != nil {
fndgLog.Errorf("Unable to serialize PSBT for "+
"contribution from %v: %v", peerKey, err)
f.failFundingFlow(fmsg.peer, msg.PendingChannelID, err)
return
}
resCtx.updates <- &lnrpc.OpenStatusUpdate{
PendingChanId: pendingChanID[:],
Update: &lnrpc.OpenStatusUpdate_PsbtFund{
PsbtFund: &lnrpc.ReadyForPsbtFunding{
FundingAddress: addr.EncodeAddress(),
FundingAmount: amt,
Psbt: buf.Bytes(),
},
},
}
psbtIntent = psbtErr.Intent
} else if err != nil {
fndgLog.Errorf("Unable to process contribution from %v: %v",
peerKey, err)
f.failFundingFlow(fmsg.peer, msg.PendingChannelID, err)
@ -1559,6 +1595,105 @@ func (f *fundingManager) handleFundingAccept(fmsg *fundingAcceptMsg) {
fndgLog.Debugf("Remote party accepted commitment constraints: %v",
spew.Sdump(remoteContribution.ChannelConfig.ChannelConstraints))
// If the user requested funding through a PSBT, we cannot directly
// continue now and need to wait for the fully funded and signed PSBT
// to arrive. To not block any other channels from opening, we wait in
// a separate goroutine.
if psbtIntent != nil {
f.wg.Add(1)
go func() {
defer f.wg.Done()
f.waitForPsbt(psbtIntent, resCtx, pendingChanID)
}()
// With the new goroutine spawned, we can now exit to unblock
// the main event loop.
return
}
// In a normal, non-PSBT funding flow, we can jump directly to the next
// step where we expect our contribution to be finalized.
f.continueFundingAccept(resCtx, pendingChanID)
}
// waitForPsbt blocks until either a signed PSBT arrives, an error occurs or
// the funding manager shuts down. In the case of a valid PSBT, the funding flow
// is continued.
//
// NOTE: This method must be called as a goroutine.
func (f *fundingManager) waitForPsbt(intent *chanfunding.PsbtIntent,
resCtx *reservationWithCtx, pendingChanID [32]byte) {
// failFlow is a helper that logs an error message with the current
// context and then fails the funding flow.
peerKey := resCtx.peer.IdentityKey()
failFlow := func(errMsg string, cause error) {
fndgLog.Errorf("Unable to handle funding accept message "+
"for peer_key=%x, pending_chan_id=%x: %s: %v",
peerKey.SerializeCompressed(), pendingChanID, errMsg,
cause)
f.failFundingFlow(resCtx.peer, pendingChanID, cause)
}
// We'll now wait until the intent has received the final and complete
// funding transaction. If the channel is closed without any error being
// sent, we know everything's going as expected.
select {
case err := <-intent.PsbtReady:
switch err {
// If the user canceled the funding reservation, we need to
// inform the other peer about us canceling the reservation.
case chanfunding.ErrUserCanceled:
failFlow("aborting PSBT flow", err)
return
// If the remote canceled the funding reservation, we don't need
// to send another fail message. But we want to inform the user
// about what happened.
case chanfunding.ErrRemoteCanceled:
fndgLog.Infof("Remote canceled, aborting PSBT flow "+
"for peer_key=%x, pending_chan_id=%x",
peerKey.SerializeCompressed(), pendingChanID)
return
// Nil error means the flow continues normally now.
case nil:
// For any other error, we'll fail the funding flow.
default:
failFlow("error waiting for PSBT flow", err)
return
}
// A non-nil error means we can continue the funding flow.
// Notify the wallet so it can prepare everything we need to
// continue.
err = resCtx.reservation.ProcessPsbt()
if err != nil {
failFlow("error continuing PSBT flow", err)
return
}
// We are now ready to continue the funding flow.
f.continueFundingAccept(resCtx, pendingChanID)
// Handle a server shutdown as well because the reservation won't
// survive a restart as it's in memory only.
case <-f.quit:
fndgLog.Errorf("Unable to handle funding accept message "+
"for peer_key=%x, pending_chan_id=%x: funding manager "+
"shutting down", peerKey.SerializeCompressed(),
pendingChanID)
return
}
}
// continueFundingAccept continues the channel funding flow once our
// contribution is finalized, the channel output is known and the funding
// transaction is signed.
func (f *fundingManager) continueFundingAccept(resCtx *reservationWithCtx,
pendingChanID [32]byte) {
// Now that we have their contribution, we can extract, then send over
// both the funding out point and our signature for their version of
// the commitment transaction to the remote peer.
@ -1586,6 +1721,7 @@ func (f *fundingManager) handleFundingAccept(fmsg *fundingAcceptMsg) {
fndgLog.Infof("Generated ChannelPoint(%v) for pending_id(%x)", outPoint,
pendingChanID[:])
var err error
fundingCreated := &lnwire.FundingCreated{
PendingChannelID: pendingChanID,
FundingPoint: *outPoint,
@ -1593,12 +1729,12 @@ func (f *fundingManager) handleFundingAccept(fmsg *fundingAcceptMsg) {
fundingCreated.CommitSig, err = lnwire.NewSigFromRawSignature(sig)
if err != nil {
fndgLog.Errorf("Unable to parse signature: %v", err)
f.failFundingFlow(fmsg.peer, msg.PendingChannelID, err)
f.failFundingFlow(resCtx.peer, pendingChanID, err)
return
}
if err := fmsg.peer.SendMessage(true, fundingCreated); err != nil {
if err := resCtx.peer.SendMessage(true, fundingCreated); err != nil {
fndgLog.Errorf("Unable to send funding complete message: %v", err)
f.failFundingFlow(fmsg.peer, msg.PendingChannelID, err)
f.failFundingFlow(resCtx.peer, pendingChanID, err)
return
}
}
@ -3070,7 +3206,8 @@ func (f *fundingManager) handleInitFundingMsg(msg *initFundingMsg) {
// Since we were unable to send the initial message to the peer
// and start the funding flow, we'll cancel this reservation.
if _, err := f.cancelReservationCtx(peerKey, chanID); err != nil {
_, err := f.cancelReservationCtx(peerKey, chanID, false)
if err != nil {
fndgLog.Errorf("unable to cancel reservation: %v", err)
}
@ -3130,7 +3267,7 @@ func (f *fundingManager) handleErrorMsg(fmsg *fundingErrorMsg) {
// First, we'll attempt to retrieve and cancel the funding workflow
// that this error was tied to. If we're unable to do so, then we'll
// exit early as this was an unwarranted error.
resCtx, err := f.cancelReservationCtx(fmsg.peerKey, chanID)
resCtx, err := f.cancelReservationCtx(fmsg.peerKey, chanID, true)
if err != nil {
fndgLog.Warnf("Received error for non-existent funding "+
"flow: %v (%v)", err, protocolErr.Error())
@ -3144,6 +3281,14 @@ func (f *fundingManager) handleErrorMsg(fmsg *fundingErrorMsg) {
)
fndgLog.Errorf(fundingErr.Error())
// If this was a PSBT funding flow, the remote likely timed out because
// we waited too long. Return a nice error message to the user in that
// case so the user knows what's the problem.
if resCtx.reservation.IsPsbt() {
fundingErr = fmt.Errorf("%w: %v", chanfunding.ErrRemoteCanceled,
fundingErr)
}
resCtx.err <- fundingErr
}
@ -3160,7 +3305,14 @@ func (f *fundingManager) pruneZombieReservations() {
continue
}
if time.Since(resCtx.lastUpdated) > f.cfg.ReservationTimeout {
// We don't want to expire PSBT funding reservations.
// These reservations are always initiated by us and the
// remote peer is likely going to cancel them after some
// idle time anyway. So no need for us to also prune
// them.
sinceLastUpdate := time.Since(resCtx.lastUpdated)
isExpired := sinceLastUpdate > f.cfg.ReservationTimeout
if !resCtx.reservation.IsPsbt() && isExpired {
zombieReservations[pendingChanID] = resCtx
}
}
@ -3169,7 +3321,7 @@ func (f *fundingManager) pruneZombieReservations() {
for pendingChanID, resCtx := range zombieReservations {
err := fmt.Errorf("reservation timed out waiting for peer "+
"(peer_id:%v, chan_id:%x)", resCtx.peer.IdentityKey(),
"(peer_id:%x, chan_id:%x)", resCtx.peer.IdentityKey(),
pendingChanID[:])
fndgLog.Warnf(err.Error())
f.failFundingFlow(resCtx.peer, pendingChanID, err)
@ -3179,7 +3331,7 @@ func (f *fundingManager) pruneZombieReservations() {
// cancelReservationCtx does all needed work in order to securely cancel the
// reservation.
func (f *fundingManager) cancelReservationCtx(peerKey *btcec.PublicKey,
pendingChanID [32]byte) (*reservationWithCtx, error) {
pendingChanID [32]byte, byRemote bool) (*reservationWithCtx, error) {
fndgLog.Infof("Cancelling funding reservation for node_key=%x, "+
"chan_id=%x", peerKey.SerializeCompressed(), pendingChanID[:])
@ -3201,6 +3353,14 @@ func (f *fundingManager) cancelReservationCtx(peerKey *btcec.PublicKey,
"peer(%x)", pendingChanID[:], peerIDKey[:])
}
// If the reservation was a PSBT funding flow and it was canceled by the
// remote peer, then we need to thread through a different error message
// to the subroutine that's waiting for the user input so it can return
// a nice error message to the user.
if ctx.reservation.IsPsbt() && byRemote {
ctx.reservation.RemoteCanceled()
}
if err := ctx.reservation.Cancel(); err != nil {
return nil, errors.Errorf("unable to cancel reservation: %v",
err)

View File

@ -590,6 +590,7 @@ func testFundingTransactionLockedOutputs(miner *rpctest.Harness,
FundingFeePerKw: feePerKw,
PushMSat: 0,
Flags: lnwire.FFAnnounceChannel,
PendingChanID: [32]byte{0, 1, 2, 3},
}
if _, err := alice.InitChannelReservation(req); err != nil {
t.Fatalf("unable to initialize funding reservation 1: %v", err)
@ -612,6 +613,7 @@ func testFundingTransactionLockedOutputs(miner *rpctest.Harness,
FundingFeePerKw: feePerKw,
PushMSat: 0,
Flags: lnwire.FFAnnounceChannel,
PendingChanID: [32]byte{1, 2, 3, 4},
}
failedReservation, err := alice.InitChannelReservation(failedReq)
if err == nil {
@ -648,6 +650,7 @@ func testFundingCancellationNotEnoughFunds(miner *rpctest.Harness,
FundingFeePerKw: feePerKw,
PushMSat: 0,
Flags: lnwire.FFAnnounceChannel,
PendingChanID: [32]byte{2, 3, 4, 5},
}
chanReservation, err := alice.InitChannelReservation(req)
if err != nil {
@ -655,6 +658,7 @@ func testFundingCancellationNotEnoughFunds(miner *rpctest.Harness,
}
// Attempt to create another channel with 44 BTC, this should fail.
req.PendingChanID = [32]byte{3, 4, 5, 6}
_, err = alice.InitChannelReservation(req)
if _, ok := err.(*chanfunding.ErrInsufficientFunds); !ok {
t.Fatalf("coin selection succeeded should have insufficient funds: %v",
@ -684,6 +688,7 @@ func testFundingCancellationNotEnoughFunds(miner *rpctest.Harness,
// attempting coin selection.
// Request to fund a new channel should now succeed.
req.PendingChanID = [32]byte{4, 5, 6, 7, 8}
if _, err := alice.InitChannelReservation(req); err != nil {
t.Fatalf("unable to initialize funding reservation: %v", err)
}

View File

@ -471,6 +471,37 @@ func (r *ChannelReservation) ProcessContribution(theirContribution *ChannelContr
return <-errChan
}
// IsPsbt returns true if there is a PSBT funding intent mapped to this
// reservation.
func (r *ChannelReservation) IsPsbt() bool {
_, ok := r.fundingIntent.(*chanfunding.PsbtIntent)
return ok
}
// ProcessPsbt continues a previously paused funding flow that involves PSBT to
// construct the funding transaction. This method can be called once the PSBT is
// finalized and the signed transaction is available.
func (r *ChannelReservation) ProcessPsbt() error {
errChan := make(chan error, 1)
r.wallet.msgChan <- &continueContributionMsg{
pendingFundingID: r.reservationID,
err: errChan,
}
return <-errChan
}
// RemoteCanceled informs the PSBT funding state machine that the remote peer
// has canceled the pending reservation, likely due to a timeout.
func (r *ChannelReservation) RemoteCanceled() {
psbtIntent, ok := r.fundingIntent.(*chanfunding.PsbtIntent)
if !ok {
return
}
psbtIntent.RemoteCanceled()
}
// ProcessSingleContribution verifies, and records the initiator's contribution
// to this pending single funder channel. Internally, no further action is
// taken other than recording the initiator's contribution to the single funder

View File

@ -15,6 +15,7 @@ import (
"github.com/btcsuite/btcd/txscript"
"github.com/btcsuite/btcd/wire"
"github.com/btcsuite/btcutil"
"github.com/btcsuite/btcutil/psbt"
"github.com/btcsuite/btcutil/txsort"
"github.com/davecgh/go-spew/spew"
"github.com/lightningnetwork/lnd/channeldb"
@ -33,6 +34,28 @@ const (
msgBufferSize = 100
)
var (
// ErrPsbtFundingRequired is the error that is returned during the
// contribution handling process if the process should be paused for
// the construction of a PSBT outside of lnd's wallet.
ErrPsbtFundingRequired = errors.New("PSBT funding required")
)
// PsbtFundingRequired is a type that implements the error interface and
// contains the information needed to construct a PSBT.
type PsbtFundingRequired struct {
// Intent is the pending PSBT funding intent that needs to be funded
// if the wrapping error is returned.
Intent *chanfunding.PsbtIntent
}
// Error returns the underlying error.
//
// NOTE: This method is part of the error interface.
func (p *PsbtFundingRequired) Error() string {
return ErrPsbtFundingRequired.Error()
}
// InitFundingReserveMsg 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
@ -150,6 +173,16 @@ type addContributionMsg struct {
err chan error
}
// continueContributionMsg represents a message that signals that the
// interrupted funding process involving a PSBT can now be continued because the
// finalized transaction is now available.
type continueContributionMsg struct {
pendingFundingID uint64
// NOTE: In order to avoid deadlocks, this channel MUST be buffered.
err chan error
}
// addSingleContributionMsg represents a message executing the second phase of
// a single funder channel reservation workflow. This messages carries the
// counterparty's "contribution" to the payment channel. As this message is
@ -395,6 +428,8 @@ out:
l.handleSingleContribution(msg)
case *addContributionMsg:
l.handleContributionMsg(msg)
case *continueContributionMsg:
l.handleChanPointReady(msg)
case *addSingleFunderSigsMsg:
l.handleSingleFunderSigs(msg)
case *addCounterPartySigsMsg:
@ -462,17 +497,74 @@ func (l *LightningWallet) RegisterFundingIntent(expectedID [32]byte,
return nil
}
// PsbtFundingVerify looks up a previously registered funding intent by its
// pending channel ID and tries to advance the state machine by verifying the
// passed PSBT.
func (l *LightningWallet) PsbtFundingVerify(pid [32]byte,
packet *psbt.Packet) error {
l.intentMtx.Lock()
defer l.intentMtx.Unlock()
intent, ok := l.fundingIntents[pid]
if !ok {
return fmt.Errorf("no funding intent found for "+
"pendingChannelID(%x)", pid[:])
}
psbtIntent, ok := intent.(*chanfunding.PsbtIntent)
if !ok {
return fmt.Errorf("incompatible funding intent")
}
err := psbtIntent.Verify(packet)
if err != nil {
return fmt.Errorf("error verifying PSBT: %v", err)
}
return nil
}
// PsbtFundingFinalize looks up a previously registered funding intent by its
// pending channel ID and tries to advance the state machine by finalizing the
// passed PSBT.
func (l *LightningWallet) PsbtFundingFinalize(pid [32]byte,
packet *psbt.Packet) error {
l.intentMtx.Lock()
defer l.intentMtx.Unlock()
intent, ok := l.fundingIntents[pid]
if !ok {
return fmt.Errorf("no funding intent found for "+
"pendingChannelID(%x)", pid[:])
}
psbtIntent, ok := intent.(*chanfunding.PsbtIntent)
if !ok {
return fmt.Errorf("incompatible funding intent")
}
err := psbtIntent.Finalize(packet)
if err != nil {
return fmt.Errorf("error finalizing PSBT: %v", err)
}
return nil
}
// CancelFundingIntent allows a caller to cancel a previously registered
// funding intent. If no intent was found, then an error will be returned.
func (l *LightningWallet) CancelFundingIntent(pid [32]byte) error {
l.intentMtx.Lock()
defer l.intentMtx.Unlock()
if _, ok := l.fundingIntents[pid]; !ok {
intent, ok := l.fundingIntents[pid]
if !ok {
return fmt.Errorf("no funding intent found for "+
"pendingChannelID(%x)", pid[:])
}
// Give the intent a chance to clean up after itself, removing coin
// locks or similar reserved resources.
intent.Cancel()
delete(l.fundingIntents, pid)
return nil
@ -556,10 +648,28 @@ func (l *LightningWallet) handleFundingReserveRequest(req *InitFundingReserveMsg
return
}
// Register the funding intent now in case we need to access it
// again later, as it's the case for the PSBT state machine for
// example.
err = l.RegisterFundingIntent(req.PendingChanID, fundingIntent)
if err != nil {
req.err <- err
req.resp <- nil
return
}
localFundingAmt = fundingIntent.LocalFundingAmt()
remoteFundingAmt = fundingIntent.RemoteFundingAmt()
}
// At this point there _has_ to be a funding intent, otherwise something
// went really wrong.
if fundingIntent == nil {
req.err <- fmt.Errorf("no funding intent present")
req.resp <- nil
return
}
// If this is a shim intent, then it may be attempting to use an
// existing set of keys for the funding workflow. In this case, we'll
// make a simple wrapper keychain.KeyRing that will proxy certain
@ -592,9 +702,7 @@ func (l *LightningWallet) handleFundingReserveRequest(req *InitFundingReserveMsg
thawHeight,
)
if err != nil {
if fundingIntent != nil {
fundingIntent.Cancel()
}
fundingIntent.Cancel()
req.err <- err
req.resp <- nil
@ -605,9 +713,7 @@ func (l *LightningWallet) handleFundingReserveRequest(req *InitFundingReserveMsg
reservation, fundingIntent, req.NodeAddr, req.NodeID, keyRing,
)
if err != nil {
if fundingIntent != nil {
fundingIntent.Cancel()
}
fundingIntent.Cancel()
req.err <- err
req.resp <- nil
@ -852,6 +958,9 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
// which type of intent we obtained from our chanfunding.Assembler,
// we'll carry out a distinct set of steps.
switch fundingIntent := pendingReservation.fundingIntent.(type) {
// The transaction was created outside of the wallet and might already
// be published. Nothing left to do other than using the correct
// outpoint.
case *chanfunding.ShimIntent:
chanPoint, err = fundingIntent.ChanPoint()
if err != nil {
@ -861,6 +970,38 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
pendingReservation.partialState.FundingOutpoint = *chanPoint
// The user has signaled that they want to use a PSBT to construct the
// funding transaction. Because we now have the multisig keys from both
// parties, we can create the multisig script that needs to be funded
// and then pause the process until the user supplies the PSBT
// containing the eventual funding transaction.
case *chanfunding.PsbtIntent:
if fundingIntent.PendingPsbt != nil {
req.err <- fmt.Errorf("PSBT funding already in" +
"progress")
return
}
// Now that we know our contribution, we can bind both the local
// and remote key which will be needed to calculate the multisig
// funding output in a next step.
pendingChanID := pendingReservation.pendingChanID
walletLog.Debugf("Advancing PSBT funding flow for "+
"pending_id(%x), binding keys local_key=%v, "+
"remote_key=%x", pendingChanID,
&ourContribution.MultiSigKey,
theirContribution.MultiSigKey.PubKey.SerializeCompressed())
fundingIntent.BindKeys(
&ourContribution.MultiSigKey,
theirContribution.MultiSigKey.PubKey,
)
// Exit early because we can't continue the funding flow yet.
req.err <- &PsbtFundingRequired{
Intent: fundingIntent,
}
return
case *chanfunding.FullIntent:
// Now that we know their public key, we can bind theirs as
// well as ours to the funding intent.
@ -915,6 +1056,69 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
"generated: %v", chanPoint, spew.Sdump(fundingTx))
}
// If we landed here and didn't exit early, it means we already have
// the channel point ready. We can jump directly to the next step.
l.handleChanPointReady(&continueContributionMsg{
pendingFundingID: req.pendingFundingID,
err: req.err,
})
}
// handleChanPointReady continues the funding process once the channel point
// is known and the funding transaction can be completed.
func (l *LightningWallet) handleChanPointReady(req *continueContributionMsg) {
l.limboMtx.Lock()
pendingReservation, ok := l.fundingLimbo[req.pendingFundingID]
l.limboMtx.Unlock()
if !ok {
req.err <- fmt.Errorf("attempted to update non-existent " +
"funding state")
return
}
ourContribution := pendingReservation.ourContribution
theirContribution := pendingReservation.theirContribution
chanPoint := pendingReservation.partialState.FundingOutpoint
// If we're in the PSBT funding flow, we now should have everything that
// is needed to construct and publish the full funding transaction.
intent := pendingReservation.fundingIntent
if psbtIntent, ok := intent.(*chanfunding.PsbtIntent); ok {
// With our keys bound, we can now construct+sign the final
// funding transaction and also obtain the chanPoint that
// creates the channel.
fundingTx, err := psbtIntent.CompileFundingTx()
if err != nil {
req.err <- fmt.Errorf("unable to construct funding "+
"tx: %v", err)
return
}
chanPointPtr, err := psbtIntent.ChanPoint()
if err != nil {
req.err <- fmt.Errorf("unable to obtain chan "+
"point: %v", err)
return
}
// Finally, we'll populate the relevant information in our
// pendingReservation so the rest of the funding flow can
// continue as normal.
pendingReservation.fundingTx = fundingTx
pendingReservation.partialState.FundingOutpoint = *chanPointPtr
chanPoint = *chanPointPtr
pendingReservation.ourFundingInputScripts = make(
[]*input.Script, 0, len(ourContribution.Inputs),
)
for _, txIn := range fundingTx.TxIn {
pendingReservation.ourFundingInputScripts = append(
pendingReservation.ourFundingInputScripts,
&input.Script{
Witness: txIn.Witness,
SigScript: txIn.SignatureScript,
},
)
}
}
// 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).
@ -930,7 +1134,7 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
// Create the txin to our commitment transaction; required to construct
// the commitment transactions.
fundingTxIn := wire.TxIn{
PreviousOutPoint: *chanPoint,
PreviousOutPoint: chanPoint,
}
// With the funding tx complete, create both commitment transactions.
@ -991,7 +1195,7 @@ func (l *LightningWallet) handleContributionMsg(req *addContributionMsg) {
chanPoint, spew.Sdump(theirCommitTx))
// Record newly available information within the open channel state.
chanState.FundingOutpoint = *chanPoint
chanState.FundingOutpoint = chanPoint
chanState.LocalCommitment.CommitTx = ourCommitTx
chanState.RemoteCommitment.CommitTx = theirCommitTx