From c418034d87b1fe1c53a75579597399d0a4b42853 Mon Sep 17 00:00:00 2001 From: Wilmer Paulino Date: Fri, 17 Oct 2025 11:19:46 -0700 Subject: [PATCH 1/8] Remove previous holder HTLC data on splice locked when necessary If while a splice is pending, the channel happens to not have any commitment updates, but did prior to the splice being negotiated, it's possible that we end up with bogus holder HTLC data for the previous commitment. After the splice becomes locked, we've successfully transitioned to the new funding transaction, but that funding transaction never had a commitment transaction negotiated for the previous state. --- lightning/src/chain/channelmonitor.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/lightning/src/chain/channelmonitor.rs b/lightning/src/chain/channelmonitor.rs index 5c26fd01738..0cf208a574b 100644 --- a/lightning/src/chain/channelmonitor.rs +++ b/lightning/src/chain/channelmonitor.rs @@ -4118,6 +4118,14 @@ impl ChannelMonitorImpl { self.funding.prev_holder_commitment_tx.clone(), ); + // It's possible that no commitment updates happened during the lifecycle of the pending + // splice's `FundingScope` that was promoted. If so, our `prev_holder_htlc_data` is + // now irrelevant, since there's no valid previous commitment that exists for the current + // funding transaction that could be broadcast. + if self.funding.prev_holder_commitment_tx.is_none() { + self.prev_holder_htlc_data.take(); + } + let no_further_updates_allowed = self.no_further_updates_allowed(); // The swap above places the previous `FundingScope` into `pending_funding`. From f2ada1a1be9a1b61538c2a6f0851b69389cecc40 Mon Sep 17 00:00:00 2001 From: Wilmer Paulino Date: Fri, 17 Oct 2025 11:19:47 -0700 Subject: [PATCH 2/8] Fix legacy SCID pruning We relied on `position` giving us the last index we need to prune, but this may return `None` when all known legacy SCIDs need to be pruned. In such cases, we ended up not pruning any of the legacy SCIDs at all. Rewritten by: Matt Corallo --- lightning/src/ln/channel.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/lightning/src/ln/channel.rs b/lightning/src/ln/channel.rs index 8f74dc24e71..c342415e1ae 100644 --- a/lightning/src/ln/channel.rs +++ b/lightning/src/ln/channel.rs @@ -13233,18 +13233,18 @@ where let end = self .funding .get_short_channel_id() - .and_then(|current_scid| { + .map(|current_scid| { let historical_scids = &self.context.historical_scids; historical_scids .iter() .zip(historical_scids.iter().skip(1).chain(core::iter::once(¤t_scid))) - .map(|(_, next_scid)| { - let funding_height = block_from_scid(*next_scid); - let retain_scid = - funding_height + CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY - 1 > height; - retain_scid + .filter(|(_, next_scid)| { + let funding_height = block_from_scid(**next_scid); + let drop_scid = + funding_height + CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY - 1 <= height; + drop_scid }) - .position(|retain_scid| retain_scid) + .count() }) .unwrap_or(0); From b84ad655e3efdbede3899dd64d8a3adad2ba1fad Mon Sep 17 00:00:00 2001 From: Wilmer Paulino Date: Fri, 17 Oct 2025 11:19:47 -0700 Subject: [PATCH 3/8] Test inflight HTLC forward and resolution after locked splice Test tweaked by: Matt Corallo --- lightning/src/ln/channel.rs | 2 +- lightning/src/ln/functional_test_utils.rs | 9 ++ lightning/src/ln/splicing_tests.rs | 151 +++++++++++++++++++++- 3 files changed, 160 insertions(+), 2 deletions(-) diff --git a/lightning/src/ln/channel.rs b/lightning/src/ln/channel.rs index c342415e1ae..6419d8cbdad 100644 --- a/lightning/src/ln/channel.rs +++ b/lightning/src/ln/channel.rs @@ -1373,7 +1373,7 @@ pub(crate) const COINBASE_MATURITY: u32 = 100; /// The number of blocks to wait for a channel_announcement to propagate such that payments using an /// older SCID can still be relayed. Once the spend of the previous funding transaction has reached /// this number of confirmations, the corresponding SCID will be forgotten. -const CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY: u32 = 144; +pub(crate) const CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY: u32 = 144; struct PendingChannelMonitorUpdate { update: ChannelMonitorUpdate, diff --git a/lightning/src/ln/functional_test_utils.rs b/lightning/src/ln/functional_test_utils.rs index d8e59dde166..6ce9ca8ea3b 100644 --- a/lightning/src/ln/functional_test_utils.rs +++ b/lightning/src/ln/functional_test_utils.rs @@ -2494,6 +2494,15 @@ pub fn expect_and_process_pending_htlcs(node: &Node<'_, '_, '_>, process_twice: assert!(!node.node.needs_pending_htlc_processing()); } +/// Processes an HTLC which is pending forward but will fail to forward when we process it here. +pub fn expect_htlc_forwarding_fails( + node: &Node<'_, '_, '_>, expected_failure: &[HTLCHandlingFailureType], +) { + expect_and_process_pending_htlcs(node, false); + let events = node.node.get_and_clear_pending_events(); + expect_htlc_failure_conditions(events, expected_failure); +} + #[macro_export] /// Performs the "commitment signed dance" - the series of message exchanges which occur after a /// commitment update. diff --git a/lightning/src/ln/splicing_tests.rs b/lightning/src/ln/splicing_tests.rs index db34969074b..2b415a1402a 100644 --- a/lightning/src/ln/splicing_tests.rs +++ b/lightning/src/ln/splicing_tests.rs @@ -14,11 +14,13 @@ use crate::chain::ChannelMonitorUpdateStatus; use crate::events::bump_transaction::sync::WalletSourceSync; use crate::events::{ClosureReason, Event, FundingInfo, HTLCHandlingFailureType}; use crate::ln::chan_utils; -use crate::ln::channelmanager::BREAKDOWN_TIMEOUT; +use crate::ln::channel::CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY; +use crate::ln::channelmanager::{PaymentId, RecipientOnionFields, BREAKDOWN_TIMEOUT}; use crate::ln::functional_test_utils::*; use crate::ln::funding::{FundingTxInput, SpliceContribution}; use crate::ln::msgs::{self, BaseMessageHandler, ChannelMessageHandler, MessageSendEvent}; use crate::ln::types::ChannelId; +use crate::routing::router::{PaymentParameters, RouteParameters}; use crate::util::errors::APIError; use crate::util::ser::Writeable; use crate::util::test_channel_signer::SignerOp; @@ -1801,3 +1803,150 @@ fn fail_quiescent_action_on_channel_close() { check_closed_broadcast(&nodes[0], 1, true); check_added_monitors(&nodes[0], 1); } + +fn do_test_splice_with_inflight_htlc_forward_and_resolution(expire_scid_pre_forward: bool) { + // Test that we are still able to forward and resolve HTLCs while the original SCIDs contained + // in the onion packets have now changed due channel splices becoming locked. + let chanmon_cfgs = create_chanmon_cfgs(3); + let node_cfgs = create_node_cfgs(3, &chanmon_cfgs); + let mut config = test_default_channel_config(); + config.channel_config.cltv_expiry_delta = CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY as u16 * 2; + let node_chanmgrs = create_node_chanmgrs( + 3, + &node_cfgs, + &[Some(config.clone()), Some(config.clone()), Some(config)], + ); + let nodes = create_network(3, &node_cfgs, &node_chanmgrs); + + let node_id_0 = nodes[0].node.get_our_node_id(); + let node_id_1 = nodes[1].node.get_our_node_id(); + let node_id_2 = nodes[2].node.get_our_node_id(); + + let (_, _, channel_id_0_1, _) = create_announced_chan_between_nodes(&nodes, 0, 1); + let (chan_upd_1_2, _, channel_id_1_2, _) = create_announced_chan_between_nodes(&nodes, 1, 2); + + let node_max_height = + nodes.iter().map(|node| node.blocks.lock().unwrap().len()).max().unwrap() as u32; + connect_blocks(&nodes[0], node_max_height - nodes[0].best_block_info().1); + connect_blocks(&nodes[1], node_max_height - nodes[1].best_block_info().1); + connect_blocks(&nodes[2], node_max_height - nodes[2].best_block_info().1); + + // Send an outbound HTLC from node 0 to 2. + let payment_amount = 1_000_000; + let payment_params = + PaymentParameters::from_node_id(node_id_2, CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY * 2) + .with_bolt11_features(nodes[2].node.bolt11_invoice_features()) + .unwrap(); + let route_params = + RouteParameters::from_payment_params_and_value(payment_params, payment_amount); + let route = get_route(&nodes[0], &route_params).unwrap(); + let (_, payment_hash, payment_secret) = + get_payment_preimage_hash(&nodes[2], Some(payment_amount), None); + let onion = RecipientOnionFields::secret_only(payment_secret); + let id = PaymentId(payment_hash.0); + nodes[0].node.send_payment_with_route(route.clone(), payment_hash, onion, id).unwrap(); + check_added_monitors(&nodes[0], 1); + + // Node 1 should now have a pending HTLC to forward to 2. + let update_add_0_1 = get_htlc_update_msgs(&nodes[0], &node_id_1); + nodes[1].node.handle_update_add_htlc(node_id_0, &update_add_0_1.update_add_htlcs[0]); + commitment_signed_dance!(nodes[1], nodes[0], update_add_0_1.commitment_signed, false); + assert!(nodes[1].node.needs_pending_htlc_processing()); + + // Splice both channels, lock them, and connect enough blocks to trigger the legacy SCID pruning + // logic while the HTLC is still pending. + let contribution = SpliceContribution::SpliceOut { + outputs: vec![TxOut { + value: Amount::from_sat(1_000), + script_pubkey: nodes[0].wallet_source.get_change_script().unwrap(), + }], + }; + let splice_tx_0_1 = splice_channel(&nodes[0], &nodes[1], channel_id_0_1, contribution); + for node in &nodes { + mine_transaction(node, &splice_tx_0_1); + } + + let contribution = SpliceContribution::SpliceOut { + outputs: vec![TxOut { + value: Amount::from_sat(1_000), + script_pubkey: nodes[1].wallet_source.get_change_script().unwrap(), + }], + }; + let splice_tx_1_2 = splice_channel(&nodes[1], &nodes[2], channel_id_1_2, contribution); + for node in &nodes { + mine_transaction(node, &splice_tx_1_2); + } + + for node in &nodes { + connect_blocks(node, ANTI_REORG_DELAY - 2); + } + let splice_locked = get_event_msg!(nodes[0], MessageSendEvent::SendSpliceLocked, node_id_1); + lock_splice(&nodes[0], &nodes[1], &splice_locked, false); + + for node in &nodes { + connect_blocks(node, 1); + } + let splice_locked = get_event_msg!(nodes[1], MessageSendEvent::SendSpliceLocked, node_id_2); + lock_splice(&nodes[1], &nodes[2], &splice_locked, false); + + if expire_scid_pre_forward { + for node in &nodes { + connect_blocks(node, CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY); + } + + // Now attempt to forward the HTLC from node 1 to 2 which will fail because the SCID is no + // longer stored and has expired. Obviously this is somewhat of an absurd case - not + // forwarding for `CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY` blocks is kinda nuts. + let fail_type = HTLCHandlingFailureType::InvalidForward { + requested_forward_scid: chan_upd_1_2.contents.short_channel_id, + }; + expect_htlc_forwarding_fails(&nodes[1], &[fail_type]); + check_added_monitors(&nodes[1], 1); + let update_fail_1_0 = get_htlc_update_msgs(&nodes[1], &node_id_0); + nodes[0].node.handle_update_fail_htlc(node_id_1, &update_fail_1_0.update_fail_htlcs[0]); + commitment_signed_dance!(nodes[0], nodes[1], update_fail_1_0.commitment_signed, false); + + let conditions = PaymentFailedConditions::new(); + expect_payment_failed_conditions(&nodes[0], payment_hash, false, conditions); + } else { + // Now attempt to forward the HTLC from node 1 to 2. + nodes[1].node.process_pending_htlc_forwards(); + check_added_monitors(&nodes[1], 1); + let update_add_1_2 = get_htlc_update_msgs(&nodes[1], &node_id_2); + nodes[2].node.handle_update_add_htlc(node_id_1, &update_add_1_2.update_add_htlcs[0]); + commitment_signed_dance!(nodes[2], nodes[1], update_add_1_2.commitment_signed, false); + assert!(nodes[2].node.needs_pending_htlc_processing()); + + // Node 2 should see the claimable payment. Fail it back to make sure we also handle the SCID + // change on the way back. + nodes[2].node.process_pending_htlc_forwards(); + expect_payment_claimable!(&nodes[2], payment_hash, payment_secret, payment_amount); + nodes[2].node.fail_htlc_backwards(&payment_hash); + let fail_type = HTLCHandlingFailureType::Receive { payment_hash }; + expect_and_process_pending_htlcs_and_htlc_handling_failed(&nodes[2], &[fail_type]); + check_added_monitors(&nodes[2], 1); + + let update_fail_1_2 = get_htlc_update_msgs(&nodes[2], &node_id_1); + nodes[1].node.handle_update_fail_htlc(node_id_2, &update_fail_1_2.update_fail_htlcs[0]); + commitment_signed_dance!(nodes[1], nodes[2], update_fail_1_2.commitment_signed, false); + let fail_type = HTLCHandlingFailureType::Forward { + node_id: Some(node_id_2), + channel_id: channel_id_1_2, + }; + expect_and_process_pending_htlcs_and_htlc_handling_failed(&nodes[1], &[fail_type]); + check_added_monitors(&nodes[1], 1); + + let update_fail_0_1 = get_htlc_update_msgs(&nodes[1], &node_id_0); + nodes[0].node.handle_update_fail_htlc(node_id_1, &update_fail_0_1.update_fail_htlcs[0]); + commitment_signed_dance!(nodes[0], nodes[1], update_fail_0_1.commitment_signed, false); + + let conditions = PaymentFailedConditions::new(); + expect_payment_failed_conditions(&nodes[0], payment_hash, true, conditions); + } +} + +#[test] +fn test_splice_with_inflight_htlc_forward_and_resolution() { + do_test_splice_with_inflight_htlc_forward_and_resolution(true); + do_test_splice_with_inflight_htlc_forward_and_resolution(false); +} From e95ebf8b9a3d43108176e21c8b4c6bd82f3aaabf Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 21 Oct 2025 13:39:09 +0000 Subject: [PATCH 4/8] Properly handle funding key rotation during splices When splicing, we're required by protocol to retain all the existing keys material except the funding key which we're allowed to rotate. In the original implementation we acknowledged that but figured we'd stick with a single `pubkey` method in the `ChannelSigner` anyway cause adding a specific method for it is annoying. Sadly, this was ultimately broken - in `FundingScope::for_splice`, we called the signer's `new_pubkeys` method (renamed from `pubkeys` after splicing initially landed), replacing all of the public keys the `Channel` would use rather than just the funding key. This can result in commitment signature mismatches if the signer changes any keys aside from the funding one. `InMemorySigner` did not do so, however, so we didn't notice the bug. Luckily-ish, in 189b8ac4a7674bbf623f903dcd144c9d1a24a128 we started generating a fresh `remote_key` when splicing (at least when upgrading from 0.1 to 0.2 or when setting `KeysManager` to use v1 `remote_key` derivation). This breaks splicing cause we can't communicate the new `remote_key` to the counterparty during the splicing handshake. Ultimately this bug is because the API we had didn't communicate to the signer that we weren't allowed to change anything except the funding key, and allowed returning a `ChannelPublicKeys` which would break the channel. Here we fix this by renaming `new_pubkeys` `pubkeys` again (partially reverting 9d291e01f98417c2f6b2d4321bbf806464c424a6 but keeping the changed requirements that `pubkeys` only be called once) and adding a new `ChannelSigner:new_funding_pubkey` method specifically for splicing. We also update `channel.rs` to correctly fetch the new funding pubkey before sending `splice_init`, storing it in the `PendingFunding` untl we build a `FundingScope`. --- lightning/src/chain/channelmonitor.rs | 4 +- lightning/src/chain/onchaintx.rs | 2 +- lightning/src/ln/chan_utils.rs | 8 +- lightning/src/ln/channel.rs | 103 ++++++++++++++-------- lightning/src/sign/mod.rs | 52 ++++++----- lightning/src/util/dyn_signer.rs | 7 +- lightning/src/util/test_channel_signer.rs | 12 ++- 7 files changed, 112 insertions(+), 76 deletions(-) diff --git a/lightning/src/chain/channelmonitor.rs b/lightning/src/chain/channelmonitor.rs index 0cf208a574b..e2f0effed80 100644 --- a/lightning/src/chain/channelmonitor.rs +++ b/lightning/src/chain/channelmonitor.rs @@ -7009,7 +7009,7 @@ mod tests { let funding_outpoint = OutPoint { txid: Txid::all_zeros(), index: u16::MAX }; let channel_id = ChannelId::v1_from_funding_outpoint(funding_outpoint); let channel_parameters = ChannelTransactionParameters { - holder_pubkeys: keys.new_pubkeys(None, &secp_ctx), + holder_pubkeys: keys.pubkeys(&secp_ctx), holder_selected_contest_delay: 66, is_outbound_from_holder: true, counterparty_parameters: Some(CounterpartyChannelTransactionParameters { @@ -7272,7 +7272,7 @@ mod tests { let funding_outpoint = OutPoint { txid: Txid::all_zeros(), index: u16::MAX }; let channel_id = ChannelId::v1_from_funding_outpoint(funding_outpoint); let channel_parameters = ChannelTransactionParameters { - holder_pubkeys: keys.new_pubkeys(None, &secp_ctx), + holder_pubkeys: keys.pubkeys(&secp_ctx), holder_selected_contest_delay: 66, is_outbound_from_holder: true, counterparty_parameters: Some(CounterpartyChannelTransactionParameters { diff --git a/lightning/src/chain/onchaintx.rs b/lightning/src/chain/onchaintx.rs index 48bd40a8347..4f38c96825d 100644 --- a/lightning/src/chain/onchaintx.rs +++ b/lightning/src/chain/onchaintx.rs @@ -1339,7 +1339,7 @@ mod tests { // Use non-anchor channels so that HTLC-Timeouts are broadcast immediately instead of sent // to the user for external funding. let chan_params = ChannelTransactionParameters { - holder_pubkeys: signer.new_pubkeys(None, &secp_ctx), + holder_pubkeys: signer.pubkeys(&secp_ctx), holder_selected_contest_delay: 66, is_outbound_from_holder: true, counterparty_parameters: Some(CounterpartyChannelTransactionParameters { diff --git a/lightning/src/ln/chan_utils.rs b/lightning/src/ln/chan_utils.rs index 72cb34657fb..eeb785e3be9 100644 --- a/lightning/src/ln/chan_utils.rs +++ b/lightning/src/ln/chan_utils.rs @@ -1018,11 +1018,11 @@ pub struct ChannelTransactionParameters { /// If a channel was funded with transaction A, and later spliced with transaction B, this field /// tracks the txid of transaction A. /// - /// See [`compute_funding_key_tweak`] and [`ChannelSigner::new_pubkeys`] for more context on how + /// See [`compute_funding_key_tweak`] and [`ChannelSigner::pubkeys`] for more context on how /// this may be used. /// /// [`compute_funding_key_tweak`]: crate::sign::compute_funding_key_tweak - /// [`ChannelSigner::new_pubkeys`]: crate::sign::ChannelSigner::new_pubkeys + /// [`ChannelSigner::pubkeys`]: crate::sign::ChannelSigner::pubkeys pub splice_parent_funding_txid: Option, /// This channel's type, as negotiated during channel open. For old objects where this field /// wasn't serialized, it will default to static_remote_key at deserialization. @@ -2245,8 +2245,8 @@ mod tests { let counterparty_signer = keys_provider.derive_channel_signer(keys_provider.generate_channel_keys_id(true, 1)); let per_commitment_secret = SecretKey::from_slice(&>::from_hex("1f1e1d1c1b1a191817161514131211100f0e0d0c0b0a09080706050403020100").unwrap()[..]).unwrap(); let per_commitment_point = PublicKey::from_secret_key(&secp_ctx, &per_commitment_secret); - let holder_pubkeys = signer.new_pubkeys(None, &secp_ctx); - let counterparty_pubkeys = counterparty_signer.new_pubkeys(None, &secp_ctx).clone(); + let holder_pubkeys = signer.pubkeys(&secp_ctx); + let counterparty_pubkeys = counterparty_signer.pubkeys(&secp_ctx).clone(); let channel_parameters = ChannelTransactionParameters { holder_pubkeys: holder_pubkeys.clone(), holder_selected_contest_delay: 0, diff --git a/lightning/src/ln/channel.rs b/lightning/src/ln/channel.rs index 6419d8cbdad..c50d0d4e121 100644 --- a/lightning/src/ln/channel.rs +++ b/lightning/src/ln/channel.rs @@ -2470,6 +2470,7 @@ impl FundingScope { fn for_splice( prev_funding: &Self, context: &ChannelContext, our_funding_contribution: SignedAmount, their_funding_contribution: SignedAmount, counterparty_funding_pubkey: PublicKey, + our_new_holder_keys: ChannelPublicKeys, ) -> Self where SP::Target: SignerProvider, @@ -2489,19 +2490,15 @@ impl FundingScope { debug_assert!(post_value_to_self_msat.is_some()); let post_value_to_self_msat = post_value_to_self_msat.unwrap(); - // Rotate the pubkeys using the prev_funding_txid as a tweak - let prev_funding_txid = prev_funding.get_funding_txid(); - let holder_pubkeys = context.new_holder_pubkeys(prev_funding_txid); - let channel_parameters = &prev_funding.channel_transaction_parameters; let mut post_channel_transaction_parameters = ChannelTransactionParameters { - holder_pubkeys, + holder_pubkeys: our_new_holder_keys, holder_selected_contest_delay: channel_parameters.holder_selected_contest_delay, // The 'outbound' attribute doesn't change, even if the splice initiator is the other node is_outbound_from_holder: channel_parameters.is_outbound_from_holder, counterparty_parameters: channel_parameters.counterparty_parameters.clone(), funding_outpoint: None, // filled later - splice_parent_funding_txid: prev_funding_txid, + splice_parent_funding_txid: prev_funding.get_funding_txid(), channel_type_features: channel_parameters.channel_type_features.clone(), channel_value_satoshis: post_channel_value, }; @@ -2637,6 +2634,7 @@ impl_writeable_tlv_based!(PendingFunding, { enum FundingNegotiation { AwaitingAck { context: FundingNegotiationContext, + new_holder_funding_key: PublicKey, }, ConstructingTransaction { funding: FundingScope, @@ -2667,7 +2665,7 @@ impl FundingNegotiation { fn is_initiator(&self) -> bool { match self { - FundingNegotiation::AwaitingAck { context } => context.is_initiator, + FundingNegotiation::AwaitingAck { context, .. } => context.is_initiator, FundingNegotiation::ConstructingTransaction { interactive_tx_constructor, .. } => { interactive_tx_constructor.is_initiator() }, @@ -3510,7 +3508,7 @@ where // TODO(dual_funding): Checks for `funding_feerate_sat_per_1000_weight`? - let pubkeys = holder_signer.new_pubkeys(None, &secp_ctx); + let pubkeys = holder_signer.pubkeys(&secp_ctx); let funding = FundingScope { value_to_self_msat, @@ -3748,7 +3746,7 @@ where Err(_) => return Err(APIError::ChannelUnavailable { err: "Failed to get destination script".to_owned()}), }; - let pubkeys = holder_signer.new_pubkeys(None, &secp_ctx); + let pubkeys = holder_signer.pubkeys(&secp_ctx); let temporary_channel_id = temporary_channel_id_fn.map(|f| f(&pubkeys)) .unwrap_or_else(|| ChannelId::temporary_from_entropy_source(entropy_source)); @@ -4111,16 +4109,6 @@ where return &mut self.holder_signer; } - /// Returns holder pubkeys to use for the channel. - fn new_holder_pubkeys(&self, prev_funding_txid: Option) -> ChannelPublicKeys { - match &self.holder_signer { - ChannelSignerType::Ecdsa(ecdsa) => ecdsa.new_pubkeys(prev_funding_txid, &self.secp_ctx), - // TODO (taproot|arik) - #[cfg(taproot)] - _ => todo!(), - } - } - /// Only allowed immediately after deserialization if get_outbound_scid_alias returns 0, /// indicating we were written by LDK prior to 0.0.106 which did not set outbound SCID aliases /// or prior to any channel actions during `Channel` initialization. @@ -6891,7 +6879,7 @@ macro_rules! maybe_create_splice_funding_failed { .map(|funding| funding.get_channel_type().clone()); let (contributed_inputs, contributed_outputs) = match funding_negotiation { - FundingNegotiation::AwaitingAck { context } => { + FundingNegotiation::AwaitingAck { context, .. } => { context.$contributed_inputs_and_outputs() }, FundingNegotiation::ConstructingTransaction { @@ -11962,17 +11950,29 @@ where change_script, }; + // Rotate the funding pubkey using the prev_funding_txid as a tweak + let prev_funding_txid = self.funding.get_funding_txid(); + let funding_pubkey = match (prev_funding_txid, &self.context.holder_signer) { + (None, _) => { + debug_assert!(false); + self.funding.get_holder_pubkeys().funding_pubkey + }, + (Some(prev_funding_txid), ChannelSignerType::Ecdsa(ecdsa)) => { + ecdsa.new_funding_pubkey(prev_funding_txid, &self.context.secp_ctx) + }, + #[cfg(taproot)] + _ => todo!(), + }; + + let funding_negotiation = + FundingNegotiation::AwaitingAck { context, new_holder_funding_key: funding_pubkey }; self.pending_splice = Some(PendingFunding { - funding_negotiation: Some(FundingNegotiation::AwaitingAck { context }), + funding_negotiation: Some(funding_negotiation), negotiated_candidates: vec![], sent_funding_txid: None, received_funding_txid: None, }); - // Rotate the pubkeys using the prev_funding_txid as a tweak - let prev_funding_txid = self.funding.get_funding_txid(); - let funding_pubkey = self.context.new_holder_pubkeys(prev_funding_txid).funding_pubkey; - msgs::SpliceInit { channel_id: self.context.channel_id, funding_contribution_satoshis: adjusted_funding_contribution.to_sat(), @@ -12056,12 +12056,29 @@ where self.validate_splice_contributions(our_funding_contribution, their_funding_contribution) .map_err(|e| ChannelError::WarnAndDisconnect(e))?; + // Rotate the pubkeys using the prev_funding_txid as a tweak + let prev_funding_txid = self.funding.get_funding_txid(); + let funding_pubkey = match (prev_funding_txid, &self.context.holder_signer) { + (None, _) => { + debug_assert!(false); + self.funding.get_holder_pubkeys().funding_pubkey + }, + (Some(prev_funding_txid), ChannelSignerType::Ecdsa(ecdsa)) => { + ecdsa.new_funding_pubkey(prev_funding_txid, &self.context.secp_ctx) + }, + #[cfg(taproot)] + _ => todo!(), + }; + let mut new_keys = self.funding.get_holder_pubkeys().clone(); + new_keys.funding_pubkey = funding_pubkey; + Ok(FundingScope::for_splice( &self.funding, &self.context, our_funding_contribution, their_funding_contribution, msg.funding_pubkey, + new_keys, )) } @@ -12206,8 +12223,7 @@ where // optimization, but for often-offline nodes it may be, as we may connect and immediately // go into splicing from both sides. - let funding_pubkey = splice_funding.get_holder_pubkeys().funding_pubkey; - + let new_funding_pubkey = splice_funding.get_holder_pubkeys().funding_pubkey; self.pending_splice = Some(PendingFunding { funding_negotiation: Some(FundingNegotiation::ConstructingTransaction { funding: splice_funding, @@ -12221,7 +12237,7 @@ where Ok(msgs::SpliceAck { channel_id: self.context.channel_id, funding_contribution_satoshis: our_funding_contribution.to_sat(), - funding_pubkey, + funding_pubkey: new_funding_pubkey, require_confirmed_inputs: None, }) } @@ -12247,13 +12263,14 @@ where let pending_splice = self.pending_splice.as_mut().expect("We should have returned an error earlier!"); // TODO: Good candidate for a let else statement once MSRV >= 1.65 - let funding_negotiation_context = if let Some(FundingNegotiation::AwaitingAck { context }) = - pending_splice.funding_negotiation.take() - { - context - } else { - panic!("We should have returned an error earlier!"); - }; + let funding_negotiation_context = + if let Some(FundingNegotiation::AwaitingAck { context, .. }) = + pending_splice.funding_negotiation.take() + { + context + } else { + panic!("We should have returned an error earlier!"); + }; let mut interactive_tx_constructor = funding_negotiation_context .into_interactive_tx_constructor( @@ -12284,13 +12301,17 @@ where fn validate_splice_ack(&self, msg: &msgs::SpliceAck) -> Result { // TODO(splicing): Add check that we are the splice (quiescence) initiator - let funding_negotiation_context = match &self + let pending_splice = self .pending_splice .as_ref() - .ok_or(ChannelError::Ignore("Channel is not in pending splice".to_owned()))? + .ok_or_else(|| ChannelError::Ignore("Channel is not in pending splice".to_owned()))?; + + let (funding_negotiation_context, new_holder_funding_key) = match &pending_splice .funding_negotiation { - Some(FundingNegotiation::AwaitingAck { context }) => context, + Some(FundingNegotiation::AwaitingAck { context, new_holder_funding_key }) => { + (context, new_holder_funding_key) + }, Some(FundingNegotiation::ConstructingTransaction { .. }) | Some(FundingNegotiation::AwaitingSignatures { .. }) => { return Err(ChannelError::WarnAndDisconnect( @@ -12309,12 +12330,16 @@ where self.validate_splice_contributions(our_funding_contribution, their_funding_contribution) .map_err(|e| ChannelError::WarnAndDisconnect(e))?; + let mut new_keys = self.funding.get_holder_pubkeys().clone(); + new_keys.funding_pubkey = *new_holder_funding_key; + Ok(FundingScope::for_splice( &self.funding, &self.context, our_funding_contribution, their_funding_contribution, msg.funding_pubkey, + new_keys, )) } @@ -16475,7 +16500,7 @@ mod tests { [0; 32], ); - let holder_pubkeys = signer.new_pubkeys(None, &secp_ctx); + let holder_pubkeys = signer.pubkeys(&secp_ctx); assert_eq!(holder_pubkeys.funding_pubkey.serialize()[..], >::from_hex("023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb").unwrap()[..]); let keys_provider = Keys { signer: signer.clone() }; diff --git a/lightning/src/sign/mod.rs b/lightning/src/sign/mod.rs index 5bd24d8e2e1..95b90bfccd0 100644 --- a/lightning/src/sign/mod.rs +++ b/lightning/src/sign/mod.rs @@ -272,12 +272,12 @@ pub enum SpendableOutputDescriptor { /// To derive the delayed payment key which is used to sign this input, you must pass the /// holder [`InMemorySigner::delayed_payment_base_key`] (i.e., the private key which /// corresponds to the [`ChannelPublicKeys::delayed_payment_basepoint`] in - /// [`ChannelSigner::new_pubkeys`]) and the provided + /// [`ChannelSigner::pubkeys`]) and the provided /// [`DelayedPaymentOutputDescriptor::per_commitment_point`] to /// [`chan_utils::derive_private_key`]. The DelayedPaymentKey can be generated without the /// secret key using [`DelayedPaymentKey::from_basepoint`] and only the /// [`ChannelPublicKeys::delayed_payment_basepoint`] which appears in - /// [`ChannelSigner::new_pubkeys`]. + /// [`ChannelSigner::pubkeys`]. /// /// To derive the [`DelayedPaymentOutputDescriptor::revocation_pubkey`] provided here (which is /// used in the witness script generation), you must pass the counterparty @@ -292,7 +292,7 @@ pub enum SpendableOutputDescriptor { /// [`chan_utils::get_revokeable_redeemscript`]. DelayedPaymentOutput(DelayedPaymentOutputDescriptor), /// An output spendable exclusively by our payment key (i.e., the private key that corresponds - /// to the `payment_point` in [`ChannelSigner::new_pubkeys`]). The output type depends on the + /// to the `payment_point` in [`ChannelSigner::pubkeys`]). The output type depends on the /// channel type negotiated. /// /// On an anchor outputs channel, the witness in the spending input is: @@ -792,19 +792,25 @@ pub trait ChannelSigner { /// and pause future signing operations until this validation completes. fn validate_counterparty_revocation(&self, idx: u64, secret: &SecretKey) -> Result<(), ()>; - /// Returns a *new* set of holder channel public keys and basepoints. They may be the same as a - /// previous value, but are also allowed to change arbitrarily. Signing methods must still - /// support signing for any keys which have ever been returned. This should only be called - /// either for new channels or new splices. + /// Returns the holder channel public keys and basepoints. This should only be called once + /// during channel creation and as such implementations are allowed undefined behavior if + /// called more than once. /// - /// `splice_parent_funding_txid` can be used to compute a tweak to rotate the funding key in the - /// 2-of-2 multisig script during a splice. See [`compute_funding_key_tweak`] for an example - /// tweak and more details. + /// This method is *not* asynchronous. Instead, the value must be computed locally or in + /// advance and cached. + fn pubkeys(&self, secp_ctx: &Secp256k1) -> ChannelPublicKeys; + + /// Returns a new funding pubkey (i.e. our public which is used in a 2-of-2 with the + /// counterparty's key to to lock the funds on-chain) for a spliced channel. + /// + /// `splice_parent_funding_txid` can be used to compute a tweak with which to rotate the base + /// key (which will then be available later in signing operations via + /// [`ChannelTransactionParameters::splice_parent_funding_txid`]). /// /// This method is *not* asynchronous. Instead, the value must be cached locally. - fn new_pubkeys( - &self, splice_parent_funding_txid: Option, secp_ctx: &Secp256k1, - ) -> ChannelPublicKeys; + fn new_funding_pubkey( + &self, splice_parent_funding_txid: Txid, secp_ctx: &Secp256k1, + ) -> PublicKey; /// Returns an arbitrary identifier describing the set of keys which are provided back to you in /// some [`SpendableOutputDescriptor`] types. This should be sufficient to identify this @@ -1457,17 +1463,13 @@ impl ChannelSigner for InMemorySigner { Ok(()) } - fn new_pubkeys( - &self, splice_parent_funding_txid: Option, secp_ctx: &Secp256k1, - ) -> ChannelPublicKeys { + fn pubkeys(&self, secp_ctx: &Secp256k1) -> ChannelPublicKeys { // Because splices always break downgrades, we go ahead and always use the new derivation // here as its just much better. - let use_v2_derivation = - self.v2_remote_key_derivation || splice_parent_funding_txid.is_some(); let payment_key = - if use_v2_derivation { &self.payment_key_v2 } else { &self.payment_key_v1 }; + if self.v2_remote_key_derivation { &self.payment_key_v2 } else { &self.payment_key_v1 }; let from_secret = |s: &SecretKey| PublicKey::from_secret_key(secp_ctx, s); - let mut pubkeys = ChannelPublicKeys { + let pubkeys = ChannelPublicKeys { funding_pubkey: from_secret(&self.funding_key.0), revocation_basepoint: RevocationBasepoint::from(from_secret(&self.revocation_base_key)), payment_point: from_secret(payment_key), @@ -1477,13 +1479,15 @@ impl ChannelSigner for InMemorySigner { htlc_basepoint: HtlcBasepoint::from(from_secret(&self.htlc_base_key)), }; - if splice_parent_funding_txid.is_some() { - pubkeys.funding_pubkey = - self.funding_key(splice_parent_funding_txid).public_key(secp_ctx); - } pubkeys } + fn new_funding_pubkey( + &self, splice_parent_funding_txid: Txid, secp_ctx: &Secp256k1, + ) -> PublicKey { + self.funding_key(Some(splice_parent_funding_txid)).public_key(secp_ctx) + } + fn channel_keys_id(&self) -> [u8; 32] { self.channel_keys_id } diff --git a/lightning/src/util/dyn_signer.rs b/lightning/src/util/dyn_signer.rs index c519484a938..cf1cac37903 100644 --- a/lightning/src/util/dyn_signer.rs +++ b/lightning/src/util/dyn_signer.rs @@ -174,9 +174,12 @@ delegate!(DynSigner, ChannelSigner, holder_tx: &HolderCommitmentTransaction, preimages: Vec ) -> Result<(), ()>, - fn new_pubkeys(, - splice_parent_funding_txid: Option, secp_ctx: &Secp256k1 + fn pubkeys(, + secp_ctx: &Secp256k1 ) -> ChannelPublicKeys, + fn new_funding_pubkey(, + splice_parent_funding_txid: Txid, secp_ctx: &Secp256k1 + ) -> PublicKey, fn channel_keys_id(,) -> [u8; 32], fn validate_counterparty_revocation(, idx: u64, secret: &SecretKey) -> Result<(), ()> ); diff --git a/lightning/src/util/test_channel_signer.rs b/lightning/src/util/test_channel_signer.rs index 6b1950169e8..652c1ffa48c 100644 --- a/lightning/src/util/test_channel_signer.rs +++ b/lightning/src/util/test_channel_signer.rs @@ -221,10 +221,14 @@ impl ChannelSigner for TestChannelSigner { Ok(()) } - fn new_pubkeys( - &self, splice_parent_funding_txid: Option, secp_ctx: &Secp256k1, - ) -> ChannelPublicKeys { - self.inner.new_pubkeys(splice_parent_funding_txid, secp_ctx) + fn pubkeys(&self, secp_ctx: &Secp256k1) -> ChannelPublicKeys { + self.inner.pubkeys(secp_ctx) + } + + fn new_funding_pubkey( + &self, splice_parent_funding_txid: Txid, secp_ctx: &Secp256k1, + ) -> PublicKey { + self.inner.new_funding_pubkey(splice_parent_funding_txid, secp_ctx) } fn channel_keys_id(&self) -> [u8; 32] { From 491b6949fec743e34c74c3a472cf0a2cb83d7ab3 Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 21 Oct 2025 22:25:14 +0000 Subject: [PATCH 5/8] Enforce that `ChanelSigner::pubkeys` is only called once In the previous commit we partially reverted 9d291e01f98417c2f6b2d4321bbf806464c424a6 renaming `ChannelSigner::new_pubkeys` to `pubkeys` again, but we still don't want to go back to requiring that `pubkeys` return the same contents on each call. Thus, here, we add test logic to check that `pubkeys` isn't called more than once. --- lightning/src/util/test_channel_signer.rs | 32 +++++++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/lightning/src/util/test_channel_signer.rs b/lightning/src/util/test_channel_signer.rs index 652c1ffa48c..bad00f65d0a 100644 --- a/lightning/src/util/test_channel_signer.rs +++ b/lightning/src/util/test_channel_signer.rs @@ -24,7 +24,9 @@ use crate::prelude::*; #[cfg(any(test, feature = "_test_utils"))] use crate::sync::MutexGuard; use crate::sync::{Arc, Mutex}; + use core::cmp; +use core::sync::atomic::{AtomicBool, Ordering}; use bitcoin::hashes::Hash; use bitcoin::sighash; @@ -68,13 +70,31 @@ pub const INITIAL_REVOKED_COMMITMENT_NUMBER: u64 = 1 << 48; /// /// Note that before we do so we should ensure its serialization format has backwards- and /// forwards-compatibility prefix/suffixes! -#[derive(Clone)] pub struct TestChannelSigner { pub inner: DynSigner, /// Channel state used for policy enforcement pub state: Arc>, pub disable_revocation_policy_check: bool, pub disable_all_state_policy_checks: bool, + have_fetched_pubkeys: AtomicBool, +} + +impl Clone for TestChannelSigner { + fn clone(&self) -> Self { + // Generally, a signer should only ever be cloned when a ChannelMonitor is cloned (which + // doesn't fetch the pubkeys at all). This isn't really a critical test, but if it + // it ever does fail we should make sure the clone is hapening in a sensible place. + assert!(!self.have_fetched_pubkeys.load(Ordering::Acquire)); + Self { + inner: self.inner.clone(), + state: Arc::clone(&self.state), + disable_revocation_policy_check: self.disable_revocation_policy_check, + disable_all_state_policy_checks: self.disable_all_state_policy_checks, + // In some tests we clone a `ChannelMonitor` multiple times, so have to initialize with + // `!have_fetched_pubkeys` to ensure the above assertion passes. + have_fetched_pubkeys: AtomicBool::new(false), + } + } } #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -129,6 +149,7 @@ impl TestChannelSigner { state, disable_revocation_policy_check: false, disable_all_state_policy_checks: false, + have_fetched_pubkeys: AtomicBool::new(false), } } @@ -141,7 +162,13 @@ impl TestChannelSigner { inner: DynSigner, state: Arc>, disable_revocation_policy_check: bool, disable_all_state_policy_checks: bool, ) -> Self { - Self { inner, state, disable_revocation_policy_check, disable_all_state_policy_checks } + Self { + inner, + state, + disable_revocation_policy_check, + disable_all_state_policy_checks, + have_fetched_pubkeys: AtomicBool::new(false), + } } #[cfg(any(test, feature = "_test_utils"))] @@ -222,6 +249,7 @@ impl ChannelSigner for TestChannelSigner { } fn pubkeys(&self, secp_ctx: &Secp256k1) -> ChannelPublicKeys { + assert!(!self.have_fetched_pubkeys.swap(true, Ordering::AcqRel)); self.inner.pubkeys(secp_ctx) } From 0f4e6c228205c8595c7c80853c43f84e55cb503f Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 21 Oct 2025 13:42:21 +0000 Subject: [PATCH 6/8] Fix `generated_by_local` arg to build commmitment during splicing `build_commitment_transaction`'s fifth argument is supposed to be whether we're the one generating the commitment (i.e. because we're signing rather than validating the commitment). During splicing, this doesn't matter because there should be no async HTLC addition/removal happening so the commitment generated wil be the same in either case, but its still good to pass the correct bool. --- lightning/src/ln/channel.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lightning/src/ln/channel.rs b/lightning/src/ln/channel.rs index c50d0d4e121..dc966af966f 100644 --- a/lightning/src/ln/channel.rs +++ b/lightning/src/ln/channel.rs @@ -6252,7 +6252,7 @@ where commitment_number, &commitment_point, false, - false, + true, logger, ); let counterparty_initial_commitment_tx = commitment_data.tx; From d12c6a3572a886c2f8dd543403992943ec8b697d Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Fri, 24 Oct 2025 21:38:02 +0000 Subject: [PATCH 7/8] Correct spliced-stale SCID expiry for upgrades from pre-0.2 HTLC If an HTLC was forwarded in 0.1, but waiting to be failed back, it will ultimately be failed by adding it to the `ChannelManager::pending_forwards` map with the channel's original SCID. If that channel is spliced between when the HTLC was forwarded (on 0.1) and when the HTLC is failed back (on 0.2), that SCID may no longer exist, causing the HTLC fail-back to be lost. Luckily, delaying when an SCID is expired is cheap - its just storing an extra `u64` or two and generating one requires an on-chain splice, so we simply delay removal of SCIDs for two months at which point any incoming HTLCs should have been expired for six weeks and the counterparty should have force-closed anyway. --- lightning/src/ln/channel.rs | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/lightning/src/ln/channel.rs b/lightning/src/ln/channel.rs index dc966af966f..f5cb6c54fe9 100644 --- a/lightning/src/ln/channel.rs +++ b/lightning/src/ln/channel.rs @@ -1373,6 +1373,27 @@ pub(crate) const COINBASE_MATURITY: u32 = 100; /// The number of blocks to wait for a channel_announcement to propagate such that payments using an /// older SCID can still be relayed. Once the spend of the previous funding transaction has reached /// this number of confirmations, the corresponding SCID will be forgotten. +/// +/// Because HTLCs added prior to 0.1 which were waiting to be failed may reference a channel's +/// pre-splice SCID, we need to ensure this is at least the maximum number of blocks before an HTLC +/// gets failed-back due to a time-out. Luckily, in LDK prior to 0.2, this is enforced directly +/// when checking the incoming HTLC, and compared against `CLTV_FAR_FAR_AWAY` (which prior to LDK +/// 0.2, and still at the time of writing, is 14 * 24 * 6, i.e. two weeks). +/// +/// Here we use four times that value to give us more time to fail an HTLC back (which does require +/// the user call [`ChannelManager::process_pending_htlc_forwards`]) just in case (if an HTLC has +/// been expired for 3 * 2 weeks our counterparty really should have closed the channel by now). +/// Holding on to stale SCIDs doesn't really cost us much as each one costs an on-chain splice to +/// generate anyway, so we might as well make this nearly arbitrarily long. +/// +/// [`ChannelManager::process_pending_htlc_forwards`]: crate::ln::channelmanager::ChannelManager::process_pending_htlc_forwards +#[cfg(not(test))] +pub(crate) const CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY: u32 = 14 * 24 * 6 * 4; + +/// In test (not `_test_utils`, though, since that tests actual upgrading), we deliberately break +/// the above condition so that we can ensure that HTLCs forwarded in 0.2 or later are handled +/// correctly even if this constant is reduced and an HTLC can outlive the original channel's SCID. +#[cfg(test)] pub(crate) const CHANNEL_ANNOUNCEMENT_PROPAGATION_DELAY: u32 = 144; struct PendingChannelMonitorUpdate { From 2c836b3072597cf8da840ef93892d82fb9397840 Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 21 Oct 2025 13:53:58 +0000 Subject: [PATCH 8/8] Add an upgrade test of splicing after upgrading from 0.1 --- lightning-tests/Cargo.toml | 2 +- .../src/upgrade_downgrade_tests.rs | 206 +++++++++++++++++- lightning/src/chain/channelmonitor.rs | 2 +- lightning/src/ln/mod.rs | 4 +- lightning/src/ln/splicing_tests.rs | 22 +- 5 files changed, 221 insertions(+), 15 deletions(-) diff --git a/lightning-tests/Cargo.toml b/lightning-tests/Cargo.toml index 23c81fae4a3..36b03a2756a 100644 --- a/lightning-tests/Cargo.toml +++ b/lightning-tests/Cargo.toml @@ -14,7 +14,7 @@ lightning-types = { path = "../lightning-types", features = ["_test_utils"] } lightning-invoice = { path = "../lightning-invoice", default-features = false } lightning-macros = { path = "../lightning-macros" } lightning = { path = "../lightning", features = ["_test_utils"] } -lightning_0_1 = { package = "lightning", version = "0.1.1", features = ["_test_utils"] } +lightning_0_1 = { package = "lightning", version = "0.1.7", features = ["_test_utils"] } lightning_0_0_125 = { package = "lightning", version = "0.0.125", features = ["_test_utils"] } bitcoin = { version = "0.32.2", default-features = false } diff --git a/lightning-tests/src/upgrade_downgrade_tests.rs b/lightning-tests/src/upgrade_downgrade_tests.rs index 0b3f5dff427..d1153b7a8fa 100644 --- a/lightning-tests/src/upgrade_downgrade_tests.rs +++ b/lightning-tests/src/upgrade_downgrade_tests.rs @@ -10,9 +10,15 @@ //! Tests which test upgrading from previous versions of LDK or downgrading to previous versions of //! LDK. +use lightning_0_1::commitment_signed_dance as commitment_signed_dance_0_1; use lightning_0_1::events::ClosureReason as ClosureReason_0_1; +use lightning_0_1::expect_pending_htlcs_forwardable_ignore as expect_pending_htlcs_forwardable_ignore_0_1; use lightning_0_1::get_monitor as get_monitor_0_1; +use lightning_0_1::ln::channelmanager::PaymentId as PaymentId_0_1; +use lightning_0_1::ln::channelmanager::RecipientOnionFields as RecipientOnionFields_0_1; use lightning_0_1::ln::functional_test_utils as lightning_0_1_utils; +use lightning_0_1::ln::msgs::ChannelMessageHandler as _; +use lightning_0_1::routing::router as router_0_1; use lightning_0_1::util::ser::Writeable as _; use lightning_0_0_125::chain::ChannelMonitorUpdateStatus as ChannelMonitorUpdateStatus_0_0_125; @@ -29,16 +35,23 @@ use lightning_0_0_125::ln::msgs::ChannelMessageHandler as _; use lightning_0_0_125::routing::router as router_0_0_125; use lightning_0_0_125::util::ser::Writeable as _; -use lightning::chain::channelmonitor::ANTI_REORG_DELAY; -use lightning::events::{ClosureReason, Event}; +use lightning::chain::channelmonitor::{ANTI_REORG_DELAY, HTLC_FAIL_BACK_BUFFER}; +use lightning::events::bump_transaction::sync::WalletSourceSync; +use lightning::events::{ClosureReason, Event, HTLCHandlingFailureType}; use lightning::ln::functional_test_utils::*; +use lightning::ln::funding::SpliceContribution; +use lightning::ln::msgs::BaseMessageHandler as _; +use lightning::ln::msgs::ChannelMessageHandler as _; +use lightning::ln::msgs::MessageSendEvent; +use lightning::ln::splicing_tests::*; +use lightning::ln::types::ChannelId; use lightning::sign::OutputSpender; -use lightning_types::payment::PaymentPreimage; +use lightning_types::payment::{PaymentHash, PaymentPreimage, PaymentSecret}; -use bitcoin::opcodes; use bitcoin::script::Builder; use bitcoin::secp256k1::Secp256k1; +use bitcoin::{opcodes, Amount, TxOut}; use std::sync::Arc; @@ -299,3 +312,188 @@ fn test_0_1_legacy_remote_key_derivation() { panic!("Wrong event"); } } + +fn do_test_0_1_htlc_forward_after_splice(fail_htlc: bool) { + // Test what happens if an HTLC set to be forwarded in 0.1 is forwarded after the inbound + // channel is spliced. In the initial splice code, this could have led to a dangling HTLC if + // the HTLC is failed as the backwards-failure would use the channel's original SCID which is + // no longer valid. + // In some later splice code, this also failed because the `KeysManager` would have tried to + // rotate the `to_remote` key, which we aren't able to do in the splicing protocol. + let (node_a_ser, node_b_ser, node_c_ser, mon_a_1_ser, mon_b_1_ser, mon_b_2_ser, mon_c_1_ser); + let (node_a_id, node_b_id, node_c_id); + let (chan_id_bytes_a, chan_id_bytes_b); + let (payment_secret_bytes, payment_hash_bytes, payment_preimage_bytes); + let (node_a_blocks, node_b_blocks, node_c_blocks); + + const EXTRA_BLOCKS_BEFORE_FAIL: u32 = 145; + + { + let chanmon_cfgs = lightning_0_1_utils::create_chanmon_cfgs(3); + let node_cfgs = lightning_0_1_utils::create_node_cfgs(3, &chanmon_cfgs); + let node_chanmgrs = + lightning_0_1_utils::create_node_chanmgrs(3, &node_cfgs, &[None, None, None]); + let nodes = lightning_0_1_utils::create_network(3, &node_cfgs, &node_chanmgrs); + + node_a_id = nodes[0].node.get_our_node_id(); + node_b_id = nodes[1].node.get_our_node_id(); + node_c_id = nodes[2].node.get_our_node_id(); + let chan_id_a = lightning_0_1_utils::create_announced_chan_between_nodes_with_value( + &nodes, 0, 1, 10_000_000, 0, + ) + .2; + chan_id_bytes_a = chan_id_a.0; + + let chan_id_b = lightning_0_1_utils::create_announced_chan_between_nodes_with_value( + &nodes, 1, 2, 50_000, 0, + ) + .2; + chan_id_bytes_b = chan_id_b.0; + + // Ensure all nodes are at the same initial height. + let node_max_height = nodes.iter().map(|node| node.best_block_info().1).max().unwrap(); + for node in &nodes { + let blocks_to_mine = node_max_height - node.best_block_info().1; + if blocks_to_mine > 0 { + lightning_0_1_utils::connect_blocks(node, blocks_to_mine); + } + } + + let (preimage, hash, secret) = + lightning_0_1_utils::get_payment_preimage_hash(&nodes[2], Some(1_000_000), None); + payment_preimage_bytes = preimage.0; + payment_hash_bytes = hash.0; + payment_secret_bytes = secret.0; + + let pay_params = router_0_1::PaymentParameters::from_node_id( + node_c_id, + lightning_0_1_utils::TEST_FINAL_CLTV, + ) + .with_bolt11_features(nodes[2].node.bolt11_invoice_features()) + .unwrap(); + + let route_params = + router_0_1::RouteParameters::from_payment_params_and_value(pay_params, 1_000_000); + let mut route = lightning_0_1_utils::get_route(&nodes[0], &route_params).unwrap(); + route.paths[0].hops[1].cltv_expiry_delta = + EXTRA_BLOCKS_BEFORE_FAIL + HTLC_FAIL_BACK_BUFFER + 1; + if fail_htlc { + // Pay more than the channel's value (and probably not enough fee) + route.paths[0].hops[1].fee_msat = 50_000_000; + } + + let onion = RecipientOnionFields_0_1::secret_only(secret); + let id = PaymentId_0_1(hash.0); + nodes[0].node.send_payment_with_route(route, hash, onion, id).unwrap(); + + lightning_0_1_utils::check_added_monitors(&nodes[0], 1); + let send_event = lightning_0_1_utils::SendEvent::from_node(&nodes[0]); + + nodes[1].node.handle_update_add_htlc(node_a_id, &send_event.msgs[0]); + commitment_signed_dance_0_1!(nodes[1], nodes[0], send_event.commitment_msg, false); + expect_pending_htlcs_forwardable_ignore_0_1!(nodes[1]); + + // We now have an HTLC pending in node B's forwarding queue with the original channel's + // SCID as the source. + // We now upgrade to 0.2 and splice before forwarding that HTLC... + node_a_ser = nodes[0].node.encode(); + node_b_ser = nodes[1].node.encode(); + node_c_ser = nodes[2].node.encode(); + mon_a_1_ser = get_monitor_0_1!(nodes[0], chan_id_a).encode(); + mon_b_1_ser = get_monitor_0_1!(nodes[1], chan_id_a).encode(); + mon_b_2_ser = get_monitor_0_1!(nodes[1], chan_id_b).encode(); + mon_c_1_ser = get_monitor_0_1!(nodes[2], chan_id_b).encode(); + + node_a_blocks = Arc::clone(&nodes[0].blocks); + node_b_blocks = Arc::clone(&nodes[1].blocks); + node_c_blocks = Arc::clone(&nodes[2].blocks); + } + + // Create a dummy node to reload over with the 0.1 state + let mut chanmon_cfgs = create_chanmon_cfgs(3); + + // Our TestChannelSigner will fail as we're jumping ahead, so disable its state-based checks + chanmon_cfgs[0].keys_manager.disable_all_state_policy_checks = true; + chanmon_cfgs[1].keys_manager.disable_all_state_policy_checks = true; + chanmon_cfgs[2].keys_manager.disable_all_state_policy_checks = true; + + chanmon_cfgs[0].tx_broadcaster.blocks = node_a_blocks; + chanmon_cfgs[1].tx_broadcaster.blocks = node_b_blocks; + chanmon_cfgs[2].tx_broadcaster.blocks = node_c_blocks; + + let node_cfgs = create_node_cfgs(3, &chanmon_cfgs); + let (persister_a, persister_b, persister_c, chain_mon_a, chain_mon_b, chain_mon_c); + let node_chanmgrs = create_node_chanmgrs(3, &node_cfgs, &[None, None, None]); + let (node_a, node_b, node_c); + let mut nodes = create_network(3, &node_cfgs, &node_chanmgrs); + + let config = test_default_channel_config(); + let a_mons = &[&mon_a_1_ser[..]]; + reload_node!(nodes[0], config.clone(), &node_a_ser, a_mons, persister_a, chain_mon_a, node_a); + let b_mons = &[&mon_b_1_ser[..], &mon_b_2_ser[..]]; + reload_node!(nodes[1], config.clone(), &node_b_ser, b_mons, persister_b, chain_mon_b, node_b); + let c_mons = &[&mon_c_1_ser[..]]; + reload_node!(nodes[2], config, &node_c_ser, c_mons, persister_c, chain_mon_c, node_c); + + reconnect_nodes(ReconnectArgs::new(&nodes[0], &nodes[1])); + let mut reconnect_b_c_args = ReconnectArgs::new(&nodes[1], &nodes[2]); + reconnect_b_c_args.send_channel_ready = (true, true); + reconnect_b_c_args.send_announcement_sigs = (true, true); + reconnect_nodes(reconnect_b_c_args); + + let contribution = SpliceContribution::SpliceOut { + outputs: vec![TxOut { + value: Amount::from_sat(1_000), + script_pubkey: nodes[0].wallet_source.get_change_script().unwrap(), + }], + }; + let splice_tx = splice_channel(&nodes[0], &nodes[1], ChannelId(chan_id_bytes_a), contribution); + for node in nodes.iter() { + mine_transaction(node, &splice_tx); + connect_blocks(node, ANTI_REORG_DELAY - 1); + } + + let splice_locked = get_event_msg!(nodes[0], MessageSendEvent::SendSpliceLocked, node_b_id); + lock_splice(&nodes[0], &nodes[1], &splice_locked, false); + + for node in nodes.iter() { + connect_blocks(node, EXTRA_BLOCKS_BEFORE_FAIL - ANTI_REORG_DELAY); + } + + // Now release the HTLC to be failed back to node A + nodes[1].node.process_pending_htlc_forwards(); + + let pay_secret = PaymentSecret(payment_secret_bytes); + let pay_hash = PaymentHash(payment_hash_bytes); + let pay_preimage = PaymentPreimage(payment_preimage_bytes); + + if fail_htlc { + let failure = HTLCHandlingFailureType::Forward { + node_id: Some(node_c_id), + channel_id: ChannelId(chan_id_bytes_b), + }; + expect_and_process_pending_htlcs_and_htlc_handling_failed(&nodes[1], &[failure]); + check_added_monitors(&nodes[1], 1); + + let updates = get_htlc_update_msgs(&nodes[1], &node_a_id); + nodes[0].node.handle_update_fail_htlc(node_b_id, &updates.update_fail_htlcs[0]); + commitment_signed_dance!(nodes[0], nodes[1], updates.commitment_signed, false); + let conditions = PaymentFailedConditions::new(); + expect_payment_failed_conditions(&nodes[0], pay_hash, false, conditions); + } else { + check_added_monitors(&nodes[1], 1); + let forward_event = SendEvent::from_node(&nodes[1]); + nodes[2].node.handle_update_add_htlc(node_b_id, &forward_event.msgs[0]); + commitment_signed_dance!(nodes[2], nodes[1], forward_event.commitment_msg, false); + + expect_and_process_pending_htlcs(&nodes[2], false); + expect_payment_claimable!(nodes[2], pay_hash, pay_secret, 1_000_000); + claim_payment(&nodes[0], &[&nodes[1], &nodes[2]], pay_preimage); + } +} + +#[test] +fn test_0_1_htlc_forward_after_splice() { + do_test_0_1_htlc_forward_after_splice(true); + do_test_0_1_htlc_forward_after_splice(false); +} diff --git a/lightning/src/chain/channelmonitor.rs b/lightning/src/chain/channelmonitor.rs index e2f0effed80..4acba6f4085 100644 --- a/lightning/src/chain/channelmonitor.rs +++ b/lightning/src/chain/channelmonitor.rs @@ -327,7 +327,7 @@ pub const ARCHIVAL_DELAY_BLOCKS: u32 = 4032; /// (2) is the same, but with an additional buffer to avoid accepting an HTLC which is immediately /// in a race condition between the user connecting a block (which would fail it) and the user /// providing us the preimage (which would claim it). -pub(crate) const HTLC_FAIL_BACK_BUFFER: u32 = CLTV_CLAIM_BUFFER + LATENCY_GRACE_PERIOD_BLOCKS; +pub const HTLC_FAIL_BACK_BUFFER: u32 = CLTV_CLAIM_BUFFER + LATENCY_GRACE_PERIOD_BLOCKS; // Deprecated, use [`HolderCommitment`] or [`HolderCommitmentTransaction`]. #[derive(Clone, PartialEq, Eq)] diff --git a/lightning/src/ln/mod.rs b/lightning/src/ln/mod.rs index bae55584575..9473142cfed 100644 --- a/lightning/src/ln/mod.rs +++ b/lightning/src/ln/mod.rs @@ -120,8 +120,8 @@ mod reorg_tests; #[cfg(test)] #[allow(unused_mut)] mod shutdown_tests; -#[cfg(test)] -mod splicing_tests; +#[cfg(any(feature = "_test_utils", test))] +pub mod splicing_tests; #[cfg(any(test, feature = "_externalize_tests"))] #[allow(unused_mut)] pub mod update_fee_tests; diff --git a/lightning/src/ln/splicing_tests.rs b/lightning/src/ln/splicing_tests.rs index 2b415a1402a..f95883e9434 100644 --- a/lightning/src/ln/splicing_tests.rs +++ b/lightning/src/ln/splicing_tests.rs @@ -7,6 +7,8 @@ // You may not use this file except in accordance with one or both of these // licenses. +#![cfg_attr(not(test), allow(unused_imports))] + use crate::chain::chaininterface::FEERATE_FLOOR_SATS_PER_KW; use crate::chain::channelmonitor::{ANTI_REORG_DELAY, LATENCY_GRACE_PERIOD_BLOCKS}; use crate::chain::transaction::OutPoint; @@ -68,7 +70,7 @@ fn test_v1_splice_in_negative_insufficient_inputs() { } } -fn negotiate_splice_tx<'a, 'b, 'c, 'd>( +pub fn negotiate_splice_tx<'a, 'b, 'c, 'd>( initiator: &'a Node<'b, 'c, 'd>, acceptor: &'a Node<'b, 'c, 'd>, channel_id: ChannelId, initiator_contribution: SpliceContribution, ) -> msgs::CommitmentSigned { @@ -83,7 +85,7 @@ fn negotiate_splice_tx<'a, 'b, 'c, 'd>( ) } -fn complete_splice_handshake<'a, 'b, 'c, 'd>( +pub fn complete_splice_handshake<'a, 'b, 'c, 'd>( initiator: &'a Node<'b, 'c, 'd>, acceptor: &'a Node<'b, 'c, 'd>, channel_id: ChannelId, initiator_contribution: SpliceContribution, ) -> ScriptBuf { @@ -120,7 +122,7 @@ fn complete_splice_handshake<'a, 'b, 'c, 'd>( new_funding_script } -fn complete_interactive_funding_negotiation<'a, 'b, 'c, 'd>( +pub fn complete_interactive_funding_negotiation<'a, 'b, 'c, 'd>( initiator: &'a Node<'b, 'c, 'd>, acceptor: &'a Node<'b, 'c, 'd>, channel_id: ChannelId, initiator_contribution: SpliceContribution, new_funding_script: ScriptBuf, ) -> msgs::CommitmentSigned { @@ -209,7 +211,7 @@ fn complete_interactive_funding_negotiation<'a, 'b, 'c, 'd>( } } -fn sign_interactive_funding_tx<'a, 'b, 'c, 'd>( +pub fn sign_interactive_funding_tx<'a, 'b, 'c, 'd>( initiator: &'a Node<'b, 'c, 'd>, acceptor: &'a Node<'b, 'c, 'd>, initial_commit_sig_for_acceptor: msgs::CommitmentSigned, is_0conf: bool, ) -> (Transaction, Option<(msgs::SpliceLocked, PublicKey)>) { @@ -277,7 +279,7 @@ fn sign_interactive_funding_tx<'a, 'b, 'c, 'd>( (tx, splice_locked) } -fn splice_channel<'a, 'b, 'c, 'd>( +pub fn splice_channel<'a, 'b, 'c, 'd>( initiator: &'a Node<'b, 'c, 'd>, acceptor: &'a Node<'b, 'c, 'd>, channel_id: ChannelId, initiator_contribution: SpliceContribution, ) -> Transaction { @@ -304,7 +306,7 @@ fn splice_channel<'a, 'b, 'c, 'd>( splice_tx } -fn lock_splice_after_blocks<'a, 'b, 'c, 'd>( +pub fn lock_splice_after_blocks<'a, 'b, 'c, 'd>( node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, num_blocks: u32, ) { connect_blocks(node_a, num_blocks); @@ -316,7 +318,7 @@ fn lock_splice_after_blocks<'a, 'b, 'c, 'd>( lock_splice(node_a, node_b, &splice_locked_for_node_b, false); } -fn lock_splice<'a, 'b, 'c, 'd>( +pub fn lock_splice<'a, 'b, 'c, 'd>( node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, splice_locked_for_node_b: &msgs::SpliceLocked, is_0conf: bool, ) { @@ -387,6 +389,7 @@ fn test_splice_state_reset_on_disconnect() { do_test_splice_state_reset_on_disconnect(true); } +#[cfg(test)] fn do_test_splice_state_reset_on_disconnect(reload: bool) { // Tests that we're able to forget our pending splice state after a disconnect such that we can // retry later. @@ -714,6 +717,7 @@ fn test_splice_out() { let _ = send_payment(&nodes[0], &[&nodes[1]], htlc_limit_msat); } +#[cfg(test)] #[derive(PartialEq)] enum SpliceStatus { Unconfirmed, @@ -731,6 +735,7 @@ fn test_splice_commitment_broadcast() { do_test_splice_commitment_broadcast(SpliceStatus::Locked, true); } +#[cfg(test)] fn do_test_splice_commitment_broadcast(splice_status: SpliceStatus, claim_htlcs: bool) { // Tests that we're able to enforce HTLCs onchain during the different stages of a splice. let chanmon_cfgs = create_chanmon_cfgs(2); @@ -926,6 +931,7 @@ fn test_splice_reestablish() { do_test_splice_reestablish(true, true); } +#[cfg(test)] fn do_test_splice_reestablish(reload: bool, async_monitor_update: bool) { // Test that we're able to reestablish the channel succesfully throughout the lifecycle of a splice. let chanmon_cfgs = create_chanmon_cfgs(2); @@ -1188,6 +1194,7 @@ fn test_propose_splice_while_disconnected() { do_test_propose_splice_while_disconnected(true, true); } +#[cfg(test)] fn do_test_propose_splice_while_disconnected(reload: bool, use_0conf: bool) { // Test that both nodes are able to propose a splice while the counterparty is disconnected, and // whoever doesn't go first due to the quiescence tie-breaker, will retry their splice after the @@ -1804,6 +1811,7 @@ fn fail_quiescent_action_on_channel_close() { check_added_monitors(&nodes[0], 1); } +#[cfg(test)] fn do_test_splice_with_inflight_htlc_forward_and_resolution(expire_scid_pre_forward: bool) { // Test that we are still able to forward and resolve HTLCs while the original SCIDs contained // in the onion packets have now changed due channel splices becoming locked.