From f3230a4386939369f0647678ccb6a5bf20d6ab8d Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Fri, 8 Dec 2023 12:50:43 -0800 Subject: [PATCH 01/29] htlcswitch: define state machine for quiescence htlcswitch: add drive method to quiescer --- htlcswitch/quiescer.go | 233 ++++++++++++++++++++++++++++++++++++ htlcswitch/quiescer_test.go | 102 ++++++++++++++++ 2 files changed, 335 insertions(+) create mode 100644 htlcswitch/quiescer.go create mode 100644 htlcswitch/quiescer_test.go diff --git a/htlcswitch/quiescer.go b/htlcswitch/quiescer.go new file mode 100644 index 00000000000..cf9ef3283aa --- /dev/null +++ b/htlcswitch/quiescer.go @@ -0,0 +1,233 @@ +//nolint:unused +package htlcswitch + +import ( + "fmt" + + "github.com/lightningnetwork/lnd/fn" + "github.com/lightningnetwork/lnd/lntypes" + "github.com/lightningnetwork/lnd/lnwire" +) + +var ( + // ErrStfuAlreadySent indicates that this channel has already sent an + // Stfu message for this negotiation. + ErrStfuAlreadySent = fmt.Errorf("stfu already sent") + + // ErrStfuAlreadyRcvd indicates that this channel has already received + // an Stfu message for this negotiation. + ErrStfuAlreadyRcvd = fmt.Errorf("stfu already received") + + // ErrNoQuiescenceInitiator indicates that the caller has requested the + // quiescence initiator for a channel that is not yet quiescent. + ErrNoQuiescenceInitiator = fmt.Errorf( + "indeterminate quiescence initiator: channel is not quiescent", + ) + + // ErrPendingRemoteUpdates indicates that we have received an Stfu while + // the remote party has issued updates that are not yet bilaterally + // committed. + ErrPendingRemoteUpdates = fmt.Errorf( + "stfu received with pending remote updates", + ) + + // ErrPendingLocalUpdates indicates that we are attempting to send an + // Stfu while we have issued updates that are not yet bilaterally + // committed. + ErrPendingLocalUpdates = fmt.Errorf( + "stfu send attempted with pending local updates", + ) +) + +type quiescerCfg struct { + // chanID marks what channel we are managing the state machine for. This + // is important because the quiescer is responsible for constructing the + // messages we send out and the ChannelID is a key field in that + // message. + chanID lnwire.ChannelID + + // channelInitiator indicates which ChannelParty originally opened the + // channel. This is used to break ties when both sides of the channel + // send Stfu claiming to be the initiator. + channelInitiator lntypes.ChannelParty + + // numPendingUpdates is a function that returns the number of pending + // originated by the party in the first argument that have yet to be + // committed to the commitment transaction held by the party in the + // second argument. + numPendingUpdates func(lntypes.ChannelParty, + lntypes.ChannelParty) uint64 + + // sendMsg is a function that can be used to send an Stfu message over + // the wire. + sendMsg func(lnwire.Stfu) error +} + +// quiescer is a state machine that tracks progression through the quiescence +// protocol. +type quiescer struct { + cfg quiescerCfg + + // localInit indicates whether our path through this state machine was + // initiated by our node. This can be true or false independently of + // remoteInit. + localInit bool + + // remoteInit indicates whether we received Stfu from our peer where the + // message indicated that the remote node believes it was the initiator. + // This can be true or false independently of localInit. + remoteInit bool + + // sent tracks whether or not we have emitted Stfu for sending. + sent bool + + // received tracks whether or not we have received Stfu from our peer. + received bool +} + +// newQuiescer creates a new quiescer for the given channel. +func newQuiescer(cfg quiescerCfg) quiescer { + return quiescer{ + cfg: cfg, + } +} + +// recvStfu is called when we receive an Stfu message from the remote. +func (q *quiescer) recvStfu(msg lnwire.Stfu) error { + // At the time of this writing, this check that we have already received + // an Stfu is not strictly necessary, according to the specification. + // However, it is fishy if we do and it is unclear how we should handle + // such a case so we will err on the side of caution. + if q.received { + return fmt.Errorf("%w for channel %v", ErrStfuAlreadyRcvd, + q.cfg.chanID) + } + + if !q.canRecvStfu() { + return fmt.Errorf("%w for channel %v", ErrPendingRemoteUpdates, + q.cfg.chanID) + } + + q.received = true + + // If the remote party sets the initiator bit to true then we will + // remember that they are making a claim to the initiator role. This + // does not necessarily mean they will get it, though. + q.remoteInit = msg.Initiator + + return nil +} + +// makeStfu is called when we are ready to send an Stfu message. It returns the +// Stfu message to be sent. +func (q *quiescer) makeStfu() fn.Result[lnwire.Stfu] { + if q.sent { + return fn.Errf[lnwire.Stfu]("%w for channel %v", + ErrStfuAlreadySent, q.cfg.chanID) + } + + if !q.canSendStfu() { + return fn.Errf[lnwire.Stfu]("%w for channel %v", + ErrPendingLocalUpdates, q.cfg.chanID) + } + + stfu := lnwire.Stfu{ + ChanID: q.cfg.chanID, + Initiator: q.localInit, + } + + return fn.Ok(stfu) +} + +// oweStfu returns true if we owe the other party an Stfu. We owe the remote an +// Stfu when we have received but not yet sent an Stfu, or we are the initiator +// but have not yet sent an Stfu. +func (q *quiescer) oweStfu() bool { + return q.received && !q.sent +} + +// needStfu returns true if the remote owes us an Stfu. They owe us an Stfu when +// we have sent but not yet received an Stfu. +func (q *quiescer) needStfu() bool { + return q.sent && !q.received +} + +// isQuiescent returns true if the state machine has been driven all the way to +// completion. If this returns true, processes that depend on channel quiescence +// may proceed. +func (q *quiescer) isQuiescent() bool { + return q.sent && q.received +} + +// quiescenceInitiator determines which ChannelParty is the initiator of +// quiescence for the purposes of downstream protocols. If the channel is not +// currently quiescent, this method will return ErrNoDownstreamLeader. +func (q *quiescer) quiescenceInitiator() fn.Result[lntypes.ChannelParty] { + switch { + case !q.isQuiescent(): + return fn.Err[lntypes.ChannelParty](ErrNoQuiescenceInitiator) + + case q.localInit && q.remoteInit: + // In the case of a tie, the channel initiator wins. + return fn.Ok(q.cfg.channelInitiator) + + case !q.localInit && !q.remoteInit: + // We assume it is impossible for both to be false if the + // channel is quiescent. + panic("impossible: one party must have initiated quiescence") + + case q.localInit: + return fn.Ok(lntypes.Local) + + case q.remoteInit: + return fn.Ok(lntypes.Remote) + } + + panic("impossible: non-exhaustive switch quiescer.downstreamLeader") +} + +// canSendUpdates returns true if we haven't yet sent an Stfu which would mark +// the end of our ability to send updates. +func (q *quiescer) canSendUpdates() bool { + return !q.sent && !q.localInit +} + +// canRecvUpdates returns true if we haven't yet received an Stfu which would +// mark the end of the remote's ability to send updates. +func (q *quiescer) canRecvUpdates() bool { + return !q.received +} + +// canSendStfu returns true if we can send an Stfu. +func (q *quiescer) canSendStfu() bool { + return q.cfg.numPendingUpdates(lntypes.Local, lntypes.Local) == 0 && + q.cfg.numPendingUpdates(lntypes.Local, lntypes.Remote) == 0 +} + +// canRecvStfu returns true if we can receive an Stfu. +func (q *quiescer) canRecvStfu() bool { + return q.cfg.numPendingUpdates(lntypes.Remote, lntypes.Local) == 0 && + q.cfg.numPendingUpdates(lntypes.Remote, lntypes.Remote) == 0 +} + +// drive drives the quiescence machine forward. It returns an error if the state +// machine is in an invalid state. +func (q *quiescer) drive() error { + if !q.oweStfu() || !q.canSendStfu() { + return nil + } + + stfu, err := q.makeStfu().Unpack() + if err != nil { + return err + } + + err = q.cfg.sendMsg(stfu) + if err != nil { + return err + } + + q.sent = true + + return nil +} diff --git a/htlcswitch/quiescer_test.go b/htlcswitch/quiescer_test.go new file mode 100644 index 00000000000..973e987d5b7 --- /dev/null +++ b/htlcswitch/quiescer_test.go @@ -0,0 +1,102 @@ +package htlcswitch + +import ( + "bytes" + "testing" + + "github.com/lightningnetwork/lnd/lntypes" + "github.com/lightningnetwork/lnd/lnwire" + "github.com/stretchr/testify/require" +) + +var cid = lnwire.ChannelID(bytes.Repeat([]byte{0x00}, 32)) + +type quiescerTestHarness struct { + pendingUpdates lntypes.Dual[uint64] + quiescer quiescer + conn <-chan lnwire.Stfu +} + +func initQuiescerTestHarness() *quiescerTestHarness { + conn := make(chan lnwire.Stfu, 1) + harness := &quiescerTestHarness{ + pendingUpdates: lntypes.Dual[uint64]{}, + conn: conn, + } + + harness.quiescer = newQuiescer(quiescerCfg{ + chanID: cid, + numPendingUpdates: func(whoseUpdate lntypes.ChannelParty, + _ lntypes.ChannelParty) uint64 { + + return harness.pendingUpdates.GetForParty(whoseUpdate) + }, + sendMsg: func(msg lnwire.Stfu) error { + conn <- msg + return nil + }, + }) + + return harness +} + +func TestQuiescerDoubleRecvInvalid(t *testing.T) { + harness := initQuiescerTestHarness() + + msg := lnwire.Stfu{ + ChanID: cid, + Initiator: true, + } + + err := harness.quiescer.recvStfu(msg) + require.NoError(t, err) + err = harness.quiescer.recvStfu(msg) + require.Error(t, err) +} + +func TestQuiescerPendingUpdatesRecvInvalid(t *testing.T) { + harness := initQuiescerTestHarness() + + msg := lnwire.Stfu{ + ChanID: cid, + Initiator: true, + } + + harness.pendingUpdates.SetForParty(lntypes.Remote, 1) + err := harness.quiescer.recvStfu(msg) + require.Error(t, err) +} + +func TestQuiescenceRemoteInit(t *testing.T) { + harness := initQuiescerTestHarness() + + msg := lnwire.Stfu{ + ChanID: cid, + Initiator: true, + } + + harness.pendingUpdates.SetForParty(lntypes.Local, 1) + + err := harness.quiescer.recvStfu(msg) + require.NoError(t, err) + + err = harness.quiescer.drive() + require.NoError(t, err) + + select { + case <-harness.conn: + t.Fatalf("stfu sent when not expected") + default: + } + + harness.pendingUpdates.SetForParty(lntypes.Local, 0) + err = harness.quiescer.drive() + require.NoError(t, err) + + select { + case msg := <-harness.conn: + require.False(t, msg.Initiator) + default: + t.Fatalf("stfu not sent when expected") + } +} From 9c4c65a0e9f08438280e9a22c621bf391f040453 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Mon, 11 Dec 2023 21:41:31 -0800 Subject: [PATCH 02/29] htlcswitch: implement stfu response htlcswitch: use quiescer drive method in link stfu implementation This commit amends the implementation presented in the last one by taking advantage of the quiescer drive method. This allows us to centralize quiescence logic better rather than diffusing it throughout the link. h/t @carlaKC --- htlcswitch/link.go | 53 +++++++++++++++++++++++++++++++++++++++ htlcswitch/linkfailure.go | 6 +++++ 2 files changed, 59 insertions(+) diff --git a/htlcswitch/link.go b/htlcswitch/link.go index e7766bc8150..77c4f369ca7 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -387,6 +387,10 @@ type channelLink struct { // our next CommitSig. incomingCommitHooks hookMap + // quiescer is the state machine that tracks where this channel is with + // respect to the quiescence protocol. + quiescer quiescer + wg sync.WaitGroup quit chan struct{} } @@ -460,6 +464,17 @@ func NewChannelLink(cfg ChannelLinkConfig, cfg.MaxFeeExposure = DefaultMaxFeeExposure } + qsm := newQuiescer(quiescerCfg{ + chanID: lnwire.NewChanIDFromOutPoint( + channel.ChannelPoint(), + ), + channelInitiator: channel.Initiator(), + numPendingUpdates: channel.NumPendingUpdates, + sendMsg: func(s lnwire.Stfu) error { + return cfg.Peer.SendMessage(false, &s) + }, + }) + return &channelLink{ cfg: cfg, channel: channel, @@ -469,6 +484,7 @@ func NewChannelLink(cfg ChannelLinkConfig, flushHooks: newHookMap(), outgoingCommitHooks: newHookMap(), incomingCommitHooks: newHookMap(), + quiescer: qsm, quit: make(chan struct{}), } } @@ -2316,6 +2332,13 @@ func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) { } } + // If we need to send out an Stfu, this would be the time to do + // so. + err = l.quiescer.drive() + if err != nil { + l.stfuFailf("%s", err.Error()) + } + // Now that we have finished processing the incoming CommitSig // and sent out our RevokeAndAck, we invoke the flushHooks if // the channel state is clean. @@ -2449,6 +2472,12 @@ func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) { // Update the mailbox's feerate as well. l.mailBox.SetFeeRate(fee) + case *lnwire.Stfu: + err := l.handleStfu(msg) + if err != nil { + l.stfuFailf("%s", err.Error()) + } + // In the case where we receive a warning message from our peer, just // log it and move on. We choose not to disconnect from our peer, // although we "MAY" do so according to the specification. @@ -2481,6 +2510,30 @@ func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) { } +// handleStfu implements the top-level logic for handling the Stfu message from +// our peer. +func (l *channelLink) handleStfu(stfu *lnwire.Stfu) error { + err := l.quiescer.recvStfu(*stfu) + if err != nil { + return err + } + + // If we can immediately send an Stfu response back, we will. + return l.quiescer.drive() +} + +// stfuFailf fails the link in the case where the requirements of the quiescence +// protocol are violated. In all cases we opt to drop the connection as only +// link state (as opposed to channel state) is affected. +func (l *channelLink) stfuFailf(format string, args ...interface{}) { + l.failf(LinkFailureError{ + code: ErrStfuViolation, + FailureAction: LinkFailureDisconnect, + PermanentFailure: false, + Warning: true, + }, format, args...) +} + // ackDownStreamPackets is responsible for removing htlcs from a link's mailbox // for packets delivered from server, and cleaning up any circuits closed by // signing a previous commitment txn. This method ensures that the circuits are diff --git a/htlcswitch/linkfailure.go b/htlcswitch/linkfailure.go index 47f8065f766..6d8fab8f74c 100644 --- a/htlcswitch/linkfailure.go +++ b/htlcswitch/linkfailure.go @@ -51,6 +51,12 @@ const ( // circuit map. This is non-fatal and will resolve itself (usually // within several minutes). ErrCircuitError + + // ErrStfuViolation indicates that the quiescence protocol has been + // violated, either because Stfu has been sent/received at an invalid + // time, or that an update has been sent/received while the channel is + // quiesced. + ErrStfuViolation ) // LinkFailureAction is an enum-like type that describes the action that should From 52ad0ab4456ccc4dad2d7c9719c05f5070d0e746 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Fri, 9 Aug 2024 15:57:06 -0700 Subject: [PATCH 03/29] lnwire: add IsChannelUpdate function to distinguish channel updates --- lnwire/message.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/lnwire/message.go b/lnwire/message.go index a758db000db..68b09692e55 100644 --- a/lnwire/message.go +++ b/lnwire/message.go @@ -63,6 +63,25 @@ const ( MsgKickoffSig = 777 ) +// IsChannelUpdate is a filter function that discerns channel update messages +// from the other messages in the Lightning Network Protocol. +func (t MessageType) IsChannelUpdate() bool { + switch t { + case MsgUpdateAddHTLC: + return true + case MsgUpdateFulfillHTLC: + return true + case MsgUpdateFailHTLC: + return true + case MsgUpdateFailMalformedHTLC: + return true + case MsgUpdateFee: + return true + default: + return false + } +} + // ErrorEncodeMessage is used when failed to encode the message payload. func ErrorEncodeMessage(err error) error { return fmt.Errorf("failed to encode message to buffer, got %w", err) From 2beb8ebb12d03001406fad06b0d68ce33212d421 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Dec 2023 13:11:05 -0800 Subject: [PATCH 04/29] htlcswitch: bounce packets when quiescent --- htlcswitch/link.go | 94 +++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 85 insertions(+), 9 deletions(-) diff --git a/htlcswitch/link.go b/htlcswitch/link.go index 77c4f369ca7..f04930e61cf 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -391,6 +391,19 @@ type channelLink struct { // respect to the quiescence protocol. quiescer quiescer + // stateQueries is a channel that is used to query the current state of + // the channelLink in a thread-safe manner by delegating the state reads + // to the main event loop. + // + // WARNING: closures passed over this channel must be safe to run in the + // main event loop as that is the context in which they will be + // executed. The most obvious problem that can arise from using this is + // if the closure itself attempts to send over this chan. This will + // create deadlock opportunities as we cannot ensure that there is + // available buffer space on this channel in the presence of other + // threads calling APIs that depend on this component. + stateQueries chan func() + wg sync.WaitGroup quit chan struct{} } @@ -485,6 +498,7 @@ func NewChannelLink(cfg ChannelLinkConfig, outgoingCommitHooks: newHookMap(), incomingCommitHooks: newHookMap(), quiescer: qsm, + stateQueries: make(chan func(), 1), quit: make(chan struct{}), } } @@ -637,9 +651,22 @@ func (l *channelLink) WaitForShutdown() { // actively accept requests to forward HTLC's. We're able to forward HTLC's if // we are eligible to update AND the channel isn't currently flushing the // outgoing half of the channel. +// +// NOTE: MUST NOT be called from the main event loop. func (l *channelLink) EligibleToForward() bool { - return l.EligibleToUpdate() && - !l.IsFlushing(Outgoing) + return runInEventLoop(l, fn.Unit{}, func(_ fn.Unit) bool { + return l.eligibleToForward() + }).UnwrapLeftOr(false) +} + +// eligibleToForward returns a bool indicating if the channel is able to +// actively accept requests to forward HTLC's. We're able to forward HTLC's if +// we are eligible to update AND the channel isn't currently flushing the +// outgoing half of the channel. +// +// NOTE: MUST be called from the main event loop. +func (l *channelLink) eligibleToForward() bool { + return l.eligibleToUpdate() && !l.IsFlushing(Outgoing) } // EligibleToUpdate returns a bool indicating if the channel is able to update @@ -647,10 +674,26 @@ func (l *channelLink) EligibleToForward() bool { // party's next revocation point. Otherwise, we can't initiate new channel // state. We also require that the short channel ID not be the all-zero source // ID, meaning that the channel has had its ID finalized. +// +// NOTE: MUST be called from the main event loop. func (l *channelLink) EligibleToUpdate() bool { + return runInEventLoop(l, fn.Unit{}, func(_ fn.Unit) bool { + return l.eligibleToUpdate() + }).UnwrapLeftOr(false) +} + +// eligibleToUpdate returns a bool indicating if the channel is able to update +// channel state. We're able to update channel state if we know the remote +// party's next revocation point. Otherwise, we can't initiate new channel +// state. We also require that the short channel ID not be the all-zero source +// ID, meaning that the channel has had its ID finalized. +// +// NOTE: MUST be called from the main event loop. +func (l *channelLink) eligibleToUpdate() bool { return l.channel.RemoteNextRevocation() != nil && - l.ShortChanID() != hop.Source && - l.isReestablished() + l.channel.ShortChanID() != hop.Source && + l.isReestablished() && + l.quiescer.canSendUpdates() } // EnableAdds sets the ChannelUpdateHandler state to allow UpdateAddHtlc's in @@ -1457,6 +1500,9 @@ func (l *channelLink) htlcManager() { ) } + case runQuery := <-l.stateQueries: + runQuery() + case <-l.quit: return } @@ -1591,13 +1637,14 @@ func (l *channelLink) handleDownstreamUpdateAdd(pkt *htlcPacket) error { return errors.New("not an UpdateAddHTLC packet") } - // If we are flushing the link in the outgoing direction we can't add - // new htlcs to the link and we need to bounce it - if l.IsFlushing(Outgoing) { + // If we are flushing the link in the outgoing direction or we have + // already sent Stfu, then we can't add new htlcs to the link and we + // need to bounce it. + if l.IsFlushing(Outgoing) || !l.quiescer.canSendUpdates() { l.mailBox.FailAdd(pkt) return NewDetailedLinkError( - &lnwire.FailPermanentChannelFailure{}, + &lnwire.FailTemporaryChannelFailure{}, OutgoingFailureLinkNotEligible, ) } @@ -1700,6 +1747,15 @@ func (l *channelLink) handleDownstreamUpdateAdd(pkt *htlcPacket) error { // // TODO(roasbeef): add sync ntfn to ensure switch always has consistent view? func (l *channelLink) handleDownstreamPkt(pkt *htlcPacket) { + if pkt.htlc.MsgType().IsChannelUpdate() && + !l.quiescer.canRecvUpdates() { + + l.log.Warnf("unable to process channel update. "+ + "ChannelID=%v is quiescent.", l.ChanID) + + return + } + switch htlc := pkt.htlc.(type) { case *lnwire.UpdateAddHTLC: // Handle add message. The returned error can be ignored, @@ -3307,7 +3363,7 @@ func (l *channelLink) updateChannelFee(feePerKw chainfee.SatPerKWeight) error { // We skip sending the UpdateFee message if the channel is not // currently eligible to forward messages. - if !l.EligibleToUpdate() { + if !l.eligibleToUpdate() { l.log.Debugf("skipping fee update for inactive channel") return nil } @@ -4204,3 +4260,23 @@ func (l *channelLink) CommitmentCustomBlob() fn.Option[tlv.Blob] { return l.channel.LocalCommitmentBlob() } + +// runInEventLoop is a helper function that runs the given function on the +// provided argument inside the main event loop of the channel link. +func runInEventLoop[A, B any](l *channelLink, a A, f func(A) B) fn.Result[B] { + req, res := fn.NewReq[A, B](a) + + // Build a closure that reads the state we care about. This is done so + // we can get a fully consistent read of state. + query := func() { + req.Resolve(f(a)) + } + + // Chuck that closure into the event loop to ensure it is executed by + // the thread that writes this state. + if !fn.SendOrQuit(l.stateQueries, query, l.quit) { + return fn.Errf[B]("channel link is shutting down") + } + + return fn.NewResult(fn.RecvResp(res, nil, l.quit)) +} From a3607eabeb4fb64c8c7b9470a7f4cb73e9edacb6 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Dec 2023 13:16:58 -0800 Subject: [PATCH 05/29] htlcswitch: drop connection if link updates after stfu --- htlcswitch/link.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/htlcswitch/link.go b/htlcswitch/link.go index f04930e61cf..ac5eb90f020 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -2000,6 +2000,13 @@ func (l *channelLink) cleanupSpuriousResponse(pkt *htlcPacket) { // updates from the upstream peer. The upstream peer is the peer whom we have a // direct channel with, updating our respective commitment chains. func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) { + // First check if the message is an update and we are capable of + // receiving updates right now. + if msg.MsgType().IsChannelUpdate() && !l.quiescer.canRecvUpdates() { + l.stfuFailf("update received after stfu: %T", msg) + return + } + switch msg := msg.(type) { case *lnwire.UpdateAddHTLC: if l.IsFlushing(Incoming) { From d8a11b132d02af6fde8ccb2f55d79f0cd9f1748c Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Dec 2023 14:15:59 -0800 Subject: [PATCH 06/29] lnwire: signal that we support quiescence --- lnwire/features.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/lnwire/features.go b/lnwire/features.go index c597b039883..c07add445be 100644 --- a/lnwire/features.go +++ b/lnwire/features.go @@ -171,6 +171,16 @@ const ( // sender-generated preimages according to BOLT XX. AMPOptional FeatureBit = 31 + // QuiescenceRequired is a required feature bit that denotes that a + // connection established with this node must support the quiescence + // protocol if it wants to have a channel relationship. + QuiescenceRequired FeatureBit = 34 + + // QuiescenceOptional is an optional feature bit that denotes that a + // connection established with this node is permitted to use the + // quiescence protocol. + QuiescenceOptional FeatureBit = 35 + // ExplicitChannelTypeRequired is a required bit that denotes that a // connection established with this node is to use explicit channel // commitment types for negotiation instead of the existing implicit @@ -327,6 +337,8 @@ var Features = map[FeatureBit]string{ WumboChannelsOptional: "wumbo-channels", AMPRequired: "amp", AMPOptional: "amp", + QuiescenceRequired: "quiescence", + QuiescenceOptional: "quiescence", PaymentMetadataOptional: "payment-metadata", PaymentMetadataRequired: "payment-metadata", ExplicitChannelTypeOptional: "explicit-commitment-type", From b930ec7666c5e5904afde50f08820dff70dc6ed5 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Mar 2024 12:05:58 -0700 Subject: [PATCH 07/29] lnrpc add new RPC 'Quiesce' to protobuf definitions --- lnrpc/devrpc/dev.pb.go | 173 ++++++++++++++++++++++++++++++---- lnrpc/devrpc/dev.pb.json.go | 25 +++++ lnrpc/devrpc/dev.proto | 15 +++ lnrpc/devrpc/dev.swagger.json | 27 ++++++ lnrpc/devrpc/dev_grpc.pb.go | 42 +++++++++ lntest/rpc/harness_rpc.go | 4 +- 6 files changed, 265 insertions(+), 21 deletions(-) diff --git a/lnrpc/devrpc/dev.pb.go b/lnrpc/devrpc/dev.pb.go index 890f4e31714..df8a310dad5 100644 --- a/lnrpc/devrpc/dev.pb.go +++ b/lnrpc/devrpc/dev.pb.go @@ -59,6 +59,100 @@ func (*ImportGraphResponse) Descriptor() ([]byte, []int) { return file_devrpc_dev_proto_rawDescGZIP(), []int{0} } +type QuiescenceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ChanId *lnrpc.ChannelPoint `protobuf:"bytes,1,opt,name=chan_id,json=chanId,proto3" json:"chan_id,omitempty"` +} + +func (x *QuiescenceRequest) Reset() { + *x = QuiescenceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_devrpc_dev_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QuiescenceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuiescenceRequest) ProtoMessage() {} + +func (x *QuiescenceRequest) ProtoReflect() protoreflect.Message { + mi := &file_devrpc_dev_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuiescenceRequest.ProtoReflect.Descriptor instead. +func (*QuiescenceRequest) Descriptor() ([]byte, []int) { + return file_devrpc_dev_proto_rawDescGZIP(), []int{1} +} + +func (x *QuiescenceRequest) GetChanId() *lnrpc.ChannelPoint { + if x != nil { + return x.ChanId + } + return nil +} + +type QuiescenceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Initiator bool `protobuf:"varint,1,opt,name=initiator,proto3" json:"initiator,omitempty"` +} + +func (x *QuiescenceResponse) Reset() { + *x = QuiescenceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_devrpc_dev_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QuiescenceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuiescenceResponse) ProtoMessage() {} + +func (x *QuiescenceResponse) ProtoReflect() protoreflect.Message { + mi := &file_devrpc_dev_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuiescenceResponse.ProtoReflect.Descriptor instead. +func (*QuiescenceResponse) Descriptor() ([]byte, []int) { + return file_devrpc_dev_proto_rawDescGZIP(), []int{2} +} + +func (x *QuiescenceResponse) GetInitiator() bool { + if x != nil { + return x.Initiator + } + return false +} + var File_devrpc_dev_proto protoreflect.FileDescriptor var file_devrpc_dev_proto_rawDesc = []byte{ @@ -66,15 +160,26 @@ var file_devrpc_dev_proto_rawDesc = []byte{ 0x74, 0x6f, 0x12, 0x06, 0x64, 0x65, 0x76, 0x72, 0x70, 0x63, 0x1a, 0x0f, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x6e, 0x69, 0x6e, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x32, 0x46, 0x0a, 0x03, 0x44, 0x65, 0x76, 0x12, 0x3f, 0x0a, 0x0b, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x13, 0x2e, 0x6c, 0x6e, 0x72, 0x70, 0x63, - 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x47, 0x72, 0x61, 0x70, 0x68, 0x1a, 0x1b, 0x2e, - 0x64, 0x65, 0x76, 0x72, 0x70, 0x63, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x47, 0x72, 0x61, - 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x2e, 0x5a, 0x2c, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x6e, 0x69, - 0x6e, 0x67, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x2f, 0x6c, 0x6e, 0x64, 0x2f, 0x6c, 0x6e, - 0x72, 0x70, 0x63, 0x2f, 0x64, 0x65, 0x76, 0x72, 0x70, 0x63, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x73, 0x65, 0x22, 0x41, 0x0a, 0x11, 0x51, 0x75, 0x69, 0x65, 0x73, 0x63, 0x65, 0x6e, 0x63, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6c, 0x6e, 0x72, 0x70, 0x63, + 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x06, 0x63, + 0x68, 0x61, 0x6e, 0x49, 0x64, 0x22, 0x32, 0x0a, 0x12, 0x51, 0x75, 0x69, 0x65, 0x73, 0x63, 0x65, + 0x6e, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x69, + 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, + 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, 0x32, 0x88, 0x01, 0x0a, 0x03, 0x44, 0x65, + 0x76, 0x12, 0x3f, 0x0a, 0x0b, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, + 0x12, 0x13, 0x2e, 0x6c, 0x6e, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x47, 0x72, 0x61, 0x70, 0x68, 0x1a, 0x1b, 0x2e, 0x64, 0x65, 0x76, 0x72, 0x70, 0x63, 0x2e, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x40, 0x0a, 0x07, 0x51, 0x75, 0x69, 0x65, 0x73, 0x63, 0x65, 0x12, 0x19, 0x2e, + 0x64, 0x65, 0x76, 0x72, 0x70, 0x63, 0x2e, 0x51, 0x75, 0x69, 0x65, 0x73, 0x63, 0x65, 0x6e, 0x63, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x64, 0x65, 0x76, 0x72, 0x70, + 0x63, 0x2e, 0x51, 0x75, 0x69, 0x65, 0x73, 0x63, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x2e, 0x5a, 0x2c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x6e, 0x69, 0x6e, 0x67, 0x6e, 0x65, 0x74, 0x77, + 0x6f, 0x72, 0x6b, 0x2f, 0x6c, 0x6e, 0x64, 0x2f, 0x6c, 0x6e, 0x72, 0x70, 0x63, 0x2f, 0x64, 0x65, + 0x76, 0x72, 0x70, 0x63, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -89,19 +194,25 @@ func file_devrpc_dev_proto_rawDescGZIP() []byte { return file_devrpc_dev_proto_rawDescData } -var file_devrpc_dev_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_devrpc_dev_proto_msgTypes = make([]protoimpl.MessageInfo, 3) var file_devrpc_dev_proto_goTypes = []interface{}{ (*ImportGraphResponse)(nil), // 0: devrpc.ImportGraphResponse - (*lnrpc.ChannelGraph)(nil), // 1: lnrpc.ChannelGraph + (*QuiescenceRequest)(nil), // 1: devrpc.QuiescenceRequest + (*QuiescenceResponse)(nil), // 2: devrpc.QuiescenceResponse + (*lnrpc.ChannelPoint)(nil), // 3: lnrpc.ChannelPoint + (*lnrpc.ChannelGraph)(nil), // 4: lnrpc.ChannelGraph } var file_devrpc_dev_proto_depIdxs = []int32{ - 1, // 0: devrpc.Dev.ImportGraph:input_type -> lnrpc.ChannelGraph - 0, // 1: devrpc.Dev.ImportGraph:output_type -> devrpc.ImportGraphResponse - 1, // [1:2] is the sub-list for method output_type - 0, // [0:1] is the sub-list for method input_type - 0, // [0:0] is the sub-list for extension type_name - 0, // [0:0] is the sub-list for extension extendee - 0, // [0:0] is the sub-list for field type_name + 3, // 0: devrpc.QuiescenceRequest.chan_id:type_name -> lnrpc.ChannelPoint + 4, // 1: devrpc.Dev.ImportGraph:input_type -> lnrpc.ChannelGraph + 1, // 2: devrpc.Dev.Quiesce:input_type -> devrpc.QuiescenceRequest + 0, // 3: devrpc.Dev.ImportGraph:output_type -> devrpc.ImportGraphResponse + 2, // 4: devrpc.Dev.Quiesce:output_type -> devrpc.QuiescenceResponse + 3, // [3:5] is the sub-list for method output_type + 1, // [1:3] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 1, // [1:1] is the sub-list for extension extendee + 0, // [0:1] is the sub-list for field type_name } func init() { file_devrpc_dev_proto_init() } @@ -122,6 +233,30 @@ func file_devrpc_dev_proto_init() { return nil } } + file_devrpc_dev_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QuiescenceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_devrpc_dev_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QuiescenceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -129,7 +264,7 @@ func file_devrpc_dev_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_devrpc_dev_proto_rawDesc, NumEnums: 0, - NumMessages: 1, + NumMessages: 3, NumExtensions: 0, NumServices: 1, }, diff --git a/lnrpc/devrpc/dev.pb.json.go b/lnrpc/devrpc/dev.pb.json.go index 954917a1a3f..2163a13de65 100644 --- a/lnrpc/devrpc/dev.pb.json.go +++ b/lnrpc/devrpc/dev.pb.json.go @@ -46,4 +46,29 @@ func RegisterDevJSONCallbacks(registry map[string]func(ctx context.Context, } callback(string(respBytes), nil) } + + registry["devrpc.Dev.Quiesce"] = func(ctx context.Context, + conn *grpc.ClientConn, reqJSON string, callback func(string, error)) { + + req := &QuiescenceRequest{} + err := marshaler.Unmarshal([]byte(reqJSON), req) + if err != nil { + callback("", err) + return + } + + client := NewDevClient(conn) + resp, err := client.Quiesce(ctx, req) + if err != nil { + callback("", err) + return + } + + respBytes, err := marshaler.Marshal(resp) + if err != nil { + callback("", err) + return + } + callback(string(respBytes), nil) + } } diff --git a/lnrpc/devrpc/dev.proto b/lnrpc/devrpc/dev.proto index 502fbadc8b9..4016ace2cf5 100644 --- a/lnrpc/devrpc/dev.proto +++ b/lnrpc/devrpc/dev.proto @@ -30,7 +30,22 @@ service Dev { used for development. */ rpc ImportGraph (lnrpc.ChannelGraph) returns (ImportGraphResponse); + + /* + Quiesce instructs a channel to initiate the quiescence (stfu) protocol. This + RPC is for testing purposes only. The commit that adds it will be removed + once interop is confirmed. + */ + rpc Quiesce (QuiescenceRequest) returns (QuiescenceResponse); } message ImportGraphResponse { } + +message QuiescenceRequest { + lnrpc.ChannelPoint chan_id = 1; +} + +message QuiescenceResponse { + bool initiator = 1; +} diff --git a/lnrpc/devrpc/dev.swagger.json b/lnrpc/devrpc/dev.swagger.json index 16e16d7be8e..01caca3680f 100644 --- a/lnrpc/devrpc/dev.swagger.json +++ b/lnrpc/devrpc/dev.swagger.json @@ -54,6 +54,14 @@ "devrpcImportGraphResponse": { "type": "object" }, + "devrpcQuiescenceResponse": { + "type": "object", + "properties": { + "initiator": { + "type": "boolean" + } + } + }, "lnrpcChannelEdge": { "type": "object", "properties": { @@ -116,6 +124,25 @@ }, "description": "Returns a new instance of the directed channel graph." }, + "lnrpcChannelPoint": { + "type": "object", + "properties": { + "funding_txid_bytes": { + "type": "string", + "format": "byte", + "description": "Txid of the funding transaction. When using REST, this field must be\nencoded as base64." + }, + "funding_txid_str": { + "type": "string", + "description": "Hex-encoded string representing the byte-reversed hash of the funding\ntransaction." + }, + "output_index": { + "type": "integer", + "format": "int64", + "title": "The index of the output of the funding transaction" + } + } + }, "lnrpcFeature": { "type": "object", "properties": { diff --git a/lnrpc/devrpc/dev_grpc.pb.go b/lnrpc/devrpc/dev_grpc.pb.go index 1744c12a3ba..1eb6266fbe4 100644 --- a/lnrpc/devrpc/dev_grpc.pb.go +++ b/lnrpc/devrpc/dev_grpc.pb.go @@ -23,6 +23,10 @@ type DevClient interface { // ImportGraph imports a ChannelGraph into the graph database. Should only be // used for development. ImportGraph(ctx context.Context, in *lnrpc.ChannelGraph, opts ...grpc.CallOption) (*ImportGraphResponse, error) + // Quiesce instructs a channel to initiate the quiescence (stfu) protocol. This + // RPC is for testing purposes only. The commit that adds it will be removed + // once interop is confirmed. + Quiesce(ctx context.Context, in *QuiescenceRequest, opts ...grpc.CallOption) (*QuiescenceResponse, error) } type devClient struct { @@ -42,6 +46,15 @@ func (c *devClient) ImportGraph(ctx context.Context, in *lnrpc.ChannelGraph, opt return out, nil } +func (c *devClient) Quiesce(ctx context.Context, in *QuiescenceRequest, opts ...grpc.CallOption) (*QuiescenceResponse, error) { + out := new(QuiescenceResponse) + err := c.cc.Invoke(ctx, "/devrpc.Dev/Quiesce", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DevServer is the server API for Dev service. // All implementations must embed UnimplementedDevServer // for forward compatibility @@ -50,6 +63,10 @@ type DevServer interface { // ImportGraph imports a ChannelGraph into the graph database. Should only be // used for development. ImportGraph(context.Context, *lnrpc.ChannelGraph) (*ImportGraphResponse, error) + // Quiesce instructs a channel to initiate the quiescence (stfu) protocol. This + // RPC is for testing purposes only. The commit that adds it will be removed + // once interop is confirmed. + Quiesce(context.Context, *QuiescenceRequest) (*QuiescenceResponse, error) mustEmbedUnimplementedDevServer() } @@ -60,6 +77,9 @@ type UnimplementedDevServer struct { func (UnimplementedDevServer) ImportGraph(context.Context, *lnrpc.ChannelGraph) (*ImportGraphResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ImportGraph not implemented") } +func (UnimplementedDevServer) Quiesce(context.Context, *QuiescenceRequest) (*QuiescenceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Quiesce not implemented") +} func (UnimplementedDevServer) mustEmbedUnimplementedDevServer() {} // UnsafeDevServer may be embedded to opt out of forward compatibility for this service. @@ -91,6 +111,24 @@ func _Dev_ImportGraph_Handler(srv interface{}, ctx context.Context, dec func(int return interceptor(ctx, in, info, handler) } +func _Dev_Quiesce_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QuiescenceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DevServer).Quiesce(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/devrpc.Dev/Quiesce", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DevServer).Quiesce(ctx, req.(*QuiescenceRequest)) + } + return interceptor(ctx, in, info, handler) +} + // Dev_ServiceDesc is the grpc.ServiceDesc for Dev service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -102,6 +140,10 @@ var Dev_ServiceDesc = grpc.ServiceDesc{ MethodName: "ImportGraph", Handler: _Dev_ImportGraph_Handler, }, + { + MethodName: "Quiesce", + Handler: _Dev_Quiesce_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "devrpc/dev.proto", diff --git a/lntest/rpc/harness_rpc.go b/lntest/rpc/harness_rpc.go index 0640581dcd8..2e08a84947c 100644 --- a/lntest/rpc/harness_rpc.go +++ b/lntest/rpc/harness_rpc.go @@ -43,7 +43,7 @@ type HarnessRPC struct { ChainKit chainrpc.ChainKitClient NeutrinoKit neutrinorpc.NeutrinoKitClient Peer peersrpc.PeersClient - DevRPC devrpc.DevClient + Dev devrpc.DevClient // Name is the HarnessNode's name. Name string @@ -75,7 +75,7 @@ func NewHarnessRPC(ctxt context.Context, t *testing.T, c *grpc.ClientConn, ChainKit: chainrpc.NewChainKitClient(c), NeutrinoKit: neutrinorpc.NewNeutrinoKitClient(c), Peer: peersrpc.NewPeersClient(c), - DevRPC: devrpc.NewDevClient(c), + Dev: devrpc.NewDevClient(c), Name: name, } From 998d39b17e9f8a0617fdaf1f5a89a030327112c2 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Mar 2024 12:16:01 -0700 Subject: [PATCH 08/29] lnd: implement new Quiesce RPC with link operation stub --- lnrpc/devrpc/config_active.go | 2 ++ lnrpc/devrpc/dev_server.go | 26 ++++++++++++++++++++++++++ subrpcserver_config.go | 4 ++++ 3 files changed, 32 insertions(+) diff --git a/lnrpc/devrpc/config_active.go b/lnrpc/devrpc/config_active.go index 6fc274f2e90..da5cd5be97b 100644 --- a/lnrpc/devrpc/config_active.go +++ b/lnrpc/devrpc/config_active.go @@ -6,6 +6,7 @@ package devrpc import ( "github.com/btcsuite/btcd/chaincfg" "github.com/lightningnetwork/lnd/channeldb" + "github.com/lightningnetwork/lnd/htlcswitch" ) // Config is the primary configuration struct for the DEV RPC server. It @@ -16,4 +17,5 @@ import ( type Config struct { ActiveNetParams *chaincfg.Params GraphDB *channeldb.ChannelGraph + Switch *htlcswitch.Switch } diff --git a/lnrpc/devrpc/dev_server.go b/lnrpc/devrpc/dev_server.go index 662c0d08d9f..4b7ebcd1e6e 100644 --- a/lnrpc/devrpc/dev_server.go +++ b/lnrpc/devrpc/dev_server.go @@ -40,6 +40,10 @@ var ( Entity: "offchain", Action: "write", }}, + "/devrpc.Dev/Quiesce": {{ + Entity: "offchain", + Action: "write", + }}, } ) @@ -342,3 +346,25 @@ func (s *Server) ImportGraph(ctx context.Context, return &ImportGraphResponse{}, nil } + +// Quiesce initiates the quiescence process for the channel with the given +// channel ID. This method will block until the channel is fully quiesced. +func (s *Server) Quiesce(_ context.Context, in *QuiescenceRequest) ( + *QuiescenceResponse, error) { + + txid, err := lnrpc.GetChanPointFundingTxid(in.ChanId) + if err != nil { + return nil, err + } + + op := wire.NewOutPoint(txid, in.ChanId.OutputIndex) + cid := lnwire.NewChanIDFromOutPoint(*op) + _, err = s.cfg.Switch.GetLink(cid) + if err != nil { + return nil, err + } + + // TODO(proofofkeags): Add Link operation for initiating quiescence and + // implement the rest of this in those terms + return nil, fmt.Errorf("TODO(proofofkeags): Implement") +} diff --git a/subrpcserver_config.go b/subrpcserver_config.go index 8ea5b0ab397..041e9755087 100644 --- a/subrpcserver_config.go +++ b/subrpcserver_config.go @@ -346,6 +346,10 @@ func (s *subRPCServerConfigs) PopulateDependencies(cfg *Config, reflect.ValueOf(graphDB), ) + subCfgValue.FieldByName("Switch").Set( + reflect.ValueOf(htlcSwitch), + ) + case *peersrpc.Config: subCfgValue := extractReflectValue(subCfg) From 7fc6c2fc4a888ff306a2bc95984dce04fa5c0d33 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Mar 2024 11:43:48 -0700 Subject: [PATCH 09/29] htlcswitch: add link operation for initiating quiescence --- htlcswitch/interfaces.go | 12 ++++++++++++ htlcswitch/link.go | 16 ++++++++++++++++ htlcswitch/mock.go | 8 ++++++++ peer/test_utils.go | 9 +++++++++ 4 files changed, 45 insertions(+) diff --git a/htlcswitch/interfaces.go b/htlcswitch/interfaces.go index 3dd70247d29..a3f4ff9a9a3 100644 --- a/htlcswitch/interfaces.go +++ b/htlcswitch/interfaces.go @@ -170,6 +170,18 @@ type ChannelUpdateHandler interface { // will only ever be called once. If no CommitSig is owed in the // argument's LinkDirection, then we will call this hook immediately. OnCommitOnce(LinkDirection, func()) + + // InitStfu allows us to initiate quiescence on this link. It returns + // a receive only channel that will block until quiescence has been + // achieved, or definitively fails. The return value is the + // ChannelParty who holds the role of initiator or Err if the operation + // fails. + // + // This operation has been added to allow channels to be quiesced via + // RPC. It may be removed or reworked in the future as RPC initiated + // quiescence is a holdover until we have downstream protocols that use + // it. + InitStfu() <-chan fn.Result[lntypes.ChannelParty] } // CommitHookID is a value that is used to uniquely identify hooks in the diff --git a/htlcswitch/link.go b/htlcswitch/link.go index ac5eb90f020..a3145a9971a 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -757,6 +757,22 @@ func (l *channelLink) OnCommitOnce(direction LinkDirection, hook func()) { } } +// InitStfu allows us to initiate quiescence on this link. It returns a receive +// only channel that will block until quiescence has been achieved, or +// definitively fails. +// +// This operation has been added to allow channels to be quiesced via RPC. It +// may be removed or reworked in the future as RPC initiated quiescence is a +// holdover until we have downstream protocols that use it. +func (l *channelLink) InitStfu() <-chan fn.Result[lntypes.ChannelParty] { + // TODO(proofofkeags): Implement + c := make(chan fn.Result[lntypes.ChannelParty], 1) + + c <- fn.Errf[lntypes.ChannelParty]("InitStfu not yet implemented") + + return c +} + // isReestablished returns true if the link has successfully completed the // channel reestablishment dance. func (l *channelLink) isReestablished() bool { diff --git a/htlcswitch/mock.go b/htlcswitch/mock.go index 6de60b38a16..37bf4c6ef2d 100644 --- a/htlcswitch/mock.go +++ b/htlcswitch/mock.go @@ -950,6 +950,14 @@ func (f *mockChannelLink) OnFlushedOnce(func()) { func (f *mockChannelLink) OnCommitOnce(LinkDirection, func()) { // TODO(proofofkeags): Implement } +func (f *mockChannelLink) InitStfu() <-chan fn.Result[lntypes.ChannelParty] { + // TODO(proofofkeags): Implement + c := make(chan fn.Result[lntypes.ChannelParty], 1) + + c <- fn.Errf[lntypes.ChannelParty]("InitStfu not implemented") + + return c +} func (f *mockChannelLink) FundingCustomBlob() fn.Option[tlv.Blob] { return fn.None[tlv.Blob]() diff --git a/peer/test_utils.go b/peer/test_utils.go index 231a476ce17..ab7d24a2287 100644 --- a/peer/test_utils.go +++ b/peer/test_utils.go @@ -24,6 +24,7 @@ import ( "github.com/lightningnetwork/lnd/keychain" "github.com/lightningnetwork/lnd/lntest/channels" "github.com/lightningnetwork/lnd/lntest/mock" + "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwallet" "github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/lnwire" @@ -473,6 +474,14 @@ func (m *mockUpdateHandler) OnCommitOnce( hook() } +func (m *mockUpdateHandler) InitStfu() <-chan fn.Result[lntypes.ChannelParty] { + // TODO(proofofkeags): Implement + c := make(chan fn.Result[lntypes.ChannelParty], 1) + + c <- fn.Errf[lntypes.ChannelParty]("InitStfu not yet implemented") + + return c +} func newMockConn(t *testing.T, expectedMessages int) *mockMessageConn { return &mockMessageConn{ From c097d20abd191756284b60f6c3c2e96e598da04c Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Mar 2024 12:30:33 -0700 Subject: [PATCH 10/29] lnd: finish Quiesce implementation using new link op --- lnrpc/devrpc/dev_server.go | 32 +++++++++++++++++++++++++++----- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/lnrpc/devrpc/dev_server.go b/lnrpc/devrpc/dev_server.go index 4b7ebcd1e6e..4957ed93165 100644 --- a/lnrpc/devrpc/dev_server.go +++ b/lnrpc/devrpc/dev_server.go @@ -18,8 +18,10 @@ import ( "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" "github.com/lightningnetwork/lnd/channeldb" "github.com/lightningnetwork/lnd/channeldb/models" + "github.com/lightningnetwork/lnd/fn" "github.com/lightningnetwork/lnd/lncfg" "github.com/lightningnetwork/lnd/lnrpc" + "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwire" "google.golang.org/grpc" "gopkg.in/macaroon-bakery.v2/bakery" @@ -60,6 +62,7 @@ type ServerShell struct { type Server struct { started int32 // To be used atomically. shutdown int32 // To be used atomically. + quit chan struct{} // Required by the grpc-gateway/v2 library for forward compatibility. // Must be after the atomically used variables to not break struct @@ -82,7 +85,8 @@ func New(cfg *Config) (*Server, lnrpc.MacaroonPerms, error) { // We don't create any new macaroons for this subserver, instead reuse // existing onchain/offchain permissions. server := &Server{ - cfg: cfg, + quit: make(chan struct{}), + cfg: cfg, } return server, macPermissions, nil @@ -107,6 +111,8 @@ func (s *Server) Stop() error { return nil } + close(s.quit) + return nil } @@ -359,12 +365,28 @@ func (s *Server) Quiesce(_ context.Context, in *QuiescenceRequest) ( op := wire.NewOutPoint(txid, in.ChanId.OutputIndex) cid := lnwire.NewChanIDFromOutPoint(*op) - _, err = s.cfg.Switch.GetLink(cid) + ln, err := s.cfg.Switch.GetLink(cid) if err != nil { return nil, err } - // TODO(proofofkeags): Add Link operation for initiating quiescence and - // implement the rest of this in those terms - return nil, fmt.Errorf("TODO(proofofkeags): Implement") + select { + case result := <-ln.InitStfu(): + mkResp := func(b lntypes.ChannelParty) *QuiescenceResponse { + return &QuiescenceResponse{ + Initiator: b.IsLocal(), + } + } + + newEither := fn.MapLeft[lntypes.ChannelParty, error]( + mkResp, + )(result.Either) + + return fn.Result[*QuiescenceResponse]{ + Either: newEither, + }.Unpack() + + case <-s.quit: + return nil, fmt.Errorf("server shutting down") + } } From 66ecd4298ff948168b47a503958d27ccedc17630 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Mar 2024 12:32:43 -0700 Subject: [PATCH 11/29] htlcswitch: implement InitStfu link operation --- htlcswitch/link.go | 30 +++++++++++++++++++++---- htlcswitch/quiescer.go | 45 ++++++++++++++++++++++++++++++++++++- htlcswitch/quiescer_test.go | 39 +++++++++++++++++++++++++++----- 3 files changed, 104 insertions(+), 10 deletions(-) diff --git a/htlcswitch/link.go b/htlcswitch/link.go index a3145a9971a..25c2dcd6ccc 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -391,6 +391,11 @@ type channelLink struct { // respect to the quiescence protocol. quiescer quiescer + // quiescenceRequests is a queue of requests to quiesce this link. + // The members of the queue are send-only channels we should call back + // with the result. + quiescenceReqs chan StfuReq + // stateQueries is a channel that is used to query the current state of // the channelLink in a thread-safe manner by delegating the state reads // to the main event loop. @@ -488,6 +493,10 @@ func NewChannelLink(cfg ChannelLinkConfig, }, }) + quiescenceReqs := make( + chan fn.Req[fn.Unit, fn.Result[lntypes.ChannelParty]], 1, + ) + return &channelLink{ cfg: cfg, channel: channel, @@ -498,6 +507,7 @@ func NewChannelLink(cfg ChannelLinkConfig, outgoingCommitHooks: newHookMap(), incomingCommitHooks: newHookMap(), quiescer: qsm, + quiescenceReqs: quiescenceReqs, stateQueries: make(chan func(), 1), quit: make(chan struct{}), } @@ -765,12 +775,17 @@ func (l *channelLink) OnCommitOnce(direction LinkDirection, hook func()) { // may be removed or reworked in the future as RPC initiated quiescence is a // holdover until we have downstream protocols that use it. func (l *channelLink) InitStfu() <-chan fn.Result[lntypes.ChannelParty] { - // TODO(proofofkeags): Implement - c := make(chan fn.Result[lntypes.ChannelParty], 1) + req, out := fn.NewReq[fn.Unit, fn.Result[lntypes.ChannelParty]]( + fn.Unit{}, + ) - c <- fn.Errf[lntypes.ChannelParty]("InitStfu not yet implemented") + select { + case l.quiescenceReqs <- req: + case <-l.quit: + req.Resolve(fn.Err[lntypes.ChannelParty](ErrLinkShuttingDown)) + } - return c + return out } // isReestablished returns true if the link has successfully completed the @@ -1516,6 +1531,13 @@ func (l *channelLink) htlcManager() { ) } + case qReq := <-l.quiescenceReqs: + l.quiescer.initStfu(qReq) + + if err := l.quiescer.drive(); err != nil { + l.stfuFailf("%s", err.Error()) + } + case runQuery := <-l.stateQueries: runQuery() diff --git a/htlcswitch/quiescer.go b/htlcswitch/quiescer.go index cf9ef3283aa..468f140703d 100644 --- a/htlcswitch/quiescer.go +++ b/htlcswitch/quiescer.go @@ -39,6 +39,8 @@ var ( ) ) +type StfuReq = fn.Req[fn.Unit, fn.Result[lntypes.ChannelParty]] + type quiescerCfg struct { // chanID marks what channel we are managing the state machine for. This // is important because the quiescer is responsible for constructing the @@ -83,6 +85,10 @@ type quiescer struct { // received tracks whether or not we have received Stfu from our peer. received bool + + // activeQuiescenceRequest is a possibly None Request that we should + // resolve when we complete quiescence. + activeQuiescenceReq fn.Option[StfuReq] } // newQuiescer creates a new quiescer for the given channel. @@ -115,6 +121,10 @@ func (q *quiescer) recvStfu(msg lnwire.Stfu) error { // does not necessarily mean they will get it, though. q.remoteInit = msg.Initiator + // Since we just received an Stfu, we may have a newly quiesced state. + // If so, we will try to resolve any outstanding StfuReqs. + q.tryResolveStfuReq() + return nil } @@ -143,7 +153,7 @@ func (q *quiescer) makeStfu() fn.Result[lnwire.Stfu] { // Stfu when we have received but not yet sent an Stfu, or we are the initiator // but have not yet sent an Stfu. func (q *quiescer) oweStfu() bool { - return q.received && !q.sent + return (q.received || q.localInit) && !q.sent } // needStfu returns true if the remote owes us an Stfu. They owe us an Stfu when @@ -229,5 +239,38 @@ func (q *quiescer) drive() error { q.sent = true + // Since we just sent an Stfu, we may have a newly quiesced state. + // If so, we will try to resolve any outstanding StfuReqs. + q.tryResolveStfuReq() + return nil } + +// tryResolveStfuReq attempts to resolve the active quiescence request if the +// state machine has reached a quiescent state. +func (q *quiescer) tryResolveStfuReq() { + q.activeQuiescenceReq.WhenSome( + func(req StfuReq) { + if q.isQuiescent() { + req.Resolve(q.quiescenceInitiator()) + q.activeQuiescenceReq = fn.None[StfuReq]() + } + }, + ) +} + +// initStfu instructs the quiescer that we intend to begin a quiescence +// negotiation where we are the initiator. We don't yet send stfu yet because +// we need to wait for the link to give us a valid opportunity to do so. +func (q *quiescer) initStfu(req StfuReq) { + if q.localInit { + req.Resolve(fn.Errf[lntypes.ChannelParty]( + "quiescence already requested", + )) + + return + } + + q.localInit = true + q.activeQuiescenceReq = fn.Some(req) +} diff --git a/htlcswitch/quiescer_test.go b/htlcswitch/quiescer_test.go index 973e987d5b7..f268706ad04 100644 --- a/htlcswitch/quiescer_test.go +++ b/htlcswitch/quiescer_test.go @@ -4,6 +4,7 @@ import ( "bytes" "testing" + "github.com/lightningnetwork/lnd/fn" "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwire" "github.com/stretchr/testify/require" @@ -17,7 +18,9 @@ type quiescerTestHarness struct { conn <-chan lnwire.Stfu } -func initQuiescerTestHarness() *quiescerTestHarness { +func initQuiescerTestHarness( + channelInitiator lntypes.ChannelParty) *quiescerTestHarness { + conn := make(chan lnwire.Stfu, 1) harness := &quiescerTestHarness{ pendingUpdates: lntypes.Dual[uint64]{}, @@ -25,7 +28,8 @@ func initQuiescerTestHarness() *quiescerTestHarness { } harness.quiescer = newQuiescer(quiescerCfg{ - chanID: cid, + chanID: cid, + channelInitiator: channelInitiator, numPendingUpdates: func(whoseUpdate lntypes.ChannelParty, _ lntypes.ChannelParty) uint64 { @@ -41,7 +45,7 @@ func initQuiescerTestHarness() *quiescerTestHarness { } func TestQuiescerDoubleRecvInvalid(t *testing.T) { - harness := initQuiescerTestHarness() + harness := initQuiescerTestHarness(lntypes.Local) msg := lnwire.Stfu{ ChanID: cid, @@ -55,7 +59,7 @@ func TestQuiescerDoubleRecvInvalid(t *testing.T) { } func TestQuiescerPendingUpdatesRecvInvalid(t *testing.T) { - harness := initQuiescerTestHarness() + harness := initQuiescerTestHarness(lntypes.Local) msg := lnwire.Stfu{ ChanID: cid, @@ -68,7 +72,7 @@ func TestQuiescerPendingUpdatesRecvInvalid(t *testing.T) { } func TestQuiescenceRemoteInit(t *testing.T) { - harness := initQuiescerTestHarness() + harness := initQuiescerTestHarness(lntypes.Local) msg := lnwire.Stfu{ ChanID: cid, @@ -100,3 +104,28 @@ func TestQuiescenceRemoteInit(t *testing.T) { t.Fatalf("stfu not sent when expected") } } + +func TestQuiescerTieBreaker(t *testing.T) { + for _, initiator := range []lntypes.ChannelParty{ + lntypes.Local, lntypes.Remote, + } { + harness := initQuiescerTestHarness(initiator) + + msg := lnwire.Stfu{ + ChanID: cid, + Initiator: true, + } + + req, res := fn.NewReq[fn.Unit, fn.Result[lntypes.ChannelParty]]( + fn.Unit{}, + ) + + harness.quiescer.initStfu(req) + require.NoError(t, harness.quiescer.recvStfu(msg)) + require.NoError(t, harness.quiescer.drive()) + + party := <-res + + require.Equal(t, fn.Ok(initiator), party) + } +} From d20381b63056cee4148130ef344441889c6ca1c8 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 12 Mar 2024 12:26:01 -0700 Subject: [PATCH 12/29] itest+lntest: add itest for Quiesce RPC method --- itest/list_on_test.go | 4 +++ itest/lnd_quiescence_test.go | 48 ++++++++++++++++++++++++++++++++++++ lntest/rpc/lnd.go | 15 +++++++++++ 3 files changed, 67 insertions(+) create mode 100644 itest/lnd_quiescence_test.go diff --git a/itest/list_on_test.go b/itest/list_on_test.go index c5fa7cebf33..3359862458d 100644 --- a/itest/list_on_test.go +++ b/itest/list_on_test.go @@ -702,4 +702,8 @@ var allTestCases = []*lntest.TestCase{ Name: "send to route failed htlc timeout", TestFunc: testSendToRouteFailHTLCTimeout, }, + { + Name: "quiescence", + TestFunc: testQuiescence, + }, } diff --git a/itest/lnd_quiescence_test.go b/itest/lnd_quiescence_test.go new file mode 100644 index 00000000000..7c2c274a212 --- /dev/null +++ b/itest/lnd_quiescence_test.go @@ -0,0 +1,48 @@ +package itest + +import ( + "github.com/btcsuite/btcd/btcutil" + "github.com/lightningnetwork/lnd/lnrpc" + "github.com/lightningnetwork/lnd/lnrpc/devrpc" + "github.com/lightningnetwork/lnd/lnrpc/routerrpc" + "github.com/lightningnetwork/lnd/lntest" + "github.com/stretchr/testify/require" +) + +// testQuiescence tests whether we can come to agreement on quiescence of a +// channel. We initiate quiescence via RPC and if it succeeds we verify that +// the expected initiator is the resulting initiator. +// +// NOTE FOR REVIEW: this could be improved by blasting the channel with HTLC +// traffic on both sides to increase the surface area of the change under test. +func testQuiescence(ht *lntest.HarnessTest) { + alice, bob := ht.Alice, ht.Bob + + chanPoint := ht.OpenChannel(bob, alice, lntest.OpenChannelParams{ + Amt: btcutil.Amount(1000000), + }) + defer ht.CloseChannel(bob, chanPoint) + + res := alice.RPC.Quiesce(&devrpc.QuiescenceRequest{ + ChanId: chanPoint, + }) + + require.True(ht, res.Initiator) + + req := &routerrpc.SendPaymentRequest{ + Dest: ht.Alice.PubKey[:], + Amt: 100, + PaymentHash: ht.Random32Bytes(), + FinalCltvDelta: finalCltvDelta, + TimeoutSeconds: 60, + FeeLimitMsat: noFeeLimitMsat, + } + + ht.SendPaymentAssertFail( + ht.Bob, req, + // This fails with insufficient balance because the bandwidth + // manager reports 0 bandwidth if a link is not eligible for + // forwarding, which is the case during quiescence. + lnrpc.PaymentFailureReason_FAILURE_REASON_INSUFFICIENT_BALANCE, + ) +} diff --git a/lntest/rpc/lnd.go b/lntest/rpc/lnd.go index f0ed52fd88f..1657caac8dd 100644 --- a/lntest/rpc/lnd.go +++ b/lntest/rpc/lnd.go @@ -5,6 +5,7 @@ import ( "strings" "github.com/lightningnetwork/lnd/lnrpc" + "github.com/lightningnetwork/lnd/lnrpc/devrpc" "github.com/stretchr/testify/require" ) @@ -730,3 +731,17 @@ func (h *HarnessRPC) LookupHtlcResolutionAssertErr( return err } + +// Quiesce makes an RPC call to the node's Quiesce method and returns the +// response. +func (h *HarnessRPC) Quiesce( + req *devrpc.QuiescenceRequest) *devrpc.QuiescenceResponse { + + ctx, cancel := context.WithTimeout(h.runCtx, DefaultTimeout) + defer cancel() + + res, err := h.Dev.Quiesce(ctx, req) + h.NoError(err, "Quiesce returned an error") + + return res +} From e63e3109015e5bf998ef06e02a8b96fb0932b6ef Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Mon, 8 Apr 2024 18:17:00 -0700 Subject: [PATCH 13/29] htlcswitch: defer processRemoteAdds when quiescent In this commit we defer processRemoteAdds using a new mechanism on the quiescer where we capture a closure that needs to be run. We do this because we need to avoid the scenario where we send back immediate resolutions to the newly added HTLCs when quiescent as it is a protocol violation. It is not enough for us to simply defer sending the messages since the purpose of quiescence itself is to have well-defined and agreed upon channel state. If, for whatever reason, the node (or connection) is restarted between when these hooks are captured and when they are ultimately run, they will be resolved by the resolveFwdPkgs logic when the link comes back up. In a future commit we will explicitly call the quiescer's resume method when it is OK for htlc traffic to commence. --- htlcswitch/link.go | 16 +++++++++++++++- htlcswitch/quiescer.go | 25 +++++++++++++++++++++++++ htlcswitch/quiescer_test.go | 22 ++++++++++++++++++++++ 3 files changed, 62 insertions(+), 1 deletion(-) diff --git a/htlcswitch/link.go b/htlcswitch/link.go index 25c2dcd6ccc..df9a1cced09 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -2506,8 +2506,22 @@ func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) { } } + // If we can send updates then we can process adds in case we + // are the exit hop and need to send back resolutions, or in + // case there are validity issues with the packets. Otherwise + // we defer the action until resume. + // + // We are free to process the settles and fails without this + // check since processing those can't result in further updates + // to this channel link. + if l.quiescer.canSendUpdates() { + l.processRemoteAdds(fwdPkg) + } else { + l.quiescer.onResume(func() { + l.processRemoteAdds(fwdPkg) + }) + } l.processRemoteSettleFails(fwdPkg) - l.processRemoteAdds(fwdPkg) // If the link failed during processing the adds, we must // return to ensure we won't attempted to update the state diff --git a/htlcswitch/quiescer.go b/htlcswitch/quiescer.go index 468f140703d..5d2ab887f94 100644 --- a/htlcswitch/quiescer.go +++ b/htlcswitch/quiescer.go @@ -89,6 +89,11 @@ type quiescer struct { // activeQuiescenceRequest is a possibly None Request that we should // resolve when we complete quiescence. activeQuiescenceReq fn.Option[StfuReq] + + // resumeQueue is a slice of hooks that will be called when the quiescer + // is resumed. These are actions that needed to be deferred while the + // channel was quiescent. + resumeQueue []func() } // newQuiescer creates a new quiescer for the given channel. @@ -274,3 +279,23 @@ func (q *quiescer) initStfu(req StfuReq) { q.localInit = true q.activeQuiescenceReq = fn.Some(req) } + +// onResume accepts a no return closure that will run when the quiescer is +// resumed. +func (q *quiescer) onResume(hook func()) { + q.resumeQueue = append(q.resumeQueue, hook) +} + +// resume runs all of the deferred actions that have accumulated while the +// channel has been quiescent and then resets the quiescer state to its initial +// state. +func (q *quiescer) resume() { + for _, hook := range q.resumeQueue { + hook() + } + q.localInit = false + q.remoteInit = false + q.sent = false + q.received = false + q.resumeQueue = nil +} diff --git a/htlcswitch/quiescer_test.go b/htlcswitch/quiescer_test.go index f268706ad04..a141177dbf1 100644 --- a/htlcswitch/quiescer_test.go +++ b/htlcswitch/quiescer_test.go @@ -129,3 +129,25 @@ func TestQuiescerTieBreaker(t *testing.T) { require.Equal(t, fn.Ok(initiator), party) } } + +func TestQuiescerResume(t *testing.T) { + harness := initQuiescerTestHarness(lntypes.Local) + + msg := lnwire.Stfu{ + ChanID: cid, + Initiator: true, + } + + require.NoError(t, harness.quiescer.recvStfu(msg)) + require.NoError(t, harness.quiescer.drive()) + + require.True(t, harness.quiescer.isQuiescent()) + var resumeHooksCalled = false + harness.quiescer.onResume(func() { + resumeHooksCalled = true + }) + + harness.quiescer.resume() + require.True(t, resumeHooksCalled) + require.False(t, harness.quiescer.isQuiescent()) +} From cb867aad7b3064d4ee954f37061fde7e96a8a39d Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 9 Apr 2024 17:48:56 -0700 Subject: [PATCH 14/29] htlcswitch: add test for deferred processing remote adds when quiescent --- htlcswitch/link_test.go | 87 +++++++++++++++++++++++++++++++++++++++++ htlcswitch/mock.go | 12 +++++- 2 files changed, 97 insertions(+), 2 deletions(-) diff --git a/htlcswitch/link_test.go b/htlcswitch/link_test.go index 018601a0454..3baf537f1fc 100644 --- a/htlcswitch/link_test.go +++ b/htlcswitch/link_test.go @@ -7508,3 +7508,90 @@ func TestLinkFlushHooksCalled(t *testing.T) { ctx.receiveRevAndAckAliceToBob() assertHookCalled(true) } + +// TestLinkQuiescenceExitHopProcessingDeferred ensures that we do not send back +// htlc resolution messages in the case where the link is quiescent AND we are +// the exit hop. This is needed because we handle exit hop processing in the +// link instead of the switch and we process htlc resolutions when we receive +// a RevokeAndAck. Because of this we need to ensure that we hold off on +// processing the remote adds when we are quiescent. Later, when the channel +// update traffic is allowed to resume, we will need to verify that the actions +// we didn't run during the initial RevokeAndAck are run. +func TestLinkQuiescenceExitHopProcessingDeferred(t *testing.T) { + // Initialize two channel state machines for testing. + alice, bob, err := createMirroredChannel( + t, btcutil.SatoshiPerBitcoin, btcutil.SatoshiPerBitcoin, + ) + require.NoError(t, err) + + // Build a single edge network to test channel quiescence. + network := newTwoHopNetwork( + t, alice.channel, bob.channel, testStartingHeight, + ) + aliceLink := network.aliceChannelLink + bobLink := network.bobChannelLink + + // Generate an invoice for Bob so that Alice can pay him. + htlcID := uint64(0) + htlc, invoice := generateHtlcAndInvoice(t, htlcID) + err = network.bobServer.registry.AddInvoice( + nil, *invoice, htlc.PaymentHash, + ) + require.NoError(t, err) + + // Establish a payment circuit for Alice + circuit := &PaymentCircuit{ + Incoming: CircuitKey{ + HtlcID: htlcID, + }, + PaymentHash: htlc.PaymentHash, + } + circuitMap := network.aliceServer.htlcSwitch.circuits + _, err = circuitMap.CommitCircuits(circuit) + require.NoError(t, err) + + // Add a switch packet to Alice's switch so that she can initialize the + // payment attempt. + err = aliceLink.handleSwitchPacket(&htlcPacket{ + incomingHTLCID: htlcID, + htlc: htlc, + circuit: circuit, + }) + require.NoError(t, err) + + // give alice enough time to fire the update_add + // TODO(proofofkeags): make this not depend on a flakey sleep. + <-time.After(time.Millisecond) + + // bob initiates stfu which he can do immediately since he doesn't have + // local updates + <-bobLink.InitStfu() + + // wait for other possible messages to play out + <-time.After(1 * time.Second) + + ensureNoUpdateAfterStfu := func(t *testing.T, trace []lnwire.Message) { + stfuReceived := false + for _, msg := range trace { + if msg.MsgType() == lnwire.MsgStfu { + stfuReceived = true + continue + } + + if stfuReceived && msg.MsgType().IsChannelUpdate() { + t.Fatalf("channel update after stfu: %v", + msg.MsgType()) + } + } + } + + network.aliceServer.protocolTraceMtx.Lock() + ensureNoUpdateAfterStfu(t, network.aliceServer.protocolTrace) + network.aliceServer.protocolTraceMtx.Unlock() + + network.bobServer.protocolTraceMtx.Lock() + ensureNoUpdateAfterStfu(t, network.bobServer.protocolTrace) + network.bobServer.protocolTraceMtx.Unlock() + + // TODO(proofofkeags): make sure these actions are run on resume. +} diff --git a/htlcswitch/mock.go b/htlcswitch/mock.go index 37bf4c6ef2d..0a3364ae271 100644 --- a/htlcswitch/mock.go +++ b/htlcswitch/mock.go @@ -153,8 +153,10 @@ type mockServer struct { t testing.TB - name string - messages chan lnwire.Message + name string + messages chan lnwire.Message + protocolTraceMtx sync.Mutex + protocolTrace []lnwire.Message id [33]byte htlcSwitch *Switch @@ -289,6 +291,10 @@ func (s *mockServer) Start() error { for { select { case msg := <-s.messages: + s.protocolTraceMtx.Lock() + s.protocolTrace = append(s.protocolTrace, msg) + s.protocolTraceMtx.Unlock() + var shouldSkip bool for _, interceptor := range s.interceptorFuncs { @@ -627,6 +633,8 @@ func (s *mockServer) readHandler(message lnwire.Message) error { targetChan = msg.ChanID case *lnwire.UpdateFee: targetChan = msg.ChanID + case *lnwire.Stfu: + targetChan = msg.ChanID default: return fmt.Errorf("unknown message type: %T", msg) } From 42315d9db91ea4177cc8ef6ba1c4586b621c4e12 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Fri, 9 Aug 2024 20:25:44 -0700 Subject: [PATCH 15/29] htlcswitch: implement noop quiescer In this commit we implement a noop quiescer that we will use when the feature hasn't been negotiated. This will make it far easier to manage quiescence operations without having a number of if statements in the link logic. --- htlcswitch/quiescer.go | 90 +++++++++++++++++++++++++++++++++--------- 1 file changed, 71 insertions(+), 19 deletions(-) diff --git a/htlcswitch/quiescer.go b/htlcswitch/quiescer.go index 5d2ab887f94..2a65dfd2e84 100644 --- a/htlcswitch/quiescer.go +++ b/htlcswitch/quiescer.go @@ -41,6 +41,43 @@ var ( type StfuReq = fn.Req[fn.Unit, fn.Result[lntypes.ChannelParty]] +type quiescer interface { + // isQuiescent returns true if the state machine has been driven all the + // way to completion. If this returns true, processes that depend on + // channel quiescence may proceed. + isQuiescent() bool + + // initStfu instructs the quiescer that we intend to begin a quiescence + // negotiation where we are the initiator. We don't yet send stfu yet + // because we need to wait for the link to give us a valid opportunity + // to do so. + initStfu(req StfuReq) + + // recvStfu is called when we receive an Stfu message from the remote. + recvStfu(stfu lnwire.Stfu) error + + // canRecvUpdates returns true if we haven't yet received an Stfu which + // would mark the end of the remote's ability to send updates. + canRecvUpdates() bool + + // canSendUpdates returns true if we haven't yet sent an Stfu which + // would mark the end of our ability to send updates. + canSendUpdates() bool + + // drive drives the quiescence machine forward. It returns an error if + // the state machine is in an invalid state. + drive() error + + // onResume accepts a no return closure that will run when the quiescer + // is resumed. + onResume(hook func()) + + // resume runs all of the deferred actions that have accumulated while + // the channel has been quiescent and then resets the quiescer state to + // its initial state. + resume() +} + type quiescerCfg struct { // chanID marks what channel we are managing the state machine for. This // is important because the quiescer is responsible for constructing the @@ -65,9 +102,9 @@ type quiescerCfg struct { sendMsg func(lnwire.Stfu) error } -// quiescer is a state machine that tracks progression through the quiescence -// protocol. -type quiescer struct { +// quiescerLive is a state machine that tracks progression through the +// quiescence protocol. +type quiescerLive struct { cfg quiescerCfg // localInit indicates whether our path through this state machine was @@ -98,13 +135,13 @@ type quiescer struct { // newQuiescer creates a new quiescer for the given channel. func newQuiescer(cfg quiescerCfg) quiescer { - return quiescer{ + return &quiescerLive{ cfg: cfg, } } // recvStfu is called when we receive an Stfu message from the remote. -func (q *quiescer) recvStfu(msg lnwire.Stfu) error { +func (q *quiescerLive) recvStfu(msg lnwire.Stfu) error { // At the time of this writing, this check that we have already received // an Stfu is not strictly necessary, according to the specification. // However, it is fishy if we do and it is unclear how we should handle @@ -135,7 +172,7 @@ func (q *quiescer) recvStfu(msg lnwire.Stfu) error { // makeStfu is called when we are ready to send an Stfu message. It returns the // Stfu message to be sent. -func (q *quiescer) makeStfu() fn.Result[lnwire.Stfu] { +func (q *quiescerLive) makeStfu() fn.Result[lnwire.Stfu] { if q.sent { return fn.Errf[lnwire.Stfu]("%w for channel %v", ErrStfuAlreadySent, q.cfg.chanID) @@ -157,27 +194,27 @@ func (q *quiescer) makeStfu() fn.Result[lnwire.Stfu] { // oweStfu returns true if we owe the other party an Stfu. We owe the remote an // Stfu when we have received but not yet sent an Stfu, or we are the initiator // but have not yet sent an Stfu. -func (q *quiescer) oweStfu() bool { +func (q *quiescerLive) oweStfu() bool { return (q.received || q.localInit) && !q.sent } // needStfu returns true if the remote owes us an Stfu. They owe us an Stfu when // we have sent but not yet received an Stfu. -func (q *quiescer) needStfu() bool { +func (q *quiescerLive) needStfu() bool { return q.sent && !q.received } // isQuiescent returns true if the state machine has been driven all the way to // completion. If this returns true, processes that depend on channel quiescence // may proceed. -func (q *quiescer) isQuiescent() bool { +func (q *quiescerLive) isQuiescent() bool { return q.sent && q.received } // quiescenceInitiator determines which ChannelParty is the initiator of // quiescence for the purposes of downstream protocols. If the channel is not // currently quiescent, this method will return ErrNoDownstreamLeader. -func (q *quiescer) quiescenceInitiator() fn.Result[lntypes.ChannelParty] { +func (q *quiescerLive) quiescenceInitiator() fn.Result[lntypes.ChannelParty] { switch { case !q.isQuiescent(): return fn.Err[lntypes.ChannelParty](ErrNoQuiescenceInitiator) @@ -203,31 +240,31 @@ func (q *quiescer) quiescenceInitiator() fn.Result[lntypes.ChannelParty] { // canSendUpdates returns true if we haven't yet sent an Stfu which would mark // the end of our ability to send updates. -func (q *quiescer) canSendUpdates() bool { +func (q *quiescerLive) canSendUpdates() bool { return !q.sent && !q.localInit } // canRecvUpdates returns true if we haven't yet received an Stfu which would // mark the end of the remote's ability to send updates. -func (q *quiescer) canRecvUpdates() bool { +func (q *quiescerLive) canRecvUpdates() bool { return !q.received } // canSendStfu returns true if we can send an Stfu. -func (q *quiescer) canSendStfu() bool { +func (q *quiescerLive) canSendStfu() bool { return q.cfg.numPendingUpdates(lntypes.Local, lntypes.Local) == 0 && q.cfg.numPendingUpdates(lntypes.Local, lntypes.Remote) == 0 } // canRecvStfu returns true if we can receive an Stfu. -func (q *quiescer) canRecvStfu() bool { +func (q *quiescerLive) canRecvStfu() bool { return q.cfg.numPendingUpdates(lntypes.Remote, lntypes.Local) == 0 && q.cfg.numPendingUpdates(lntypes.Remote, lntypes.Remote) == 0 } // drive drives the quiescence machine forward. It returns an error if the state // machine is in an invalid state. -func (q *quiescer) drive() error { +func (q *quiescerLive) drive() error { if !q.oweStfu() || !q.canSendStfu() { return nil } @@ -253,7 +290,7 @@ func (q *quiescer) drive() error { // tryResolveStfuReq attempts to resolve the active quiescence request if the // state machine has reached a quiescent state. -func (q *quiescer) tryResolveStfuReq() { +func (q *quiescerLive) tryResolveStfuReq() { q.activeQuiescenceReq.WhenSome( func(req StfuReq) { if q.isQuiescent() { @@ -267,7 +304,7 @@ func (q *quiescer) tryResolveStfuReq() { // initStfu instructs the quiescer that we intend to begin a quiescence // negotiation where we are the initiator. We don't yet send stfu yet because // we need to wait for the link to give us a valid opportunity to do so. -func (q *quiescer) initStfu(req StfuReq) { +func (q *quiescerLive) initStfu(req StfuReq) { if q.localInit { req.Resolve(fn.Errf[lntypes.ChannelParty]( "quiescence already requested", @@ -282,14 +319,14 @@ func (q *quiescer) initStfu(req StfuReq) { // onResume accepts a no return closure that will run when the quiescer is // resumed. -func (q *quiescer) onResume(hook func()) { +func (q *quiescerLive) onResume(hook func()) { q.resumeQueue = append(q.resumeQueue, hook) } // resume runs all of the deferred actions that have accumulated while the // channel has been quiescent and then resets the quiescer state to its initial // state. -func (q *quiescer) resume() { +func (q *quiescerLive) resume() { for _, hook := range q.resumeQueue { hook() } @@ -299,3 +336,18 @@ func (q *quiescer) resume() { q.received = false q.resumeQueue = nil } + +type quiescerNoop struct{} + +var _ quiescer = (*quiescerNoop)(nil) + +func (q *quiescerNoop) initStfu(req StfuReq) { + req.Resolve(fn.Errf[lntypes.ChannelParty]("quiescence not supported")) +} +func (q *quiescerNoop) recvStfu(_ lnwire.Stfu) error { return nil } +func (q *quiescerNoop) canRecvUpdates() bool { return true } +func (q *quiescerNoop) canSendUpdates() bool { return true } +func (q *quiescerNoop) drive() error { return nil } +func (q *quiescerNoop) isQuiescent() bool { return false } +func (q *quiescerNoop) onResume(hook func()) { hook() } +func (q *quiescerNoop) resume() {} From 68ef311368eb5b8b66641daf2aa923b3ca5fbbdf Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Fri, 9 Aug 2024 20:33:34 -0700 Subject: [PATCH 16/29] htlcswitch+peer: allow the disabling of quiescence Here we add a flag where we can disable quiescence. This will be used in the case where the feature is not negotiated with our peer. --- htlcswitch/link.go | 29 +++++++++++++++++++---------- peer/brontide.go | 6 ++++++ 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/htlcswitch/link.go b/htlcswitch/link.go index df9a1cced09..7c386e85df7 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -282,6 +282,10 @@ type ChannelLinkConfig struct { // invalid. DisallowRouteBlinding bool + // DisallowQuiescence is a flag that can be used to disable the + // quiescence protocol. + DisallowQuiescence bool + // MaxFeeExposure is the threshold in milli-satoshis after which we'll // restrict the flow of HTLCs and fee updates. MaxFeeExposure lnwire.MilliSatoshi @@ -482,16 +486,21 @@ func NewChannelLink(cfg ChannelLinkConfig, cfg.MaxFeeExposure = DefaultMaxFeeExposure } - qsm := newQuiescer(quiescerCfg{ - chanID: lnwire.NewChanIDFromOutPoint( - channel.ChannelPoint(), - ), - channelInitiator: channel.Initiator(), - numPendingUpdates: channel.NumPendingUpdates, - sendMsg: func(s lnwire.Stfu) error { - return cfg.Peer.SendMessage(false, &s) - }, - }) + var qsm quiescer + if !cfg.DisallowQuiescence { + qsm = newQuiescer(quiescerCfg{ + chanID: lnwire.NewChanIDFromOutPoint( + channel.ChannelPoint(), + ), + channelInitiator: channel.Initiator(), + numPendingUpdates: channel.NumPendingUpdates, + sendMsg: func(s lnwire.Stfu) error { + return cfg.Peer.SendMessage(false, &s) + }, + }) + } else { + qsm = &quiescerNoop{} + } quiescenceReqs := make( chan fn.Req[fn.Unit, fn.Result[lntypes.ChannelParty]], 1, diff --git a/peer/brontide.go b/peer/brontide.go index 28b3e0dcd4a..8b97eb185e6 100644 --- a/peer/brontide.go +++ b/peer/brontide.go @@ -411,6 +411,10 @@ type Config struct { // invalid. DisallowRouteBlinding bool + // DisallowQuiescence is a flag that indicates whether the Brontide + // should have the quiescence feature disabled. + DisallowQuiescence bool + // MaxFeeExposure limits the number of outstanding fees in a channel. // This value will be passed to created links. MaxFeeExposure lnwire.MilliSatoshi @@ -1318,6 +1322,8 @@ func (p *Brontide) addLink(chanPoint *wire.OutPoint, GetAliases: p.cfg.GetAliases, PreviouslySentShutdown: shutdownMsg, DisallowRouteBlinding: p.cfg.DisallowRouteBlinding, + DisallowQuiescence: p.cfg.DisallowQuiescence || + !p.remoteFeatures.HasFeature(lnwire.QuiescenceOptional), MaxFeeExposure: p.cfg.MaxFeeExposure, } From a98b790cf7629fb7a72423d1804d989d1037c8f1 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Fri, 13 Sep 2024 16:27:59 -0700 Subject: [PATCH 17/29] lnwire: remove no longer used initiator field --- lnwire/dyn_propose.go | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/lnwire/dyn_propose.go b/lnwire/dyn_propose.go index b0cc1198e94..4744e89fd39 100644 --- a/lnwire/dyn_propose.go +++ b/lnwire/dyn_propose.go @@ -52,12 +52,6 @@ type DynPropose struct { // re-negotiate. ChanID ChannelID - // Initiator is a byte that identifies whether this message was sent as - // the initiator of a dynamic commitment negotiation or the responder - // of a dynamic commitment negotiation. bool true indicates it is the - // initiator - Initiator bool - // DustLimit, if not nil, proposes a change to the dust_limit_satoshis // for the sender's commitment transaction. DustLimit fn.Option[btcutil.Amount] @@ -191,10 +185,6 @@ func (dp *DynPropose) Encode(w *bytes.Buffer, _ uint32) error { return err } - if err := WriteBool(w, dp.Initiator); err != nil { - return err - } - return WriteBytes(w, dp.ExtraData) } @@ -205,7 +195,7 @@ func (dp *DynPropose) Encode(w *bytes.Buffer, _ uint32) error { // This is a part of the lnwire.Message interface. func (dp *DynPropose) Decode(r io.Reader, _ uint32) error { // Parse out the only required field. - if err := ReadElements(r, &dp.ChanID, &dp.Initiator); err != nil { + if err := ReadElements(r, &dp.ChanID); err != nil { return err } From a269a836a744e6f505f2aff6c25c02018193ac67 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Thu, 12 Sep 2024 14:52:09 -0700 Subject: [PATCH 18/29] lnwire: add signature to DynAck --- lnwire/dyn_ack.go | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/lnwire/dyn_ack.go b/lnwire/dyn_ack.go index 24f23a228dd..b3c37da75ab 100644 --- a/lnwire/dyn_ack.go +++ b/lnwire/dyn_ack.go @@ -24,6 +24,10 @@ type DynAck struct { // a dynamic commitment negotiation ChanID ChannelID + // Sig is a signature that acknowledges and approves the parameters + // that were requested in the DynPropose + Sig Sig + // LocalNonce is an optional field that is transmitted when accepting // a dynamic commitment upgrade to Taproot Channels. This nonce will be // used to verify the first commitment transaction signature. This will @@ -50,6 +54,10 @@ func (da *DynAck) Encode(w *bytes.Buffer, _ uint32) error { return err } + if err := WriteSig(w, da.Sig); err != nil { + return err + } + var tlvRecords []tlv.Record da.LocalNonce.WhenSome(func(nonce Musig2Nonce) { tlvRecords = append( @@ -84,7 +92,7 @@ func (da *DynAck) Encode(w *bytes.Buffer, _ uint32) error { // This is a part of the lnwire.Message interface. func (da *DynAck) Decode(r io.Reader, _ uint32) error { // Parse out main message. - if err := ReadElements(r, &da.ChanID); err != nil { + if err := ReadElements(r, &da.ChanID, &da.Sig); err != nil { return err } From f89f44d87856d62d3c154953d594095f639647c4 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Thu, 12 Sep 2024 17:24:54 -0700 Subject: [PATCH 19/29] lnwire: add DynCommit message to match spec --- lnwire/dyn_commit.go | 280 ++++++++++++++++++++++++++++++++++++++++++ lnwire/fuzz_test.go | 13 +- lnwire/lnwire_test.go | 53 ++++++++ lnwire/message.go | 5 + 4 files changed, 350 insertions(+), 1 deletion(-) create mode 100644 lnwire/dyn_commit.go diff --git a/lnwire/dyn_commit.go b/lnwire/dyn_commit.go new file mode 100644 index 00000000000..7c0f81743d9 --- /dev/null +++ b/lnwire/dyn_commit.go @@ -0,0 +1,280 @@ +package lnwire + +import ( + "bytes" + "io" + + "github.com/btcsuite/btcd/btcec/v2" + "github.com/btcsuite/btcd/btcutil" + "github.com/lightningnetwork/lnd/fn" + "github.com/lightningnetwork/lnd/lnwallet/chainfee" + "github.com/lightningnetwork/lnd/tlv" +) + +// DynAck is the message used to accept the parameters of a dynamic commitment +// negotiation. Additional optional parameters will need to be present depending +// on the details of the dynamic commitment upgrade. +type DynCommit struct { + // ChanID is the ChannelID of the channel that is currently undergoing + // a dynamic commitment negotiation + ChanID ChannelID + + // Sig is a signature that acknowledges and approves the parameters + // that were requested in the DynPropose + Sig Sig + + // DustLimit, if not nil, proposes a change to the dust_limit_satoshis + // for the sender's commitment transaction. + DustLimit fn.Option[btcutil.Amount] + + // MaxValueInFlight, if not nil, proposes a change to the + // max_htlc_value_in_flight_msat limit of the sender. + MaxValueInFlight fn.Option[MilliSatoshi] + + // ChannelReserve, if not nil, proposes a change to the + // channel_reserve_satoshis requirement of the recipient. + ChannelReserve fn.Option[btcutil.Amount] + + // CsvDelay, if not nil, proposes a change to the to_self_delay + // requirement of the recipient. + CsvDelay fn.Option[uint16] + + // MaxAcceptedHTLCs, if not nil, proposes a change to the + // max_accepted_htlcs limit of the sender. + MaxAcceptedHTLCs fn.Option[uint16] + + // FundingKey, if not nil, proposes a change to the funding_pubkey + // parameter of the sender. + FundingKey fn.Option[btcec.PublicKey] + + // ChannelType, if not nil, proposes a change to the channel_type + // parameter. + ChannelType fn.Option[ChannelType] + + // KickoffFeerate proposes the fee rate in satoshis per kw that it + // is offering for a ChannelType conversion that requires a kickoff + // transaction. + KickoffFeerate fn.Option[chainfee.SatPerKWeight] + + // ExtraData is the set of data that was appended to this message to + // fill out the full maximum transport message size. These fields can + // be used to specify optional data such as custom TLV fields. + ExtraData ExtraOpaqueData +} + +// A compile time check to ensure DynAck implements the lnwire.Message +// interface. +var _ Message = (*DynCommit)(nil) + +// Encode serializes the target DynAck into the passed io.Writer. Serialization +// will observe the rules defined by the passed protocol version. +// +// This is a part of the lnwire.Message interface. +func (dc *DynCommit) Encode(w *bytes.Buffer, _ uint32) error { + if err := WriteChannelID(w, dc.ChanID); err != nil { + return err + } + + if err := WriteSig(w, dc.Sig); err != nil { + return err + } + + var tlvRecords []tlv.Record + dc.DustLimit.WhenSome(func(dl btcutil.Amount) { + protoSats := uint64(dl) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPDustLimitSatoshis, &protoSats, + ), + ) + }) + dc.MaxValueInFlight.WhenSome(func(max MilliSatoshi) { + protoSats := uint64(max) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPMaxHtlcValueInFlightMsat, &protoSats, + ), + ) + }) + dc.ChannelReserve.WhenSome(func(min btcutil.Amount) { + channelReserve := uint64(min) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPChannelReserveSatoshis, &channelReserve, + ), + ) + }) + dc.CsvDelay.WhenSome(func(wait uint16) { + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPToSelfDelay, &wait, + ), + ) + }) + dc.MaxAcceptedHTLCs.WhenSome(func(max uint16) { + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPMaxAcceptedHtlcs, &max, + ), + ) + }) + dc.FundingKey.WhenSome(func(key btcec.PublicKey) { + keyScratch := &key + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPFundingPubkey, &keyScratch, + ), + ) + }) + dc.ChannelType.WhenSome(func(ty ChannelType) { + tlvRecords = append( + tlvRecords, tlv.MakeDynamicRecord( + DPChannelType, &ty, + ty.featureBitLen, + channelTypeEncoder, channelTypeDecoder, + ), + ) + }) + dc.KickoffFeerate.WhenSome(func(kickoffFeerate chainfee.SatPerKWeight) { + protoSats := uint32(kickoffFeerate) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPKickoffFeerate, &protoSats, + ), + ) + }) + tlv.SortRecords(tlvRecords) + + tlvStream, err := tlv.NewStream(tlvRecords...) + if err != nil { + return err + } + + var extraBytesWriter bytes.Buffer + if err := tlvStream.Encode(&extraBytesWriter); err != nil { + return err + } + + dc.ExtraData = ExtraOpaqueData(extraBytesWriter.Bytes()) + + return WriteBytes(w, dc.ExtraData) +} + +// Decode deserializes the serialized DynCommit stored in the passed io.Reader +// into the target DynAck using the deserialization rules defined by the passed +// protocol version. +// +// This is a part of the lnwire.Message interface. +func (dc *DynCommit) Decode(r io.Reader, _ uint32) error { + // Parse out main message. + if err := ReadElements(r, &dc.ChanID, &dc.Sig); err != nil { + return err + } + + // Parse out TLV records. + var tlvRecords ExtraOpaqueData + if err := ReadElement(r, &tlvRecords); err != nil { + return err + } + + // Prepare receiving buffers to be filled by TLV extraction. + var dustLimitScratch uint64 + dustLimit := tlv.MakePrimitiveRecord( + DPDustLimitSatoshis, &dustLimitScratch, + ) + + var maxValueScratch uint64 + maxValue := tlv.MakePrimitiveRecord( + DPMaxHtlcValueInFlightMsat, &maxValueScratch, + ) + + var reserveScratch uint64 + reserve := tlv.MakePrimitiveRecord( + DPChannelReserveSatoshis, &reserveScratch, + ) + + var csvDelayScratch uint16 + csvDelay := tlv.MakePrimitiveRecord(DPToSelfDelay, &csvDelayScratch) + + var maxHtlcsScratch uint16 + maxHtlcs := tlv.MakePrimitiveRecord( + DPMaxAcceptedHtlcs, &maxHtlcsScratch, + ) + + var fundingKeyScratch *btcec.PublicKey + fundingKey := tlv.MakePrimitiveRecord( + DPFundingPubkey, &fundingKeyScratch, + ) + + var chanTypeScratch ChannelType + chanType := tlv.MakeDynamicRecord( + DPChannelType, &chanTypeScratch, chanTypeScratch.featureBitLen, + channelTypeEncoder, channelTypeDecoder, + ) + + var kickoffFeerateScratch uint32 + kickoffFeerate := tlv.MakePrimitiveRecord( + DPKickoffFeerate, &kickoffFeerateScratch, + ) + + // Create set of Records to read TLV bytestream into. + records := []tlv.Record{ + dustLimit, maxValue, reserve, csvDelay, maxHtlcs, fundingKey, + chanType, kickoffFeerate, + } + tlv.SortRecords(records) + + // Read TLV stream into record set. + extraBytesReader := bytes.NewReader(tlvRecords) + tlvStream, err := tlv.NewStream(records...) + if err != nil { + return err + } + typeMap, err := tlvStream.DecodeWithParsedTypesP2P(extraBytesReader) + if err != nil { + return err + } + + // Check the results of the TLV Stream decoding and appropriately set + // message fields. + if val, ok := typeMap[DPDustLimitSatoshis]; ok && val == nil { + dc.DustLimit = fn.Some(btcutil.Amount(dustLimitScratch)) + } + if val, ok := typeMap[DPMaxHtlcValueInFlightMsat]; ok && val == nil { + dc.MaxValueInFlight = fn.Some(MilliSatoshi(maxValueScratch)) + } + if val, ok := typeMap[DPChannelReserveSatoshis]; ok && val == nil { + dc.ChannelReserve = fn.Some(btcutil.Amount(reserveScratch)) + } + if val, ok := typeMap[DPToSelfDelay]; ok && val == nil { + dc.CsvDelay = fn.Some(csvDelayScratch) + } + if val, ok := typeMap[DPMaxAcceptedHtlcs]; ok && val == nil { + dc.MaxAcceptedHTLCs = fn.Some(maxHtlcsScratch) + } + if val, ok := typeMap[DPFundingPubkey]; ok && val == nil { + dc.FundingKey = fn.Some(*fundingKeyScratch) + } + if val, ok := typeMap[DPChannelType]; ok && val == nil { + dc.ChannelType = fn.Some(chanTypeScratch) + } + if val, ok := typeMap[DPKickoffFeerate]; ok && val == nil { + dc.KickoffFeerate = fn.Some( + chainfee.SatPerKWeight(kickoffFeerateScratch), + ) + } + + if len(tlvRecords) != 0 { + dc.ExtraData = tlvRecords + } + + return nil +} + +// MsgType returns the MessageType code which uniquely identifies this message +// as a DynCommit on the wire. +// +// This is part of the lnwire.Message interface. +func (dc *DynCommit) MsgType() MessageType { + return MsgDynCommit +} diff --git a/lnwire/fuzz_test.go b/lnwire/fuzz_test.go index d6ec3b61b35..9d19bdb81ac 100644 --- a/lnwire/fuzz_test.go +++ b/lnwire/fuzz_test.go @@ -650,7 +650,7 @@ func FuzzDynReject(f *testing.F) { func FuzzDynAck(f *testing.F) { f.Fuzz(func(t *testing.T, data []byte) { - // Prefix with DynReject. + // Prefix with DynAck. data = prefixWithMsgType(data, MsgDynAck) // Pass the message into our general fuzz harness for wire @@ -659,6 +659,17 @@ func FuzzDynAck(f *testing.F) { }) } +func FuzzDynCommit(f *testing.F) { + f.Fuzz(func(t *testing.T, data []byte) { + // Prefix with DynCommit + data = prefixWithMsgType(data, MsgDynCommit) + + // Pass the message into our general fuzz harness for wire + // messages! + harness(t, data) + }) +} + func FuzzKickoffSig(f *testing.F) { f.Fuzz(func(t *testing.T, data []byte) { // Prefix with KickoffSig diff --git a/lnwire/lnwire_test.go b/lnwire/lnwire_test.go index 6b9630f58ad..88c2fb34a81 100644 --- a/lnwire/lnwire_test.go +++ b/lnwire/lnwire_test.go @@ -925,6 +925,53 @@ func TestLightningWireProtocol(t *testing.T) { v[0] = reflect.ValueOf(da) }, + MsgDynCommit: func(v []reflect.Value, r *rand.Rand) { + var dc DynCommit + + rand.Read(dc.ChanID[:]) + rand.Read(dc.Sig.bytes[:]) + if rand.Uint32()%2 == 0 { + v := btcutil.Amount(rand.Uint32()) + dc.DustLimit = fn.Some(v) + } + + if rand.Uint32()%2 == 0 { + v := MilliSatoshi(rand.Uint32()) + dc.MaxValueInFlight = fn.Some(v) + } + + if rand.Uint32()%2 == 0 { + v := btcutil.Amount(rand.Uint32()) + dc.ChannelReserve = fn.Some(v) + } + + if rand.Uint32()%2 == 0 { + v := uint16(rand.Uint32()) + dc.CsvDelay = fn.Some(v) + } + + if rand.Uint32()%2 == 0 { + v := uint16(rand.Uint32()) + dc.MaxAcceptedHTLCs = fn.Some(v) + } + + if rand.Uint32()%2 == 0 { + v, _ := btcec.NewPrivateKey() + dc.FundingKey = fn.Some(*v.PubKey()) + } + + if rand.Uint32()%2 == 0 { + v := ChannelType(*NewRawFeatureVector()) + dc.ChannelType = fn.Some(v) + } + + if rand.Uint32()%2 == 0 { + v := chainfee.SatPerKWeight(rand.Uint32()) + dc.KickoffFeerate = fn.Some(v) + } + + v[0] = reflect.ValueOf(dc) + }, MsgKickoffSig: func(v []reflect.Value, r *rand.Rand) { ks := KickoffSig{ ExtraData: make([]byte, 0), @@ -1779,6 +1826,12 @@ func TestLightningWireProtocol(t *testing.T) { return mainScenario(&m) }, }, + { + msgType: MsgDynCommit, + scenario: func(m DynCommit) bool { + return mainScenario(&m) + }, + }, { msgType: MsgKickoffSig, scenario: func(m KickoffSig) bool { diff --git a/lnwire/message.go b/lnwire/message.go index 68b09692e55..34effc85155 100644 --- a/lnwire/message.go +++ b/lnwire/message.go @@ -40,6 +40,7 @@ const ( MsgDynPropose = 111 MsgDynAck = 113 MsgDynReject = 115 + MsgDynCommit = 117 MsgUpdateAddHTLC = 128 MsgUpdateFulfillHTLC = 130 MsgUpdateFailHTLC = 131 @@ -131,6 +132,8 @@ func (t MessageType) String() string { return "DynAck" case MsgDynReject: return "DynReject" + case MsgDynCommit: + return "DynCommit" case MsgKickoffSig: return "KickoffSig" case MsgUpdateAddHTLC: @@ -266,6 +269,8 @@ func makeEmptyMessage(msgType MessageType) (Message, error) { msg = &DynAck{} case MsgDynReject: msg = &DynReject{} + case MsgDynCommit: + msg = &DynCommit{} case MsgKickoffSig: msg = &KickoffSig{} case MsgUpdateAddHTLC: From 58eb9c9c3dca849fd95ddf3c40d61d8e1c8685d8 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Thu, 12 Sep 2024 17:30:06 -0700 Subject: [PATCH 20/29] lnwire: add function to easily construct DynCommit --- lnwire/dyn_commit.go | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/lnwire/dyn_commit.go b/lnwire/dyn_commit.go index 7c0f81743d9..3524ac3f2f8 100644 --- a/lnwire/dyn_commit.go +++ b/lnwire/dyn_commit.go @@ -278,3 +278,20 @@ func (dc *DynCommit) Decode(r io.Reader, _ uint32) error { func (dc *DynCommit) MsgType() MessageType { return MsgDynCommit } + +// NegotiateDynCommit constructs a DynCommit message from the prior DynPropose +// and DynAck messages exchanged during the negotiation. +func NegotiateDynCommit(propose DynPropose, ack DynAck) DynCommit { + return DynCommit{ + ChanID: propose.ChanID, + Sig: ack.Sig, + DustLimit: propose.DustLimit, + MaxValueInFlight: propose.MaxValueInFlight, + ChannelReserve: propose.ChannelReserve, + CsvDelay: propose.CsvDelay, + MaxAcceptedHTLCs: propose.MaxAcceptedHTLCs, + FundingKey: propose.FundingKey, + ChannelType: propose.ChannelType, + KickoffFeerate: propose.KickoffFeerate, + } +} From e072626295972017aa017920e6086a380865bea2 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Fri, 13 Sep 2024 16:25:19 -0700 Subject: [PATCH 21/29] lnwire: add convenience functions for protocol validation --- lnwire/dyn_ack.go | 43 ++++++++++++++ lnwire/dyn_propose.go | 132 ++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 175 insertions(+) diff --git a/lnwire/dyn_ack.go b/lnwire/dyn_ack.go index b3c37da75ab..5e9adadce00 100644 --- a/lnwire/dyn_ack.go +++ b/lnwire/dyn_ack.go @@ -2,9 +2,12 @@ package lnwire import ( "bytes" + "fmt" "io" "github.com/btcsuite/btcd/btcec/v2/schnorr/musig2" + "github.com/btcsuite/btcd/chaincfg/chainhash" + "github.com/decred/dcrd/dcrec/secp256k1/v4" "github.com/lightningnetwork/lnd/fn" "github.com/lightningnetwork/lnd/tlv" ) @@ -144,3 +147,43 @@ func (da *DynAck) Decode(r io.Reader, _ uint32) error { func (da *DynAck) MsgType() MessageType { return MsgDynAck } + +// Validate does DynAck signature validation for a given prior DynPropose +// message. +func (da *DynAck) Validate(propose DynPropose, nextHeight uint64, + pubkey *secp256k1.PublicKey) error { + + cSig, err := da.Sig.ToSignature() + if err != nil { + return err + } + + var msg bytes.Buffer + err = WriteChannelID(&msg, da.ChanID) + if err != nil { + return err + } + + err = WriteElement(&msg, nextHeight) + if err != nil { + return err + } + + tlvData, err := propose.SerializeTlvData() + if err != nil { + return err + } + + msg.Write(tlvData) + + digest := chainhash.DoubleHashB(msg.Bytes()) + + if !cSig.Verify(digest, pubkey) { + return fmt.Errorf( + "invalid signature for dyn_ack: %v @ %v by %v", + propose, nextHeight, pubkey, + ) + } + + return nil +} diff --git a/lnwire/dyn_propose.go b/lnwire/dyn_propose.go index 4744e89fd39..b112239546f 100644 --- a/lnwire/dyn_propose.go +++ b/lnwire/dyn_propose.go @@ -7,6 +7,7 @@ import ( "github.com/btcsuite/btcd/btcec/v2" "github.com/btcsuite/btcd/btcutil" "github.com/lightningnetwork/lnd/fn" + "github.com/lightningnetwork/lnd/keychain" "github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/tlv" ) @@ -307,3 +308,134 @@ func (dp *DynPropose) Decode(r io.Reader, _ uint32) error { func (dp *DynPropose) MsgType() MessageType { return MsgDynPropose } + +// SerializeTlvData takes just the TLV data of DynPropose (which covers all of +// the parameters on deck for changing) and serializes just this component. The +// main purpose of this is to make it easier to validate the DynAck signature. +func (dp *DynPropose) SerializeTlvData() ([]byte, error) { + var tlvRecords []tlv.Record + dp.DustLimit.WhenSome(func(dl btcutil.Amount) { + protoSats := uint64(dl) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPDustLimitSatoshis, &protoSats, + ), + ) + }) + dp.MaxValueInFlight.WhenSome(func(max MilliSatoshi) { + protoSats := uint64(max) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPMaxHtlcValueInFlightMsat, &protoSats, + ), + ) + }) + dp.ChannelReserve.WhenSome(func(min btcutil.Amount) { + channelReserve := uint64(min) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPChannelReserveSatoshis, &channelReserve, + ), + ) + }) + dp.CsvDelay.WhenSome(func(wait uint16) { + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPToSelfDelay, &wait, + ), + ) + }) + dp.MaxAcceptedHTLCs.WhenSome(func(max uint16) { + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPMaxAcceptedHtlcs, &max, + ), + ) + }) + dp.FundingKey.WhenSome(func(key btcec.PublicKey) { + keyScratch := &key + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPFundingPubkey, &keyScratch, + ), + ) + }) + dp.ChannelType.WhenSome(func(ty ChannelType) { + tlvRecords = append( + tlvRecords, tlv.MakeDynamicRecord( + DPChannelType, &ty, + ty.featureBitLen, + channelTypeEncoder, channelTypeDecoder, + ), + ) + }) + dp.KickoffFeerate.WhenSome(func(kickoffFeerate chainfee.SatPerKWeight) { + protoSats := uint32(kickoffFeerate) + tlvRecords = append( + tlvRecords, tlv.MakePrimitiveRecord( + DPKickoffFeerate, &protoSats, + ), + ) + }) + tlv.SortRecords(tlvRecords) + + tlvStream, err := tlv.NewStream(tlvRecords...) + if err != nil { + return nil, err + } + + var outBuf bytes.Buffer + err = tlvStream.Encode(&outBuf) + if err != nil { + return nil, err + } + + return outBuf.Bytes(), nil +} + +// Accept provides a convenience method for taking a DynPropose and issuing a +// corresponding DynAck using the provided MessageSignerRing. +func (dp *DynPropose) Accept(nextHeight uint64, + signer keychain.MessageSignerRing) (DynAck, error) { + + var msg bytes.Buffer + err := WriteChannelID(&msg, dp.ChanID) + if err != nil { + return DynAck{}, err + } + + err = WriteElement(&msg, nextHeight) + if err != nil { + return DynAck{}, err + } + + tlvData, err := dp.SerializeTlvData() + if err != nil { + return DynAck{}, err + } + + msg.Write(tlvData) + + nodeKeyLoc := keychain.KeyLocator{ + Family: keychain.KeyFamilyNodeKey, + Index: 0, + } + + rawSig, err := signer.SignMessageCompact(nodeKeyLoc, msg.Bytes(), false) + if err != nil { + return DynAck{}, err + } + + var sigFixed [64]byte + copy(sigFixed[:], rawSig[0:64]) + + sig := Sig{ + bytes: sigFixed, + sigType: sigTypeECDSA, + } + + return DynAck{ + ChanID: dp.ChanID, + Sig: sig, + }, nil +} From dd55c6e94f8843fb959ab955bb8b204ebf508b06 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Mon, 7 Oct 2024 15:37:30 +0200 Subject: [PATCH 22/29] lnwire: remove kickoff feerate from propose/commit --- lnwire/dyn_commit.go | 27 +-------------------------- lnwire/dyn_propose.go | 38 +------------------------------------- lnwire/lnwire_test.go | 14 -------------- 3 files changed, 2 insertions(+), 77 deletions(-) diff --git a/lnwire/dyn_commit.go b/lnwire/dyn_commit.go index 3524ac3f2f8..0e0b3f2a55e 100644 --- a/lnwire/dyn_commit.go +++ b/lnwire/dyn_commit.go @@ -7,7 +7,6 @@ import ( "github.com/btcsuite/btcd/btcec/v2" "github.com/btcsuite/btcd/btcutil" "github.com/lightningnetwork/lnd/fn" - "github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/tlv" ) @@ -51,11 +50,6 @@ type DynCommit struct { // parameter. ChannelType fn.Option[ChannelType] - // KickoffFeerate proposes the fee rate in satoshis per kw that it - // is offering for a ChannelType conversion that requires a kickoff - // transaction. - KickoffFeerate fn.Option[chainfee.SatPerKWeight] - // ExtraData is the set of data that was appended to this message to // fill out the full maximum transport message size. These fields can // be used to specify optional data such as custom TLV fields. @@ -135,14 +129,6 @@ func (dc *DynCommit) Encode(w *bytes.Buffer, _ uint32) error { ), ) }) - dc.KickoffFeerate.WhenSome(func(kickoffFeerate chainfee.SatPerKWeight) { - protoSats := uint32(kickoffFeerate) - tlvRecords = append( - tlvRecords, tlv.MakePrimitiveRecord( - DPKickoffFeerate, &protoSats, - ), - ) - }) tlv.SortRecords(tlvRecords) tlvStream, err := tlv.NewStream(tlvRecords...) @@ -212,15 +198,10 @@ func (dc *DynCommit) Decode(r io.Reader, _ uint32) error { channelTypeEncoder, channelTypeDecoder, ) - var kickoffFeerateScratch uint32 - kickoffFeerate := tlv.MakePrimitiveRecord( - DPKickoffFeerate, &kickoffFeerateScratch, - ) - // Create set of Records to read TLV bytestream into. records := []tlv.Record{ dustLimit, maxValue, reserve, csvDelay, maxHtlcs, fundingKey, - chanType, kickoffFeerate, + chanType, } tlv.SortRecords(records) @@ -258,11 +239,6 @@ func (dc *DynCommit) Decode(r io.Reader, _ uint32) error { if val, ok := typeMap[DPChannelType]; ok && val == nil { dc.ChannelType = fn.Some(chanTypeScratch) } - if val, ok := typeMap[DPKickoffFeerate]; ok && val == nil { - dc.KickoffFeerate = fn.Some( - chainfee.SatPerKWeight(kickoffFeerateScratch), - ) - } if len(tlvRecords) != 0 { dc.ExtraData = tlvRecords @@ -292,6 +268,5 @@ func NegotiateDynCommit(propose DynPropose, ack DynAck) DynCommit { MaxAcceptedHTLCs: propose.MaxAcceptedHTLCs, FundingKey: propose.FundingKey, ChannelType: propose.ChannelType, - KickoffFeerate: propose.KickoffFeerate, } } diff --git a/lnwire/dyn_propose.go b/lnwire/dyn_propose.go index b112239546f..699d1a25708 100644 --- a/lnwire/dyn_propose.go +++ b/lnwire/dyn_propose.go @@ -8,7 +8,6 @@ import ( "github.com/btcsuite/btcd/btcutil" "github.com/lightningnetwork/lnd/fn" "github.com/lightningnetwork/lnd/keychain" - "github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/tlv" ) @@ -40,10 +39,6 @@ const ( // DPChannelType is the TLV type number that identifies the record for // DynPropose.ChannelType. DPChannelType tlv.Type = 6 - - // DPKickoffFeerate is the TLV type number that identifies the record - // for DynPropose.KickoffFeerate. - DPKickoffFeerate tlv.Type = 7 ) // DynPropose is a message that is sent during a dynamic commitments negotiation @@ -81,11 +76,6 @@ type DynPropose struct { // parameter. ChannelType fn.Option[ChannelType] - // KickoffFeerate proposes the fee rate in satoshis per kw that it - // is offering for a ChannelType conversion that requires a kickoff - // transaction. - KickoffFeerate fn.Option[chainfee.SatPerKWeight] - // ExtraData is the set of data that was appended to this message to // fill out the full maximum transport message size. These fields can // be used to specify optional data such as custom TLV fields. @@ -161,14 +151,6 @@ func (dp *DynPropose) Encode(w *bytes.Buffer, _ uint32) error { ), ) }) - dp.KickoffFeerate.WhenSome(func(kickoffFeerate chainfee.SatPerKWeight) { - protoSats := uint32(kickoffFeerate) - tlvRecords = append( - tlvRecords, tlv.MakePrimitiveRecord( - DPKickoffFeerate, &protoSats, - ), - ) - }) tlv.SortRecords(tlvRecords) tlvStream, err := tlv.NewStream(tlvRecords...) @@ -241,15 +223,10 @@ func (dp *DynPropose) Decode(r io.Reader, _ uint32) error { channelTypeEncoder, channelTypeDecoder, ) - var kickoffFeerateScratch uint32 - kickoffFeerate := tlv.MakePrimitiveRecord( - DPKickoffFeerate, &kickoffFeerateScratch, - ) - // Create set of Records to read TLV bytestream into. records := []tlv.Record{ dustLimit, maxValue, reserve, csvDelay, maxHtlcs, fundingKey, - chanType, kickoffFeerate, + chanType, } tlv.SortRecords(records) @@ -288,11 +265,6 @@ func (dp *DynPropose) Decode(r io.Reader, _ uint32) error { if val, ok := typeMap[DPChannelType]; ok && val == nil { dp.ChannelType = fn.Some(chanTypeScratch) } - if val, ok := typeMap[DPKickoffFeerate]; ok && val == nil { - dp.KickoffFeerate = fn.Some( - chainfee.SatPerKWeight(kickoffFeerateScratch), - ) - } if len(tlvRecords) != 0 { dp.ExtraData = tlvRecords @@ -369,14 +341,6 @@ func (dp *DynPropose) SerializeTlvData() ([]byte, error) { ), ) }) - dp.KickoffFeerate.WhenSome(func(kickoffFeerate chainfee.SatPerKWeight) { - protoSats := uint32(kickoffFeerate) - tlvRecords = append( - tlvRecords, tlv.MakePrimitiveRecord( - DPKickoffFeerate, &protoSats, - ), - ) - }) tlv.SortRecords(tlvRecords) tlvStream, err := tlv.NewStream(tlvRecords...) diff --git a/lnwire/lnwire_test.go b/lnwire/lnwire_test.go index 88c2fb34a81..d04e4465a42 100644 --- a/lnwire/lnwire_test.go +++ b/lnwire/lnwire_test.go @@ -23,7 +23,6 @@ import ( "github.com/btcsuite/btcd/chaincfg/chainhash" "github.com/btcsuite/btcd/wire" "github.com/lightningnetwork/lnd/fn" - "github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/tlv" "github.com/lightningnetwork/lnd/tor" "github.com/stretchr/testify/assert" @@ -862,11 +861,6 @@ func TestLightningWireProtocol(t *testing.T) { dp.ChannelType = fn.Some(v) } - if rand.Uint32()%2 == 0 { - v := chainfee.SatPerKWeight(rand.Uint32()) - dp.KickoffFeerate = fn.Some(v) - } - v[0] = reflect.ValueOf(dp) }, MsgDynReject: func(v []reflect.Value, r *rand.Rand) { @@ -906,9 +900,6 @@ func TestLightningWireProtocol(t *testing.T) { features.Set(FeatureBit(DPChannelType)) } - if rand.Uint32()%2 == 0 { - features.Set(FeatureBit(DPKickoffFeerate)) - } dr.UpdateRejections = *features v[0] = reflect.ValueOf(dr) @@ -965,11 +956,6 @@ func TestLightningWireProtocol(t *testing.T) { dc.ChannelType = fn.Some(v) } - if rand.Uint32()%2 == 0 { - v := chainfee.SatPerKWeight(rand.Uint32()) - dc.KickoffFeerate = fn.Some(v) - } - v[0] = reflect.ValueOf(dc) }, MsgKickoffSig: func(v []reflect.Value, r *rand.Rand) { From 56a34cbbd2de7a9f11f4fe0c9034bec8205ae1ca Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Mon, 19 Aug 2024 15:07:52 -0600 Subject: [PATCH 23/29] multi: pack LocalChanCfg/RemoteChanCfg into Dual --- chanbackup/single.go | 4 +- chanbackup/single_test.go | 7 +- channeldb/channel.go | 19 +-- channeldb/channel_test.go | 8 +- channeldb/db.go | 2 +- channeldb/db_test.go | 20 ++- chanrestore.go | 7 +- contractcourt/breach_arbitrator_test.go | 17 ++- contractcourt/chain_watcher.go | 23 +-- contractcourt/chain_watcher_test.go | 3 +- contractcourt/commit_sweep_resolver.go | 2 +- funding/manager.go | 29 ++-- htlcswitch/link.go | 4 +- htlcswitch/test_utils.go | 12 +- lnwallet/aux_leaf_store.go | 4 +- lnwallet/channel.go | 185 +++++++++++++----------- lnwallet/channel_test.go | 137 +++++++++--------- lnwallet/commitment.go | 22 +-- lnwallet/test_utils.go | 12 +- lnwallet/transactions_test.go | 12 +- lnwallet/wallet.go | 14 +- peer/test_utils.go | 12 +- routing/localchans/manager.go | 4 +- routing/localchans/manager_test.go | 7 +- rpcserver.go | 24 +-- 25 files changed, 326 insertions(+), 264 deletions(-) diff --git a/chanbackup/single.go b/chanbackup/single.go index b741320b078..88f9ded27b6 100644 --- a/chanbackup/single.go +++ b/chanbackup/single.go @@ -296,8 +296,8 @@ func NewSingle(channel *channeldb.OpenChannel, RemoteNodePub: channel.IdentityPub, Addresses: nodeAddrs, Capacity: channel.Capacity, - LocalChanCfg: channel.LocalChanCfg, - RemoteChanCfg: channel.RemoteChanCfg, + LocalChanCfg: channel.ChanCfgs.Local, + RemoteChanCfg: channel.ChanCfgs.Remote, ShaChainRootDesc: shaChainRootDesc, } diff --git a/chanbackup/single_test.go b/chanbackup/single_test.go index d2212bd859c..dd5bd1a816a 100644 --- a/chanbackup/single_test.go +++ b/chanbackup/single_test.go @@ -16,6 +16,7 @@ import ( "github.com/lightningnetwork/lnd/fn" "github.com/lightningnetwork/lnd/keychain" "github.com/lightningnetwork/lnd/lnencrypt" + "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/shachain" "github.com/stretchr/testify/require" @@ -241,11 +242,13 @@ func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) { ), ThawHeight: rand.Uint32(), IdentityPub: pub, - LocalChanCfg: localCfg, - RemoteChanCfg: remoteCfg, LocalCommitment: localCommit, RevocationProducer: shaChainProducer, TapscriptRoot: tapscriptRootOption, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: localCfg, + Remote: remoteCfg, + }, }, nil } diff --git a/channeldb/channel.go b/channeldb/channel.go index c21716a4560..5a59be91562 100644 --- a/channeldb/channel.go +++ b/channeldb/channel.go @@ -938,11 +938,8 @@ type OpenChannel struct { // opening. InitialRemoteBalance lnwire.MilliSatoshi - // LocalChanCfg is the channel configuration for the local node. - LocalChanCfg ChannelConfig - - // RemoteChanCfg is the channel configuration for the remote node. - RemoteChanCfg ChannelConfig + // ChanCfgs is the channel configuration for the local and remote nodes. + ChanCfgs lntypes.Dual[ChannelConfig] // LocalCommitment is the current local commitment state for the local // party. This is stored distinct from the state of the remote party @@ -1823,7 +1820,7 @@ func (c *OpenChannel) ChanSyncMsg() (*lnwire.ChannelReestablish, error) { } nextNonce, err := NewMusigVerificationNonce( - c.LocalChanCfg.MultiSigKey.PubKey, + c.ChanCfgs.Local.MultiSigKey.PubKey, nextLocalCommitHeight, taprootRevProducer, ) if err != nil { @@ -4112,7 +4109,7 @@ func putChannelCloseSummary(tx kvdb.RwTx, chanID []byte, summary.RemoteCurrentRevocation = lastChanState.RemoteCurrentRevocation summary.RemoteNextRevocation = lastChanState.RemoteNextRevocation - summary.LocalChanConfig = lastChanState.LocalChanCfg + summary.LocalChanConfig = lastChanState.ChanCfgs.Local var b bytes.Buffer if err := serializeChannelCloseSummary(&b, summary); err != nil { @@ -4302,10 +4299,10 @@ func putChanInfo(chanBucket kvdb.RwBucket, channel *OpenChannel) error { } } - if err := writeChanConfig(&w, &channel.LocalChanCfg); err != nil { + if err := writeChanConfig(&w, &channel.ChanCfgs.Local); err != nil { return err } - if err := writeChanConfig(&w, &channel.RemoteChanCfg); err != nil { + if err := writeChanConfig(&w, &channel.ChanCfgs.Remote); err != nil { return err } @@ -4484,10 +4481,10 @@ func fetchChanInfo(chanBucket kvdb.RBucket, channel *OpenChannel) error { } } - if err := readChanConfig(r, &channel.LocalChanCfg); err != nil { + if err := readChanConfig(r, &channel.ChanCfgs.Local); err != nil { return err } - if err := readChanConfig(r, &channel.RemoteChanCfg); err != nil { + if err := readChanConfig(r, &channel.ChanCfgs.Remote); err != nil { return err } diff --git a/channeldb/channel_test.go b/channeldb/channel_test.go index e92692201d8..91ad13e97ec 100644 --- a/channeldb/channel_test.go +++ b/channeldb/channel_test.go @@ -339,8 +339,10 @@ func createTestChannelState(t *testing.T, cdb *ChannelStateDB) *OpenChannel { IsPending: true, IdentityPub: pubKey, Capacity: btcutil.Amount(10000), - LocalChanCfg: localCfg, - RemoteChanCfg: remoteCfg, + ChanCfgs: lntypes.Dual[ChannelConfig]{ + Local: localCfg, + Remote: remoteCfg, + }, TotalMSatSent: 8, TotalMSatReceived: 2, LocalCommitment: ChannelCommitment{ @@ -1025,7 +1027,7 @@ func TestFetchClosedChannels(t *testing.T) { TimeLockedBalance: state.RemoteCommitment.LocalBalance.ToSatoshis() + 10000, CloseType: RemoteForceClose, IsPending: true, - LocalChanConfig: state.LocalChanCfg, + LocalChanConfig: state.ChanCfgs.Local, } if err := state.CloseChannel(summary); err != nil { t.Fatalf("unable to close channel: %v", err) diff --git a/channeldb/db.go b/channeldb/db.go index 92e0498eceb..715eb38510f 100644 --- a/channeldb/db.go +++ b/channeldb/db.go @@ -1430,7 +1430,7 @@ func (c *ChannelStateDB) AbandonChannel(chanPoint *wire.OutPoint, ShortChanID: dbChan.ShortChanID(), RemoteCurrentRevocation: dbChan.RemoteCurrentRevocation, RemoteNextRevocation: dbChan.RemoteNextRevocation, - LocalChanConfig: dbChan.LocalChanCfg, + LocalChanConfig: dbChan.ChanCfgs.Local, } // Finally, we'll close the channel in the DB, and return back to the diff --git a/channeldb/db_test.go b/channeldb/db_test.go index d8113db8306..8dda4927588 100644 --- a/channeldb/db_test.go +++ b/channeldb/db_test.go @@ -296,6 +296,16 @@ func genRandomChannelShell() (*ChannelShell, error) { CsvDelay: uint16(rand.Int63()), } + localChanCfg := ChannelConfig{ + CommitmentParams: commitParams, + PaymentBasePoint: keychain.KeyDescriptor{ + KeyLocator: keychain.KeyLocator{ + Family: keychain.KeyFamily(rand.Int63()), + Index: uint32(rand.Int63()), + }, + }, + } + return &ChannelShell{ NodeAddrs: []net.Addr{&net.TCPAddr{ IP: net.ParseIP("127.0.0.1"), @@ -309,14 +319,8 @@ func genRandomChannelShell() (*ChannelShell, error) { uint64(rand.Int63()), ), IdentityPub: pub, - LocalChanCfg: ChannelConfig{ - CommitmentParams: commitParams, - PaymentBasePoint: keychain.KeyDescriptor{ - KeyLocator: keychain.KeyLocator{ - Family: keychain.KeyFamily(rand.Int63()), - Index: uint32(rand.Int63()), - }, - }, + ChanCfgs: lntypes.Dual[ChannelConfig]{ + Local: localChanCfg, }, RemoteCurrentRevocation: pub, IsPending: false, diff --git a/chanrestore.go b/chanrestore.go index 5b221c105a5..1fe80119954 100644 --- a/chanrestore.go +++ b/chanrestore.go @@ -12,6 +12,7 @@ import ( "github.com/lightningnetwork/lnd/channeldb" "github.com/lightningnetwork/lnd/contractcourt" "github.com/lightningnetwork/lnd/keychain" + "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/shachain" ) @@ -187,12 +188,14 @@ func (c *chanDBRestorer) openChannelShell(backup chanbackup.Single) ( ShortChannelID: backup.ShortChannelID, IdentityPub: backup.RemoteNodePub, IsPending: false, - LocalChanCfg: backup.LocalChanCfg, - RemoteChanCfg: backup.RemoteChanCfg, RemoteCurrentRevocation: backup.RemoteNodePub, RevocationStore: shachain.NewRevocationStore(), RevocationProducer: shaChainProducer, ThawHeight: backup.LeaseExpiry, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: backup.LocalChanCfg, + Remote: backup.RemoteChanCfg, + }, }, } diff --git a/contractcourt/breach_arbitrator_test.go b/contractcourt/breach_arbitrator_test.go index bd4ad856831..faf36f5e6f9 100644 --- a/contractcourt/breach_arbitrator_test.go +++ b/contractcourt/breach_arbitrator_test.go @@ -28,6 +28,7 @@ import ( "github.com/lightningnetwork/lnd/lntest/channels" "github.com/lightningnetwork/lnd/lntest/mock" "github.com/lightningnetwork/lnd/lntest/wait" + "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwallet" "github.com/lightningnetwork/lnd/lnwallet/chainfee" "github.com/lightningnetwork/lnd/lnwire" @@ -1636,7 +1637,7 @@ func testBreachSpends(t *testing.T, test breachTest) { ShortChanID: state.ShortChanID(), RemoteCurrentRevocation: state.RemoteCurrentRevocation, RemoteNextRevocation: state.RemoteNextRevocation, - LocalChanConfig: state.LocalChanCfg, + LocalChanConfig: state.ChanCfgs.Local, }) require.NoError(t, err, "unable to close channel") @@ -1850,7 +1851,7 @@ func TestBreachDelayedJusticeConfirmation(t *testing.T) { ShortChanID: state.ShortChanID(), RemoteCurrentRevocation: state.RemoteCurrentRevocation, RemoteNextRevocation: state.RemoteNextRevocation, - LocalChanConfig: state.LocalChanCfg, + LocalChanConfig: state.ChanCfgs.Local, }) require.NoError(t, err, "unable to close channel") @@ -2333,8 +2334,10 @@ func createInitChannels(t *testing.T) ( ) aliceChannelState := &channeldb.OpenChannel{ - LocalChanCfg: aliceCfg, - RemoteChanCfg: bobCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: aliceCfg, + Remote: bobCfg, + }, IdentityPub: aliceKeyPub, FundingOutpoint: *prevOut, ShortChannelID: shortChanID, @@ -2351,8 +2354,10 @@ func createInitChannels(t *testing.T) ( FundingTxn: channels.TestFundingTx, } bobChannelState := &channeldb.OpenChannel{ - LocalChanCfg: bobCfg, - RemoteChanCfg: aliceCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: bobCfg, + Remote: aliceCfg, + }, IdentityPub: bobKeyPub, FundingOutpoint: *prevOut, ShortChannelID: shortChanID, diff --git a/contractcourt/chain_watcher.go b/contractcourt/chain_watcher.go index ef4a4e20081..bdb84c5eeda 100644 --- a/contractcourt/chain_watcher.go +++ b/contractcourt/chain_watcher.go @@ -250,13 +250,13 @@ func newChainWatcher(cfg chainWatcherConfig) (*chainWatcher, error) { chanState := cfg.chanState if chanState.IsInitiator { stateHint = lnwallet.DeriveStateHintObfuscator( - chanState.LocalChanCfg.PaymentBasePoint.PubKey, - chanState.RemoteChanCfg.PaymentBasePoint.PubKey, + chanState.ChanCfgs.Local.PaymentBasePoint.PubKey, + chanState.ChanCfgs.Remote.PaymentBasePoint.PubKey, ) } else { stateHint = lnwallet.DeriveStateHintObfuscator( - chanState.RemoteChanCfg.PaymentBasePoint.PubKey, - chanState.LocalChanCfg.PaymentBasePoint.PubKey, + chanState.ChanCfgs.Remote.PaymentBasePoint.PubKey, + chanState.ChanCfgs.Local.PaymentBasePoint.PubKey, ) } @@ -306,8 +306,8 @@ func (c *chainWatcher) Start() error { } } - localKey := chanState.LocalChanCfg.MultiSigKey.PubKey - remoteKey := chanState.RemoteChanCfg.MultiSigKey.PubKey + localKey := chanState.ChanCfgs.Local.MultiSigKey.PubKey + remoteKey := chanState.ChanCfgs.Remote.MultiSigKey.PubKey var ( err error @@ -426,7 +426,8 @@ func (c *chainWatcher) handleUnknownLocalState( // revoke our own commitment. commitKeyRing := lnwallet.DeriveCommitmentKeys( commitPoint, lntypes.Local, c.cfg.chanState.ChanType, - &c.cfg.chanState.LocalChanCfg, &c.cfg.chanState.RemoteChanCfg, + &c.cfg.chanState.ChanCfgs.Local, + &c.cfg.chanState.ChanCfgs.Remote, ) auxResult, err := fn.MapOptionZ( @@ -475,7 +476,7 @@ func (c *chainWatcher) handleUnknownLocalState( localScript, err := lnwallet.CommitScriptToSelf( c.cfg.chanState.ChanType, c.cfg.chanState.IsInitiator, commitKeyRing.ToLocalKey, commitKeyRing.RevocationKey, - uint32(c.cfg.chanState.LocalChanCfg.CsvDelay), leaseExpiry, + uint32(c.cfg.chanState.ChanCfgs.Local.CsvDelay), leaseExpiry, localAuxLeaf, ) if err != nil { @@ -1106,7 +1107,7 @@ func (c *chainWatcher) dispatchCooperativeClose(commitSpend *chainntnfs.SpendDet IsPending: true, RemoteCurrentRevocation: c.cfg.chanState.RemoteCurrentRevocation, RemoteNextRevocation: c.cfg.chanState.RemoteNextRevocation, - LocalChanConfig: c.cfg.chanState.LocalChanCfg, + LocalChanConfig: c.cfg.chanState.ChanCfgs.Local, } // Attempt to add a channel sync message to the close summary. @@ -1171,7 +1172,7 @@ func (c *chainWatcher) dispatchLocalForceClose( CloseHeight: uint32(commitSpend.SpendingHeight), RemoteCurrentRevocation: c.cfg.chanState.RemoteCurrentRevocation, RemoteNextRevocation: c.cfg.chanState.RemoteNextRevocation, - LocalChanConfig: c.cfg.chanState.LocalChanCfg, + LocalChanConfig: c.cfg.chanState.ChanCfgs.Local, } // If our commitment output isn't dust or we have active HTLC's on the @@ -1312,7 +1313,7 @@ func (c *chainWatcher) dispatchContractBreach(spendEvent *chainntnfs.SpendDetail ShortChanID: c.cfg.chanState.ShortChanID(), RemoteCurrentRevocation: c.cfg.chanState.RemoteCurrentRevocation, RemoteNextRevocation: c.cfg.chanState.RemoteNextRevocation, - LocalChanConfig: c.cfg.chanState.LocalChanCfg, + LocalChanConfig: c.cfg.chanState.ChanCfgs.Local, } // Attempt to add a channel sync message to the close summary. diff --git a/contractcourt/chain_watcher_test.go b/contractcourt/chain_watcher_test.go index 489a6051854..60f7128787b 100644 --- a/contractcourt/chain_watcher_test.go +++ b/contractcourt/chain_watcher_test.go @@ -348,7 +348,8 @@ func TestChainWatcherDataLossProtect(t *testing.T) { // key for this output. sweepTweak := input.SingleTweakBytes( dlpPoint, - aliceChannel.State().LocalChanCfg.PaymentBasePoint.PubKey, + //nolint:lll + aliceChannel.State().ChanCfgs.Local.PaymentBasePoint.PubKey, ) commitResolution := uniClose.CommitResolution resolutionTweak := commitResolution.SelfOutputSignDesc.SingleTweak diff --git a/contractcourt/commit_sweep_resolver.go b/contractcourt/commit_sweep_resolver.go index 4b47a342948..3526ae5574b 100644 --- a/contractcourt/commit_sweep_resolver.go +++ b/contractcourt/commit_sweep_resolver.go @@ -471,7 +471,7 @@ func (c *commitSweepResolver) SupplementState(state *channeldb.OpenChannel) { if state.ChanType.HasLeaseExpiration() { c.leaseExpiry = state.ThawHeight } - c.localChanCfg = state.LocalChanCfg + c.localChanCfg = state.ChanCfgs.Local c.channelInitiator = state.IsInitiator c.chanType = state.ChanType } diff --git a/funding/manager.go b/funding/manager.go index 1fa90c6932b..aa52f51e8dd 100644 --- a/funding/manager.go +++ b/funding/manager.go @@ -2552,7 +2552,7 @@ func (f *Manager) fundeeProcessFundingCreated(peer lnpeer.Peer, SettledBalance: localBalance, RemoteCurrentRevocation: completeChan.RemoteCurrentRevocation, RemoteNextRevocation: completeChan.RemoteNextRevocation, - LocalChanConfig: completeChan.LocalChanCfg, + LocalChanConfig: completeChan.ChanCfgs.Local, } // Close the channel with us as the initiator because we are @@ -2917,7 +2917,7 @@ func (f *Manager) fundingTimeout(c *channeldb.OpenChannel, CloseType: channeldb.FundingCanceled, RemoteCurrentRevocation: c.RemoteCurrentRevocation, RemoteNextRevocation: c.RemoteNextRevocation, - LocalChanConfig: c.LocalChanCfg, + LocalChanConfig: c.ChanCfgs.Local, } // Close the channel with us as the initiator because we are timing the @@ -3017,8 +3017,8 @@ func (f *Manager) waitForFundingWithTimeout( // makeFundingScript re-creates the funding script for the funding transaction // of the target channel. func makeFundingScript(channel *channeldb.OpenChannel) ([]byte, error) { - localKey := channel.LocalChanCfg.MultiSigKey.PubKey - remoteKey := channel.RemoteChanCfg.MultiSigKey.PubKey + localKey := channel.ChanCfgs.Local.MultiSigKey.PubKey + remoteKey := channel.ChanCfgs.Remote.MultiSigKey.PubKey if channel.ChanType.IsTaproot() { pkScript, _, err := input.GenTaprootFundingScript( @@ -3510,7 +3510,7 @@ func (f *Manager) extractAnnounceParams(c *channeldb.OpenChannel) ( // we'll use this value within our ChannelUpdate. This constraint is // originally set by the remote node, as it will be the one that will // need to determine the smallest HTLC it deems economically relevant. - fwdMinHTLC := c.LocalChanCfg.MinHTLC + fwdMinHTLC := c.ChanCfgs.Local.MinHTLC // We don't necessarily want to go as low as the remote party allows. // Check it against our default forwarding policy. @@ -3521,7 +3521,7 @@ func (f *Manager) extractAnnounceParams(c *channeldb.OpenChannel) ( // We'll obtain the max HTLC value we can forward in our direction, as // we'll use this value within our ChannelUpdate. This value must be <= // channel capacity and <= the maximum in-flight msats set by the peer. - fwdMaxHTLC := c.LocalChanCfg.MaxPendingAmount + fwdMaxHTLC := c.ChanCfgs.Local.MaxPendingAmount capacityMSat := lnwire.NewMSatFromSatoshis(c.Capacity) if fwdMaxHTLC > capacityMSat { fwdMaxHTLC = capacityMSat @@ -3549,8 +3549,8 @@ func (f *Manager) addToGraph(completeChan *channeldb.OpenChannel, ann, err := f.newChanAnnouncement( f.cfg.IDKey, completeChan.IdentityPub, - &completeChan.LocalChanCfg.MultiSigKey, - completeChan.RemoteChanCfg.MultiSigKey.PubKey, *shortChanID, + &completeChan.ChanCfgs.Local.MultiSigKey, + completeChan.ChanCfgs.Remote.MultiSigKey.PubKey, *shortChanID, chanID, fwdMinHTLC, fwdMaxHTLC, ourPolicy, completeChan.ChanType, ) @@ -3743,8 +3743,8 @@ func (f *Manager) annAfterSixConfs(completeChan *channeldb.OpenChannel, // public and usable for other nodes for routing. err = f.announceChannel( f.cfg.IDKey, completeChan.IdentityPub, - &completeChan.LocalChanCfg.MultiSigKey, - completeChan.RemoteChanCfg.MultiSigKey.PubKey, + &completeChan.ChanCfgs.Local.MultiSigKey, + completeChan.ChanCfgs.Remote.MultiSigKey.PubKey, *shortChanID, chanID, completeChan.ChanType, ) if err != nil { @@ -3864,7 +3864,7 @@ func genFirstStateMusigNonce(channel *channeldb.OpenChannel, // We use the _next_ commitment height here as we need to generate the // nonce for the next state the remote party will sign for us. verNonce, err := channeldb.NewMusigVerificationNonce( - channel.LocalChanCfg.MultiSigKey.PubKey, + channel.ChanCfgs.Local.MultiSigKey.PubKey, channel.LocalCommitment.CommitHeight+1, musig2ShaChain, ) @@ -4257,7 +4257,7 @@ func (f *Manager) ensureInitialForwardingPolicy(chanID lnwire.ChannelID, "falling back to default values: %v", err) forwardingPolicy = f.defaultForwardingPolicy( - channel.LocalChanCfg.ChannelStateBounds, + channel.ChanCfgs.Local.ChannelStateBounds, ) needDBUpdate = true } @@ -4267,11 +4267,12 @@ func (f *Manager) ensureInitialForwardingPolicy(chanID lnwire.ChannelID, // still pending while updating to this version, we'll need to set the // values to the default values. if forwardingPolicy.MinHTLCOut == 0 { - forwardingPolicy.MinHTLCOut = channel.LocalChanCfg.MinHTLC + forwardingPolicy.MinHTLCOut = channel.ChanCfgs.Local.MinHTLC needDBUpdate = true } if forwardingPolicy.MaxHTLC == 0 { - forwardingPolicy.MaxHTLC = channel.LocalChanCfg.MaxPendingAmount + forwardingPolicy.MaxHTLC = + channel.ChanCfgs.Local.MaxPendingAmount needDBUpdate = true } diff --git a/htlcswitch/link.go b/htlcswitch/link.go index 7c386e85df7..c094c6dd9e5 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -2932,8 +2932,8 @@ func (l *channelLink) getFeeRate() chainfee.SatPerKWeight { // // NOTE: Part of the dustHandler interface. func (l *channelLink) getDustClosure() dustClosure { - localDustLimit := l.channel.State().LocalChanCfg.DustLimit - remoteDustLimit := l.channel.State().RemoteChanCfg.DustLimit + localDustLimit := l.channel.State().ChanCfgs.Local.DustLimit + remoteDustLimit := l.channel.State().ChanCfgs.Remote.DustLimit chanType := l.channel.State().ChanType return dustHelper(chanType, localDustLimit, remoteDustLimit) diff --git a/htlcswitch/test_utils.go b/htlcswitch/test_utils.go index 450d5a19d85..77d861e734f 100644 --- a/htlcswitch/test_utils.go +++ b/htlcswitch/test_utils.go @@ -301,8 +301,10 @@ func createTestChannel(t *testing.T, alicePrivKey, bobPrivKey []byte, } aliceChannelState := &channeldb.OpenChannel{ - LocalChanCfg: aliceCfg, - RemoteChanCfg: bobCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: aliceCfg, + Remote: bobCfg, + }, IdentityPub: aliceKeyPub, FundingOutpoint: *prevOut, ChanType: channeldb.SingleFunderTweaklessBit, @@ -320,8 +322,10 @@ func createTestChannel(t *testing.T, alicePrivKey, bobPrivKey []byte, } bobChannelState := &channeldb.OpenChannel{ - LocalChanCfg: bobCfg, - RemoteChanCfg: aliceCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: bobCfg, + Remote: aliceCfg, + }, IdentityPub: bobKeyPub, FundingOutpoint: *prevOut, ChanType: channeldb.SingleFunderTweaklessBit, diff --git a/lnwallet/aux_leaf_store.go b/lnwallet/aux_leaf_store.go index 4558c2f81ce..68bfbc1641e 100644 --- a/lnwallet/aux_leaf_store.go +++ b/lnwallet/aux_leaf_store.go @@ -112,8 +112,8 @@ func NewAuxChanState(chanState *channeldb.OpenChannel) AuxChanState { ShortChannelID: chanState.ShortChannelID, IsInitiator: chanState.IsInitiator, Capacity: chanState.Capacity, - LocalChanCfg: chanState.LocalChanCfg, - RemoteChanCfg: chanState.RemoteChanCfg, + LocalChanCfg: chanState.ChanCfgs.Local, + RemoteChanCfg: chanState.ChanCfgs.Remote, ThawHeight: chanState.ThawHeight, TapscriptRoot: chanState.TapscriptRoot, CustomBlob: chanState.CustomBlob, diff --git a/lnwallet/channel.go b/lnwallet/channel.go index 9417c8e6048..d042a2a0352 100644 --- a/lnwallet/channel.go +++ b/lnwallet/channel.go @@ -501,7 +501,7 @@ func (lc *LightningChannel) diskHtlcToPayDesc(feeRate chainfee.SatPerKWeight, // on-disk state snapshot. isDustLocal := HtlcIsDust( chanType, htlc.Incoming, lntypes.Local, feeRate, - htlc.Amt.ToSatoshis(), lc.channelState.LocalChanCfg.DustLimit, + htlc.Amt.ToSatoshis(), lc.channelState.ChanCfgs.Local.DustLimit, ) localCommitKeys := commitKeys.GetForParty(lntypes.Local) if !isDustLocal && localCommitKeys != nil { @@ -518,7 +518,8 @@ func (lc *LightningChannel) diskHtlcToPayDesc(feeRate chainfee.SatPerKWeight, } isDustRemote := HtlcIsDust( chanType, htlc.Incoming, lntypes.Remote, feeRate, - htlc.Amt.ToSatoshis(), lc.channelState.RemoteChanCfg.DustLimit, + htlc.Amt.ToSatoshis(), + lc.channelState.ChanCfgs.Remote.DustLimit, ) remoteCommitKeys := commitKeys.GetForParty(lntypes.Remote) if !isDustRemote && remoteCommitKeys != nil { @@ -641,16 +642,16 @@ func (lc *LightningChannel) diskCommitToMemCommit( commitKeys.SetForParty(lntypes.Local, DeriveCommitmentKeys( localCommitPoint, lntypes.Local, lc.channelState.ChanType, - &lc.channelState.LocalChanCfg, - &lc.channelState.RemoteChanCfg, + &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, )) } if remoteCommitPoint != nil { commitKeys.SetForParty(lntypes.Remote, DeriveCommitmentKeys( remoteCommitPoint, lntypes.Remote, lc.channelState.ChanType, - &lc.channelState.LocalChanCfg, - &lc.channelState.RemoteChanCfg, + &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, )) } @@ -702,9 +703,9 @@ func (lc *LightningChannel) diskCommitToMemCommit( customBlob: diskCommit.CustomBlob, } if whoseCommit.IsLocal() { - commit.dustLimit = lc.channelState.LocalChanCfg.DustLimit + commit.dustLimit = lc.channelState.ChanCfgs.Local.DustLimit } else { - commit.dustLimit = lc.channelState.RemoteChanCfg.DustLimit + commit.dustLimit = lc.channelState.ChanCfgs.Remote.DustLimit } return commit, nil @@ -1001,8 +1002,8 @@ func (lc *LightningChannel) createSignDesc() error { ) chanState := lc.channelState - localKey := chanState.LocalChanCfg.MultiSigKey.PubKey - remoteKey := chanState.RemoteChanCfg.MultiSigKey.PubKey + localKey := chanState.ChanCfgs.Local.MultiSigKey.PubKey + remoteKey := chanState.ChanCfgs.Remote.MultiSigKey.PubKey if chanState.ChanType.IsTaproot() { fundingPkScript, _, err = input.GenTaprootFundingScript( @@ -1032,7 +1033,7 @@ func (lc *LightningChannel) createSignDesc() error { Value: int64(lc.channelState.Capacity), } lc.signDesc = &input.SignDescriptor{ - KeyDesc: lc.channelState.LocalChanCfg.MultiSigKey, + KeyDesc: lc.channelState.ChanCfgs.Local.MultiSigKey, WitnessScript: multiSigScript, Output: &lc.fundingOutput, HashType: txscript.SigHashAll, @@ -1507,8 +1508,8 @@ func (lc *LightningChannel) restoreCommitState( pendingRemoteKeyChain = DeriveCommitmentKeys( pendingCommitPoint, lntypes.Remote, lc.channelState.ChanType, - &lc.channelState.LocalChanCfg, - &lc.channelState.RemoteChanCfg, + &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, ) } @@ -1845,7 +1846,7 @@ func (lc *LightningChannel) restorePendingLocalUpdates( &logUpdate, lc.updateLogs.Remote, pendingHeight, chainfee.SatPerKWeight(pendingCommit.FeePerKw), pendingRemoteKeys, - lc.channelState.RemoteChanCfg.DustLimit, + lc.channelState.ChanCfgs.Remote.DustLimit, auxResult.AuxLeaves, ) if err != nil { @@ -2060,7 +2061,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, // keys we'll need to reconstruct the commitment state, keyRing := DeriveCommitmentKeys( commitmentPoint, lntypes.Remote, chanState.ChanType, - &chanState.LocalChanCfg, &chanState.RemoteChanCfg, + &chanState.ChanCfgs.Local, &chanState.ChanCfgs.Remote, ) // Next, reconstruct the scripts as they were present at this state @@ -2101,7 +2102,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, return l.LocalAuxLeaf }, )(auxResult.AuxLeaves) - theirDelay := uint32(chanState.RemoteChanCfg.CsvDelay) + theirDelay := uint32(chanState.ChanCfgs.Remote.CsvDelay) theirScript, err := CommitScriptToSelf( chanState.ChanType, isRemoteInitiator, keyRing.ToLocalKey, keyRing.RevocationKey, theirDelay, leaseExpiry, localAuxLeaf, @@ -2150,7 +2151,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, // // If our balance exceeds the remote party's dust limit, instantiate // the sign descriptor for our output. - if ourAmt >= int64(chanState.RemoteChanCfg.DustLimit) { + if ourAmt >= int64(chanState.ChanCfgs.Remote.DustLimit) { // As we're about to sweep our own output w/o a delay, we'll // obtain the witness script for the success/delay path. witnessScript, err := ourScript.WitnessScriptForPath( @@ -2162,7 +2163,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, br.LocalOutputSignDesc = &input.SignDescriptor{ SingleTweak: keyRing.LocalCommitKeyTweak, - KeyDesc: chanState.LocalChanCfg.PaymentBasePoint, + KeyDesc: chanState.ChanCfgs.Local.PaymentBasePoint, //nolint:lll WitnessScript: witnessScript, Output: &wire.TxOut{ PkScript: ourScript.PkScript(), @@ -2227,7 +2228,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, // Similarly, if their balance exceeds the remote party's dust limit, // assemble the sign descriptor for their output, which we can sweep. - if theirAmt >= int64(chanState.RemoteChanCfg.DustLimit) { + if theirAmt >= int64(chanState.ChanCfgs.Remote.DustLimit) { // As we're trying to defend the channel against a breach // attempt from the remote party, we want to obain the // revocation witness script here. @@ -2239,7 +2240,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, } br.RemoteOutputSignDesc = &input.SignDescriptor{ - KeyDesc: chanState.LocalChanCfg. + KeyDesc: chanState.ChanCfgs.Local. RevocationBasePoint, DoubleTweak: commitmentSecret, WitnessScript: witnessScript, @@ -2324,7 +2325,7 @@ func createHtlcRetribution(chanState *channeldb.OpenChannel, var emptyRetribution HtlcRetribution - theirDelay := uint32(chanState.RemoteChanCfg.CsvDelay) + theirDelay := uint32(chanState.ChanCfgs.Remote.CsvDelay) isRemoteInitiator := !chanState.IsInitiator // We'll generate the original second level witness script now, as @@ -2382,7 +2383,7 @@ func createHtlcRetribution(chanState *channeldb.OpenChannel, } signDesc := input.SignDescriptor{ - KeyDesc: chanState.LocalChanCfg. + KeyDesc: chanState.ChanCfgs.Local. RevocationBasePoint, DoubleTweak: commitmentSecret, WitnessScript: scriptInfo.WitnessScriptToSign(), @@ -2588,7 +2589,7 @@ func createBreachRetributionLegacy(revokedLog *channeldb.ChannelCommitment, chanState.ChanType, htlc.Incoming, lntypes.Remote, chainfee.SatPerKWeight(revokedLog.FeePerKw), htlc.Amt.ToSatoshis(), - chanState.RemoteChanCfg.DustLimit, + chanState.ChanCfgs.Remote.DustLimit, ) { continue @@ -2744,10 +2745,10 @@ func (lc *LightningChannel) fetchCommitmentView( keyRing *CommitmentKeyRing) (*commitment, error) { commitChain := lc.commitChains.Local - dustLimit := lc.channelState.LocalChanCfg.DustLimit + dustLimit := lc.channelState.ChanCfgs.Local.DustLimit if whoseCommitChain.IsRemote() { commitChain = lc.commitChains.Remote - dustLimit = lc.channelState.RemoteChanCfg.DustLimit + dustLimit = lc.channelState.ChanCfgs.Remote.DustLimit } nextHeight := commitChain.tip().height + 1 @@ -3120,8 +3121,8 @@ func genRemoteHtlcSigJobs(keyRing *CommitmentKeyRing, var ( isRemoteInitiator = !chanState.IsInitiator - localChanCfg = chanState.LocalChanCfg - remoteChanCfg = chanState.RemoteChanCfg + localChanCfg = chanState.ChanCfgs.Local + remoteChanCfg = chanState.ChanCfgs.Remote chanType = chanState.ChanType ) @@ -3742,10 +3743,10 @@ func (lc *LightningChannel) validateCommitmentSanity(theirLogCounter, // If the added HTLCs will decrease the balance, make sure they won't // dip the local and remote balances below the channel reserves. ourReserve := lnwire.NewMSatFromSatoshis( - lc.channelState.LocalChanCfg.ChanReserve, + lc.channelState.ChanCfgs.Local.ChanReserve, ) theirReserve := lnwire.NewMSatFromSatoshis( - lc.channelState.RemoteChanCfg.ChanReserve, + lc.channelState.ChanCfgs.Remote.ChanReserve, ) // Calculate the commitment fee to log the information if needed. @@ -3829,7 +3830,7 @@ func (lc *LightningChannel) validateCommitmentSanity(theirLogCounter, // First check that the remote updates won't violate it's channel // constraints. err = validateUpdates( - filteredView.Updates.Remote, &lc.channelState.RemoteChanCfg, + filteredView.Updates.Remote, &lc.channelState.ChanCfgs.Remote, ) if err != nil { return err @@ -3838,7 +3839,7 @@ func (lc *LightningChannel) validateCommitmentSanity(theirLogCounter, // Secondly check that our updates won't violate our channel // constraints. err = validateUpdates( - filteredView.Updates.Local, &lc.channelState.LocalChanCfg, + filteredView.Updates.Local, &lc.channelState.ChanCfgs.Local, ) if err != nil { return err @@ -3946,7 +3947,8 @@ func (lc *LightningChannel) SignNextCommitment() (*NewCommitState, error) { // construct the commitment state. keyRing := DeriveCommitmentKeys( commitPoint, lntypes.Remote, lc.channelState.ChanType, - &lc.channelState.LocalChanCfg, &lc.channelState.RemoteChanCfg, + &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, ) // Create a new commitment view which will calculate the evaluated @@ -4543,10 +4545,10 @@ func (lc *LightningChannel) computeView(view *HtlcView, lnwire.MilliSatoshi, lntypes.WeightUnit, *HtlcView, error) { commitChain := lc.commitChains.Local - dustLimit := lc.channelState.LocalChanCfg.DustLimit + dustLimit := lc.channelState.ChanCfgs.Local.DustLimit if whoseCommitChain.IsRemote() { commitChain = lc.commitChains.Remote - dustLimit = lc.channelState.RemoteChanCfg.DustLimit + dustLimit = lc.channelState.ChanCfgs.Remote.DustLimit } // Since the fetched htlc view will include all updates added after the @@ -4692,7 +4694,7 @@ func genHtlcSigValidationJobs(chanState *channeldb.OpenChannel, var ( isLocalInitiator = chanState.IsInitiator - localChanCfg = chanState.LocalChanCfg + localChanCfg = chanState.ChanCfgs.Local chanType = chanState.ChanType ) @@ -5122,7 +5124,8 @@ func (lc *LightningChannel) ReceiveNewCommitment(commitSigs *CommitSigs) error { commitPoint := input.ComputeCommitmentPoint(commitSecret[:]) keyRing := DeriveCommitmentKeys( commitPoint, lntypes.Local, lc.channelState.ChanType, - &lc.channelState.LocalChanCfg, &lc.channelState.RemoteChanCfg, + &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, ) // With the current commitment point re-calculated, construct the new @@ -5254,7 +5257,7 @@ func (lc *LightningChannel) ReceiveNewCommitment(commitSigs *CommitSigs) error { return err } - verifyKey := lc.channelState.RemoteChanCfg.MultiSigKey.PubKey + verifyKey := lc.channelState.ChanCfgs.Remote.MultiSigKey.PubKey cSig, err := commitSigs.CommitSig.ToSignature() if err != nil { @@ -5893,11 +5896,11 @@ func (lc *LightningChannel) GetDustSum(whoseCommit lntypes.ChannelParty, var dustSum lnwire.MilliSatoshi - dustLimit := lc.channelState.LocalChanCfg.DustLimit + dustLimit := lc.channelState.ChanCfgs.Local.DustLimit commit := lc.channelState.LocalCommitment if whoseCommit.IsRemote() { // Calculate dust sum on the remote's commitment. - dustLimit = lc.channelState.RemoteChanCfg.DustLimit + dustLimit = lc.channelState.ChanCfgs.Remote.DustLimit commit = lc.channelState.RemoteCommitment } @@ -5968,8 +5971,8 @@ func (lc *LightningChannel) MayAddOutgoingHtlc(amt lnwire.MilliSatoshi) error { // In absence of a specific amount, we want to use minimum htlc value // for the channel. However certain implementations may set this value // to zero, so we only use this value if it is non-zero. - case lc.channelState.LocalChanCfg.MinHTLC != 0: - mockHtlcAmt = lc.channelState.LocalChanCfg.MinHTLC + case lc.channelState.ChanCfgs.Local.MinHTLC != 0: + mockHtlcAmt = lc.channelState.ChanCfgs.Local.MinHTLC // As a last resort, we just add a non-zero amount. default: @@ -6591,8 +6594,8 @@ func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) { inputs := SignedCommitTxInputs{ CommitTx: localCommit.CommitTx, CommitSig: localCommit.CommitSig, - OurKey: lc.channelState.LocalChanCfg.MultiSigKey, - TheirKey: lc.channelState.RemoteChanCfg.MultiSigKey, + OurKey: lc.channelState.ChanCfgs.Local.MultiSigKey, + TheirKey: lc.channelState.ChanCfgs.Remote.MultiSigKey, SignDesc: lc.signDesc, } @@ -6694,7 +6697,7 @@ func NewUnilateralCloseSummary(chanState *channeldb.OpenChannel, commitType := lntypes.Remote keyRing := DeriveCommitmentKeys( commitPoint, commitType, chanState.ChanType, - &chanState.LocalChanCfg, &chanState.RemoteChanCfg, + &chanState.ChanCfgs.Local, &chanState.ChanCfgs.Remote, ) auxResult, err := fn.MapOptionZ( @@ -6724,8 +6727,8 @@ func NewUnilateralCloseSummary(chanState *channeldb.OpenChannel, } htlcResolutions, err := extractHtlcResolutions( chainfee.SatPerKWeight(remoteCommit.FeePerKw), commitType, - signer, remoteCommit.Htlcs, keyRing, &chanState.LocalChanCfg, - &chanState.RemoteChanCfg, commitSpend.SpendingTx, + signer, remoteCommit.Htlcs, keyRing, &chanState.ChanCfgs.Local, + &chanState.ChanCfgs.Remote, commitSpend.SpendingTx, chanState.ChanType, isRemoteInitiator, leaseExpiry, auxResult.AuxLeaves, ) @@ -6766,7 +6769,7 @@ func NewUnilateralCloseSummary(chanState *channeldb.OpenChannel, // non-trimmed balance. var commitResolution *CommitOutputResolution if selfPoint != nil { - localPayBase := chanState.LocalChanCfg.PaymentBasePoint + localPayBase := chanState.ChanCfgs.Local.PaymentBasePoint // As the remote party has force closed, we just need the // success witness script. @@ -6856,7 +6859,7 @@ func NewUnilateralCloseSummary(chanState *channeldb.OpenChannel, RemoteCurrentRevocation: chanState.RemoteCurrentRevocation, RemoteNextRevocation: chanState.RemoteNextRevocation, ShortChanID: chanState.ShortChanID(), - LocalChanConfig: chanState.LocalChanCfg, + LocalChanConfig: chanState.ChanCfgs.Local, } // Attempt to add a channel sync message to the close summary. @@ -7709,7 +7712,6 @@ func (lc *LightningChannel) ForceClose() (*LocalForceCloseSummary, error) { return summary, nil } - // NewLocalForceCloseSummary generates a LocalForceCloseSummary from the given // channel state. The passed commitTx must be a fully signed commitment // transaction corresponding to localCommit. @@ -7723,7 +7725,7 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, // commitment transaction. We'll need this to find the corresponding // output in the commitment transaction and potentially for creating // the sign descriptor. - csvTimeout := uint32(chanState.LocalChanCfg.CsvDelay) + csvTimeout := uint32(chanState.ChanCfgs.Local.CsvDelay) // We use the passed state num to derive our scripts, since in case // this is after recovery, our latest channels state might not be up to @@ -7735,7 +7737,7 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, commitPoint := input.ComputeCommitmentPoint(revocation[:]) keyRing := DeriveCommitmentKeys( commitPoint, lntypes.Local, chanState.ChanType, - &chanState.LocalChanCfg, &chanState.RemoteChanCfg, + &chanState.ChanCfgs.Local, &chanState.ChanCfgs.Remote, ) auxResult, err := fn.MapOptionZ( @@ -7810,7 +7812,7 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, Index: delayIndex, }, SelfOutputSignDesc: input.SignDescriptor{ - KeyDesc: chanState.LocalChanCfg.DelayBasePoint, + KeyDesc: chanState.ChanCfgs.Local.DelayBasePoint, //nolint:lll SingleTweak: keyRing.LocalCommitKeyTweak, WitnessScript: witnessScript, Output: &wire.TxOut{ @@ -7880,8 +7882,8 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, localCommit := chanState.LocalCommitment htlcResolutions, err := extractHtlcResolutions( chainfee.SatPerKWeight(localCommit.FeePerKw), lntypes.Local, - signer, localCommit.Htlcs, keyRing, &chanState.LocalChanCfg, - &chanState.RemoteChanCfg, commitTx, chanState.ChanType, + signer, localCommit.Htlcs, keyRing, &chanState.ChanCfgs.Local, + &chanState.ChanCfgs.Remote, commitTx, chanState.ChanType, chanState.IsInitiator, leaseExpiry, auxResult.AuxLeaves, ) if err != nil { @@ -8026,9 +8028,11 @@ func (lc *LightningChannel) CreateCloseProposal(proposedFee btcutil.Amount, } closeTx, err := CreateCooperativeCloseTx( - fundingTxIn(lc.channelState), lc.channelState.LocalChanCfg.DustLimit, - lc.channelState.RemoteChanCfg.DustLimit, ourBalance, theirBalance, - localDeliveryScript, remoteDeliveryScript, closeTxOpts..., + fundingTxIn(lc.channelState), + lc.channelState.ChanCfgs.Local.DustLimit, + lc.channelState.ChanCfgs.Remote.DustLimit, ourBalance, + theirBalance, localDeliveryScript, remoteDeliveryScript, + closeTxOpts..., ) if err != nil { return nil, nil, 0, err @@ -8130,9 +8134,11 @@ func (lc *LightningChannel) CompleteCooperativeClose( // on this active channel back to both parties. In this current model, // the initiator pays full fees for the cooperative close transaction. closeTx, err := CreateCooperativeCloseTx( - fundingTxIn(lc.channelState), lc.channelState.LocalChanCfg.DustLimit, - lc.channelState.RemoteChanCfg.DustLimit, ourBalance, theirBalance, - localDeliveryScript, remoteDeliveryScript, closeTxOpts..., + fundingTxIn(lc.channelState), + lc.channelState.ChanCfgs.Local.DustLimit, + lc.channelState.ChanCfgs.Remote.DustLimit, ourBalance, + theirBalance, localDeliveryScript, remoteDeliveryScript, + closeTxOpts..., ) if err != nil { return nil, 0, err @@ -8180,9 +8186,9 @@ func (lc *LightningChannel) CompleteCooperativeClose( } else { // For regular channels, we'll need to , construct the witness // stack minding the order of the pubkeys+sigs on the stack. - ourKey := lc.channelState.LocalChanCfg.MultiSigKey.PubKey. + ourKey := lc.channelState.ChanCfgs.Local.MultiSigKey.PubKey. SerializeCompressed() - theirKey := lc.channelState.RemoteChanCfg.MultiSigKey.PubKey. + theirKey := lc.channelState.ChanCfgs.Remote.MultiSigKey.PubKey. SerializeCompressed() witness := input.SpendMultiSig( lc.signDesc.WitnessScript, ourKey, localSig, theirKey, @@ -8250,7 +8256,8 @@ func (lc *LightningChannel) NewAnchorResolutions() (*AnchorResolutions, localCommitPoint := input.ComputeCommitmentPoint(revocation[:]) localKeyRing := DeriveCommitmentKeys( localCommitPoint, lntypes.Local, lc.channelState.ChanType, - &lc.channelState.LocalChanCfg, &lc.channelState.RemoteChanCfg, + &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, ) localRes, err := NewAnchorResolution( lc.channelState, lc.channelState.LocalCommitment.CommitTx, @@ -8264,8 +8271,8 @@ func (lc *LightningChannel) NewAnchorResolutions() (*AnchorResolutions, // Add anchor for remote commitment tx, if any. remoteKeyRing := DeriveCommitmentKeys( lc.channelState.RemoteCurrentRevocation, lntypes.Remote, - lc.channelState.ChanType, &lc.channelState.LocalChanCfg, - &lc.channelState.RemoteChanCfg, + lc.channelState.ChanType, &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, ) remoteRes, err := NewAnchorResolution( lc.channelState, lc.channelState.RemoteCommitment.CommitTx, @@ -8285,8 +8292,9 @@ func (lc *LightningChannel) NewAnchorResolutions() (*AnchorResolutions, if remotePendingCommit != nil { pendingRemoteKeyRing := DeriveCommitmentKeys( lc.channelState.RemoteNextRevocation, lntypes.Remote, - lc.channelState.ChanType, &lc.channelState.LocalChanCfg, - &lc.channelState.RemoteChanCfg, + lc.channelState.ChanType, + &lc.channelState.ChanCfgs.Local, + &lc.channelState.ChanCfgs.Remote, ) remotePendingRes, err := NewAnchorResolution( lc.channelState, @@ -8318,8 +8326,8 @@ func NewAnchorResolution(chanState *channeldb.OpenChannel, // will differ depending on if this is our local or remote // commitment. localAnchor, remoteAnchor, err := CommitScriptAnchors( - chanState.ChanType, &chanState.LocalChanCfg, - &chanState.RemoteChanCfg, keyRing, + chanState.ChanType, &chanState.ChanCfgs.Local, + &chanState.ChanCfgs.Remote, keyRing, ) if err != nil { return nil, err @@ -8356,7 +8364,7 @@ func NewAnchorResolution(chanState *channeldb.OpenChannel, // Instantiate the sign descriptor that allows sweeping of the anchor. signDesc := &input.SignDescriptor{ - KeyDesc: chanState.LocalChanCfg.MultiSigKey, + KeyDesc: chanState.ChanCfgs.Local.MultiSigKey, WitnessScript: anchorWitnessScript, Output: &wire.TxOut{ PkScript: localAnchor.PkScript(), @@ -8386,14 +8394,15 @@ func NewAnchorResolution(chanState *channeldb.OpenChannel, // tweak. signDesc.SingleTweak = keyRing.LocalCommitKeyTweak - signDesc.KeyDesc = chanState.LocalChanCfg.DelayBasePoint + signDesc.KeyDesc = + chanState.ChanCfgs.Local.DelayBasePoint } else { // When we're playing the force close of a remote // commitment, as this is a "tweakless" channel type, // we don't need a tweak value at all. // //nolint:lll - signDesc.KeyDesc = chanState.LocalChanCfg.PaymentBasePoint + signDesc.KeyDesc = chanState.ChanCfgs.Local.PaymentBasePoint } // Finally, as this is a keyspend method, we'll need to also @@ -8507,7 +8516,7 @@ func (lc *LightningChannel) availableCommitmentBalance(view *HtlcView, // We can never spend from the channel reserve, so we'll subtract it // from our available balance. ourReserve := lnwire.NewMSatFromSatoshis( - lc.channelState.LocalChanCfg.ChanReserve, + lc.channelState.ChanCfgs.Local.ChanReserve, ) if ourReserve <= ourBalance { ourBalance -= ourReserve @@ -8561,7 +8570,7 @@ func (lc *LightningChannel) availableCommitmentBalance(view *HtlcView, // enough balance to pay for the fee of our HTLC. We'll start by also // subtracting our counterparty's reserve from their balance. theirReserve := lnwire.NewMSatFromSatoshis( - lc.channelState.RemoteChanCfg.ChanReserve, + lc.channelState.ChanCfgs.Remote.ChanReserve, ) if theirReserve <= theirBalance { theirBalance -= theirReserve @@ -8572,7 +8581,7 @@ func (lc *LightningChannel) availableCommitmentBalance(view *HtlcView, // We'll use the dustlimit and htlcFee to find the largest HTLC value // that will be considered dust on the commitment. dustlimit := lnwire.NewMSatFromSatoshis( - lc.channelState.LocalChanCfg.DustLimit, + lc.channelState.ChanCfgs.Local.DustLimit, ) // For an extra HTLC fee to be paid on our commitment, the HTLC must be @@ -8585,7 +8594,7 @@ func (lc *LightningChannel) availableCommitmentBalance(view *HtlcView, // dust limit and the fee for adding an HTLC success transaction. if whoseCommitChain.IsRemote() { dustlimit = lnwire.NewMSatFromSatoshis( - lc.channelState.RemoteChanCfg.DustLimit, + lc.channelState.ChanCfgs.Remote.DustLimit, ) htlcFee = lnwire.NewMSatFromSatoshis( HtlcSuccessFee(lc.channelState.ChanType, feePerKw), @@ -8814,7 +8823,7 @@ func (lc *LightningChannel) generateRevocation(height uint64) (*lnwire.RevokeAnd // verification nonce for this target state. if lc.channelState.ChanType.IsTaproot() { nextVerificationNonce, err := channeldb.NewMusigVerificationNonce( //nolint:lll - lc.channelState.LocalChanCfg.MultiSigKey.PubKey, + lc.channelState.ChanCfgs.Local.MultiSigKey.PubKey, revHeight, lc.taprootNonceProducer, ) if err != nil { @@ -9049,7 +9058,7 @@ func (lc *LightningChannel) LocalBalanceDust() (bool, btcutil.Amount) { localBalance += 2 * AnchorSize } - localDust := chanState.LocalChanCfg.DustLimit + localDust := chanState.ChanCfgs.Local.DustLimit return localBalance <= localDust, localDust } @@ -9071,7 +9080,7 @@ func (lc *LightningChannel) RemoteBalanceDust() (bool, btcutil.Amount) { remoteBalance += 2 * AnchorSize } - remoteDust := chanState.RemoteChanCfg.DustLimit + remoteDust := chanState.ChanCfgs.Remote.DustLimit return remoteBalance <= remoteDust, remoteDust } @@ -9120,8 +9129,8 @@ func (lc *LightningChannel) CalcFee(feeRate chainfee.SatPerKWeight) btcutil.Amou // Moreover it returns the share of the total balance in the range of [0,1] // which can be allocated to fees. When our desired fee allocation would lead to // a maximum fee rate below the current commitment fee rate we floor the maximum -// at the current fee rate which leads to different fee allocations than -// initially requested via `maxAllocation`. +// fee rate at the current fee rate which leads to different fee allocations +// than initially requested via `maxAllocation`. // // NOTE: This should only be used for channels in which the local commitment is // the initiator. @@ -9364,7 +9373,7 @@ func (lc *LightningChannel) ActiveHtlcs() []channeldb.HTLC { // LocalChanReserve returns our local ChanReserve requirement for the remote party. func (lc *LightningChannel) LocalChanReserve() btcutil.Amount { - return lc.channelState.LocalChanCfg.ChanReserve + return lc.channelState.ChanCfgs.Local.ChanReserve } // NextLocalHtlcIndex returns the next unallocated local htlc index. To ensure @@ -9381,7 +9390,7 @@ func (lc *LightningChannel) NextLocalHtlcIndex() (uint64, error) { // FwdMinHtlc returns the minimum HTLC value required by the remote node, i.e. // the minimum value HTLC we can forward on this channel. func (lc *LightningChannel) FwdMinHtlc() lnwire.MilliSatoshi { - return lc.channelState.LocalChanCfg.MinHTLC + return lc.channelState.ChanCfgs.Local.MinHTLC } // unsignedLocalUpdates retrieves the unsigned local updates that we should @@ -9434,7 +9443,7 @@ func (lc *LightningChannel) GenMusigNonces() (*musig2.Nonces, error) { // verification nonces we'll send to the party to create our _next_ // state. lc.pendingVerificationNonce, err = channeldb.NewMusigVerificationNonce( - lc.channelState.LocalChanCfg.MultiSigKey.PubKey, + lc.channelState.ChanCfgs.Local.MultiSigKey.PubKey, lc.currentHeight+1, lc.taprootNonceProducer, ) if err != nil { @@ -9468,8 +9477,8 @@ func (lc *LightningChannel) InitRemoteMusigNonces(remoteNonce *musig2.Nonces, // commitment of the remote party. localNonce := lc.pendingVerificationNonce - localChanCfg := lc.channelState.LocalChanCfg - remoteChanCfg := lc.channelState.RemoteChanCfg + localChanCfg := lc.channelState.ChanCfgs.Local + remoteChanCfg := lc.channelState.ChanCfgs.Remote // TODO(roasbeef): propagate rename of signing and verification nonces @@ -9525,8 +9534,8 @@ func (lc *LightningChannel) MultiSigKeys() (keychain.KeyDescriptor, lc.RLock() defer lc.RUnlock() - return lc.channelState.LocalChanCfg.MultiSigKey, - lc.channelState.RemoteChanCfg.MultiSigKey + return lc.channelState.ChanCfgs.Local.MultiSigKey, + lc.channelState.ChanCfgs.Remote.MultiSigKey } // LocalCommitmentBlob returns the custom blob of the local commitment. diff --git a/lnwallet/channel_test.go b/lnwallet/channel_test.go index 49e4321beb6..92ed99f9acd 100644 --- a/lnwallet/channel_test.go +++ b/lnwallet/channel_test.go @@ -1027,7 +1027,8 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { } // The rest of the close summary should have been populated properly. - aliceDelayPoint := aliceChannel.channelState.LocalChanCfg.DelayBasePoint + aliceDelayPoint := + aliceChannel.channelState.ChanCfgs.Local.DelayBasePoint if !aliceCommitResolution.SelfOutputSignDesc.KeyDesc.PubKey.IsEqual( aliceDelayPoint.PubKey, ) { @@ -1055,11 +1056,11 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { // Alice's listed CSV delay should also match the delay that was // pre-committed to at channel opening. if aliceCommitResolution.MaturityDelay != - uint32(aliceChannel.channelState.LocalChanCfg.CsvDelay) { + uint32(aliceChannel.channelState.ChanCfgs.Local.CsvDelay) { t.Fatalf("alice: incorrect local CSV delay in ForceCloseSummary, "+ "expected %v, got %v", - aliceChannel.channelState.LocalChanCfg.CsvDelay, + aliceChannel.channelState.ChanCfgs.Local.CsvDelay, aliceCommitResolution.MaturityDelay) } @@ -1103,7 +1104,7 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { }) htlcResolution.SweepSignDesc.InputIndex = 0 - csvDelay := uint32(aliceChannel.channelState.LocalChanCfg.CsvDelay) + csvDelay := uint32(aliceChannel.channelState.ChanCfgs.Local.CsvDelay) if testCase.chanType.IsTaproot() { sweepTx.TxIn[0].Sequence = input.LockTimeToSequence( false, csvDelay, @@ -1205,7 +1206,8 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { sweepTx.TxIn[0].Witness, err = input.HtlcSpendSuccess( aliceChannel.Signer, &inHtlcResolution.SweepSignDesc, sweepTx, - uint32(aliceChannel.channelState.LocalChanCfg.CsvDelay), + //nolint:lll + uint32(aliceChannel.channelState.ChanCfgs.Local.CsvDelay), ) } require.NoError(t, err, "unable to gen witness for timeout output") @@ -1235,7 +1237,7 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { if bobCommitResolution == nil { t.Fatalf("bob fails to include to-self output in ForceCloseSummary") } - bobDelayPoint := bobChannel.channelState.LocalChanCfg.DelayBasePoint + bobDelayPoint := bobChannel.channelState.ChanCfgs.Local.DelayBasePoint if !bobCommitResolution.SelfOutputSignDesc.KeyDesc.PubKey.IsEqual(bobDelayPoint.PubKey) { t.Fatalf("bob incorrect pubkey in SelfOutputSignDesc") } @@ -1248,11 +1250,11 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { int64(bobCommitResolution.SelfOutputSignDesc.Output.Value)) } if bobCommitResolution.MaturityDelay != - uint32(bobChannel.channelState.LocalChanCfg.CsvDelay) { + uint32(bobChannel.channelState.ChanCfgs.Local.CsvDelay) { t.Fatalf("bob: incorrect local CSV delay in ForceCloseSummary, "+ "expected %v, got %v", - bobChannel.channelState.LocalChanCfg.CsvDelay, + bobChannel.channelState.ChanCfgs.Local.CsvDelay, bobCommitResolution.MaturityDelay) } @@ -1298,10 +1300,10 @@ func TestForceCloseDustOutput(t *testing.T) { // We set both node's channel reserves to 0, to make sure // they can create small dust outputs without going under // their channel reserves. - aliceChannel.channelState.LocalChanCfg.ChanReserve = 0 - bobChannel.channelState.LocalChanCfg.ChanReserve = 0 - aliceChannel.channelState.RemoteChanCfg.ChanReserve = 0 - bobChannel.channelState.RemoteChanCfg.ChanReserve = 0 + aliceChannel.channelState.ChanCfgs.Local.ChanReserve = 0 + bobChannel.channelState.ChanCfgs.Local.ChanReserve = 0 + aliceChannel.channelState.ChanCfgs.Remote.ChanReserve = 0 + bobChannel.channelState.ChanCfgs.Remote.ChanReserve = 0 htlcAmount := lnwire.NewMSatFromSatoshis(500) @@ -1344,7 +1346,7 @@ func TestForceCloseDustOutput(t *testing.T) { "ForceCloseSummary") } if !commitResolution.SelfOutputSignDesc.KeyDesc.PubKey.IsEqual( - aliceChannel.channelState.LocalChanCfg.DelayBasePoint.PubKey, + aliceChannel.channelState.ChanCfgs.Local.DelayBasePoint.PubKey, ) { t.Fatalf("alice incorrect pubkey in SelfOutputSignDesc") } @@ -1357,10 +1359,11 @@ func TestForceCloseDustOutput(t *testing.T) { } if commitResolution.MaturityDelay != - uint32(aliceChannel.channelState.LocalChanCfg.CsvDelay) { + uint32(aliceChannel.channelState.ChanCfgs.Local.CsvDelay) { + t.Fatalf("alice: incorrect local CSV delay in ForceCloseSummary, "+ "expected %v, got %v", - aliceChannel.channelState.LocalChanCfg.CsvDelay, + aliceChannel.channelState.ChanCfgs.Local.CsvDelay, commitResolution.MaturityDelay) } @@ -1721,8 +1724,8 @@ func TestChannelBalanceDustLimit(t *testing.T) { // To allow Alice's balance to get beneath her dust limit, set the // channel reserve to be 0. - aliceChannel.channelState.LocalChanCfg.ChanReserve = 0 - bobChannel.channelState.RemoteChanCfg.ChanReserve = 0 + aliceChannel.channelState.ChanCfgs.Local.ChanReserve = 0 + bobChannel.channelState.ChanCfgs.Remote.ChanReserve = 0 // This amount should leave an amount larger than Alice's dust limit // once fees have been subtracted, but smaller than Bob's dust limit. @@ -2244,10 +2247,10 @@ func TestCooperativeCloseDustAdherence(t *testing.T) { ) setDustLimit := func(dustVal btcutil.Amount) { - aliceChannel.channelState.LocalChanCfg.DustLimit = dustVal - aliceChannel.channelState.RemoteChanCfg.DustLimit = dustVal - bobChannel.channelState.LocalChanCfg.DustLimit = dustVal - bobChannel.channelState.RemoteChanCfg.DustLimit = dustVal + aliceChannel.channelState.ChanCfgs.Local.DustLimit = dustVal + aliceChannel.channelState.ChanCfgs.Remote.DustLimit = dustVal + bobChannel.channelState.ChanCfgs.Local.DustLimit = dustVal + bobChannel.channelState.ChanCfgs.Remote.DustLimit = dustVal } resetChannelState := func() { @@ -2935,7 +2938,7 @@ func TestAddHTLCNegativeBalance(t *testing.T) { // We set the channel reserve to 0, such that we can add HTLCs all the // way to a negative balance. - aliceChannel.channelState.LocalChanCfg.ChanReserve = 0 + aliceChannel.channelState.ChanCfgs.Local.ChanReserve = 0 // First, we'll add 3 HTLCs of 1 BTC each to Alice's commitment. const numHTLCs = 3 @@ -5140,7 +5143,7 @@ func TestChanAvailableBandwidth(t *testing.T) { require.NoError(t, err, "unable to create test channels") aliceReserve := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.LocalChanCfg.ChanReserve, + aliceChannel.channelState.ChanCfgs.Local.ChanReserve, ) feeRate := chainfee.SatPerKWeight( aliceChannel.channelState.LocalCommitment.FeePerKw, @@ -5286,14 +5289,14 @@ func TestChanAvailableBalanceNearHtlcFee(t *testing.T) { bobBalance := lnwire.NewMSatFromSatoshis(5 * btcutil.SatoshiPerBitcoin) aliceReserve := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.LocalChanCfg.ChanReserve, + aliceChannel.channelState.ChanCfgs.Local.ChanReserve, ) bobReserve := lnwire.NewMSatFromSatoshis( - bobChannel.channelState.LocalChanCfg.ChanReserve, + bobChannel.channelState.ChanCfgs.Local.ChanReserve, ) aliceDustlimit := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.LocalChanCfg.DustLimit, + aliceChannel.channelState.ChanCfgs.Local.DustLimit, ) feeRate := chainfee.SatPerKWeight( aliceChannel.channelState.LocalCommitment.FeePerKw, @@ -5473,10 +5476,10 @@ func TestChanCommitWeightDustHtlcs(t *testing.T) { require.NoError(t, err, "unable to create test channels") aliceDustlimit := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.LocalChanCfg.DustLimit, + aliceChannel.channelState.ChanCfgs.Local.DustLimit, ) bobDustlimit := lnwire.NewMSatFromSatoshis( - bobChannel.channelState.LocalChanCfg.DustLimit, + bobChannel.channelState.ChanCfgs.Local.DustLimit, ) feeRate := chainfee.SatPerKWeight( @@ -6343,14 +6346,14 @@ func TestMaxAcceptedHTLCs(t *testing.T) { // Set the remote's required MaxAcceptedHtlcs. This means that Alice // can only offer the remote up to numHTLCs HTLCs. - aliceChannel.channelState.LocalChanCfg.MaxAcceptedHtlcs = numHTLCs - bobChannel.channelState.RemoteChanCfg.MaxAcceptedHtlcs = numHTLCs + aliceChannel.channelState.ChanCfgs.Local.MaxAcceptedHtlcs = numHTLCs + bobChannel.channelState.ChanCfgs.Remote.MaxAcceptedHtlcs = numHTLCs // Similarly, set the remote config's MaxAcceptedHtlcs. This means // that the remote will be aware that Bob will only accept up to // numHTLCs at a time. - aliceChannel.channelState.RemoteChanCfg.MaxAcceptedHtlcs = numHTLCs - bobChannel.channelState.LocalChanCfg.MaxAcceptedHtlcs = numHTLCs + aliceChannel.channelState.ChanCfgs.Remote.MaxAcceptedHtlcs = numHTLCs + bobChannel.channelState.ChanCfgs.Local.MaxAcceptedHtlcs = numHTLCs // Each HTLC amount is 0.1 BTC. htlcAmt := lnwire.NewMSatFromSatoshis(0.1 * btcutil.SatoshiPerBitcoin) @@ -6470,14 +6473,14 @@ func TestMaxAsynchronousHtlcs(t *testing.T) { // Set the remote's required MaxAcceptedHtlcs. This means that Alice // can only offer the remote up to numHTLCs HTLCs. - aliceChannel.channelState.LocalChanCfg.MaxAcceptedHtlcs = numHTLCs - bobChannel.channelState.RemoteChanCfg.MaxAcceptedHtlcs = numHTLCs + aliceChannel.channelState.ChanCfgs.Local.MaxAcceptedHtlcs = numHTLCs + bobChannel.channelState.ChanCfgs.Remote.MaxAcceptedHtlcs = numHTLCs // Similarly, set the remote config's MaxAcceptedHtlcs. This means // that the remote will be aware that Bob will only accept up to // numHTLCs at a time. - aliceChannel.channelState.RemoteChanCfg.MaxAcceptedHtlcs = numHTLCs - bobChannel.channelState.LocalChanCfg.MaxAcceptedHtlcs = numHTLCs + aliceChannel.channelState.ChanCfgs.Remote.MaxAcceptedHtlcs = numHTLCs + bobChannel.channelState.ChanCfgs.Local.MaxAcceptedHtlcs = numHTLCs // Each HTLC amount is 0.1 BTC. htlcAmt := lnwire.NewMSatFromSatoshis(0.1 * btcutil.SatoshiPerBitcoin) @@ -6585,8 +6588,8 @@ func TestMaxPendingAmount(t *testing.T) { // We set the max pending amount of Alice's config. This mean that she // cannot offer Bob HTLCs with a total value above this limit at a given // time. - aliceChannel.channelState.LocalChanCfg.MaxPendingAmount = maxPending - bobChannel.channelState.RemoteChanCfg.MaxPendingAmount = maxPending + aliceChannel.channelState.ChanCfgs.Local.MaxPendingAmount = maxPending + bobChannel.channelState.ChanCfgs.Remote.MaxPendingAmount = maxPending // First, we'll add 2 HTLCs of 1.5 BTC each to Alice's commitment. // This won't trigger Alice's ErrMaxPendingAmount error. @@ -6669,20 +6672,24 @@ func TestChanReserve(t *testing.T) { // Alice will need to keep her reserve above aliceMinReserve, // so set this limit to here local config. - aliceChannel.channelState.LocalChanCfg.ChanReserve = aliceMinReserve + aliceChannel.channelState.ChanCfgs.Local.ChanReserve = + aliceMinReserve // During channel opening Bob will also get to know Alice's // minimum reserve, and this will be found in his remote // config. - bobChannel.channelState.RemoteChanCfg.ChanReserve = aliceMinReserve + bobChannel.channelState.ChanCfgs.Remote.ChanReserve = + aliceMinReserve // We set Bob's channel reserve to a value that is larger than // his current balance in the channel. This will ensure that // after a channel is first opened, Bob can still receive HTLCs // even though his balance is less than his channel reserve. bobMinReserve := btcutil.Amount(6 * btcutil.SatoshiPerBitcoin) - bobChannel.channelState.LocalChanCfg.ChanReserve = bobMinReserve - aliceChannel.channelState.RemoteChanCfg.ChanReserve = bobMinReserve + bobChannel.channelState.ChanCfgs.Local.ChanReserve = + bobMinReserve + aliceChannel.channelState.ChanCfgs.Remote.ChanReserve = + bobMinReserve return aliceChannel, bobChannel } @@ -6865,8 +6872,8 @@ func TestChanReserveRemoteInitiator(t *testing.T) { commitFee := aliceChannel.channelState.LocalCommitment.CommitFee aliceMinReserve := 5*btcutil.SatoshiPerBitcoin - commitFee - aliceChannel.channelState.LocalChanCfg.ChanReserve = aliceMinReserve - bobChannel.channelState.RemoteChanCfg.ChanReserve = aliceMinReserve + aliceChannel.channelState.ChanCfgs.Local.ChanReserve = aliceMinReserve + bobChannel.channelState.ChanCfgs.Remote.ChanReserve = aliceMinReserve // Now let Bob attempt to add an HTLC of 0.1 BTC. He has plenty of // money available to spend, but Alice, which is the initiator, cannot @@ -6920,8 +6927,8 @@ func TestChanReserveLocalInitiatorDustHtlc(t *testing.T) { commitFee := aliceChannel.channelState.LocalCommitment.CommitFee aliceMinReserve := 5*btcutil.SatoshiPerBitcoin - commitFee - htlcSat - aliceChannel.channelState.LocalChanCfg.ChanReserve = aliceMinReserve - bobChannel.channelState.RemoteChanCfg.ChanReserve = aliceMinReserve + aliceChannel.channelState.ChanCfgs.Local.ChanReserve = aliceMinReserve + bobChannel.channelState.ChanCfgs.Remote.ChanReserve = aliceMinReserve htlcDustAmt := lnwire.NewMSatFromSatoshis(htlcSat) htlc, _ := createHTLC(0, htlcDustAmt) @@ -6950,8 +6957,8 @@ func TestMinHTLC(t *testing.T) { // Setting the min value in Alice's local config means that the // remote will not accept any HTLCs of value less than specified. - aliceChannel.channelState.LocalChanCfg.MinHTLC = minValue - bobChannel.channelState.RemoteChanCfg.MinHTLC = minValue + aliceChannel.channelState.ChanCfgs.Local.MinHTLC = minValue + bobChannel.channelState.ChanCfgs.Remote.MinHTLC = minValue // First, we will add an HTLC of 0.5 BTC. This will not trigger // ErrBelowMinHTLC. @@ -6995,8 +7002,8 @@ func TestInvalidHTLCAmt(t *testing.T) { // We'll set the min HTLC values for each party to zero, which // technically would permit zero-value HTLCs. - aliceChannel.channelState.LocalChanCfg.MinHTLC = 0 - bobChannel.channelState.RemoteChanCfg.MinHTLC = 0 + aliceChannel.channelState.ChanCfgs.Local.MinHTLC = 0 + bobChannel.channelState.ChanCfgs.Remote.MinHTLC = 0 // Create a zero-value HTLC. htlcAmt := lnwire.MilliSatoshi(0) @@ -7034,10 +7041,10 @@ func TestNewBreachRetributionSkipsDustHtlcs(t *testing.T) { // We'll modify the dust settings on both channels to be a predictable // value for the prurpose of the test. dustValue := btcutil.Amount(200) - aliceChannel.channelState.LocalChanCfg.DustLimit = dustValue - aliceChannel.channelState.RemoteChanCfg.DustLimit = dustValue - bobChannel.channelState.LocalChanCfg.DustLimit = dustValue - bobChannel.channelState.RemoteChanCfg.DustLimit = dustValue + aliceChannel.channelState.ChanCfgs.Local.DustLimit = dustValue + aliceChannel.channelState.ChanCfgs.Remote.DustLimit = dustValue + bobChannel.channelState.ChanCfgs.Local.DustLimit = dustValue + bobChannel.channelState.ChanCfgs.Remote.DustLimit = dustValue // We'll now create a series of dust HTLC's, and send then from Alice // to Bob, finally locking both of them in. @@ -9405,7 +9412,7 @@ func TestMayAddOutgoingHtlc(t *testing.T) { // Hard set alice's min htlc to zero and test the case where we just // fall back to a non-zero value. - aliceChannel.channelState.LocalChanCfg.MinHTLC = 0 + aliceChannel.channelState.ChanCfgs.Local.MinHTLC = 0 require.NoError(t, aliceChannel.MayAddOutgoingHtlc(0)) } @@ -9738,12 +9745,12 @@ func testGetDustSum(t *testing.T, chantype channeldb.ChannelType) { func deriveDummyRetributionParams(chanState *channeldb.OpenChannel) (uint32, *CommitmentKeyRing, chainhash.Hash) { - config := chanState.RemoteChanCfg + config := chanState.ChanCfgs.Remote commitHash := chanState.RemoteCommitment.CommitTx.TxHash() keyRing := DeriveCommitmentKeys( config.RevocationBasePoint.PubKey, lntypes.Remote, - chanState.ChanType, &chanState.LocalChanCfg, - &chanState.RemoteChanCfg, + chanState.ChanType, &chanState.ChanCfgs.Local, + &chanState.ChanCfgs.Remote, ) leaseExpiry := chanState.ThawHeight return leaseExpiry, keyRing, commitHash @@ -10099,7 +10106,7 @@ func testNewBreachRetribution(t *testing.T, chanType channeldb.ChannelType) { breachHeight := uint32(101) stateNum := uint64(0) chainHash := aliceChannel.channelState.ChainHash - theirDelay := uint32(aliceChannel.channelState.RemoteChanCfg.CsvDelay) + theirDelay := uint32(aliceChannel.channelState.ChanCfgs.Remote.CsvDelay) breachTx := aliceChannel.channelState.RemoteCommitment.CommitTx // Create a breach retribution at height 0, which should give us an @@ -10452,7 +10459,7 @@ func TestAsynchronousSendingContraint(t *testing.T) { ) require.NoError(t, err) - aliceReserve := aliceChannel.channelState.LocalChanCfg.ChanReserve + aliceReserve := aliceChannel.channelState.ChanCfgs.Local.ChanReserve capacity := aliceChannel.channelState.Capacity @@ -10487,7 +10494,7 @@ func TestAsynchronousSendingContraint(t *testing.T) { // We need to take the remote dustlimit amount, because it the greater // one. htlcAmt2 := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.RemoteChanCfg.DustLimit + htlcFee, + aliceChannel.channelState.ChanCfgs.Remote.DustLimit + htlcFee, ) htlc2, _ := createHTLC(0, htlcAmt2) @@ -10585,7 +10592,7 @@ func TestAsynchronousSendingWithFeeBuffer(t *testing.T) { ) require.NoError(t, err) - aliceReserve := aliceChannel.channelState.LocalChanCfg.ChanReserve + aliceReserve := aliceChannel.channelState.ChanCfgs.Local.ChanReserve capacity := aliceChannel.channelState.Capacity @@ -10619,7 +10626,7 @@ func TestAsynchronousSendingWithFeeBuffer(t *testing.T) { // make sure this htlc is non-dust for alice. htlcFee := HtlcSuccessFee(channeldb.SingleFunderTweaklessBit, feePerKw) htlcAmt2 := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.LocalChanCfg.DustLimit + htlcFee, + aliceChannel.channelState.ChanCfgs.Local.DustLimit + htlcFee, ) htlc2, _ := createHTLC(0, htlcAmt2) _, err = bobChannel.AddHTLC(htlc2, nil) @@ -10703,7 +10710,7 @@ func TestAsynchronousSendingWithFeeBuffer(t *testing.T) { // Update the non-dust amount because we updated the fee by 100%. htlcFee = HtlcSuccessFee(channeldb.SingleFunderTweaklessBit, feePerKw*2) htlcAmt3 := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.LocalChanCfg.DustLimit + htlcFee, + aliceChannel.channelState.ChanCfgs.Local.DustLimit + htlcFee, ) htlc3, _ := createHTLC(1, htlcAmt3) _, err = bobChannel.AddHTLC(htlc3, nil) @@ -10780,7 +10787,7 @@ func TestEnforceFeeBuffer(t *testing.T) { ) require.NoError(t, err) - aliceReserve := aliceChannel.channelState.LocalChanCfg.ChanReserve + aliceReserve := aliceChannel.channelState.ChanCfgs.Local.ChanReserve capacity := aliceChannel.channelState.Capacity diff --git a/lnwallet/commitment.go b/lnwallet/commitment.go index 6d61729a41c..e1229fccb4e 100644 --- a/lnwallet/commitment.go +++ b/lnwallet/commitment.go @@ -649,14 +649,14 @@ func NewCommitmentBuilder(chanState *channeldb.OpenChannel, func createStateHintObfuscator(state *channeldb.OpenChannel) [StateHintSize]byte { if state.IsInitiator { return DeriveStateHintObfuscator( - state.LocalChanCfg.PaymentBasePoint.PubKey, - state.RemoteChanCfg.PaymentBasePoint.PubKey, + state.ChanCfgs.Local.PaymentBasePoint.PubKey, + state.ChanCfgs.Remote.PaymentBasePoint.PubKey, ) } return DeriveStateHintObfuscator( - state.RemoteChanCfg.PaymentBasePoint.PubKey, - state.LocalChanCfg.PaymentBasePoint.PubKey, + state.ChanCfgs.Remote.PaymentBasePoint.PubKey, + state.ChanCfgs.Local.PaymentBasePoint.PubKey, ) } @@ -696,9 +696,9 @@ func (cb *CommitmentBuilder) createUnsignedCommitmentTx(ourBalance, filteredHTLCView *HtlcView, keyRing *CommitmentKeyRing, prevCommit *commitment) (*unsignedCommitmentTx, error) { - dustLimit := cb.chanState.LocalChanCfg.DustLimit + dustLimit := cb.chanState.ChanCfgs.Local.DustLimit if whoseCommit.IsRemote() { - dustLimit = cb.chanState.RemoteChanCfg.DustLimit + dustLimit = cb.chanState.ChanCfgs.Remote.DustLimit } numHTLCs := int64(0) @@ -785,7 +785,8 @@ func (cb *CommitmentBuilder) createUnsignedCommitmentTx(ourBalance, if whoseCommit.IsLocal() { commitTx, err = CreateCommitTx( cb.chanState.ChanType, fundingTxIn(cb.chanState), keyRing, - &cb.chanState.LocalChanCfg, &cb.chanState.RemoteChanCfg, + &cb.chanState.ChanCfgs.Local, + &cb.chanState.ChanCfgs.Remote, ourBalance.ToSatoshis(), theirBalance.ToSatoshis(), numHTLCs, cb.chanState.IsInitiator, leaseExpiry, auxResult.AuxLeaves, @@ -793,7 +794,8 @@ func (cb *CommitmentBuilder) createUnsignedCommitmentTx(ourBalance, } else { commitTx, err = CreateCommitTx( cb.chanState.ChanType, fundingTxIn(cb.chanState), keyRing, - &cb.chanState.RemoteChanCfg, &cb.chanState.LocalChanCfg, + &cb.chanState.ChanCfgs.Remote, + &cb.chanState.ChanCfgs.Local, theirBalance.ToSatoshis(), ourBalance.ToSatoshis(), numHTLCs, !cb.chanState.IsInitiator, leaseExpiry, auxResult.AuxLeaves, @@ -1284,13 +1286,13 @@ func findOutputIndexesFromRemote(revocationPreimage *chainhash.Hash, // which will be used to generate the output scripts. keyRing := DeriveCommitmentKeys( commitmentPoint, lntypes.Remote, chanState.ChanType, - &chanState.LocalChanCfg, &chanState.RemoteChanCfg, + &chanState.ChanCfgs.Local, &chanState.ChanCfgs.Remote, ) // Since it's remote commitment chain, we'd used the mirrored values. // // We use the remote's channel config for the csv delay. - theirDelay := uint32(chanState.RemoteChanCfg.CsvDelay) + theirDelay := uint32(chanState.ChanCfgs.Remote.CsvDelay) // If we are the initiator of this channel, then it's be false from the // remote's PoV. diff --git a/lnwallet/test_utils.go b/lnwallet/test_utils.go index d4f0d05aeff..c427724c8b4 100644 --- a/lnwallet/test_utils.go +++ b/lnwallet/test_utils.go @@ -317,8 +317,10 @@ func CreateTestChannels(t *testing.T, chanType channeldb.ChannelType, ) aliceChannelState := &channeldb.OpenChannel{ - LocalChanCfg: aliceCfg, - RemoteChanCfg: bobCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: aliceCfg, + Remote: bobCfg, + }, IdentityPub: aliceKeys[0].PubKey(), FundingOutpoint: *prevOut, ShortChannelID: shortChanID, @@ -335,8 +337,10 @@ func CreateTestChannels(t *testing.T, chanType channeldb.ChannelType, FundingTxn: testTx, } bobChannelState := &channeldb.OpenChannel{ - LocalChanCfg: bobCfg, - RemoteChanCfg: aliceCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: bobCfg, + Remote: aliceCfg, + }, IdentityPub: bobKeys[0].PubKey(), FundingOutpoint: *prevOut, ShortChannelID: shortChanID, diff --git a/lnwallet/transactions_test.go b/lnwallet/transactions_test.go index 8786c2d5dc6..9612e28b67b 100644 --- a/lnwallet/transactions_test.go +++ b/lnwallet/transactions_test.go @@ -971,8 +971,10 @@ func createTestChannelsForVectors(tc *testContext, chanType channeldb.ChannelTyp ) remoteChannelState := &channeldb.OpenChannel{ - LocalChanCfg: remoteCfg, - RemoteChanCfg: localCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: remoteCfg, + Remote: localCfg, + }, IdentityPub: remoteDummy2.PubKey(), FundingOutpoint: *prevOut, ShortChannelID: shortChanID, @@ -989,8 +991,10 @@ func createTestChannelsForVectors(tc *testContext, chanType channeldb.ChannelTyp FundingTxn: tc.fundingTx.MsgTx(), } localChannelState := &channeldb.OpenChannel{ - LocalChanCfg: localCfg, - RemoteChanCfg: remoteCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: localCfg, + Remote: remoteCfg, + }, IdentityPub: localDummy2.PubKey(), FundingOutpoint: *prevOut, ShortChannelID: shortChanID, diff --git a/lnwallet/wallet.go b/lnwallet/wallet.go index a9018437d5c..78f961fbfdd 100644 --- a/lnwallet/wallet.go +++ b/lnwallet/wallet.go @@ -2318,8 +2318,8 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs // As we've completed the funding process, we'll no convert the // contribution structs into their underlying channel config objects to // he stored within the database. - res.partialState.LocalChanCfg = res.ourContribution.toChanConfig() - res.partialState.RemoteChanCfg = res.theirContribution.toChanConfig() + res.partialState.ChanCfgs.Local = res.ourContribution.toChanConfig() + res.partialState.ChanCfgs.Remote = res.theirContribution.toChanConfig() // We'll also record the finalized funding txn, which will allow us to // rebroadcast on startup in case we fail. @@ -2525,8 +2525,10 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) { // Add the complete funding transaction to the DB, in it's open bucket // which will be used for the lifetime of this channel. - chanState.LocalChanCfg = pendingReservation.ourContribution.toChanConfig() - chanState.RemoteChanCfg = pendingReservation.theirContribution.toChanConfig() + chanState.ChanCfgs.Local = + pendingReservation.ourContribution.toChanConfig() + chanState.ChanCfgs.Remote = + pendingReservation.theirContribution.toChanConfig() chanState.RevocationKeyLocator = pendingReservation.nextRevocationKeyLoc @@ -2620,8 +2622,8 @@ func (l *LightningWallet) ValidateChannel(channelState *channeldb.OpenChannel, return err } - localKey := channelState.LocalChanCfg.MultiSigKey.PubKey - remoteKey := channelState.RemoteChanCfg.MultiSigKey.PubKey + localKey := channelState.ChanCfgs.Local.MultiSigKey.PubKey + remoteKey := channelState.ChanCfgs.Remote.MultiSigKey.PubKey // We'll also need the multi-sig witness script itself so the // chanvalidate package can check it for correctness against the diff --git a/peer/test_utils.go b/peer/test_utils.go index ab7d24a2287..fc7e5757cd1 100644 --- a/peer/test_utils.go +++ b/peer/test_utils.go @@ -244,8 +244,10 @@ func createTestPeerWithChannel(t *testing.T, updateChan func(a, ) aliceChannelState := &channeldb.OpenChannel{ - LocalChanCfg: aliceCfg, - RemoteChanCfg: bobCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: aliceCfg, + Remote: bobCfg, + }, IdentityPub: aliceKeyPub, FundingOutpoint: *prevOut, ShortChannelID: shortChanID, @@ -262,8 +264,10 @@ func createTestPeerWithChannel(t *testing.T, updateChan func(a, FundingTxn: channels.TestFundingTx, } bobChannelState := &channeldb.OpenChannel{ - LocalChanCfg: bobCfg, - RemoteChanCfg: aliceCfg, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: bobCfg, + Remote: aliceCfg, + }, IdentityPub: bobKeyPub, FundingOutpoint: *prevOut, ChanType: channeldb.SingleFunderTweaklessBit, diff --git a/routing/localchans/manager.go b/routing/localchans/manager.go index f0f9b88de00..dd6202936d2 100644 --- a/routing/localchans/manager.go +++ b/routing/localchans/manager.go @@ -279,9 +279,9 @@ func (r *Manager) getHtlcAmtLimits(tx kvdb.RTx, chanPoint wire.OutPoint) ( // capacity AND less than or equal to the max in-flight HTLC value. // Since the latter is always less than or equal to the former, just // return the max in-flight value. - maxAmt := ch.LocalChanCfg.ChannelStateBounds.MaxPendingAmount + maxAmt := ch.ChanCfgs.Local.ChannelStateBounds.MaxPendingAmount - return ch.LocalChanCfg.MinHTLC, maxAmt, nil + return ch.ChanCfgs.Local.MinHTLC, maxAmt, nil } // makeFailureItem creates a lnrpc.FailedUpdate object. diff --git a/routing/localchans/manager_test.go b/routing/localchans/manager_test.go index 7594eef04ad..9a9fad2fdce 100644 --- a/routing/localchans/manager_test.go +++ b/routing/localchans/manager_test.go @@ -11,6 +11,7 @@ import ( "github.com/lightningnetwork/lnd/discovery" "github.com/lightningnetwork/lnd/kvdb" "github.com/lightningnetwork/lnd/lnrpc" + "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/routing" "github.com/stretchr/testify/require" @@ -131,8 +132,10 @@ func TestManager(t *testing.T) { } return &channeldb.OpenChannel{ - LocalChanCfg: channeldb.ChannelConfig{ - ChannelStateBounds: bounds, + ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ + Local: channeldb.ChannelConfig{ + ChannelStateBounds: bounds, + }, }, }, nil } diff --git a/rpcserver.go b/rpcserver.go index 9bd69e43684..15ae37a186a 100644 --- a/rpcserver.go +++ b/rpcserver.go @@ -3839,8 +3839,10 @@ func (r *rpcServer) fetchPendingOpenChannels() (pendingOpenChannels, error) { Capacity: int64(pendingChan.Capacity), LocalBalance: int64(localCommitment.LocalBalance.ToSatoshis()), RemoteBalance: int64(localCommitment.RemoteBalance.ToSatoshis()), - LocalChanReserveSat: int64(pendingChan.LocalChanCfg.ChanReserve), - RemoteChanReserveSat: int64(pendingChan.RemoteChanCfg.ChanReserve), + //nolint:lll + LocalChanReserveSat: int64(pendingChan.ChanCfgs.Local.ChanReserve), + //nolint:lll + RemoteChanReserveSat: int64(pendingChan.ChanCfgs.Remote.ChanReserve), Initiator: rpcInitiator(pendingChan.IsInitiator), CommitmentType: rpcCommitmentType(pendingChan.ChanType), Private: isPrivate(pendingChan), @@ -4130,8 +4132,10 @@ func (r *rpcServer) fetchWaitingCloseChannels( Capacity: int64(waitingClose.Capacity), LocalBalance: int64(waitingClose.LocalCommitment.LocalBalance.ToSatoshis()), RemoteBalance: int64(waitingClose.LocalCommitment.RemoteBalance.ToSatoshis()), - LocalChanReserveSat: int64(waitingClose.LocalChanCfg.ChanReserve), - RemoteChanReserveSat: int64(waitingClose.RemoteChanCfg.ChanReserve), + //nolint:lll + LocalChanReserveSat: int64(waitingClose.ChanCfgs.Local.ChanReserve), + //nolint:lll + RemoteChanReserveSat: int64(waitingClose.ChanCfgs.Remote.ChanReserve), Initiator: rpcInitiator(waitingClose.IsInitiator), CommitmentType: rpcCommitmentType(waitingClose.ChanType), NumForwardingPackages: int64(len(fwdPkgs)), @@ -4708,10 +4712,10 @@ func createRPCOpenChannel(r *rpcServer, dbChannel *channeldb.OpenChannel, CommitmentType: commitmentType, ThawHeight: dbChannel.ThawHeight, LocalConstraints: createChannelConstraint( - &dbChannel.LocalChanCfg, + &dbChannel.ChanCfgs.Local, ), RemoteConstraints: createChannelConstraint( - &dbChannel.RemoteChanCfg, + &dbChannel.ChanCfgs.Remote, ), AliasScids: make([]uint64, 0, len(channelAliases)), PeerScidAlias: peerScidAlias.ToUint64(), @@ -4720,9 +4724,11 @@ func createRPCOpenChannel(r *rpcServer, dbChannel *channeldb.OpenChannel, Memo: string(dbChannel.Memo), CustomChannelData: customChanBytes, // TODO: remove the following deprecated fields - CsvDelay: uint32(dbChannel.LocalChanCfg.CsvDelay), - LocalChanReserveSat: int64(dbChannel.LocalChanCfg.ChanReserve), - RemoteChanReserveSat: int64(dbChannel.RemoteChanCfg.ChanReserve), + CsvDelay: uint32(dbChannel.ChanCfgs.Local.CsvDelay), + //nolint:lll + LocalChanReserveSat: int64(dbChannel.ChanCfgs.Local.ChanReserve), + //nolint:lll + RemoteChanReserveSat: int64(dbChannel.ChanCfgs.Remote.ChanReserve), } // Look up our channel peer's node alias if the caller requests it. From 30dbe642834f0aa9c28626a64f765f7bf8e75563 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Tue, 20 Aug 2024 19:38:05 -0600 Subject: [PATCH 24/29] multi: pack ChannelCommitments into Dual in OpenChannel --- chanbackup/backup.go | 2 +- chanbackup/single_test.go | 4 +- channeldb/channel.go | 50 +++-- channeldb/channel_test.go | 90 ++++----- channeldb/db.go | 15 +- contractcourt/breach_arbitrator_test.go | 22 ++- contractcourt/chain_arbitrator.go | 4 +- contractcourt/chain_watcher.go | 3 +- contractcourt/chain_watcher_test.go | 6 +- funding/manager.go | 7 +- htlcswitch/link.go | 7 +- htlcswitch/test_utils.go | 26 +-- lnrpc/invoicesrpc/addinvoice.go | 6 +- lnrpc/invoicesrpc/addinvoice_test.go | 25 ++- lnrpc/walletrpc/walletkit_server.go | 8 +- lnwallet/channel.go | 64 ++++--- lnwallet/channel_test.go | 243 +++++++++++++----------- lnwallet/commitment.go | 2 +- lnwallet/reservation.go | 28 +-- lnwallet/test_utils.go | 22 ++- lnwallet/transactions_test.go | 24 ++- lnwallet/wallet.go | 33 ++-- peer/test_utils.go | 22 ++- pilot.go | 4 +- rpcserver.go | 64 ++++--- 25 files changed, 426 insertions(+), 355 deletions(-) diff --git a/chanbackup/backup.go b/chanbackup/backup.go index 5d9d769e878..357ff1e176c 100644 --- a/chanbackup/backup.go +++ b/chanbackup/backup.go @@ -66,7 +66,7 @@ func buildCloseTxInputs( log.Debugf("Crafting CloseTxInputs for ChannelPoint(%v)", targetChan.FundingOutpoint) - localCommit := targetChan.LocalCommitment + localCommit := targetChan.Commitments.Local if localCommit.CommitTx == nil { log.Infof("CommitTx is nil for ChannelPoint(%v), "+ diff --git a/chanbackup/single_test.go b/chanbackup/single_test.go index dd5bd1a816a..4730220915d 100644 --- a/chanbackup/single_test.go +++ b/chanbackup/single_test.go @@ -242,7 +242,9 @@ func genRandomOpenChannelShell() (*channeldb.OpenChannel, error) { ), ThawHeight: rand.Uint32(), IdentityPub: pub, - LocalCommitment: localCommit, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: localCommit, + }, RevocationProducer: shaChainProducer, TapscriptRoot: tapscriptRootOption, ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ diff --git a/channeldb/channel.go b/channeldb/channel.go index 5a59be91562..d775e3abf2d 100644 --- a/channeldb/channel.go +++ b/channeldb/channel.go @@ -941,17 +941,11 @@ type OpenChannel struct { // ChanCfgs is the channel configuration for the local and remote nodes. ChanCfgs lntypes.Dual[ChannelConfig] - // LocalCommitment is the current local commitment state for the local - // party. This is stored distinct from the state of the remote party - // as there are certain asymmetric parameters which affect the - // structure of each commitment. - LocalCommitment ChannelCommitment - - // RemoteCommitment is the current remote commitment state for the - // remote party. This is stored distinct from the state of the local - // party as there are certain asymmetric parameters which affect the - // structure of each commitment. - RemoteCommitment ChannelCommitment + // Commitments is the pair of ChannelCommitments for both the + // local and remote parties. They are stored distinctly as there are + // certain asymmetric parameters which affect the structure of each + // commitment. + Commitments lntypes.Dual[ChannelCommitment] // RemoteCurrentRevocation is the current revocation for their // commitment transaction. However, since this the derived public key, @@ -1048,13 +1042,13 @@ func (c *OpenChannel) String() string { indexStr := "height=%v, local_htlc_index=%v, local_log_index=%v, " + "remote_htlc_index=%v, remote_log_index=%v" - commit := c.LocalCommitment + commit := c.Commitments.Local local := fmt.Sprintf(indexStr, commit.CommitHeight, commit.LocalHtlcIndex, commit.LocalLogIndex, commit.RemoteHtlcIndex, commit.RemoteLogIndex, ) - commit = c.RemoteCommitment + commit = c.Commitments.Remote remote := fmt.Sprintf(indexStr, commit.CommitHeight, commit.LocalHtlcIndex, commit.LocalLogIndex, commit.RemoteHtlcIndex, commit.RemoteLogIndex, @@ -1760,13 +1754,13 @@ func (c *OpenChannel) ChanSyncMsg() (*lnwire.ChannelReestablish, error) { // one. If the receiver thinks that our commitment height is actually // *equal* to this value, then they'll re-send the last commitment that // they sent but we never fully processed. - localHeight := c.LocalCommitment.CommitHeight + localHeight := c.Commitments.Local.CommitHeight nextLocalCommitHeight := localHeight + 1 // The second value we'll send is the height of the remote commitment // from our PoV. If the receiver thinks that their height is actually // *one plus* this value, then they'll re-send their last revocation. - remoteChainTipHeight := c.RemoteCommitment.CommitHeight + remoteChainTipHeight := c.Commitments.Remote.CommitHeight // If this channel has undergone a commitment update, then in order to // prove to the remote party our knowledge of their prior commitment @@ -2410,7 +2404,7 @@ func (c *OpenChannel) UpdateCommitment(newCommitment *ChannelCommitment, return nil, err } - c.LocalCommitment = *newCommitment + c.Commitments.Local = *newCommitment return finalHtlcs, nil } @@ -2471,7 +2465,7 @@ func (c *OpenChannel) ActiveHtlcs() []HTLC { // transactions. So we'll iterate through their set of HTLC's to note // which ones are present on their commitment. remoteHtlcs := make(map[[32]byte]struct{}) - for _, htlc := range c.RemoteCommitment.Htlcs { + for _, htlc := range c.Commitments.Remote.Htlcs { log.Tracef("RemoteCommitment has htlc: id=%v, update=%v "+ "incoming=%v", htlc.HtlcIndex, htlc.LogIndex, htlc.Incoming) @@ -2483,7 +2477,7 @@ func (c *OpenChannel) ActiveHtlcs() []HTLC { // Now that we know which HTLC's they have, we'll only mark the HTLC's // as active if *we* know them as well. activeHtlcs := make([]HTLC, 0, len(remoteHtlcs)) - for _, htlc := range c.LocalCommitment.Htlcs { + for _, htlc := range c.Commitments.Local.Htlcs { log.Tracef("LocalCommitment has htlc: id=%v, update=%v "+ "incoming=%v", htlc.HtlcIndex, htlc.LogIndex, htlc.Incoming) @@ -3327,7 +3321,7 @@ func (c *OpenChannel) AdvanceCommitChainTail(fwdPkg *FwdPkg, // With the commitment pointer swapped, we can now add the // revoked (prior) state to the revocation log. err = putRevocationLog( - logBucket, &c.RemoteCommitment, ourOutputIndex, + logBucket, &c.Commitments.Remote, ourOutputIndex, theirOutputIndex, c.Db.parent.noRevLogAmtData, ) if err != nil { @@ -3409,7 +3403,7 @@ func (c *OpenChannel) AdvanceCommitChainTail(fwdPkg *FwdPkg, // With the db transaction complete, we'll swap over the in-memory // pointer of the new remote commitment, which was previously the tip // of the commit chain. - c.RemoteCommitment = *newRemoteCommit + c.Commitments.Remote = *newRemoteCommit return nil } @@ -3464,7 +3458,7 @@ func (c *OpenChannel) NextLocalHtlcIndex() (uint64, error) { } // Otherwise, fallback to using the local htlc index of their commitment. - return c.RemoteCommitment.LocalHtlcIndex, nil + return c.Commitments.Remote.LocalHtlcIndex, nil } // LoadFwdPkgs scans the forwarding log for any packages that haven't been @@ -3559,7 +3553,7 @@ func (c *OpenChannel) revocationLogTailCommitHeight() (uint64, error) { // If we haven't created any state updates yet, then we'll exit early as // there's nothing to be found on disk in the revocation bucket. - if c.RemoteCommitment.CommitHeight == 0 { + if c.Commitments.Remote.CommitHeight == 0 { return height, nil } @@ -3981,7 +3975,7 @@ func (c *OpenChannel) Snapshot() *ChannelSnapshot { c.RLock() defer c.RUnlock() - localCommit := c.LocalCommitment + localCommit := c.Commitments.Local snapshot := &ChannelSnapshot{ RemoteIdentity: *c.IdentityPub, ChannelPoint: c.FundingOutpoint, @@ -4033,7 +4027,7 @@ func (c *OpenChannel) LatestCommitments() (*ChannelCommitment, *ChannelCommitmen return nil, nil, err } - return &c.LocalCommitment, &c.RemoteCommitment, nil + return &c.Commitments.Local, &c.Commitments.Remote, nil } // RemoteRevocationStore returns the most up to date commitment version of the @@ -4412,14 +4406,14 @@ func putChanCommitments(chanBucket kvdb.RwBucket, channel *OpenChannel) error { } err := putChanCommitment( - chanBucket, &channel.LocalCommitment, true, + chanBucket, &channel.Commitments.Local, true, ) if err != nil { return err } return putChanCommitment( - chanBucket, &channel.RemoteCommitment, false, + chanBucket, &channel.Commitments.Remote, false, ) } @@ -4590,11 +4584,11 @@ func fetchChanCommitments(chanBucket kvdb.RBucket, channel *OpenChannel) error { return nil } - channel.LocalCommitment, err = fetchChanCommitment(chanBucket, true) + channel.Commitments.Local, err = fetchChanCommitment(chanBucket, true) if err != nil { return err } - channel.RemoteCommitment, err = fetchChanCommitment(chanBucket, false) + channel.Commitments.Remote, err = fetchChanCommitment(chanBucket, false) if err != nil { return err } diff --git a/channeldb/channel_test.go b/channeldb/channel_test.go index 91ad13e97ec..af17297902e 100644 --- a/channeldb/channel_test.go +++ b/channeldb/channel_test.go @@ -118,7 +118,7 @@ func openChannelOption() testChannelOption { // commitment. func localHtlcsOption(htlcs []HTLC) testChannelOption { return func(params *testChannelParams) { - params.channel.LocalCommitment.Htlcs = htlcs + params.channel.Commitments.Local.Htlcs = htlcs } } @@ -126,7 +126,7 @@ func localHtlcsOption(htlcs []HTLC) testChannelOption { // commitment. func remoteHtlcsOption(htlcs []HTLC) testChannelOption { return func(params *testChannelParams) { - params.channel.RemoteCommitment.Htlcs = htlcs + params.channel.Commitments.Remote.Htlcs = htlcs } } @@ -331,39 +331,41 @@ func createTestChannelState(t *testing.T, cdb *ChannelStateDB) *OpenChannel { copy(tapscriptRoot[:], bytes.Repeat([]byte{1}, 32)) return &OpenChannel{ - ChanType: SingleFunderBit | FrozenBit, - ChainHash: key, - FundingOutpoint: op, - ShortChannelID: chanID, - IsInitiator: true, - IsPending: true, - IdentityPub: pubKey, - Capacity: btcutil.Amount(10000), + ChanType: SingleFunderBit | FrozenBit, + ChainHash: key, + FundingOutpoint: op, + ShortChannelID: chanID, + IsInitiator: true, + IsPending: true, + IdentityPub: pubKey, + Capacity: btcutil.Amount(10000), ChanCfgs: lntypes.Dual[ChannelConfig]{ Local: localCfg, Remote: remoteCfg, }, TotalMSatSent: 8, TotalMSatReceived: 2, - LocalCommitment: ChannelCommitment{ - CommitHeight: 0, - LocalBalance: lnwire.MilliSatoshi(9000), - RemoteBalance: lnwire.MilliSatoshi(3000), - CommitFee: btcutil.Amount(rand.Int63()), - FeePerKw: btcutil.Amount(5000), - CommitTx: channels.TestFundingTx, - CommitSig: bytes.Repeat([]byte{1}, 71), - CustomBlob: fn.Some([]byte{1, 2, 3}), - }, - RemoteCommitment: ChannelCommitment{ - CommitHeight: 0, - LocalBalance: lnwire.MilliSatoshi(3000), - RemoteBalance: lnwire.MilliSatoshi(9000), - CommitFee: btcutil.Amount(rand.Int63()), - FeePerKw: btcutil.Amount(5000), - CommitTx: channels.TestFundingTx, - CommitSig: bytes.Repeat([]byte{1}, 71), - CustomBlob: fn.Some([]byte{4, 5, 6}), + Commitments: lntypes.Dual[ChannelCommitment]{ + Local: ChannelCommitment{ + CommitHeight: 0, + LocalBalance: lnwire.MilliSatoshi(9000), + RemoteBalance: lnwire.MilliSatoshi(3000), + CommitFee: btcutil.Amount(rand.Int63()), + FeePerKw: btcutil.Amount(5000), + CommitTx: channels.TestFundingTx, + CommitSig: bytes.Repeat([]byte{1}, 71), + CustomBlob: fn.Some([]byte{1, 2, 3}), + }, + Remote: ChannelCommitment{ + CommitHeight: 0, + LocalBalance: lnwire.MilliSatoshi(3000), + RemoteBalance: lnwire.MilliSatoshi(9000), + CommitFee: btcutil.Amount(rand.Int63()), + FeePerKw: btcutil.Amount(5000), + CommitTx: channels.TestFundingTx, + CommitSig: bytes.Repeat([]byte{1}, 71), + CustomBlob: fn.Some([]byte{4, 5, 6}), + }, }, NumConfsRequired: 4, RemoteCurrentRevocation: privKey.PubKey(), @@ -661,7 +663,7 @@ func TestChannelStateTransition(t *testing.T) { // Additionally, modify the signature and commitment transaction. newSequence := uint32(129498) newSig := bytes.Repeat([]byte{3}, 71) - newTx := channel.LocalCommitment.CommitTx.Copy() + newTx := channel.Commitments.Local.CommitTx.Copy() newTx.TxIn[0].Sequence = newSequence commitment := ChannelCommitment{ CommitHeight: 1, @@ -714,11 +716,9 @@ func TestChannelStateTransition(t *testing.T) { // have been updated. updatedChannel, err := cdb.FetchOpenChannels(channel.IdentityPub) require.NoError(t, err, "unable to fetch updated channel") - assertCommitmentEqual( - t, &commitment, &updatedChannel[0].LocalCommitment, + t, &commitment, &updatedChannel[0].Commitments.Local, ) - numDiskUpdates, err := updatedChannel[0].CommitmentHeight() require.NoError(t, err, "unable to read commitment height from disk") @@ -790,7 +790,7 @@ func TestChannelStateTransition(t *testing.T) { // We'll save the old remote commitment as this will be added to the // revocation log shortly. - oldRemoteCommit := channel.RemoteCommitment + oldRemoteCommit := channel.Commitments.Remote // Next, write to the log which tracks the necessary revocation state // needed to rectify any fishy behavior by the remote party. Modify the @@ -842,7 +842,7 @@ func TestChannelStateTransition(t *testing.T) { t.Fatal("update number doesn't match") } - oldRemoteCommit = channel.RemoteCommitment + oldRemoteCommit = channel.Commitments.Remote // Next modify the posted diff commitment slightly, then create a new // commitment diff and advance the tail. @@ -1019,15 +1019,17 @@ func TestFetchClosedChannels(t *testing.T) { // Next, close the channel by including a close channel summary in the // database. summary := &ChannelCloseSummary{ - ChanPoint: state.FundingOutpoint, - ClosingTXID: rev, - RemotePub: state.IdentityPub, - Capacity: state.Capacity, - SettledBalance: state.LocalCommitment.LocalBalance.ToSatoshis(), - TimeLockedBalance: state.RemoteCommitment.LocalBalance.ToSatoshis() + 10000, - CloseType: RemoteForceClose, - IsPending: true, - LocalChanConfig: state.ChanCfgs.Local, + ChanPoint: state.FundingOutpoint, + ClosingTXID: rev, + RemotePub: state.IdentityPub, + Capacity: state.Capacity, + SettledBalance: state.Commitments.Local. + LocalBalance.ToSatoshis(), + TimeLockedBalance: state.Commitments.Remote. + LocalBalance.ToSatoshis() + 10000, + CloseType: RemoteForceClose, + IsPending: true, + LocalChanConfig: state.ChanCfgs.Local, } if err := state.CloseChannel(summary); err != nil { t.Fatalf("unable to close channel: %v", err) diff --git a/channeldb/db.go b/channeldb/db.go index 715eb38510f..2b7b0669ab8 100644 --- a/channeldb/db.go +++ b/channeldb/db.go @@ -1420,13 +1420,14 @@ func (c *ChannelStateDB) AbandonChannel(chanPoint *wire.OutPoint, // channel as possible. We also ensure that we set Pending to false, to // indicate that this channel has been "fully" closed. summary := &ChannelCloseSummary{ - CloseType: Abandoned, - ChanPoint: *chanPoint, - ChainHash: dbChan.ChainHash, - CloseHeight: bestHeight, - RemotePub: dbChan.IdentityPub, - Capacity: dbChan.Capacity, - SettledBalance: dbChan.LocalCommitment.LocalBalance.ToSatoshis(), + CloseType: Abandoned, + ChanPoint: *chanPoint, + ChainHash: dbChan.ChainHash, + CloseHeight: bestHeight, + RemotePub: dbChan.IdentityPub, + Capacity: dbChan.Capacity, + SettledBalance: dbChan.Commitments.Local. + LocalBalance.ToSatoshis(), ShortChanID: dbChan.ShortChanID(), RemoteCurrentRevocation: dbChan.RemoteCurrentRevocation, RemoteNextRevocation: dbChan.RemoteNextRevocation, diff --git a/contractcourt/breach_arbitrator_test.go b/contractcourt/breach_arbitrator_test.go index faf36f5e6f9..a606641622f 100644 --- a/contractcourt/breach_arbitrator_test.go +++ b/contractcourt/breach_arbitrator_test.go @@ -2347,11 +2347,13 @@ func createInitChannels(t *testing.T) ( RemoteCurrentRevocation: bobCommitPoint, RevocationProducer: alicePreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: aliceCommit, - RemoteCommitment: aliceCommit, - Db: dbAlice.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), - FundingTxn: channels.TestFundingTx, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: aliceCommit, + Remote: aliceCommit, + }, + Db: dbAlice.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), + FundingTxn: channels.TestFundingTx, } bobChannelState := &channeldb.OpenChannel{ ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ @@ -2367,10 +2369,12 @@ func createInitChannels(t *testing.T) ( RemoteCurrentRevocation: aliceCommitPoint, RevocationProducer: bobPreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: bobCommit, - RemoteCommitment: bobCommit, - Db: dbBob.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: bobCommit, + Remote: bobCommit, + }, + Db: dbBob.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), } aliceSigner := input.NewMockSigner( diff --git a/contractcourt/chain_arbitrator.go b/contractcourt/chain_arbitrator.go index c29178b4384..129e318afa6 100644 --- a/contractcourt/chain_arbitrator.go +++ b/contractcourt/chain_arbitrator.go @@ -466,8 +466,8 @@ func newActiveChannelArbitrator(channel *channeldb.OpenChannel, // Finally, we'll need to construct a series of htlc Sets based on all // currently known valid commitments. htlcSets := make(map[HtlcSetKey]htlcSet) - htlcSets[LocalHtlcSet] = newHtlcSet(channel.LocalCommitment.Htlcs) - htlcSets[RemoteHtlcSet] = newHtlcSet(channel.RemoteCommitment.Htlcs) + htlcSets[LocalHtlcSet] = newHtlcSet(channel.Commitments.Local.Htlcs) + htlcSets[RemoteHtlcSet] = newHtlcSet(channel.Commitments.Remote.Htlcs) pendingRemoteCommitment, err := channel.RemoteCommitChainTip() if err != nil && err != channeldb.ErrNoPendingCommit { diff --git a/contractcourt/chain_watcher.go b/contractcourt/chain_watcher.go index bdb84c5eeda..c5c07d8051e 100644 --- a/contractcourt/chain_watcher.go +++ b/contractcourt/chain_watcher.go @@ -436,7 +436,8 @@ func (c *chainWatcher) handleUnknownLocalState( func(s lnwallet.AuxLeafStore) fn.Result[lnwallet.CommitDiffAuxResult] { return s.FetchLeavesFromCommit( lnwallet.NewAuxChanState(c.cfg.chanState), - c.cfg.chanState.LocalCommitment, *commitKeyRing, + c.cfg.chanState.Commitments.Local, + *commitKeyRing, ) }, ).Unpack() diff --git a/contractcourt/chain_watcher_test.go b/contractcourt/chain_watcher_test.go index 60f7128787b..c70fbbfffe7 100644 --- a/contractcourt/chain_watcher_test.go +++ b/contractcourt/chain_watcher_test.go @@ -54,7 +54,7 @@ func TestChainWatcherRemoteUnilateralClose(t *testing.T) { // If we simulate an immediate broadcast of the current commitment by // Bob, then the chain watcher should detect this case. - bobCommit := bobChannel.State().LocalCommitment.CommitTx + bobCommit := bobChannel.State().Commitments.Local.CommitTx bobTxHash := bobCommit.TxHash() bobSpend := &chainntnfs.SpendDetail{ SpenderTxHash: &bobTxHash, @@ -321,7 +321,7 @@ func TestChainWatcherDataLossProtect(t *testing.T) { // Now we'll trigger the channel close event to trigger the // scenario. - bobCommit := bobChannel.State().LocalCommitment.CommitTx + bobCommit := bobChannel.State().Commitments.Local.CommitTx bobTxHash := bobCommit.TxHash() bobSpend := &chainntnfs.SpendDetail{ SpenderTxHash: &bobTxHash, @@ -481,7 +481,7 @@ func TestChainWatcherLocalForceCloseDetect(t *testing.T) { // Next, we'll obtain Alice's commitment transaction and // trigger a force close. This should cause her to detect a // local force close, and dispatch a local close event. - aliceCommit := aliceChannel.State().LocalCommitment.CommitTx + aliceCommit := aliceChannel.State().Commitments.Local.CommitTx // Since this is Alice's commitment, her output is always first // since she's the one creating the HTLCs (lower balance). In diff --git a/funding/manager.go b/funding/manager.go index aa52f51e8dd..108157365f9 100644 --- a/funding/manager.go +++ b/funding/manager.go @@ -2542,7 +2542,8 @@ func (f *Manager) fundeeProcessFundingCreated(peer lnpeer.Peer, // we use this convenience method to delete the pending OpenChannel // from the database. deleteFromDatabase := func() { - localBalance := completeChan.LocalCommitment.LocalBalance.ToSatoshis() + localBalance := completeChan.Commitments.Local. + LocalBalance.ToSatoshis() closeInfo := &channeldb.ChannelCloseSummary{ ChanPoint: completeChan.FundingOutpoint, ChainHash: completeChan.ChainHash, @@ -2907,7 +2908,7 @@ func (f *Manager) fundingTimeout(c *channeldb.OpenChannel, // We'll get a timeout if the number of blocks mined since the channel // was initiated reaches MaxWaitNumBlocksFundingConf and we are not the // channel initiator. - localBalance := c.LocalCommitment.LocalBalance.ToSatoshis() + localBalance := c.Commitments.Local.LocalBalance.ToSatoshis() closeInfo := &channeldb.ChannelCloseSummary{ ChainHash: c.ChainHash, ChanPoint: c.FundingOutpoint, @@ -3865,7 +3866,7 @@ func genFirstStateMusigNonce(channel *channeldb.OpenChannel, // nonce for the next state the remote party will sign for us. verNonce, err := channeldb.NewMusigVerificationNonce( channel.ChanCfgs.Local.MultiSigKey.PubKey, - channel.LocalCommitment.CommitHeight+1, + channel.Commitments.Local.CommitHeight+1, musig2ShaChain, ) if err != nil { diff --git a/htlcswitch/link.go b/htlcswitch/link.go index c094c6dd9e5..9fb3384b765 100644 --- a/htlcswitch/link.go +++ b/htlcswitch/link.go @@ -2505,7 +2505,8 @@ func (l *channelLink) handleUpstreamMsg(msg lnwire.Message) { chanID := l.ChanID() err = l.cfg.TowerClient.BackupState( - &chanID, state.RemoteCommitment.CommitHeight-1, + &chanID, + state.Commitments.Remote.CommitHeight-1, ) if err != nil { l.failf(LinkFailureError{ @@ -2946,10 +2947,10 @@ func (l *channelLink) getDustClosure() dustClosure { // NOTE: Part of the dustHandler interface. func (l *channelLink) getCommitFee(remote bool) btcutil.Amount { if remote { - return l.channel.State().RemoteCommitment.CommitFee + return l.channel.State().Commitments.Remote.CommitFee } - return l.channel.State().LocalCommitment.CommitFee + return l.channel.State().Commitments.Local.CommitFee } // exceedsFeeExposureLimit returns whether or not the new proposed fee-rate diff --git a/htlcswitch/test_utils.go b/htlcswitch/test_utils.go index 77d861e734f..9f30f4e854f 100644 --- a/htlcswitch/test_utils.go +++ b/htlcswitch/test_utils.go @@ -313,12 +313,14 @@ func createTestChannel(t *testing.T, alicePrivKey, bobPrivKey []byte, RemoteCurrentRevocation: bobCommitPoint, RevocationProducer: alicePreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: aliceCommit, - RemoteCommitment: aliceCommit, - ShortChannelID: chanID, - Db: dbAlice.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(chanID), - FundingTxn: channels.TestFundingTx, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: aliceCommit, + Remote: aliceCommit, + }, + ShortChannelID: chanID, + Db: dbAlice.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(chanID), + FundingTxn: channels.TestFundingTx, } bobChannelState := &channeldb.OpenChannel{ @@ -334,11 +336,13 @@ func createTestChannel(t *testing.T, alicePrivKey, bobPrivKey []byte, RemoteCurrentRevocation: aliceCommitPoint, RevocationProducer: bobPreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: bobCommit, - RemoteCommitment: bobCommit, - ShortChannelID: chanID, - Db: dbBob.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(chanID), + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: bobCommit, + Remote: bobCommit, + }, + ShortChannelID: chanID, + Db: dbBob.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(chanID), } if err := aliceChannelState.SyncPending(bobAddr, broadcastHeight); err != nil { diff --git a/lnrpc/invoicesrpc/addinvoice.go b/lnrpc/invoicesrpc/addinvoice.go index df274f2da79..c83d3687674 100644 --- a/lnrpc/invoicesrpc/addinvoice.go +++ b/lnrpc/invoicesrpc/addinvoice.go @@ -729,7 +729,7 @@ func newHopHintInfo(c *channeldb.OpenChannel, isActive bool) *HopHintInfo { IsActive: isActive, FundingOutpoint: c.FundingOutpoint, RemotePubkey: c.IdentityPub, - RemoteBalance: c.LocalCommitment.RemoteBalance, + RemoteBalance: c.Commitments.Local.RemoteBalance, ShortChannelID: c.ShortChannelID.ToUint64(), ConfirmedScidZC: c.ZeroConfRealScid().ToUint64(), ScidAliasFeature: c.ChanType.HasScidAliasFeature(), @@ -844,8 +844,8 @@ func getPotentialHints(cfg *SelectHopHintsCfg) ([]*channeldb.OpenChannel, // Sort the channels in descending remote balance. compareRemoteBalance := func(i, j int) bool { - iBalance := privateChannels[i].LocalCommitment.RemoteBalance - jBalance := privateChannels[j].LocalCommitment.RemoteBalance + iBalance := privateChannels[i].Commitments.Local.RemoteBalance + jBalance := privateChannels[j].Commitments.Local.RemoteBalance return iBalance > jBalance } sort.Slice(privateChannels, compareRemoteBalance) diff --git a/lnrpc/invoicesrpc/addinvoice_test.go b/lnrpc/invoicesrpc/addinvoice_test.go index 76a529f8c67..723e31628f5 100644 --- a/lnrpc/invoicesrpc/addinvoice_test.go +++ b/lnrpc/invoicesrpc/addinvoice_test.go @@ -9,6 +9,7 @@ import ( "github.com/btcsuite/btcd/wire" "github.com/lightningnetwork/lnd/channeldb" "github.com/lightningnetwork/lnd/channeldb/models" + "github.com/lightningnetwork/lnd/lntypes" "github.com/lightningnetwork/lnd/lnwire" "github.com/lightningnetwork/lnd/zpay32" "github.com/stretchr/testify/mock" @@ -588,8 +589,10 @@ var populateHopHintsTestCases = []struct { remoteBalance := lnwire.MilliSatoshi(10_000_000) allChannels := []*channeldb.OpenChannel{ { - LocalCommitment: channeldb.ChannelCommitment{ - RemoteBalance: remoteBalance, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ //nolint:lll + Local: channeldb.ChannelCommitment{ + RemoteBalance: remoteBalance, + }, }, FundingOutpoint: fundingOutpoint, ShortChannelID: lnwire.NewShortChanIDFromInt(9), @@ -642,8 +645,10 @@ var populateHopHintsTestCases = []struct { // enough bandwidth we should never use this one. {}, { - LocalCommitment: channeldb.ChannelCommitment{ - RemoteBalance: remoteBalance, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ //nolint:lll + Local: channeldb.ChannelCommitment{ + RemoteBalance: remoteBalance, + }, }, FundingOutpoint: fundingOutpoint, ShortChannelID: lnwire.NewShortChanIDFromInt(9), @@ -840,16 +845,20 @@ func setupMockTwoChannels(h *hopHintsConfigMock) (lnwire.ChannelID, // After sorting we will first process chanID1 and then // chanID2. { - LocalCommitment: channeldb.ChannelCommitment{ - RemoteBalance: remoteBalance2, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: channeldb.ChannelCommitment{ + RemoteBalance: remoteBalance2, + }, }, FundingOutpoint: fundingOutpoint2, ShortChannelID: lnwire.NewShortChanIDFromInt(2), IdentityPub: getTestPubKey(), }, { - LocalCommitment: channeldb.ChannelCommitment{ - RemoteBalance: remoteBalance1, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: channeldb.ChannelCommitment{ + RemoteBalance: remoteBalance1, + }, }, FundingOutpoint: fundingOutpoint1, ShortChannelID: lnwire.NewShortChanIDFromInt(9), diff --git a/lnrpc/walletrpc/walletkit_server.go b/lnrpc/walletrpc/walletkit_server.go index 2d577db6875..55378ff746b 100644 --- a/lnrpc/walletrpc/walletkit_server.go +++ b/lnrpc/walletrpc/walletkit_server.go @@ -1190,13 +1190,13 @@ func (w *WalletKit) BumpForceCloseFee(_ context.Context, // bumping the fee. commitSet := fn.NewSet[chainhash.Hash]() - if channel.LocalCommitment.CommitTx != nil { - localTxID := channel.LocalCommitment.CommitTx.TxHash() + if channel.Commitments.Local.CommitTx != nil { + localTxID := channel.Commitments.Local.CommitTx.TxHash() commitSet.Add(localTxID) } - if channel.RemoteCommitment.CommitTx != nil { - remoteTxID := channel.RemoteCommitment.CommitTx.TxHash() + if channel.Commitments.Remote.CommitTx != nil { + remoteTxID := channel.Commitments.Remote.CommitTx.TxHash() commitSet.Add(remoteTxID) } diff --git a/lnwallet/channel.go b/lnwallet/channel.go index d042a2a0352..feb8716d865 100644 --- a/lnwallet/channel.go +++ b/lnwallet/channel.go @@ -904,8 +904,8 @@ func NewLightningChannel(signer input.Signer, optFunc(opts) } - localCommit := state.LocalCommitment - remoteCommit := state.RemoteCommitment + localCommit := state.Commitments.Local + remoteCommit := state.Commitments.Remote // First, initialize the update logs with their current counter values // from the local and remote commitments. @@ -2207,7 +2207,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, KeyRing: keyRing, CsvDelay: ourDelay, BreachCsvDelay: fn.Some(theirDelay), - CommitFee: chanState.RemoteCommitment.CommitFee, + CommitFee: chanState.Commitments.Remote.CommitFee, } if revokedLog != nil { resolveReq.CommitBlob = revokedLog.CustomBlob.ValOpt() @@ -2286,7 +2286,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, KeyRing: keyRing, CsvDelay: theirDelay, BreachCsvDelay: fn.Some(theirDelay), - CommitFee: chanState.RemoteCommitment.CommitFee, + CommitFee: chanState.Commitments.Remote.CommitFee, } if revokedLog != nil { resolveReq.CommitBlob = revokedLog.CustomBlob.ValOpt() @@ -5776,7 +5776,7 @@ func (lc *LightningChannel) ReceiveRevocation(revMsg *lnwire.RevokeAndAck) ( remoteChainTail, ) - remoteHTLCs := lc.channelState.RemoteCommitment.Htlcs + remoteHTLCs := lc.channelState.Commitments.Remote.Htlcs return fwdPkg, remoteHTLCs, nil } @@ -5897,11 +5897,11 @@ func (lc *LightningChannel) GetDustSum(whoseCommit lntypes.ChannelParty, var dustSum lnwire.MilliSatoshi dustLimit := lc.channelState.ChanCfgs.Local.DustLimit - commit := lc.channelState.LocalCommitment + commit := lc.channelState.Commitments.Local if whoseCommit.IsRemote() { // Calculate dust sum on the remote's commitment. dustLimit = lc.channelState.ChanCfgs.Remote.DustLimit - commit = lc.channelState.RemoteCommitment + commit = lc.channelState.Commitments.Remote } chanType := lc.channelState.ChanType @@ -6589,7 +6589,7 @@ func GetSignedCommitTx(inputs SignedCommitTxInputs, func (lc *LightningChannel) getSignedCommitTx() (*wire.MsgTx, error) { // Fetch the current commitment transaction, along with their signature // for the transaction. - localCommit := lc.channelState.LocalCommitment + localCommit := lc.channelState.Commitments.Local inputs := SignedCommitTxInputs{ CommitTx: localCommit.CommitTx, @@ -6822,7 +6822,7 @@ func NewUnilateralCloseSummary(chanState *channeldb.OpenChannel, ChanPoint: chanState.FundingOutpoint, ShortChanID: chanState.ShortChanID(), Initiator: chanState.IsInitiator, - CommitBlob: chanState.RemoteCommitment.CustomBlob, + CommitBlob: chanState.Commitments.Remote.CustomBlob, FundingBlob: chanState.CustomBlob, Type: input.TaprootRemoteCommitSpend, CloseType: RemoteForceClose, @@ -6831,7 +6831,7 @@ func NewUnilateralCloseSummary(chanState *channeldb.OpenChannel, SignDesc: commitResolution.SelfOutputSignDesc, KeyRing: keyRing, CsvDelay: maturityDelay, - CommitFee: chanState.RemoteCommitment.CommitFee, + CommitFee: chanState.Commitments.Remote.CommitFee, } resolveBlob := fn.MapOptionZ( auxResolver, @@ -7697,7 +7697,7 @@ func (lc *LightningChannel) ForceClose() (*LocalForceCloseSummary, error) { return nil, err } - localCommitment := lc.channelState.LocalCommitment + localCommitment := lc.channelState.Commitments.Local summary, err := NewLocalForceCloseSummary( lc.channelState, lc.Signer, commitTx, localCommitment.CommitHeight, lc.leafStore, lc.auxResolver, @@ -7744,7 +7744,7 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, leafStore, func(s AuxLeafStore) fn.Result[CommitDiffAuxResult] { return s.FetchLeavesFromCommit( NewAuxChanState(chanState), - chanState.LocalCommitment, *keyRing, + chanState.Commitments.Local, *keyRing, ) }, ).Unpack() @@ -7854,7 +7854,7 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, ChanPoint: chanState.FundingOutpoint, ShortChanID: chanState.ShortChanID(), Initiator: chanState.IsInitiator, - CommitBlob: chanState.LocalCommitment.CustomBlob, + CommitBlob: chanState.Commitments.Local.CustomBlob, FundingBlob: chanState.CustomBlob, Type: input.TaprootLocalCommitSpend, CloseType: LocalForceClose, @@ -7863,7 +7863,7 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, SignDesc: commitResolution.SelfOutputSignDesc, KeyRing: keyRing, CsvDelay: csvTimeout, - CommitFee: chanState.LocalCommitment.CommitFee, + CommitFee: chanState.Commitments.Local.CommitFee, }) }, ) @@ -7879,7 +7879,7 @@ func NewLocalForceCloseSummary(chanState *channeldb.OpenChannel, // outgoing HTLC's that we'll need to claim as well. If this is after // recovery there is not much we can do with HTLCs, so we'll always // use what we have in our latest state when extracting resolutions. - localCommit := chanState.LocalCommitment + localCommit := chanState.Commitments.Local htlcResolutions, err := extractHtlcResolutions( chainfee.SatPerKWeight(localCommit.FeePerKw), lntypes.Local, signer, localCommit.Htlcs, keyRing, &chanState.ChanCfgs.Local, @@ -7999,9 +7999,9 @@ func (lc *LightningChannel) CreateCloseProposal(proposedFee btcutil.Amount, ourBalance, theirBalance, err := CoopCloseBalance( lc.channelState.ChanType, lc.channelState.IsInitiator, proposedFee, - lc.channelState.LocalCommitment.LocalBalance.ToSatoshis(), - lc.channelState.LocalCommitment.RemoteBalance.ToSatoshis(), - lc.channelState.LocalCommitment.CommitFee, + lc.channelState.Commitments.Local.LocalBalance.ToSatoshis(), + lc.channelState.Commitments.Local.RemoteBalance.ToSatoshis(), + lc.channelState.Commitments.Local.CommitFee, ) if err != nil { return nil, nil, 0, err @@ -8102,9 +8102,9 @@ func (lc *LightningChannel) CompleteCooperativeClose( ourBalance, theirBalance, err := CoopCloseBalance( lc.channelState.ChanType, lc.channelState.IsInitiator, proposedFee, - lc.channelState.LocalCommitment.LocalBalance.ToSatoshis(), - lc.channelState.LocalCommitment.RemoteBalance.ToSatoshis(), - lc.channelState.LocalCommitment.CommitFee, + lc.channelState.Commitments.Local.LocalBalance.ToSatoshis(), + lc.channelState.Commitments.Local.RemoteBalance.ToSatoshis(), + lc.channelState.Commitments.Local.CommitFee, ) if err != nil { return nil, 0, err @@ -8260,7 +8260,7 @@ func (lc *LightningChannel) NewAnchorResolutions() (*AnchorResolutions, &lc.channelState.ChanCfgs.Remote, ) localRes, err := NewAnchorResolution( - lc.channelState, lc.channelState.LocalCommitment.CommitTx, + lc.channelState, lc.channelState.Commitments.Local.CommitTx, localKeyRing, lntypes.Local, ) if err != nil { @@ -8275,7 +8275,7 @@ func (lc *LightningChannel) NewAnchorResolutions() (*AnchorResolutions, &lc.channelState.ChanCfgs.Remote, ) remoteRes, err := NewAnchorResolution( - lc.channelState, lc.channelState.RemoteCommitment.CommitTx, + lc.channelState, lc.channelState.Commitments.Remote.CommitTx, remoteKeyRing, lntypes.Remote, ) if err != nil { @@ -9049,7 +9049,7 @@ func (lc *LightningChannel) LocalBalanceDust() (bool, btcutil.Amount) { defer lc.RUnlock() chanState := lc.channelState - localBalance := chanState.LocalCommitment.LocalBalance.ToSatoshis() + localBalance := chanState.Commitments.Local.LocalBalance.ToSatoshis() // If this is an anchor channel, and we're the initiator, then we'll // regain the stats allocated to the anchor outputs with the co-op @@ -9071,7 +9071,7 @@ func (lc *LightningChannel) RemoteBalanceDust() (bool, btcutil.Amount) { defer lc.RUnlock() chanState := lc.channelState - remoteBalance := chanState.RemoteCommitment.RemoteBalance.ToSatoshis() + remoteBalance := chanState.Commitments.Remote.RemoteBalance.ToSatoshis() // If this is an anchor channel, and they're the initiator, then we'll // regain the stats allocated to the anchor outputs with the co-op @@ -9092,7 +9092,7 @@ func (lc *LightningChannel) CommitBalances() (btcutil.Amount, btcutil.Amount) { defer lc.RUnlock() chanState := lc.channelState - localCommit := lc.channelState.LocalCommitment + localCommit := lc.channelState.Commitments.Local localBalance := localCommit.LocalBalance.ToSatoshis() remoteBalance := localCommit.RemoteBalance.ToSatoshis() @@ -9113,7 +9113,7 @@ func (lc *LightningChannel) CommitFee() btcutil.Amount { lc.RLock() defer lc.RUnlock() - return lc.channelState.LocalCommitment.CommitFee + return lc.channelState.Commitments.Local.CommitFee } // CalcFee returns the commitment fee to use for the given fee rate @@ -9271,7 +9271,9 @@ func (lc *LightningChannel) CommitFeeRate() chainfee.SatPerKWeight { lc.RLock() defer lc.RUnlock() - return chainfee.SatPerKWeight(lc.channelState.LocalCommitment.FeePerKw) + return chainfee.SatPerKWeight( + lc.channelState.Commitments.Local.FeePerKw, + ) } // WorstCaseFeeRate returns the higher feerate from either the local commitment @@ -9280,8 +9282,8 @@ func (lc *LightningChannel) WorstCaseFeeRate() chainfee.SatPerKWeight { lc.RLock() defer lc.RUnlock() - localFeeRate := lc.channelState.LocalCommitment.FeePerKw - remoteFeeRate := lc.channelState.RemoteCommitment.FeePerKw + localFeeRate := lc.channelState.Commitments.Local.FeePerKw + remoteFeeRate := lc.channelState.Commitments.Remote.FeePerKw if localFeeRate > remoteFeeRate { return chainfee.SatPerKWeight(localFeeRate) @@ -9544,7 +9546,7 @@ func (lc *LightningChannel) LocalCommitmentBlob() fn.Option[tlv.Blob] { defer lc.RUnlock() chanState := lc.channelState - localBalance := chanState.LocalCommitment.CustomBlob + localBalance := chanState.Commitments.Local.CustomBlob return fn.MapOption(func(b tlv.Blob) tlv.Blob { newBlob := make([]byte, len(b)) diff --git a/lnwallet/channel_test.go b/lnwallet/channel_test.go index 92ed99f9acd..dd5897a96e5 100644 --- a/lnwallet/channel_test.go +++ b/lnwallet/channel_test.go @@ -212,21 +212,21 @@ func testAddSettleWorkflow(t *testing.T, tweakless bool, // an anchor output. numOutputs = 5 } - if len(aliceChanState.LocalCommitment.CommitTx.TxOut) != numOutputs { + if len(aliceChanState.Commitments.Local.CommitTx.TxOut) != numOutputs { t.Fatalf("alice should have three commitment outputs, instead "+ "have %v", - len(aliceChanState.LocalCommitment.CommitTx.TxOut)) + len(aliceChanState.Commitments.Local.CommitTx.TxOut)) } - if len(bobChanState.LocalCommitment.CommitTx.TxOut) != numOutputs { + if len(bobChanState.Commitments.Local.CommitTx.TxOut) != numOutputs { t.Fatalf("bob should have three commitment outputs, instead "+ "have %v", - len(bobChanState.LocalCommitment.CommitTx.TxOut)) + len(bobChanState.Commitments.Local.CommitTx.TxOut)) } assertOutputExistsByValue(t, - aliceChannel.channelState.LocalCommitment.CommitTx, + aliceChannel.channelState.Commitments.Local.CommitTx, htlcAmt.ToSatoshis()) assertOutputExistsByValue(t, - bobChannel.channelState.LocalCommitment.CommitTx, + bobChannel.channelState.Commitments.Local.CommitTx, htlcAmt.ToSatoshis()) // Now we'll repeat a similar exchange, this time with Bob settling the @@ -804,10 +804,10 @@ func testCoopClose(t *testing.T, testCase *coopCloseTestCase) { bobDeliveryScript := testHdSeed[:] aliceFeeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) bobFeeRate := chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) // We'll start with both Alice and Bob creating a new close proposal @@ -847,14 +847,14 @@ func testCoopClose(t *testing.T, testCase *coopCloseTestCase) { // Finally, make sure the final balances are correct from both's // perspective. - aliceBalance := aliceChannel.channelState.LocalCommitment. + aliceBalance := aliceChannel.channelState.Commitments.Local. LocalBalance.ToSatoshis() // The commit balance have had the initiator's (Alice) commitfee and // any anchors subtracted, so add that back to the final expected // balance. Alice also pays the coop close fee, so that must be // subtracted. - commitFee := aliceChannel.channelState.LocalCommitment.CommitFee + commitFee := aliceChannel.channelState.Commitments.Local.CommitFee expBalanceAlice := aliceBalance + commitFee + testCase.anchorAmt - bobFee if aliceTxBalance != expBalanceAlice { @@ -864,7 +864,7 @@ func testCoopClose(t *testing.T, testCase *coopCloseTestCase) { // Bob is not the initiator, so his final balance should simply be // equal to the latest commitment balance. - expBalanceBob := bobChannel.channelState.LocalCommitment. + expBalanceBob := bobChannel.channelState.Commitments.Local. LocalBalance.ToSatoshis() if bobTxBalance != expBalanceBob { t.Fatalf("expected bob's balance to be %v got %v", @@ -930,7 +930,7 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { ) require.NoError(t, err, "unable to create test channels") - bobAmount := bobChannel.channelState.LocalCommitment.LocalBalance + bobAmount := bobChannel.channelState.Commitments.Local.LocalBalance // First, we'll add an outgoing HTLC from Alice to Bob, such that it // will still be present within the broadcast commitment transaction. @@ -1040,7 +1040,7 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { totalCommitWeight := testCase.expectedCommitWeight + (input.HTLCWeight * 2) feePerKw := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) commitFee := feePerKw.FeeForWeight(totalCommitWeight) @@ -1143,7 +1143,8 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { // Finally, the txid of the commitment transaction and the one returned // as the closing transaction should also match. closeTxHash := closeSummary.CloseTx.TxHash() - commitTxHash := aliceChannel.channelState.LocalCommitment.CommitTx.TxHash() + commitTxHash := + aliceChannel.channelState.Commitments.Local.CommitTx.TxHash() if !bytes.Equal(closeTxHash[:], commitTxHash[:]) { t.Fatalf("alice: incorrect close transaction txid") } @@ -1259,7 +1260,8 @@ func testForceClose(t *testing.T, testCase *forceCloseTestCase) { } closeTxHash = closeSummary.CloseTx.TxHash() - commitTxHash = bobChannel.channelState.LocalCommitment.CommitTx.TxHash() + commitTxHash = + bobChannel.channelState.Commitments.Local.CommitTx.TxHash() if !bytes.Equal(closeTxHash[:], commitTxHash[:]) { t.Fatalf("bob: incorrect close transaction txid") } @@ -1307,8 +1309,8 @@ func TestForceCloseDustOutput(t *testing.T) { htlcAmount := lnwire.NewMSatFromSatoshis(500) - aliceAmount := aliceChannel.channelState.LocalCommitment.LocalBalance - bobAmount := bobChannel.channelState.LocalCommitment.LocalBalance + aliceAmount := aliceChannel.channelState.Commitments.Local.LocalBalance + bobAmount := bobChannel.channelState.Commitments.Local.LocalBalance // Have Bobs' to-self output be below her dust limit and check // ForceCloseSummary again on both peers. @@ -1332,8 +1334,7 @@ func TestForceCloseDustOutput(t *testing.T) { t.Fatalf("Can't update the channel state: %v", err) } - aliceAmount = aliceChannel.channelState.LocalCommitment.LocalBalance - bobAmount = bobChannel.channelState.LocalCommitment.RemoteBalance + aliceAmount = aliceChannel.channelState.Commitments.Local.LocalBalance closeSummary, err := aliceChannel.ForceClose() require.NoError(t, err, "unable to force close channel") @@ -1354,7 +1355,7 @@ func TestForceCloseDustOutput(t *testing.T) { int64(aliceAmount.ToSatoshis()) { t.Fatalf("alice incorrect output value in SelfOutputSignDesc, "+ "expected %v, got %v", - aliceChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis(), + aliceChannel.channelState.Commitments.Local.LocalBalance.ToSatoshis(), //nolint:lll commitResolution.SelfOutputSignDesc.Output.Value) } @@ -1368,7 +1369,8 @@ func TestForceCloseDustOutput(t *testing.T) { } closeTxHash := closeSummary.CloseTx.TxHash() - commitTxHash := aliceChannel.channelState.LocalCommitment.CommitTx.TxHash() + commitTxHash := + aliceChannel.channelState.Commitments.Local.CommitTx.TxHash() if !bytes.Equal(closeTxHash[:], commitTxHash[:]) { t.Fatalf("alice: incorrect close transaction txid") } @@ -1385,7 +1387,8 @@ func TestForceCloseDustOutput(t *testing.T) { } closeTxHash = closeSummary.CloseTx.TxHash() - commitTxHash = bobChannel.channelState.LocalCommitment.CommitTx.TxHash() + commitTxHash = + bobChannel.channelState.Commitments.Local.CommitTx.TxHash() if !bytes.Equal(closeTxHash[:], commitTxHash[:]) { t.Fatalf("bob: incorrect close transaction txid") } @@ -1405,7 +1408,8 @@ func TestDustHTLCFees(t *testing.T) { ) require.NoError(t, err, "unable to create test channels") - aliceStartingBalance := aliceChannel.channelState.LocalCommitment.LocalBalance + aliceStartingBalance := + aliceChannel.channelState.Commitments.Local.LocalBalance // This HTLC amount should be lower than the dust limits of both nodes. htlcAmount := lnwire.NewMSatFromSatoshis(100) @@ -1424,17 +1428,19 @@ func TestDustHTLCFees(t *testing.T) { // properly. Namely, the local+remote+commitfee values should add up to // the total capacity of the channel. This same should hold for both // sides. - totalSatoshisAlice := (aliceChannel.channelState.LocalCommitment.LocalBalance + - aliceChannel.channelState.LocalCommitment.RemoteBalance + - lnwire.NewMSatFromSatoshis(aliceChannel.channelState.LocalCommitment.CommitFee)) + aliceChanState := &aliceChannel.channelState.Commitments.Local + totalSatoshisAlice := aliceChanState.LocalBalance + + aliceChanState.RemoteBalance + + lnwire.NewMSatFromSatoshis(aliceChanState.CommitFee) if totalSatoshisAlice+htlcAmount != lnwire.NewMSatFromSatoshis(aliceChannel.Capacity) { t.Fatalf("alice's funds leaked: total satoshis are %v, but channel "+ "capacity is %v", int64(totalSatoshisAlice), int64(aliceChannel.Capacity)) } - totalSatoshisBob := (bobChannel.channelState.LocalCommitment.LocalBalance + - bobChannel.channelState.LocalCommitment.RemoteBalance + - lnwire.NewMSatFromSatoshis(bobChannel.channelState.LocalCommitment.CommitFee)) + bobChanState := bobChannel.channelState.Commitments.Local + totalSatoshisBob := bobChanState.LocalBalance + + bobChanState.RemoteBalance + + lnwire.NewMSatFromSatoshis(bobChanState.CommitFee) if totalSatoshisBob+htlcAmount != lnwire.NewMSatFromSatoshis(bobChannel.Capacity) { t.Fatalf("bob's funds leaked: total satoshis are %v, but channel "+ "capacity is %v", int64(totalSatoshisBob), @@ -1444,20 +1450,20 @@ func TestDustHTLCFees(t *testing.T) { // The commitment fee paid should be the same, as there have been no // new material outputs added. defaultFee := calcStaticFee(channeldb.SingleFunderTweaklessBit, 0) - if aliceChannel.channelState.LocalCommitment.CommitFee != defaultFee { + if aliceChanState.CommitFee != defaultFee { t.Fatalf("dust htlc amounts not subtracted from commitment fee "+ "expected %v, got %v", defaultFee, - aliceChannel.channelState.LocalCommitment.CommitFee) + aliceChanState.CommitFee) } - if bobChannel.channelState.LocalCommitment.CommitFee != defaultFee { + if bobChanState.CommitFee != defaultFee { t.Fatalf("dust htlc amounts not subtracted from commitment fee "+ "expected %v, got %v", defaultFee, - bobChannel.channelState.LocalCommitment.CommitFee) + bobChanState.CommitFee) } // Alice's final balance should reflect the HTLC deficit even though // the HTLC was paid to fees as it was trimmed. - aliceEndBalance := aliceChannel.channelState.LocalCommitment.LocalBalance + aliceEndBalance := aliceChanState.LocalBalance aliceExpectedBalance := aliceStartingBalance - htlcAmount if aliceEndBalance != aliceExpectedBalance { t.Fatalf("alice not credited for dust: expected %v, got %v", @@ -1486,7 +1492,7 @@ func TestHTLCDustLimit(t *testing.T) { htlcSat := (btcutil.Amount(500) + HtlcTimeoutFee( aliceChannel.channelState.ChanType, chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ), )) htlcAmount := lnwire.NewMSatFromSatoshis(htlcSat) @@ -1515,10 +1521,10 @@ func TestHTLCDustLimit(t *testing.T) { 2, len(bobCommitment.txn.TxOut)) } defaultFee := calcStaticFee(channeldb.SingleFunderTweaklessBit, 0) - if bobChannel.channelState.LocalCommitment.CommitFee != defaultFee { + if bobChannel.channelState.Commitments.Local.CommitFee != defaultFee { t.Fatalf("dust htlc amount was subtracted from commitment fee "+ "expected %v, got %v", defaultFee, - bobChannel.channelState.LocalCommitment.CommitFee) + bobChannel.channelState.Commitments.Local.CommitFee) } // Settle HTLC and create a new commitment state. @@ -1731,12 +1737,12 @@ func TestChannelBalanceDustLimit(t *testing.T) { // once fees have been subtracted, but smaller than Bob's dust limit. // We account in fees for the HTLC we will be adding. defaultFee := calcStaticFee(channeldb.SingleFunderTweaklessBit, 1) - aliceBalance := aliceChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis() + aliceBalance := aliceChannel.channelState.Commitments.Local.LocalBalance.ToSatoshis() //nolint:lll htlcSat := aliceBalance - defaultFee htlcSat += HtlcSuccessFee( aliceChannel.channelState.ChanType, chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ), ) @@ -2153,11 +2159,12 @@ func TestCancelHTLC(t *testing.T) { // of the new HTLC. aliceExpectedBalance := btcutil.Amount(btcutil.SatoshiPerBitcoin*4) - calcStaticFee(channeldb.SingleFunderTweaklessBit, 1) - if aliceChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis() != + aliceChanState := &aliceChannel.channelState.Commitments.Local + if aliceChanState.LocalBalance.ToSatoshis() != aliceExpectedBalance { t.Fatalf("Alice's balance is wrong: expected %v, got %v", aliceExpectedBalance, - aliceChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis()) + aliceChanState.LocalBalance.ToSatoshis()) } // Now, with the HTLC committed on both sides, trigger a cancellation @@ -2198,32 +2205,33 @@ func TestCancelHTLC(t *testing.T) { expectedBalance := btcutil.Amount(btcutil.SatoshiPerBitcoin * 5) staticFee := calcStaticFee(channeldb.SingleFunderTweaklessBit, 0) - if aliceChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis() != + if aliceChanState.LocalBalance.ToSatoshis() != expectedBalance-staticFee { t.Fatalf("balance is wrong: expected %v, got %v", - aliceChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis(), + aliceChanState.LocalBalance.ToSatoshis(), expectedBalance-staticFee) } - if aliceChannel.channelState.LocalCommitment.RemoteBalance.ToSatoshis() != + if aliceChanState.RemoteBalance.ToSatoshis() != expectedBalance { t.Fatalf("balance is wrong: expected %v, got %v", - aliceChannel.channelState.LocalCommitment.RemoteBalance.ToSatoshis(), + aliceChanState.RemoteBalance.ToSatoshis(), expectedBalance) } - if bobChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis() != + bobChanState := bobChannel.channelState.Commitments.Local + if bobChanState.LocalBalance.ToSatoshis() != expectedBalance { t.Fatalf("balance is wrong: expected %v, got %v", - bobChannel.channelState.LocalCommitment.LocalBalance.ToSatoshis(), + bobChanState.LocalBalance.ToSatoshis(), expectedBalance) } - if bobChannel.channelState.LocalCommitment.RemoteBalance.ToSatoshis() != + if bobChanState.RemoteBalance.ToSatoshis() != expectedBalance-staticFee { t.Fatalf("balance is wrong: expected %v, got %v", - bobChannel.channelState.LocalCommitment.RemoteBalance.ToSatoshis(), + bobChanState.RemoteBalance.ToSatoshis(), expectedBalance-staticFee) } } @@ -2240,10 +2248,10 @@ func TestCooperativeCloseDustAdherence(t *testing.T) { require.NoError(t, err, "unable to create test channels") aliceFeeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) bobFeeRate := chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) setDustLimit := func(dustVal btcutil.Amount) { @@ -2259,10 +2267,12 @@ func TestCooperativeCloseDustAdherence(t *testing.T) { } setBalances := func(aliceBalance, bobBalance lnwire.MilliSatoshi) { - aliceChannel.channelState.LocalCommitment.LocalBalance = aliceBalance - aliceChannel.channelState.LocalCommitment.RemoteBalance = bobBalance - bobChannel.channelState.LocalCommitment.LocalBalance = bobBalance - bobChannel.channelState.LocalCommitment.RemoteBalance = aliceBalance + aliceChanState := &aliceChannel.channelState.Commitments.Local + aliceChanState.LocalBalance = aliceBalance + aliceChanState.RemoteBalance = bobBalance + bobChanState := &bobChannel.channelState.Commitments.Local + bobChanState.LocalBalance = bobBalance + bobChanState.RemoteBalance = aliceBalance } aliceDeliveryScript := bobsPrivKey[:] @@ -2333,7 +2343,7 @@ func TestCooperativeCloseDustAdherence(t *testing.T) { t.Fatalf("close tx has wrong number of outputs: expected %v "+ "got %v", 1, len(closeTx.TxOut)) } - commitFee := aliceChannel.channelState.LocalCommitment.CommitFee + commitFee := aliceChannel.channelState.Commitments.Local.CommitFee aliceExpectedBalance := aliceBal.ToSatoshis() - aliceFee + commitFee if closeTx.TxOut[0].Value != int64(aliceExpectedBalance) { t.Fatalf("alice's balance is incorrect: expected %v, got %v", @@ -2403,7 +2413,7 @@ func TestUpdateFeeAdjustments(t *testing.T) { // First, we'll grab the current base fee rate as we'll be using this // to make relative adjustments int he fee rate. - baseFeeRate := aliceChannel.channelState.LocalCommitment.FeePerKw + baseFeeRate := aliceChannel.channelState.Commitments.Local.FeePerKw // We'll first try to increase the fee rate 5x, this should be able to // be committed without any issue. @@ -2533,7 +2543,9 @@ func TestUpdateFeeConcurrentSig(t *testing.T) { err = bobChannel.ReceiveNewCommitment(aliceNewCommits.CommitSigs) require.NoError(t, err, "bob unable to process alice's new commitment") - if chainfee.SatPerKWeight(bobChannel.channelState.LocalCommitment.FeePerKw) == fee { + if chainfee.SatPerKWeight( + bobChannel.channelState.Commitments.Local.FeePerKw, + ) == fee { t.Fatalf("bob's feePerKw was unexpectedly locked in") } @@ -2542,7 +2554,9 @@ func TestUpdateFeeConcurrentSig(t *testing.T) { _, _, _, err = bobChannel.RevokeCurrentCommitment() require.NoError(t, err, "unable to generate bob revocation") - if chainfee.SatPerKWeight(bobChannel.channelState.LocalCommitment.FeePerKw) != fee { + if chainfee.SatPerKWeight( + bobChannel.channelState.Commitments.Local.FeePerKw, + ) != fee { t.Fatalf("bob's feePerKw was not locked in") } } @@ -2597,7 +2611,7 @@ func TestUpdateFeeSenderCommits(t *testing.T) { require.NoError(t, err, "bob unable to process alice's new commitment") if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) == fee { t.Fatalf("bob's feePerKw was unexpectedly locked in") @@ -2609,7 +2623,7 @@ func TestUpdateFeeSenderCommits(t *testing.T) { require.NoError(t, err, "unable to generate bob revocation") if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) != fee { t.Fatalf("bob's feePerKw was not locked in") @@ -2632,7 +2646,7 @@ func TestUpdateFeeSenderCommits(t *testing.T) { require.NoError(t, err, "alice unable to process bob's new commitment") if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) == fee { t.Fatalf("alice's feePerKw was unexpectedly locked in") @@ -2644,7 +2658,7 @@ func TestUpdateFeeSenderCommits(t *testing.T) { require.NoError(t, err, "unable to revoke alice channel") if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) != fee { t.Fatalf("alice's feePerKw was not locked in") @@ -2726,7 +2740,7 @@ func TestUpdateFeeReceiverCommits(t *testing.T) { require.NoError(t, err, "alice unable to process bob's new commitment") if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) == fee { t.Fatalf("bob's feePerKw was unexpectedly locked in") @@ -2739,7 +2753,7 @@ func TestUpdateFeeReceiverCommits(t *testing.T) { require.NoError(t, err, "unable to revoke alice channel") if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) != fee { t.Fatalf("bob's feePerKw was not locked in") @@ -2761,7 +2775,7 @@ func TestUpdateFeeReceiverCommits(t *testing.T) { require.NoError(t, err, "alice unable to process bob's new commitment") if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) == fee { t.Fatalf("alice's feePerKw was unexpectedly locked in") @@ -2773,7 +2787,7 @@ func TestUpdateFeeReceiverCommits(t *testing.T) { require.NoError(t, err, "unable to generate bob revocation") if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) != fee { t.Fatalf("Alice's feePerKw was not locked in") @@ -2851,7 +2865,7 @@ func TestUpdateFeeMultipleUpdates(t *testing.T) { require.NoError(t, err, "bob unable to process alice's new commitment") if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) == fee { t.Fatalf("bob's feePerKw was unexpectedly locked in") @@ -2875,7 +2889,7 @@ func TestUpdateFeeMultipleUpdates(t *testing.T) { require.NoError(t, err, "unable to generate bob revocation") if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) != fee { t.Fatalf("bob's feePerKw was not locked in") @@ -2900,7 +2914,7 @@ func TestUpdateFeeMultipleUpdates(t *testing.T) { } if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) == fee { t.Fatalf("alice's feePerKw was unexpectedly locked in") @@ -2912,7 +2926,7 @@ func TestUpdateFeeMultipleUpdates(t *testing.T) { require.NoError(t, err, "unable to revoke alice channel") if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) != fee { t.Fatalf("alice's feePerKw was not locked in") @@ -4626,7 +4640,7 @@ func TestFeeUpdateRejectInsaneFee(t *testing.T) { // Next, we'll try to add a fee rate to Alice which is 1,000,000x her // starting fee rate. startingFeeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) newFeeRate := startingFeeRate * 1000000 @@ -4653,7 +4667,7 @@ func TestChannelRetransmissionFeeUpdate(t *testing.T) { // First, we'll fetch the current fee rate present within the // commitment transactions. startingFeeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) // Next, we'll start a commitment update, with Alice sending a new @@ -4767,13 +4781,13 @@ func TestChannelRetransmissionFeeUpdate(t *testing.T) { // Both parties should now have the latest fee rate locked-in. if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) != newFeeRate { t.Fatalf("alice's feePerKw was not locked in") } if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) != newFeeRate { t.Fatalf("bob's feePerKw was not locked in") @@ -4856,7 +4870,7 @@ func TestFeeUpdateOldDiskFormat(t *testing.T) { // First, we'll fetch the current fee rate present within the // commitment transactions. startingFeeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) newFeeRate := startingFeeRate @@ -4965,13 +4979,13 @@ func TestFeeUpdateOldDiskFormat(t *testing.T) { // Both parties should now have the latest fee rate locked-in. if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) != newFeeRate { t.Fatalf("alice's feePerKw was not locked in") } if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) != newFeeRate { t.Fatalf("bob's feePerKw was not locked in") @@ -4996,13 +5010,13 @@ func TestFeeUpdateOldDiskFormat(t *testing.T) { // ...and the final fee rate locked in. if chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) != newFeeRate { t.Fatalf("alice's feePerKw was not locked in") } if chainfee.SatPerKWeight( - bobChannel.channelState.LocalCommitment.FeePerKw, + bobChannel.channelState.Commitments.Local.FeePerKw, ) != newFeeRate { t.Fatalf("bob's feePerKw was not locked in") @@ -5146,7 +5160,7 @@ func TestChanAvailableBandwidth(t *testing.T) { aliceChannel.channelState.ChanCfgs.Local.ChanReserve, ) feeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) assertBandwidthEstimateCorrect := func(aliceInitiate bool, @@ -5299,7 +5313,7 @@ func TestChanAvailableBalanceNearHtlcFee(t *testing.T) { aliceChannel.channelState.ChanCfgs.Local.DustLimit, ) feeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) // When calculating the fee buffer sending an htlc we need to account @@ -5312,7 +5326,7 @@ func TestChanAvailableBalanceNearHtlcFee(t *testing.T) { feeRate.FeeForWeight(input.HTLCWeight), ) commitFee := lnwire.NewMSatFromSatoshis( - aliceChannel.channelState.LocalCommitment.CommitFee, + aliceChannel.channelState.Commitments.Local.CommitFee, ) htlcTimeoutFee := lnwire.NewMSatFromSatoshis( HtlcTimeoutFee(aliceChannel.channelState.ChanType, feeRate), @@ -5483,7 +5497,7 @@ func TestChanCommitWeightDustHtlcs(t *testing.T) { ) feeRate := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) htlcTimeoutFee := lnwire.NewMSatFromSatoshis( HtlcTimeoutFee(aliceChannel.channelState.ChanType, feeRate), @@ -5569,7 +5583,8 @@ func TestChanCommitWeightDustHtlcs(t *testing.T) { // In addition, we expect this weight to result in the fee we currently // see being paid on the remote commitent. calcFee := feeRate.FeeForWeight(weight2) - remoteCommitFee := aliceChannel.channelState.RemoteCommitment.CommitFee + remoteCommitFee := + aliceChannel.channelState.Commitments.Remote.CommitFee require.Equal(t, calcFee, remoteCommitFee) // Settle the HTLC, bringing commitment weight back to base. @@ -5591,7 +5606,7 @@ func TestChanCommitWeightDustHtlcs(t *testing.T) { // Ensure the current remote commit has the expected commitfee. calcFee = feeRate.FeeForWeight(weight2) - remoteCommitFee = bobChannel.channelState.RemoteCommitment.CommitFee + remoteCommitFee = bobChannel.channelState.Commitments.Remote.CommitFee require.Equal(t, calcFee, remoteCommitFee) settleHtlc(preimg) @@ -6043,7 +6058,7 @@ func TestChannelUnilateralCloseHtlcResolution(t *testing.T) { aliceCloseSummary, err := NewUnilateralCloseSummary( aliceChannel.channelState, aliceChannel.Signer, spendDetail, - aliceChannel.channelState.RemoteCommitment, + aliceChannel.channelState.Commitments.Remote, aliceChannel.channelState.RemoteCurrentRevocation, fn.Some[AuxLeafStore](&MockAuxLeafStore{}), fn.Some[AuxContractResolver](&MockAuxContractResolver{}), @@ -6194,7 +6209,7 @@ func TestChannelUnilateralClosePendingCommit(t *testing.T) { aliceWrongCloseSummary, err := NewUnilateralCloseSummary( aliceChannel.channelState, aliceChannel.Signer, spendDetail, - aliceChannel.channelState.RemoteCommitment, + aliceChannel.channelState.Commitments.Remote, aliceChannel.channelState.RemoteCurrentRevocation, fn.Some[AuxLeafStore](&MockAuxLeafStore{}), fn.Some[AuxContractResolver](&MockAuxContractResolver{}), @@ -6625,8 +6640,10 @@ func assertChannelBalances(t *testing.T, alice, bob *LightningChannel, _, _, line, _ := runtime.Caller(1) - aliceSelfBalance := alice.channelState.LocalCommitment.LocalBalance.ToSatoshis() - aliceBobBalance := alice.channelState.LocalCommitment.RemoteBalance.ToSatoshis() + aliceSelfBalance := + alice.channelState.Commitments.Local.LocalBalance.ToSatoshis() + aliceBobBalance := + alice.channelState.Commitments.Local.RemoteBalance.ToSatoshis() if aliceSelfBalance != aliceBalance { t.Fatalf("line #%v: wrong alice self balance: expected %v, got %v", line, aliceBalance, aliceSelfBalance) @@ -6636,8 +6653,10 @@ func assertChannelBalances(t *testing.T, alice, bob *LightningChannel, line, bobBalance, aliceBobBalance) } - bobSelfBalance := bob.channelState.LocalCommitment.LocalBalance.ToSatoshis() - bobAliceBalance := bob.channelState.LocalCommitment.RemoteBalance.ToSatoshis() + bobSelfBalance := + bob.channelState.Commitments.Local.LocalBalance.ToSatoshis() + bobAliceBalance := + bob.channelState.Commitments.Local.RemoteBalance.ToSatoshis() if bobSelfBalance != bobBalance { t.Fatalf("line #%v: wrong bob self balance: expected %v, got %v", line, bobBalance, bobSelfBalance) @@ -6721,7 +6740,7 @@ func TestChanReserve(t *testing.T) { t.Fatalf("unable to complete state update: %v", err) } - commitFee := aliceChannel.channelState.LocalCommitment.CommitFee + commitFee := aliceChannel.channelState.Commitments.Local.CommitFee assertChannelBalances( t, aliceChannel, bobChannel, btcutil.SatoshiPerBitcoin*4.5-commitFee, btcutil.SatoshiPerBitcoin*5, @@ -6803,7 +6822,7 @@ func TestChanReserve(t *testing.T) { t.Fatalf("unable to complete state update: %v", err) } - commitFee = aliceChannel.channelState.LocalCommitment.CommitFee + commitFee = aliceChannel.channelState.Commitments.Local.CommitFee assertChannelBalances( t, aliceChannel, bobChannel, btcutil.SatoshiPerBitcoin*3-commitFee, btcutil.SatoshiPerBitcoin*5, @@ -6819,7 +6838,7 @@ func TestChanReserve(t *testing.T) { t.Fatalf("unable to complete state update: %v", err) } - commitFee = aliceChannel.channelState.LocalCommitment.CommitFee + commitFee = aliceChannel.channelState.Commitments.Local.CommitFee assertChannelBalances( t, aliceChannel, bobChannel, btcutil.SatoshiPerBitcoin*3-commitFee, btcutil.SatoshiPerBitcoin*7, @@ -6843,7 +6862,7 @@ func TestChanReserve(t *testing.T) { t.Fatalf("unable to complete state update: %v", err) } - commitFee = aliceChannel.channelState.LocalCommitment.CommitFee + commitFee = aliceChannel.channelState.Commitments.Local.CommitFee assertChannelBalances( t, aliceChannel, bobChannel, btcutil.SatoshiPerBitcoin*3-commitFee, btcutil.SatoshiPerBitcoin*6, @@ -6869,7 +6888,7 @@ func TestChanReserveRemoteInitiator(t *testing.T) { // to add any more HTLCs to the commitment. Although a reserve this // high is unrealistic, a channel can easily get into a situation // where the initiator cannot pay for the fee of any more HTLCs. - commitFee := aliceChannel.channelState.LocalCommitment.CommitFee + commitFee := aliceChannel.channelState.Commitments.Local.CommitFee aliceMinReserve := 5*btcutil.SatoshiPerBitcoin - commitFee aliceChannel.channelState.ChanCfgs.Local.ChanReserve = aliceMinReserve @@ -6917,14 +6936,14 @@ func TestChanReserveLocalInitiatorDustHtlc(t *testing.T) { htlcSat := btcutil.Amount(500) + HtlcTimeoutFee( aliceChannel.channelState.ChanType, chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ), ) // Set Alice's channel reserve to be low enough to carry the value of // the HTLC, but not low enough to allow the extra fee from adding the // HTLC to the commitment. - commitFee := aliceChannel.channelState.LocalCommitment.CommitFee + commitFee := aliceChannel.channelState.Commitments.Local.CommitFee aliceMinReserve := 5*btcutil.SatoshiPerBitcoin - commitFee - htlcSat aliceChannel.channelState.ChanCfgs.Local.ChanReserve = aliceMinReserve @@ -7078,7 +7097,8 @@ func TestNewBreachRetributionSkipsDustHtlcs(t *testing.T) { // At this point, we'll capture the current state number, as well as // the current commitment. - revokedStateNum := aliceChannel.channelState.LocalCommitment.CommitHeight + revokedStateNum := + aliceChannel.channelState.Commitments.Local.CommitHeight // We'll now have Bob settle those HTLC's to Alice and then advance // forward to a new state. @@ -7098,7 +7118,7 @@ func TestNewBreachRetributionSkipsDustHtlcs(t *testing.T) { // At this point, we'll now simulate a contract breach by Bob using the // NewBreachRetribution method. - breachTx := aliceChannel.channelState.RemoteCommitment.CommitTx + breachTx := aliceChannel.channelState.Commitments.Remote.CommitTx breachRet, err := NewBreachRetribution( aliceChannel.channelState, revokedStateNum, 100, breachTx, fn.Some[AuxLeafStore](&MockAuxLeafStore{}), @@ -8005,7 +8025,7 @@ func TestChannelMaxFeeRate(t *testing.T) { ) currentFeeRate := chainfee.SatPerKWeight( - tc.channel.channelState.LocalCommitment.FeePerKw, + tc.channel.channelState.Commitments.Local.FeePerKw, ) // When the fee allocation would push our max fee rate below our @@ -9746,7 +9766,7 @@ func deriveDummyRetributionParams(chanState *channeldb.OpenChannel) (uint32, *CommitmentKeyRing, chainhash.Hash) { config := chanState.ChanCfgs.Remote - commitHash := chanState.RemoteCommitment.CommitTx.TxHash() + commitHash := chanState.Commitments.Remote.CommitTx.TxHash() keyRing := DeriveCommitmentKeys( config.RevocationBasePoint.PubKey, lntypes.Remote, chanState.ChanType, &chanState.ChanCfgs.Local, @@ -10036,7 +10056,7 @@ func TestCreateBreachRetributionLegacy(t *testing.T) { ) // Use the remote commitment as our revocation log. - revokedLog := aliceChannel.channelState.RemoteCommitment + revokedLog := aliceChannel.channelState.Commitments.Remote ourOp := revokedLog.CommitTx.TxOut[0] theirOp := revokedLog.CommitTx.TxOut[1] @@ -10107,7 +10127,7 @@ func testNewBreachRetribution(t *testing.T, chanType channeldb.ChannelType) { stateNum := uint64(0) chainHash := aliceChannel.channelState.ChainHash theirDelay := uint32(aliceChannel.channelState.ChanCfgs.Remote.CsvDelay) - breachTx := aliceChannel.channelState.RemoteCommitment.CommitTx + breachTx := aliceChannel.channelState.Commitments.Remote.CommitTx // Create a breach retribution at height 0, which should give us an // error as there are no past delta state saved as revocation logs yet. @@ -10129,7 +10149,7 @@ func testNewBreachRetribution(t *testing.T, chanType channeldb.ChannelType) { // We now force a state transition which will give us a revocation log // at height 0. - txid := aliceChannel.channelState.RemoteCommitment.CommitTx.TxHash() + txid := aliceChannel.channelState.Commitments.Remote.CommitTx.TxHash() err = ForceStateTransition(aliceChannel, bobChannel) require.NoError(t, err) @@ -10465,7 +10485,7 @@ func TestAsynchronousSendingContraint(t *testing.T) { // Static fee rate of 6000 sats/kw. feePerKw := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) additionalHtlc := feePerKw.FeeForWeight(input.HTLCWeight) @@ -10598,7 +10618,7 @@ func TestAsynchronousSendingWithFeeBuffer(t *testing.T) { // Static fee rate of 6000 sats/kw. feePerKw := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) // Calculate the fee buffer for the current commitment tx including @@ -10736,7 +10756,8 @@ func TestAsynchronousSendingWithFeeBuffer(t *testing.T) { // All of alice's balance is used up in fees and htlcs so the local // balance equals exactly the local reserve. - require.Equal(t, aliceChannel.channelState.LocalCommitment.LocalBalance, + require.Equal(t, + aliceChannel.channelState.Commitments.Local.LocalBalance, lnwire.NewMSatFromSatoshis(aliceReserve)) } @@ -10793,7 +10814,7 @@ func TestEnforceFeeBuffer(t *testing.T) { // Static fee rate of 6000 sats/kw. feePerKw := chainfee.SatPerKWeight( - aliceChannel.channelState.LocalCommitment.FeePerKw, + aliceChannel.channelState.Commitments.Local.FeePerKw, ) // Commitment Fee of the channel state (with 1 pending htlc). @@ -10880,7 +10901,7 @@ func TestEnforceFeeBuffer(t *testing.T) { feeHTLCMsat := lnwire.NewMSatFromSatoshis(feeHTLC) aliceBalance := aliceReserveMsat + bufferAmt - 2*feeHTLCMsat - expectedAmt := aliceChannel.channelState.LocalCommitment.LocalBalance + expectedAmt := aliceChannel.channelState.Commitments.Local.LocalBalance require.Equal(t, aliceBalance, expectedAmt) } diff --git a/lnwallet/commitment.go b/lnwallet/commitment.go index e1229fccb4e..477d84f2cc3 100644 --- a/lnwallet/commitment.go +++ b/lnwallet/commitment.go @@ -1279,7 +1279,7 @@ func findOutputIndexesFromRemote(revocationPreimage *chainhash.Hash, ourIndex := uint32(channeldb.OutputIndexEmpty) theirIndex := uint32(channeldb.OutputIndexEmpty) - chanCommit := chanState.RemoteCommitment + chanCommit := chanState.Commitments.Remote _, commitmentPoint := btcec.PrivKeyFromBytes(revocationPreimage[:]) // With the commitment point generated, we can now derive the king ring diff --git a/lnwallet/reservation.go b/lnwallet/reservation.go index 4c4f58f8be0..0f5d212d771 100644 --- a/lnwallet/reservation.go +++ b/lnwallet/reservation.go @@ -483,17 +483,23 @@ func NewChannelReservation(capacity, localFundingAmt btcutil.Amount, IsInitiator: initiator, ChannelFlags: req.Flags, Capacity: capacity, - LocalCommitment: channeldb.ChannelCommitment{ - LocalBalance: ourBalance, - RemoteBalance: theirBalance, - FeePerKw: btcutil.Amount(req.CommitFeePerKw), - CommitFee: commitFee, - }, - RemoteCommitment: channeldb.ChannelCommitment{ - LocalBalance: ourBalance, - RemoteBalance: theirBalance, - FeePerKw: btcutil.Amount(req.CommitFeePerKw), - CommitFee: commitFee, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: channeldb.ChannelCommitment{ + LocalBalance: ourBalance, + RemoteBalance: theirBalance, + FeePerKw: btcutil.Amount( + req.CommitFeePerKw, + ), + CommitFee: commitFee, + }, + Remote: channeldb.ChannelCommitment{ + LocalBalance: ourBalance, + RemoteBalance: theirBalance, + FeePerKw: btcutil.Amount( + req.CommitFeePerKw, + ), + CommitFee: commitFee, + }, }, ThawHeight: thawHeight, Db: wallet.Cfg.Database, diff --git a/lnwallet/test_utils.go b/lnwallet/test_utils.go index c427724c8b4..d125f454de8 100644 --- a/lnwallet/test_utils.go +++ b/lnwallet/test_utils.go @@ -330,11 +330,13 @@ func CreateTestChannels(t *testing.T, chanType channeldb.ChannelType, RemoteCurrentRevocation: bobCommitPoint, RevocationProducer: alicePreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: aliceLocalCommit, - RemoteCommitment: aliceRemoteCommit, - Db: dbAlice.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), - FundingTxn: testTx, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: aliceLocalCommit, + Remote: aliceRemoteCommit, + }, + Db: dbAlice.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), + FundingTxn: testTx, } bobChannelState := &channeldb.OpenChannel{ ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ @@ -350,10 +352,12 @@ func CreateTestChannels(t *testing.T, chanType channeldb.ChannelType, RemoteCurrentRevocation: aliceCommitPoint, RevocationProducer: bobPreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: bobLocalCommit, - RemoteCommitment: bobRemoteCommit, - Db: dbBob.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: bobLocalCommit, + Remote: bobRemoteCommit, + }, + Db: dbBob.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), } // If the channel type has a tapscript root, then we'll also specify diff --git a/lnwallet/transactions_test.go b/lnwallet/transactions_test.go index 9612e28b67b..4361f8db7d7 100644 --- a/lnwallet/transactions_test.go +++ b/lnwallet/transactions_test.go @@ -984,11 +984,13 @@ func createTestChannelsForVectors(tc *testContext, chanType channeldb.ChannelTyp RemoteCurrentRevocation: localCommitPoint, RevocationProducer: remotePreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: remoteCommit, - RemoteCommitment: remoteCommit, - Db: dbRemote.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), - FundingTxn: tc.fundingTx.MsgTx(), + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: remoteCommit, + Remote: remoteCommit, + }, + Db: dbRemote.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), + FundingTxn: tc.fundingTx.MsgTx(), } localChannelState := &channeldb.OpenChannel{ ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ @@ -1004,11 +1006,13 @@ func createTestChannelsForVectors(tc *testContext, chanType channeldb.ChannelTyp RemoteCurrentRevocation: remoteCommitPoint, RevocationProducer: localPreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: localCommit, - RemoteCommitment: localCommit, - Db: dbLocal.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), - FundingTxn: tc.fundingTx.MsgTx(), + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: localCommit, + Remote: localCommit, + }, + Db: dbLocal.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), + FundingTxn: tc.fundingTx.MsgTx(), } // Create mock signers that can sign for the keys that are used. diff --git a/lnwallet/wallet.go b/lnwallet/wallet.go index 78f961fbfdd..898e11adfff 100644 --- a/lnwallet/wallet.go +++ b/lnwallet/wallet.go @@ -1852,13 +1852,13 @@ func (l *LightningWallet) handleChanPointReady(req *continueContributionMsg) { return desc.CustomFundingBlob })(req.auxFundingDesc) - chanState.LocalCommitment.CustomBlob = fn.MapOption( + chanState.Commitments.Local.CustomBlob = fn.MapOption( func(desc AuxFundingDesc) tlv.Blob { return desc.CustomLocalCommitBlob }, )(req.auxFundingDesc) - chanState.RemoteCommitment.CustomBlob = fn.MapOption( + chanState.Commitments.Remote.CustomBlob = fn.MapOption( func(desc AuxFundingDesc) tlv.Blob { return desc.CustomRemoteCommitBlob }, @@ -1931,8 +1931,9 @@ func (l *LightningWallet) handleChanPointReady(req *continueContributionMsg) { } // With the funding tx complete, create both commitment transactions. - localBalance := pendingReservation.partialState.LocalCommitment.LocalBalance.ToSatoshis() - remoteBalance := pendingReservation.partialState.LocalCommitment.RemoteBalance.ToSatoshis() + localCommitment := pendingReservation.partialState.Commitments.Local + localBalance := localCommitment.LocalBalance.ToSatoshis() + remoteBalance := localCommitment.RemoteBalance.ToSatoshis() var leaseExpiry uint32 if pendingReservation.partialState.ChanType.HasLeaseExpiration() { leaseExpiry = pendingReservation.partialState.ThawHeight @@ -2007,8 +2008,8 @@ func (l *LightningWallet) handleChanPointReady(req *continueContributionMsg) { // Record newly available information within the open channel state. chanState.FundingOutpoint = chanPoint - chanState.LocalCommitment.CommitTx = ourCommitTx - chanState.RemoteCommitment.CommitTx = theirCommitTx + chanState.Commitments.Local.CommitTx = ourCommitTx + chanState.Commitments.Remote.CommitTx = theirCommitTx // Next, we'll obtain the funding witness script, and the funding // output itself so we can generate a valid signature for the remote @@ -2283,7 +2284,7 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs // At this point, we can also record and verify their signature for our // commitment transaction. res.theirCommitmentSig = msg.theirCommitmentSig - commitTx := res.partialState.LocalCommitment.CommitTx + commitTx := res.partialState.Commitments.Local.CommitTx err := l.verifyCommitSig(res, msg.theirCommitmentSig, commitTx) if err != nil { @@ -2294,7 +2295,7 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs } theirCommitSigBytes := msg.theirCommitmentSig.Serialize() - res.partialState.LocalCommitment.CommitSig = theirCommitSigBytes + res.partialState.Commitments.Local.CommitSig = theirCommitSigBytes // Funding complete, this entry can be removed from limbo. l.limboMtx.Lock() @@ -2374,12 +2375,12 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) { chanState.CustomBlob = fn.MapOption(func(desc AuxFundingDesc) tlv.Blob { return desc.CustomFundingBlob })(req.auxFundingDesc) - chanState.LocalCommitment.CustomBlob = fn.MapOption( + chanState.Commitments.Local.CustomBlob = fn.MapOption( func(desc AuxFundingDesc) tlv.Blob { return desc.CustomLocalCommitBlob }, )(req.auxFundingDesc) - chanState.RemoteCommitment.CustomBlob = fn.MapOption( + chanState.Commitments.Remote.CustomBlob = fn.MapOption( func(desc AuxFundingDesc) tlv.Blob { return desc.CustomRemoteCommitBlob }, @@ -2392,8 +2393,10 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) { // Now that we have the funding outpoint, we can generate both versions // of the commitment transaction, and generate a signature for the // remote node's commitment transactions. - localBalance := pendingReservation.partialState.LocalCommitment.LocalBalance.ToSatoshis() - remoteBalance := pendingReservation.partialState.LocalCommitment.RemoteBalance.ToSatoshis() + localBalance := pendingReservation.partialState.Commitments. + Local.LocalBalance.ToSatoshis() + remoteBalance := pendingReservation.partialState.Commitments. + Local.RemoteBalance.ToSatoshis() var leaseExpiry uint32 if pendingReservation.partialState.ChanType.HasLeaseExpiration() { leaseExpiry = pendingReservation.partialState.ThawHeight @@ -2445,8 +2448,8 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) { // without further synchronization. txsort.InPlaceSort(ourCommitTx) txsort.InPlaceSort(theirCommitTx) - chanState.LocalCommitment.CommitTx = ourCommitTx - chanState.RemoteCommitment.CommitTx = theirCommitTx + chanState.Commitments.Local.CommitTx = ourCommitTx + chanState.Commitments.Remote.CommitTx = theirCommitTx walletLog.Debugf("Local commit tx for ChannelPoint(%v): %v", req.fundingOutpoint, spew.Sdump(ourCommitTx)) @@ -2465,7 +2468,7 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) { } theirCommitSigBytes := req.theirCommitmentSig.Serialize() - chanState.LocalCommitment.CommitSig = theirCommitSigBytes + chanState.Commitments.Local.CommitSig = theirCommitSigBytes channelValue := int64(pendingReservation.partialState.Capacity) theirKey := pendingReservation.theirContribution.MultiSigKey diff --git a/peer/test_utils.go b/peer/test_utils.go index fc7e5757cd1..be388a38f5b 100644 --- a/peer/test_utils.go +++ b/peer/test_utils.go @@ -257,11 +257,13 @@ func createTestPeerWithChannel(t *testing.T, updateChan func(a, RemoteCurrentRevocation: bobCommitPoint, RevocationProducer: alicePreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: aliceCommit, - RemoteCommitment: aliceCommit, - Db: dbAlice.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), - FundingTxn: channels.TestFundingTx, + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: aliceCommit, + Remote: aliceCommit, + }, + Db: dbAlice.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), + FundingTxn: channels.TestFundingTx, } bobChannelState := &channeldb.OpenChannel{ ChanCfgs: lntypes.Dual[channeldb.ChannelConfig]{ @@ -276,10 +278,12 @@ func createTestPeerWithChannel(t *testing.T, updateChan func(a, RemoteCurrentRevocation: aliceCommitPoint, RevocationProducer: bobPreimageProducer, RevocationStore: shachain.NewRevocationStore(), - LocalCommitment: bobCommit, - RemoteCommitment: bobCommit, - Db: dbBob.ChannelStateDB(), - Packager: channeldb.NewChannelPackager(shortChanID), + Commitments: lntypes.Dual[channeldb.ChannelCommitment]{ + Local: bobCommit, + Remote: bobCommit, + }, + Db: dbBob.ChannelStateDB(), + Packager: channeldb.NewChannelPackager(shortChanID), } // Set custom values on the channel states. diff --git a/pilot.go b/pilot.go index 2a37b080d08..a203f57280e 100644 --- a/pilot.go +++ b/pilot.go @@ -265,7 +265,7 @@ func initAutoPilot(svr *server, cfg *lncfg.AutoPilot, chanState := make([]autopilot.LocalChannel, len(activeChannels)) for i, channel := range activeChannels { - localCommit := channel.LocalCommitment + localCommit := channel.Commitments.Local balance := localCommit.LocalBalance.ToSatoshis() chanState[i] = autopilot.LocalChannel{ @@ -287,7 +287,7 @@ func initAutoPilot(svr *server, cfg *lncfg.AutoPilot, return nil, err } - localCommit := channel.LocalCommitment + localCommit := channel.Commitments.Local return &autopilot.LocalChannel{ ChanID: channel.ShortChanID(), Balance: localCommit.LocalBalance.ToSatoshis(), diff --git a/rpcserver.go b/rpcserver.go index 15ae37a186a..731f2a384f4 100644 --- a/rpcserver.go +++ b/rpcserver.go @@ -3676,7 +3676,7 @@ func (r *rpcServer) ChannelBalance(ctx context.Context, } for _, channel := range openChannels { - c := channel.LocalCommitment + c := channel.Commitments.Local localBalance += c.LocalBalance remoteBalance += c.RemoteBalance @@ -3690,7 +3690,8 @@ func (r *rpcServer) ChannelBalance(ctx context.Context, } // Encode the custom data for this open channel. - openChanData := channel.LocalCommitment.CustomBlob.UnwrapOr(nil) + openChanData := + channel.Commitments.Local.CustomBlob.UnwrapOr(nil) err = wire.WriteVarBytes(&customDataBuf, 0, openChanData) if err != nil { return nil, err @@ -3709,12 +3710,13 @@ func (r *rpcServer) ChannelBalance(ctx context.Context, } for _, channel := range pendingChannels { - c := channel.LocalCommitment + c := channel.Commitments.Local pendingOpenLocalBalance += c.LocalBalance pendingOpenRemoteBalance += c.RemoteBalance // Encode the custom data for this pending channel. - openChanData := channel.LocalCommitment.CustomBlob.UnwrapOr(nil) + openChanData := + channel.Commitments.Local.CustomBlob.UnwrapOr(nil) err = wire.WriteVarBytes(&customDataBuf, 0, openChanData) if err != nil { return nil, err @@ -3815,7 +3817,7 @@ func (r *rpcServer) fetchPendingOpenChannels() (pendingOpenChannels, error) { witnessWeight = input.WitnessCommitmentTxWeight } - localCommitment := pendingChan.LocalCommitment + localCommitment := pendingChan.Commitments.Local utx := btcutil.NewTx(localCommitment.CommitTx) commitBaseWeight := blockchain.GetTransactionWeight(utx) commitWeight := commitBaseWeight + witnessWeight @@ -3834,11 +3836,14 @@ func (r *rpcServer) fetchPendingOpenChannels() (pendingOpenChannels, error) { result[i] = &lnrpc.PendingChannelsResponse_PendingOpenChannel{ Channel: &lnrpc.PendingChannelsResponse_PendingChannel{ - RemoteNodePub: hex.EncodeToString(pub), - ChannelPoint: pendingChan.FundingOutpoint.String(), - Capacity: int64(pendingChan.Capacity), - LocalBalance: int64(localCommitment.LocalBalance.ToSatoshis()), - RemoteBalance: int64(localCommitment.RemoteBalance.ToSatoshis()), + RemoteNodePub: hex.EncodeToString(pub), + //nolint:lll + ChannelPoint: pendingChan.FundingOutpoint.String(), + Capacity: int64(pendingChan.Capacity), + //nolint:lll + LocalBalance: int64(localCommitment.LocalBalance.ToSatoshis()), + //nolint:lll + RemoteBalance: int64(localCommitment.RemoteBalance.ToSatoshis()), //nolint:lll LocalChanReserveSat: int64(pendingChan.ChanCfgs.Local.ChanReserve), //nolint:lll @@ -3932,9 +3937,8 @@ func (r *rpcServer) fetchPendingForceCloseChannels() (pendingForceClose, } channel.NumForwardingPackages = int64(len(fwdPkgs)) - channel.RemoteBalance = int64( - historical.LocalCommitment.RemoteBalance.ToSatoshis(), - ) + //nolint:lll + channel.RemoteBalance = int64(historical.Commitments.Local.RemoteBalance.ToSatoshis()) channel.Private = isPrivate(historical) channel.Memo = string(historical.Memo) @@ -4065,24 +4069,25 @@ func (r *rpcServer) fetchWaitingCloseChannels( var commitments lnrpc.PendingChannelsResponse_Commitments // Report local commit. May not be present when DLP is active. - if waitingClose.LocalCommitment.CommitTx != nil { + if waitingClose.Commitments.Local.CommitTx != nil { commitments.LocalTxid = - waitingClose.LocalCommitment.CommitTx.TxHash(). - String() + waitingClose.Commitments.Local.CommitTx. + TxHash().String() commitments.LocalCommitFeeSat = uint64( - waitingClose.LocalCommitment.CommitFee, + waitingClose.Commitments.Local.CommitFee, ) } // Report remote commit. May not be present when DLP is active. - if waitingClose.RemoteCommitment.CommitTx != nil { + if waitingClose.Commitments.Remote.CommitTx != nil { commitments.RemoteTxid = - waitingClose.RemoteCommitment.CommitTx.TxHash(). - String() + waitingClose.Commitments.Remote.CommitTx. + TxHash().String() commitments.RemoteCommitFeeSat = uint64( - waitingClose.RemoteCommitment.CommitFee, + waitingClose.Commitments.Remote. + CommitFee, ) } @@ -4127,11 +4132,13 @@ func (r *rpcServer) fetchWaitingCloseChannels( } channel := &lnrpc.PendingChannelsResponse_PendingChannel{ - RemoteNodePub: hex.EncodeToString(pub), - ChannelPoint: chanPoint.String(), - Capacity: int64(waitingClose.Capacity), - LocalBalance: int64(waitingClose.LocalCommitment.LocalBalance.ToSatoshis()), - RemoteBalance: int64(waitingClose.LocalCommitment.RemoteBalance.ToSatoshis()), + RemoteNodePub: hex.EncodeToString(pub), + ChannelPoint: chanPoint.String(), + Capacity: int64(waitingClose.Capacity), + //nolint:lll + LocalBalance: int64(waitingClose.Commitments.Local.LocalBalance.ToSatoshis()), + //nolint:lll + RemoteBalance: int64(waitingClose.Commitments.Local.RemoteBalance.ToSatoshis()), //nolint:lll LocalChanReserveSat: int64(waitingClose.ChanCfgs.Local.ChanReserve), //nolint:lll @@ -4608,7 +4615,8 @@ func isPrivate(dbChannel *channeldb.OpenChannel) bool { // It encodes that data as a pair of var bytes blobs. func encodeCustomChanData(lnChan *channeldb.OpenChannel) ([]byte, error) { customOpenChanData := lnChan.CustomBlob.UnwrapOr(nil) - customLocalCommitData := lnChan.LocalCommitment.CustomBlob.UnwrapOr(nil) + customLocalCommitData := + lnChan.Commitments.Local.CustomBlob.UnwrapOr(nil) // We'll encode our custom channel data as two blobs. The first is a // set of var bytes encoding of the open chan data, the second is an @@ -4649,7 +4657,7 @@ func createRPCOpenChannel(r *rpcServer, dbChannel *channeldb.OpenChannel, witnessWeight = input.WitnessCommitmentTxWeight } - localCommit := dbChannel.LocalCommitment + localCommit := dbChannel.Commitments.Local utx := btcutil.NewTx(localCommit.CommitTx) commitBaseWeight := blockchain.GetTransactionWeight(utx) commitWeight := commitBaseWeight + witnessWeight From 8d34873724136660eb60be094b416a35a8a361dc Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Wed, 21 Aug 2024 11:07:31 -0600 Subject: [PATCH 25/29] channeldb+contractcourt: tidy type signature for LatestCommitments --- channeldb/channel.go | 8 +++++--- contractcourt/chain_watcher.go | 16 ++++++++-------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/channeldb/channel.go b/channeldb/channel.go index d775e3abf2d..b85a8a52ac9 100644 --- a/channeldb/channel.go +++ b/channeldb/channel.go @@ -4012,7 +4012,9 @@ func (c *OpenChannel) Snapshot() *ChannelSnapshot { // remote party. These commitments are read from disk to ensure that only the // latest fully committed state is returned. The first commitment returned is // the local commitment, and the second returned is the remote commitment. -func (c *OpenChannel) LatestCommitments() (*ChannelCommitment, *ChannelCommitment, error) { +// +//nolint:lll +func (c *OpenChannel) LatestCommitments() fn.Result[*lntypes.Dual[ChannelCommitment]] { err := kvdb.View(c.Db.backend, func(tx kvdb.RTx) error { chanBucket, err := fetchChanBucket( tx, c.IdentityPub, &c.FundingOutpoint, c.ChainHash, @@ -4024,10 +4026,10 @@ func (c *OpenChannel) LatestCommitments() (*ChannelCommitment, *ChannelCommitmen return fetchChanCommitments(chanBucket, c) }, func() {}) if err != nil { - return nil, nil, err + return fn.Err[*lntypes.Dual[ChannelCommitment]](err) } - return &c.Commitments.Local, &c.Commitments.Remote, nil + return fn.Ok(&c.Commitments) } // RemoteRevocationStore returns the most up to date commitment version of the diff --git a/contractcourt/chain_watcher.go b/contractcourt/chain_watcher.go index c5c07d8051e..31edfe12d91 100644 --- a/contractcourt/chain_watcher.go +++ b/contractcourt/chain_watcher.go @@ -553,7 +553,7 @@ type chainSet struct { func newChainSet(chanState *channeldb.OpenChannel) (*chainSet, error) { // First, we'll grab the current unrevoked commitments for ourselves // and the remote party. - localCommit, remoteCommit, err := chanState.LatestCommitments() + commitments, err := chanState.LatestCommitments().Unpack() if err != nil { return nil, fmt.Errorf("unable to fetch channel state for "+ "chan_point=%v", chanState.FundingOutpoint) @@ -561,14 +561,14 @@ func newChainSet(chanState *channeldb.OpenChannel) (*chainSet, error) { log.Tracef("ChannelPoint(%v): local_commit_type=%v, local_commit=%v", chanState.FundingOutpoint, chanState.ChanType, - spew.Sdump(localCommit)) + spew.Sdump(commitments.Local)) log.Tracef("ChannelPoint(%v): remote_commit_type=%v, remote_commit=%v", chanState.FundingOutpoint, chanState.ChanType, - spew.Sdump(remoteCommit)) + spew.Sdump(commitments.Remote)) // Fetch the current known commit height for the remote party, and // their pending commitment chain tip if it exists. - remoteStateNum := remoteCommit.CommitHeight + remoteStateNum := commitments.Remote.CommitHeight remoteChainTip, err := chanState.RemoteCommitChainTip() if err != nil && err != channeldb.ErrNoPendingCommit { return nil, fmt.Errorf("unable to obtain chain tip for "+ @@ -581,8 +581,8 @@ func newChainSet(chanState *channeldb.OpenChannel) (*chainSet, error) { // duty. commitSet := CommitSet{ HtlcSets: map[HtlcSetKey][]channeldb.HTLC{ - LocalHtlcSet: localCommit.Htlcs, - RemoteHtlcSet: remoteCommit.Htlcs, + LocalHtlcSet: commitments.Local.Htlcs, + RemoteHtlcSet: commitments.Remote.Htlcs, }, } @@ -612,8 +612,8 @@ func newChainSet(chanState *channeldb.OpenChannel) (*chainSet, error) { return &chainSet{ remoteStateNum: remoteStateNum, commitSet: commitSet, - localCommit: *localCommit, - remoteCommit: *remoteCommit, + localCommit: commitments.Local, + remoteCommit: commitments.Remote, remotePendingCommit: remotePendingCommit, }, nil } From 5d4020ffaac00d8f0e85f6e45051ee13a598b6c2 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Wed, 21 Aug 2024 11:42:25 -0600 Subject: [PATCH 26/29] channeldb: add ChannelEpochs component for historical params --- channeldb/channel_epochs.go | 387 ++++++++++++++++++++++++++++++++++++ lntypes/channel_party.go | 24 +++ 2 files changed, 411 insertions(+) create mode 100644 channeldb/channel_epochs.go diff --git a/channeldb/channel_epochs.go b/channeldb/channel_epochs.go new file mode 100644 index 00000000000..8c35880bcf7 --- /dev/null +++ b/channeldb/channel_epochs.go @@ -0,0 +1,387 @@ +package channeldb + +import ( + "io" + + "github.com/btcsuite/btcd/btcutil" + "github.com/lightningnetwork/lnd/fn" + "github.com/lightningnetwork/lnd/lntypes" + "github.com/lightningnetwork/lnd/tlv" +) + +// CommitChainEpoch refers to a single period of time where a particular set +// of commitment parameters are in effect. +type CommitChainEpoch struct { + // LastHeight is the last commit height that marks the end of this + // epoch. + LastHeight uint64 + + // normalizedParams are the commitment parameters that affect the + // rendering of the commitment transaction. They are referred to as + // "normalized" because they indicate the parameters that actually + // apply to the party in question, irrespective of which party specified + // the parameter. + normalizedParams CommitmentParams +} + +// encode encodes a CommitChainEpoch to a writer. +func (c CommitChainEpoch) encode(w io.Writer) error { + return WriteElements( + w, c.LastHeight, c.normalizedParams.DustLimit, + c.normalizedParams.CsvDelay, + ) +} + +// decodeCommitChainEpoch decodes a CommitChainEpoch from a reader. +func decodeCommitChainEpoch(r io.Reader) (CommitChainEpoch, error) { + var lastHeight uint64 + var dustLimit btcutil.Amount + var csvDelay uint16 + + err := ReadElements(r, &lastHeight, &dustLimit, &csvDelay) + if err != nil { + return CommitChainEpoch{}, err + } + + return CommitChainEpoch{ + LastHeight: lastHeight, + normalizedParams: CommitmentParams{ + DustLimit: dustLimit, + CsvDelay: csvDelay, + }, + }, nil +} + +// CommitChainEpochHistory is a data structure designed to maintain the +// CommitmentParams history of both commitment chains. +type CommitChainEpochHistory struct { + // normalizedCurrent is the current commitment parameters that are + // in effect. They are separate from the history because we do not + // yet have the final heights that close these epochs out. + normalizedCurrent lntypes.Dual[CommitmentParams] + + // historical is a pair of lists of CommitChainEpochs that are sorted + // by LastHeight. + historical lntypes.Dual[[]CommitChainEpoch] +} + +// Size returns the size of the CommitChainEpochHistory in bytes. +func (c *CommitChainEpochHistory) Size() uint64 { + commitParamSize := uint64(2 * 8) // DustLimit + CsvDelay + epochSize := uint64(8) + commitParamSize + + currentSize := 2 * commitParamSize + localHistorySize := 2 + uint64(len(c.historical.Local))*epochSize + remoteHistorySize := 2 + uint64(len(c.historical.Remote))*epochSize + + return currentSize + localHistorySize + remoteHistorySize +} + +// encode encodes a CommitChainEpochHistory to a writer. +func (c *CommitChainEpochHistory) encode(w io.Writer) error { + // Write the normalized current params, always writing local before + // remote and dust limit before csv delay. + err := WriteElements( + w, c.normalizedCurrent.Local.DustLimit, + c.normalizedCurrent.Local.CsvDelay, + c.normalizedCurrent.Remote.DustLimit, + c.normalizedCurrent.Remote.CsvDelay, + ) + if err != nil { + return err + } + + // Write the length so we can handle deserialization. + err = WriteElement(w, uint16(len(c.historical.Local))) + if err != nil { + return err + } + + // Write the local epochs. + for _, epoch := range c.historical.Local { + err = epoch.encode(w) + if err != nil { + return err + } + } + + // Write the length so we can handle deserialization. + err = WriteElement(w, uint16(len(c.historical.Remote))) + if err != nil { + return err + } + + // Write the remote epochs. + for _, epoch := range c.historical.Remote { + err = epoch.encode(w) + if err != nil { + return err + } + } + + return nil +} + +// ECommitChainEpochHistory defines a tlv encoder for CommitChainEpochHistory. +func ECommitChainEpochHistory(w io.Writer, val interface{}, + buf *[8]byte) error { + + if hist, ok := val.(*CommitChainEpochHistory); ok { + return hist.encode(w) + } + + return tlv.NewTypeForEncodingErr(val, "*CommitChainEpochHistory") +} + +// decodeCommitChainEpochHistory decodes a CommitChainEpochHistory from a +// reader. +func decodeCommitChainEpochHistory(r io.Reader) (CommitChainEpochHistory, + error) { + + var normalizedCurrent lntypes.Dual[CommitmentParams] + err := ReadElements(r, &normalizedCurrent.Local.DustLimit, + &normalizedCurrent.Local.CsvDelay, + &normalizedCurrent.Remote.DustLimit, + &normalizedCurrent.Remote.CsvDelay, + ) + if err != nil { + return CommitChainEpochHistory{}, err + } + + historical := lntypes.Dual[[]CommitChainEpoch]{} + + var localEpochsLen uint16 + err = ReadElement(r, &localEpochsLen) + if err != nil { + return CommitChainEpochHistory{}, err + } + + if localEpochsLen > 0 { + historical.Local = make([]CommitChainEpoch, localEpochsLen) + for i := range historical.Local { + historical.Local[i], err = decodeCommitChainEpoch(r) + if err != nil { + return CommitChainEpochHistory{}, err + } + } + } + + var remoteEpochsLen uint16 + err = ReadElement(r, &remoteEpochsLen) + if err != nil { + return CommitChainEpochHistory{}, err + } + + if remoteEpochsLen > 0 { + historical.Remote = make([]CommitChainEpoch, remoteEpochsLen) + for i := range historical.Remote { + historical.Remote[i], err = decodeCommitChainEpoch(r) + if err != nil { + return CommitChainEpochHistory{}, err + } + } + } + + return CommitChainEpochHistory{ + normalizedCurrent: normalizedCurrent, + historical: historical, + }, nil +} + +// DCommitChainEpochHistory defines a tlv decoder for CommitChainEpochHistory. +func DCommitChainEpochHistory(r io.Reader, val interface{}, + buf *[8]byte, l uint64) error { + + if hist, ok := val.(*CommitChainEpochHistory); ok { + decoded, err := decodeCommitChainEpochHistory(r) + if err != nil { + return err + } + + *hist = decoded + + return nil + } + + return tlv.NewTypeForDecodingErr(val, "*CommitChainEpochHistory", l, l) +} + +// BeginChainEpochHistory initializes a new CommitChainEpochHistory with the +// original CommitmentParams specified in each party's ChannelConfig. +// +// NOTE: This function is only intended to be used during the funding workflow. +func BeginChainEpochHistory( + origCfgParams lntypes.Dual[CommitmentParams]) CommitChainEpochHistory { + + return CommitChainEpochHistory{ + normalizedCurrent: origCfgParams, + historical: lntypes.Dual[[]CommitChainEpoch]{}, + } +} + +// Record returns a TLV record that can be used to encode/decode a +// CommitChainEpochHistory to/from a TLV stream. +// +// NOTE: This is a part of the RecordProducer interface. +func (c *CommitChainEpochHistory) Record() tlv.Record { + return tlv.MakeDynamicRecord( + 0, c, c.Size, ECommitChainEpochHistory, + DCommitChainEpochHistory, + ) +} + +// Push allows a ChannelParty to change the CommitmentParams for the channel and +// mark the last commit heights for each party that the old CommitmentParams +// applied to. To use this function correctly you must call it with the +// understanding that the party making changes to its ChannelConfig will pass +// in the CommitmentParams from that config change unaltered. Finally the +// current commitment heights of both commit chains are passed in to mark the +// last height for each chain that the current channel epoch applies to. +func (c *CommitChainEpochHistory) Push(whoSpecified lntypes.ChannelParty, + params CommitmentParams, currentHeights lntypes.Dual[uint64]) { + + // Here we define a function that marks a set of normalized commitment + // parameters with the current commit height when the epoch concluded + // to create the CommitChainEpoch structure. + closeEpoch := func(normalizedCurrent CommitmentParams, + currentHeight uint64) CommitChainEpoch { + + return CommitChainEpoch{ + LastHeight: currentHeight, + normalizedParams: normalizedCurrent, + } + } + + // Using the function we just defined we now apply it to both the local + // and remote components of the current epoch which will define the last + // height that this set of normalized commitment parameters held for. + closed := lntypes.ZipWithDual( + c.normalizedCurrent, currentHeights, closeEpoch, + ) + + // Since go is unprincipled, we can't treat append as an actual function + // so we make a wrapper for our use case. + push := func(as []CommitChainEpoch, + a CommitChainEpoch) []CommitChainEpoch { + + return append(as, a) + } + + // We now take the closed epoch we just created and add it to the end of + // our history of channel epochs. + c.historical = lntypes.ZipWithDual( + c.historical, closed, push, + ) + + // Now we begin the task of assembling the new normalized current + // commitment parameters for both parties. + newCurrent := lntypes.Dual[CommitmentParams]{} + + // The party issuing the commitment parameter change is referred to as + // "main" here. It could be either the local or remote party but the + // point is that the new Csv for the main party will always be the same + // as the Csv from the last epoch, since a change to the Csv at the + // config level is an imposition on the other party's commitment + // transaction. However, the dust limit will be the dust limit set in + // the argument. + mainCsv := closed.GetForParty(whoSpecified).normalizedParams.CsvDelay + mainParams := CommitmentParams{ + DustLimit: params.DustLimit, + CsvDelay: mainCsv, + } + newCurrent.SetForParty(whoSpecified, mainParams) + + // The other party is referred to as counter here and the key here is + // that for the counterparty, their dust limit will remain the same as + // it was before, while their Csv will update since it is imposed by the + // main party. + counterParty := whoSpecified.CounterParty() + counterDustLimit := closed.GetForParty(counterParty). + normalizedParams.DustLimit + + counterParams := CommitmentParams{ + DustLimit: counterDustLimit, + CsvDelay: params.CsvDelay, + } + newCurrent.SetForParty(counterParty, counterParams) + + // Now that we have set the values appropriately for the newCurrent + // we set the normalizedCurrent values to be the newly computed current + // commitment values. + c.normalizedCurrent = newCurrent +} + +// NormalizedParamsAt queries the CommitChainEpochHistory for the normalized +// commitment parameters for the ChannelParty's commitment transaction at a +// given height. The parameters are referred to as "normalized" because they +// indicate the parameters that apply to that party's commitment transaction +// irrespective of which party is responsible for setting those parameters. +func (c *CommitChainEpochHistory) NormalizedParamsAt( + whoseCommit lntypes.ChannelParty, + height uint64) CommitmentParams { + + // Try to find the epoch that applies to the height. + histEpoch := search(c.historical.GetForParty(whoseCommit), height) + + // Extract just the portion of the epoch that specifies the parameters. + histParams := fn.MapOption(func(e CommitChainEpoch) CommitmentParams { + return e.normalizedParams + })(histEpoch) + + // If we didn't find it, then we use the current parameters. + curr := c.normalizedCurrent.GetForParty(whoseCommit) + + return histParams.UnwrapOr(curr) +} + +// search is responsible for finding the epoch that encloses the specified +// height. This means that the LastHeight of that epoch must be greater than +// or equal to the query height AND the previous epoch's LastHeight must be +// less than the query height. +func search(epochs []CommitChainEpoch, h uint64) fn.Option[CommitChainEpoch] { + // We implement a simple binary search here. + half := len(epochs) / 2 + + switch { + // We have a couple of edge cases here. If we somehow end up with an + // empty epoch history we are querying we return None. + case len(epochs) == 0: + return fn.None[CommitChainEpoch]() + + // If we have a single epoch in the history then that epoch is the + // correct one iff its LastHeight is greater than or equal to the + // query height. + case len(epochs) == 1: + if h <= epochs[0].LastHeight { + return fn.Some(epochs[0]) + } else { + return fn.None[CommitChainEpoch]() + } + + // Otherwise we begin our dividing of the slice. If our height falls + // between the LastHeight of the last epoch in the former half of the + // history and the LastHeight of the first epoch in the latter half of + // the history, then the first epoch of the latter half is the correct + // epoch. + case epochs[half-1].LastHeight < h && + h <= epochs[half].LastHeight: + + return fn.Some(epochs[half]) + + // Now that we've excluded the between case, our query height is in + // either half. If it's less than the LastHeight of the last epoch in + // the former half, then we will compute the search again on that half. + case h <= epochs[half-1].LastHeight: + return search(epochs[:half], h) + + // Otherwise it's in the latter half and so we will compute the search + // on that half. + case h > epochs[half].LastHeight: + return search(epochs[half:], h) + + // We should have excausted all cases, so this indicates something + // severely wrong with the algorithm and we choose to hard-eject. + default: + panic("non-exhaustive cases in commit epochs search") + } +} diff --git a/lntypes/channel_party.go b/lntypes/channel_party.go index 82cbd1045e7..ff9611855b5 100644 --- a/lntypes/channel_party.go +++ b/lntypes/channel_party.go @@ -109,6 +109,21 @@ func (d *Dual[A]) ModifyForParty(p ChannelParty, f func(A) A) A { } } +// RefForParty allows us to get a direct pointer access to the field that is +// associated with the given ChannelParty. +func (d *Dual[A]) RefForParty(p ChannelParty) *A { + switch p { + case Local: + return &d.Local + case Remote: + return &d.Remote + default: + panic(fmt.Sprintf( + "switch default triggered in ForParty: %v", p, + )) + } +} + // MapDual applies the function argument to both the Local and Remote fields of // the Dual[A] and returns a Dual[B] with that function applied. func MapDual[A, B any](d Dual[A], f func(A) B) Dual[B] { @@ -118,4 +133,13 @@ func MapDual[A, B any](d Dual[A], f func(A) B) Dual[B] { } } +// ZipWithDual allows us to combine two Duals into a single Dual using the +// provided function. +func ZipWithDual[A, B, C any](a Dual[A], b Dual[B], f func(A, B) C) Dual[C] { + return Dual[C]{ + Local: f(a.Local, b.Local), + Remote: f(a.Remote, b.Remote), + } +} + var BothParties []ChannelParty = []ChannelParty{Local, Remote} From dc3aef4321009819305c69efca7076cac14e49c0 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Wed, 4 Sep 2024 14:09:24 -0700 Subject: [PATCH 27/29] channeldb+lnwallet: add epoch history to OpenChannel This commit takes the CommitChainEpochHistory defined in the last commit and adds it to the OpenChannel structure. As of this commit it is essentially redundant with the ChanCfgs but it will capture the history of the ChanCfgs when we add the ability to update them. --- channeldb/channel.go | 45 +++++++++++++++++++++++++++++++++++++++ channeldb/channel_test.go | 6 ++++++ lnwallet/channel.go | 1 + lnwallet/test_utils.go | 12 +++++++++++ lnwallet/wallet.go | 20 +++++++++++++++++ 5 files changed, 84 insertions(+) diff --git a/channeldb/channel.go b/channeldb/channel.go index b85a8a52ac9..f8dc623b0f0 100644 --- a/channeldb/channel.go +++ b/channeldb/channel.go @@ -260,6 +260,10 @@ type openChannelTlvData struct { // customBlob is an optional TLV encoded blob of data representing // custom channel funding information. customBlob tlv.OptionalRecordT[tlv.TlvType7, tlv.Blob] + + // commitChainEpochHistory is the optional TLV encoded blob of data + // representing the commit chain epoch history for the channel. + commitChainEpochHistory tlv.OptionalRecordT[tlv.TlvType8, CommitChainEpochHistory] //nolint:lll } // encode serializes the openChannelTlvData to the given io.Writer. @@ -281,6 +285,11 @@ func (c *openChannelTlvData) encode(w io.Writer) error { c.customBlob.WhenSome(func(blob tlv.RecordT[tlv.TlvType7, tlv.Blob]) { tlvRecords = append(tlvRecords, blob.Record()) }) + c.commitChainEpochHistory.WhenSome( + func(hist tlv.RecordT[tlv.TlvType8, CommitChainEpochHistory]) { + tlvRecords = append(tlvRecords, hist.Record()) + }, + ) // Create the tlv stream. tlvStream, err := tlv.NewStream(tlvRecords...) @@ -296,6 +305,7 @@ func (c *openChannelTlvData) decode(r io.Reader) error { memo := c.memo.Zero() tapscriptRoot := c.tapscriptRoot.Zero() blob := c.customBlob.Zero() + commitChainEpochHistory := c.commitChainEpochHistory.Zero() // Create the tlv stream. tlvStream, err := tlv.NewStream( @@ -306,6 +316,7 @@ func (c *openChannelTlvData) decode(r io.Reader) error { memo.Record(), tapscriptRoot.Record(), blob.Record(), + commitChainEpochHistory.Record(), ) if err != nil { return err @@ -325,6 +336,10 @@ func (c *openChannelTlvData) decode(r io.Reader) error { if _, ok := tlvs[c.customBlob.TlvType()]; ok { c.customBlob = tlv.SomeRecordT(blob) } + if _, ok := tlvs[c.commitChainEpochHistory.TlvType()]; ok { + c.commitChainEpochHistory = + tlv.SomeRecordT(commitChainEpochHistory) + } return nil } @@ -947,6 +962,10 @@ type OpenChannel struct { // commitment. Commitments lntypes.Dual[ChannelCommitment] + // CommitChainEpochHistory is the history of the CommitmentParams for + // each side of the channel. + CommitChainEpochHistory CommitChainEpochHistory + // RemoteCurrentRevocation is the current revocation for their // commitment transaction. However, since this the derived public key, // we don't yet have the private key so we aren't yet able to verify @@ -1207,6 +1226,11 @@ func (c *OpenChannel) amendTlvData(auxData openChannelTlvData) { auxData.customBlob.WhenSomeV(func(blob tlv.Blob) { c.CustomBlob = fn.Some(blob) }) + auxData.commitChainEpochHistory.WhenSomeV( + func(history CommitChainEpochHistory) { + c.CommitChainEpochHistory = history + }, + ) } // extractTlvData creates a new openChannelTlvData from the given channel. @@ -1224,6 +1248,11 @@ func (c *OpenChannel) extractTlvData() openChannelTlvData { realScid: tlv.NewRecordT[tlv.TlvType4]( c.confirmedScid, ), + commitChainEpochHistory: tlv.SomeRecordT( + tlv.NewRecordT[tlv.TlvType8]( + c.CommitChainEpochHistory, + ), + ), } if len(c.Memo) != 0 { @@ -4508,6 +4537,22 @@ func fetchChanInfo(chanBucket kvdb.RBucket, channel *OpenChannel) error { // open channel. channel.amendTlvData(auxData) + // Now that we've extracted the aux data, we can initialize the + // CommitChainEpochHistory. If we don't find it in the aux data, + // then we initialize it with the original CommitmentParams from + // the ChannelConfig. + histVal := auxData.commitChainEpochHistory.ValOpt() + channel.CommitChainEpochHistory = histVal.UnwrapOr( + BeginChainEpochHistory( + lntypes.MapDual( + channel.ChanCfgs, + func(cfg ChannelConfig) CommitmentParams { + return cfg.CommitmentParams + }, + ), + ), + ) + channel.Packager = NewChannelPackager(channel.ShortChannelID) // Finally, read the optional shutdown scripts. diff --git a/channeldb/channel_test.go b/channeldb/channel_test.go index af17297902e..a5167e1ee46 100644 --- a/channeldb/channel_test.go +++ b/channeldb/channel_test.go @@ -343,6 +343,12 @@ func createTestChannelState(t *testing.T, cdb *ChannelStateDB) *OpenChannel { Local: localCfg, Remote: remoteCfg, }, + CommitChainEpochHistory: BeginChainEpochHistory( + lntypes.Dual[CommitmentParams]{ + Local: localRenderingParams, + Remote: remoteRenderingParams, + }, + ), TotalMSatSent: 8, TotalMSatReceived: 2, Commitments: lntypes.Dual[ChannelCommitment]{ diff --git a/lnwallet/channel.go b/lnwallet/channel.go index feb8716d865..65ce30d6079 100644 --- a/lnwallet/channel.go +++ b/lnwallet/channel.go @@ -7712,6 +7712,7 @@ func (lc *LightningChannel) ForceClose() (*LocalForceCloseSummary, error) { return summary, nil } + // NewLocalForceCloseSummary generates a LocalForceCloseSummary from the given // channel state. The passed commitTx must be a fully signed commitment // transaction corresponding to localCommit. diff --git a/lnwallet/test_utils.go b/lnwallet/test_utils.go index d125f454de8..6ccc8dc1c73 100644 --- a/lnwallet/test_utils.go +++ b/lnwallet/test_utils.go @@ -321,6 +321,12 @@ func CreateTestChannels(t *testing.T, chanType channeldb.ChannelType, Local: aliceCfg, Remote: bobCfg, }, + CommitChainEpochHistory: channeldb.BeginChainEpochHistory( + lntypes.Dual[channeldb.CommitmentParams]{ + Local: aliceCfg.CommitmentParams, + Remote: bobCfg.CommitmentParams, + }, + ), IdentityPub: aliceKeys[0].PubKey(), FundingOutpoint: *prevOut, ShortChannelID: shortChanID, @@ -343,6 +349,12 @@ func CreateTestChannels(t *testing.T, chanType channeldb.ChannelType, Local: bobCfg, Remote: aliceCfg, }, + CommitChainEpochHistory: channeldb.BeginChainEpochHistory( + lntypes.Dual[channeldb.CommitmentParams]{ + Local: bobCfg.CommitmentParams, + Remote: aliceCfg.CommitmentParams, + }, + ), IdentityPub: bobKeys[0].PubKey(), FundingOutpoint: *prevOut, ShortChannelID: shortChanID, diff --git a/lnwallet/wallet.go b/lnwallet/wallet.go index 898e11adfff..c9d9a527e18 100644 --- a/lnwallet/wallet.go +++ b/lnwallet/wallet.go @@ -2321,6 +2321,16 @@ func (l *LightningWallet) handleFundingCounterPartySigs(msg *addCounterPartySigs // he stored within the database. res.partialState.ChanCfgs.Local = res.ourContribution.toChanConfig() res.partialState.ChanCfgs.Remote = res.theirContribution.toChanConfig() + res.partialState.CommitChainEpochHistory = + channeldb.BeginChainEpochHistory( + lntypes.MapDual( + res.partialState.ChanCfgs, + //nolint:lll + func(cfg channeldb.ChannelConfig) channeldb.CommitmentParams { + return cfg.CommitmentParams + }, + ), + ) // We'll also record the finalized funding txn, which will allow us to // rebroadcast on startup in case we fail. @@ -2532,6 +2542,16 @@ func (l *LightningWallet) handleSingleFunderSigs(req *addSingleFunderSigsMsg) { pendingReservation.ourContribution.toChanConfig() chanState.ChanCfgs.Remote = pendingReservation.theirContribution.toChanConfig() + chanState.CommitChainEpochHistory = + channeldb.BeginChainEpochHistory( + lntypes.MapDual( + chanState.ChanCfgs, + //nolint:lll + func(cfg channeldb.ChannelConfig) channeldb.CommitmentParams { + return cfg.CommitmentParams + }, + ), + ) chanState.RevocationKeyLocator = pendingReservation.nextRevocationKeyLoc From 6678f0845a8ba27ecad32d11585df99eaca7b4c5 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Wed, 4 Sep 2024 16:16:53 -0700 Subject: [PATCH 28/29] lnwallet: use CommitChainEpochHistory to determine CsvDelay during Breach This commit changes the way we create breach retributions to use the CsvDelay we compute from the CommitChainEpochHistory so as to account for the possibility that the channel parameters have changed since opening. --- lnwallet/channel.go | 31 ++++++++++++++++++++----------- lnwallet/channel_test.go | 6 +++--- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/lnwallet/channel.go b/lnwallet/channel.go index 65ce30d6079..502fd7b2324 100644 --- a/lnwallet/channel.go +++ b/lnwallet/channel.go @@ -2102,7 +2102,12 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, return l.LocalAuxLeaf }, )(auxResult.AuxLeaves) - theirDelay := uint32(chanState.ChanCfgs.Remote.CsvDelay) + theirDelay := uint32( + chanState.CommitChainEpochHistory.NormalizedParamsAt( + lntypes.Remote, stateNum, + ).CsvDelay, + ) + theirScript, err := CommitScriptToSelf( chanState.ChanType, isRemoteInitiator, keyRing.ToLocalKey, keyRing.RevocationKey, theirDelay, leaseExpiry, localAuxLeaf, @@ -2122,7 +2127,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, // we need. if revokedLog != nil { br, ourAmt, theirAmt, err = createBreachRetribution( - revokedLog, spendTx, chanState, keyRing, + revokedLog, spendTx, chanState, stateNum, keyRing, commitmentSecret, leaseExpiry, auxResult.AuxLeaves, ) if err != nil { @@ -2137,8 +2142,8 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, // data can still function. This branch can be deleted once we // are confident that no legacy format is in use. br, ourAmt, theirAmt, err = createBreachRetributionLegacy( - revokedLogLegacy, chanState, keyRing, commitmentSecret, - ourScript, theirScript, leaseExpiry, + revokedLogLegacy, chanState, stateNum, keyRing, + commitmentSecret, ourScript, theirScript, leaseExpiry, ) if err != nil { return nil, err @@ -2317,7 +2322,7 @@ func NewBreachRetribution(chanState *channeldb.OpenChannel, stateNum uint64, // createHtlcRetribution is a helper function to construct an HtlcRetribution // based on the passed params. -func createHtlcRetribution(chanState *channeldb.OpenChannel, +func createHtlcRetribution(chanState *channeldb.OpenChannel, stateNum uint64, keyRing *CommitmentKeyRing, commitHash chainhash.Hash, commitmentSecret *btcec.PrivateKey, leaseExpiry uint32, htlc *channeldb.HTLCEntry, @@ -2325,7 +2330,11 @@ func createHtlcRetribution(chanState *channeldb.OpenChannel, var emptyRetribution HtlcRetribution - theirDelay := uint32(chanState.ChanCfgs.Remote.CsvDelay) + theirDelay := uint32( + chanState.CommitChainEpochHistory.NormalizedParamsAt( + lntypes.Remote, stateNum, + ).CsvDelay, + ) isRemoteInitiator := !chanState.IsInitiator // We'll generate the original second level witness script now, as @@ -2438,7 +2447,7 @@ func createHtlcRetribution(chanState *channeldb.OpenChannel, // see if these fields are present there. If they are not, then // ErrRevLogDataMissing is returned. func createBreachRetribution(revokedLog *channeldb.RevocationLog, - spendTx *wire.MsgTx, chanState *channeldb.OpenChannel, + spendTx *wire.MsgTx, chanState *channeldb.OpenChannel, stateNum uint64, keyRing *CommitmentKeyRing, commitmentSecret *btcec.PrivateKey, leaseExpiry uint32, auxLeaves fn.Option[CommitAuxLeaves]) (*BreachRetribution, int64, int64, @@ -2450,7 +2459,7 @@ func createBreachRetribution(revokedLog *channeldb.RevocationLog, htlcRetributions := make([]HtlcRetribution, len(revokedLog.HTLCEntries)) for i, htlc := range revokedLog.HTLCEntries { hr, err := createHtlcRetribution( - chanState, keyRing, commitHash.Val, + chanState, stateNum, keyRing, commitHash.Val, commitmentSecret, leaseExpiry, htlc, auxLeaves, ) if err != nil { @@ -2554,8 +2563,8 @@ func createBreachRetribution(revokedLog *channeldb.RevocationLog, // BreachRetribution using a ChannelCommitment. Returns the constructed // retribution, our amount, their amount, and a possible non-nil error. func createBreachRetributionLegacy(revokedLog *channeldb.ChannelCommitment, - chanState *channeldb.OpenChannel, keyRing *CommitmentKeyRing, - commitmentSecret *btcec.PrivateKey, + chanState *channeldb.OpenChannel, stateNum uint64, + keyRing *CommitmentKeyRing, commitmentSecret *btcec.PrivateKey, ourScript, theirScript input.ScriptDescriptor, leaseExpiry uint32) (*BreachRetribution, int64, int64, error) { @@ -2601,7 +2610,7 @@ func createBreachRetributionLegacy(revokedLog *channeldb.ChannelCommitment, } hr, err := createHtlcRetribution( - chanState, keyRing, commitHash, + chanState, stateNum, keyRing, commitHash, commitmentSecret, leaseExpiry, entry, fn.None[CommitAuxLeaves](), ) diff --git a/lnwallet/channel_test.go b/lnwallet/channel_test.go index dd5897a96e5..b42adcfa1d8 100644 --- a/lnwallet/channel_test.go +++ b/lnwallet/channel_test.go @@ -9807,7 +9807,7 @@ func TestCreateHtlcRetribution(t *testing.T) { // Create the htlc retribution. hr, err := createHtlcRetribution( - aliceChannel.channelState, keyRing, commitHash, + aliceChannel.channelState, 0,keyRing, commitHash, dummyPrivate, leaseExpiry, htlc, fn.None[CommitAuxLeaves](), ) // Expect no error. @@ -10012,7 +10012,7 @@ func TestCreateBreachRetribution(t *testing.T) { br, our, their, err := createBreachRetribution( tc.revocationLog, tx, - aliceChannel.channelState, keyRing, + aliceChannel.channelState, 0, keyRing, dummyPrivate, leaseExpiry, fn.None[CommitAuxLeaves](), ) @@ -10071,7 +10071,7 @@ func TestCreateBreachRetributionLegacy(t *testing.T) { // Create the breach retribution using the legacy format. br, ourAmt, theirAmt, err := createBreachRetributionLegacy( - &revokedLog, aliceChannel.channelState, keyRing, + &revokedLog, aliceChannel.channelState, 0, keyRing, dummyPrivate, ourScript, theirScript, leaseExpiry, ) require.NoError(t, err) From 72f6fe8cfa15aee2ad93dc01307c02dae8ce3ae1 Mon Sep 17 00:00:00 2001 From: Keagan McClelland Date: Wed, 17 Apr 2024 16:04:45 -0600 Subject: [PATCH 29/29] htlcswitch: create dynamic commit negotiation state machine api --- htlcswitch/dyncomm_negotiator.go | 83 ++++++++++++++++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 htlcswitch/dyncomm_negotiator.go diff --git a/htlcswitch/dyncomm_negotiator.go b/htlcswitch/dyncomm_negotiator.go new file mode 100644 index 00000000000..41446b2b3f3 --- /dev/null +++ b/htlcswitch/dyncomm_negotiator.go @@ -0,0 +1,83 @@ +package htlcswitch + +import ( + "github.com/btcsuite/btcd/btcec/v2" + "github.com/btcsuite/btcd/btcutil" + "github.com/lightningnetwork/lnd/fn" + "github.com/lightningnetwork/lnd/lnwire" +) + +type DynUpdate func(*lnwire.DynPropose) + +func WithDustLimit(amt btcutil.Amount) DynUpdate { + return func(dp *lnwire.DynPropose) { + dp.DustLimit = fn.Some(amt) + } +} + +func WithMaxValueInFlight(amt lnwire.MilliSatoshi) DynUpdate { + return func(dp *lnwire.DynPropose) { + dp.MaxValueInFlight = fn.Some(amt) + } +} + +func WithChannelReserve(amt btcutil.Amount) DynUpdate { + return func(dp *lnwire.DynPropose) { + dp.ChannelReserve = fn.Some(amt) + } +} + +func WithCsvDelay(blks uint16) DynUpdate { + return func(dp *lnwire.DynPropose) { + dp.CsvDelay = fn.Some(blks) + } +} + +func WithMaxAcceptedHTLCs(max uint16) DynUpdate { + return func(dp *lnwire.DynPropose) { + dp.MaxAcceptedHTLCs = fn.Some(max) + } +} + +func WithFundingKey(key btcec.PublicKey) DynUpdate { + return func(dp *lnwire.DynPropose) { + dp.FundingKey = fn.Some(key) + } +} + +func WithChannelType(ty lnwire.ChannelType) DynUpdate { + + return func(dp *lnwire.DynPropose) { + dp.ChannelType = fn.Some(ty) + } +} + +type dyncommNegotiator struct {} + +type DynResponse = fn.Either[lnwire.DynAck, lnwire.DynReject] + +func (*dyncommNegotiator) recvDynPropose(msg lnwire.DynPropose) ( + DynResponse, error) { + + panic("NOT IMPLEMENTED: dyncommNegotiator.recvDynPropose") +} + +func (*dyncommNegotiator) recvDynAck(msg lnwire.DynAck) (lnwire.DynPropose, + error) { + + panic("NOT IMPLEMENTED: dyncommNegotiator.recvDynAck") +} + +func (*dyncommNegotiator) recvDynReject(msg lnwire.DynReject) error { + panic("NOT IMPLEMENTED: dyncommNegotiator.recvDynReject") +} + +func (*dyncommNegotiator) sendDynPropose( + updates ...DynUpdate) lnwire.DynPropose { + + panic("NOT IMPLEMENTED: dyncommNegotiator.sendDynPropose") +} + +func (*dyncommNegotiator) reset() { + panic("NOT IMPLEMENTED: dyncommNegotiator.reset") +} \ No newline at end of file