From 8dbdd15f2bae6ca522c91a2f6b7515d05721752f Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Thu, 20 Nov 2025 00:44:22 +0000 Subject: [PATCH 1/6] Handle mon update completion actions even with update(s) is blocked If we complete a `ChannelMonitorUpdate` persistence but there are blocked `ChannelMonitorUpdate`s in the channel, we'll skip all the post-monitor-update logic entirely. While its correct that we can't resume the channel (as it expected the monitor updates it generated to complete, even if they ended up blocked), the post-update actions are a `channelmanager.rs` concept - they cannot be tied to blocked updates because `channelmanager.rs` doesn't even see blocked updates. This can lead to a channel getting stuck waiting on itself. In a production environment, an LDK user saw a case where: (a) an MPP payment was received over several channels, let's call them A + B. (b) channel B got into `AwaitingRAA` due to unrelated operations, (c) the MPP payment was claimed, with async monitor updating, (d) the `revoke_and_ack` we were waiting on was delivered, but the resulting `ChannelMonitorUpdate` was blocked due to the pending claim having inserted an RAA-blocking action, (e) the preimage `ChannelMonitorUpdate` generated for channel B completed persistence, which did nothing due to the blocked `ChannelMonitorUpdate`. (f) the `Event::PaymentClaimed` event was handled but it, too, failed to unblock the channel. Instead, here, we simply process post-update actions when an update completes, even if there are pending blocked updates. We do not fully unblock the channel, of course. Backport of 8f4a4d239d0440f2bfb1e425724ecaf4d94da7d9 Conflicts due to upstream macro rewrite resolved in: * lightning/src/ln/channelmanager.rs --- lightning/src/ln/chanmon_update_fail_tests.rs | 117 +++++++++- lightning/src/ln/channelmanager.rs | 208 ++++++++++-------- lightning/src/ln/functional_test_utils.rs | 135 ++++++------ 3 files changed, 298 insertions(+), 162 deletions(-) diff --git a/lightning/src/ln/chanmon_update_fail_tests.rs b/lightning/src/ln/chanmon_update_fail_tests.rs index 1a9af4f2071..96ae8f922ac 100644 --- a/lightning/src/ln/chanmon_update_fail_tests.rs +++ b/lightning/src/ln/chanmon_update_fail_tests.rs @@ -19,12 +19,13 @@ use crate::chain::transaction::OutPoint; use crate::chain::{ChannelMonitorUpdateStatus, Listen, Watch}; use crate::events::{ClosureReason, Event, HTLCHandlingFailureType, PaymentPurpose}; use crate::ln::channel::AnnouncementSigsState; -use crate::ln::channelmanager::{PaymentId, RAACommitmentOrder, RecipientOnionFields}; +use crate::ln::channelmanager::{PaymentId, RAACommitmentOrder, RecipientOnionFields, Retry}; use crate::ln::msgs; use crate::ln::msgs::{ BaseMessageHandler, ChannelMessageHandler, MessageSendEvent, RoutingMessageHandler, }; use crate::ln::types::ChannelId; +use crate::routing::router::{PaymentParameters, RouteParameters}; use crate::sign::NodeSigner; use crate::util::native_async::FutureQueue; use crate::util::persist::{ @@ -3542,7 +3543,7 @@ fn do_test_blocked_chan_preimage_release(completion_mode: BlockedUpdateComplMode assert!(a.is_none()); nodes[1].node.handle_revoke_and_ack(node_a_id, &raa); - check_added_monitors(&nodes[1], 0); + check_added_monitors(&nodes[1], 1); assert!(nodes[1].node.get_and_clear_pending_msg_events().is_empty()); } @@ -3561,8 +3562,8 @@ fn do_test_blocked_chan_preimage_release(completion_mode: BlockedUpdateComplMode panic!(); } - // The event processing should release the last RAA updates on both channels. - check_added_monitors(&nodes[1], 2); + // The event processing should release the last RAA update. + check_added_monitors(&nodes[1], 1); // When we fetch the next update the message getter will generate the next update for nodes[2], // generating a further monitor update. @@ -5058,3 +5059,111 @@ fn native_async_persist() { panic!(); } } + +#[test] +fn test_mpp_claim_to_holding_cell() { + // Previously, if an MPP payment was claimed while one channel was AwaitingRAA (causing the + // HTLC claim to go into the holding cell), and the RAA came in before the async monitor + // update with the preimage completed, the channel could hang waiting on itself. + // This tests that behavior. + let chanmon_cfgs = create_chanmon_cfgs(4); + let node_cfgs = create_node_cfgs(4, &chanmon_cfgs); + let node_chanmgrs = create_node_chanmgrs(4, &node_cfgs, &[None, None, None, None]); + let nodes = create_network(4, &node_cfgs, &node_chanmgrs); + + let node_b_id = nodes[1].node.get_our_node_id(); + let node_c_id = nodes[2].node.get_our_node_id(); + let node_d_id = nodes[3].node.get_our_node_id(); + + // First open channels in a diamond and deliver the MPP payment. + let chan_1_scid = create_announced_chan_between_nodes(&nodes, 0, 1).0.contents.short_channel_id; + let chan_2_scid = create_announced_chan_between_nodes(&nodes, 0, 2).0.contents.short_channel_id; + let (chan_3_update, _, chan_3_id, ..) = create_announced_chan_between_nodes(&nodes, 1, 3); + let chan_3_scid = chan_3_update.contents.short_channel_id; + let (chan_4_update, _, chan_4_id, ..) = create_announced_chan_between_nodes(&nodes, 2, 3); + let chan_4_scid = chan_4_update.contents.short_channel_id; + + let (mut route, paymnt_hash_1, preimage_1, payment_secret) = + get_route_and_payment_hash!(&nodes[0], nodes[3], 500_000); + let path = route.paths[0].clone(); + route.paths.push(path); + route.paths[0].hops[0].pubkey = node_b_id; + route.paths[0].hops[0].short_channel_id = chan_1_scid; + route.paths[0].hops[1].short_channel_id = chan_3_scid; + route.paths[0].hops[1].fee_msat = 250_000; + route.paths[1].hops[0].pubkey = node_c_id; + route.paths[1].hops[0].short_channel_id = chan_2_scid; + route.paths[1].hops[1].short_channel_id = chan_4_scid; + route.paths[1].hops[1].fee_msat = 250_000; + let paths = &[&[&nodes[1], &nodes[3]][..], &[&nodes[2], &nodes[3]][..]]; + send_along_route_with_secret(&nodes[0], route, paths, 500_000, paymnt_hash_1, payment_secret); + + // Put the C <-> D channel into AwaitingRaa + let (preimage_2, paymnt_hash_2, payment_secret_2) = get_payment_preimage_hash!(nodes[3]); + let onion = RecipientOnionFields::secret_only(payment_secret_2); + let id = PaymentId([42; 32]); + let pay_params = PaymentParameters::from_node_id(node_d_id, TEST_FINAL_CLTV); + let route_params = RouteParameters::from_payment_params_and_value(pay_params, 400_000); + nodes[2].node.send_payment(paymnt_hash_2, onion, id, route_params, Retry::Attempts(0)).unwrap(); + check_added_monitors(&nodes[2], 1); + + let mut payment_event = SendEvent::from_node(&nodes[2]); + nodes[3].node.handle_update_add_htlc(node_c_id, &payment_event.msgs[0]); + nodes[3].node.handle_commitment_signed_batch_test(node_c_id, &payment_event.commitment_msg); + check_added_monitors(&nodes[3], 1); + + let (raa, cs) = get_revoke_commit_msgs(&nodes[3], &node_c_id); + nodes[2].node.handle_revoke_and_ack(node_d_id, &raa); + check_added_monitors(&nodes[2], 1); + + nodes[2].node.handle_commitment_signed_batch_test(node_d_id, &cs); + check_added_monitors(&nodes[2], 1); + + let cs_raa = get_event_msg!(nodes[2], MessageSendEvent::SendRevokeAndACK, node_d_id); + + // Now claim the payment, completing both channel monitor updates async + // In the current code, the C <-> D channel happens to be the `durable_preimage_channel`, + // improving coverage somewhat but it isn't strictly critical to the test. + chanmon_cfgs[3].persister.set_update_ret(ChannelMonitorUpdateStatus::InProgress); + chanmon_cfgs[3].persister.set_update_ret(ChannelMonitorUpdateStatus::InProgress); + nodes[3].node.claim_funds(preimage_1); + check_added_monitors(&nodes[3], 2); + + // Complete the B <-> D monitor update, freeing the first fulfill. + let (latest_id, _) = get_latest_mon_update_id(&nodes[3], chan_3_id); + nodes[3].chain_monitor.chain_monitor.channel_monitor_updated(chan_3_id, latest_id).unwrap(); + let mut b_claim = get_htlc_update_msgs(&nodes[3], &node_b_id); + + // When we deliver the pre-claim RAA, node D will shove the monitor update into the blocked + // state since we have a pending MPP payment which is blocking RAA monitor updates. + nodes[3].node.handle_revoke_and_ack(node_c_id, &cs_raa); + check_added_monitors(&nodes[3], 0); + + // Finally, complete the C <-> D monitor update. Previously, this unlock failed to be processed + // due to the existence of the blocked RAA update above. + let (latest_id, _) = get_latest_mon_update_id(&nodes[3], chan_4_id); + nodes[3].chain_monitor.chain_monitor.channel_monitor_updated(chan_4_id, latest_id).unwrap(); + // Once we process monitor events (in this case by checking for the `PaymentClaimed` event, the + // RAA monitor update blocked above will be released. + expect_payment_claimed!(nodes[3], paymnt_hash_1, 500_000); + check_added_monitors(&nodes[3], 1); + + // After the RAA monitor update completes, the C <-> D channel will be able to generate its + // fulfill updates as well. + let mut c_claim = get_htlc_update_msgs(&nodes[3], &node_c_id); + check_added_monitors(&nodes[3], 1); + + // Finally, clear all the pending payments. + let path = [&[&nodes[1], &nodes[3]][..], &[&nodes[2], &nodes[3]][..]]; + let mut args = ClaimAlongRouteArgs::new(&nodes[0], &path[..], preimage_1); + let b_claim_msgs = (b_claim.update_fulfill_htlcs.pop().unwrap(), b_claim.commitment_signed); + let c_claim_msgs = (c_claim.update_fulfill_htlcs.pop().unwrap(), c_claim.commitment_signed); + let claims = vec![(b_claim_msgs, node_b_id), (c_claim_msgs, node_c_id)]; + pass_claimed_payment_along_route_from_ev(250_000, claims, args); + + expect_payment_sent(&nodes[0], preimage_1, None, true, true); + + expect_and_process_pending_htlcs(&nodes[3], false); + expect_payment_claimable!(nodes[3], paymnt_hash_2, payment_secret_2, 400_000); + claim_payment(&nodes[2], &[&nodes[3]], preimage_2); +} diff --git a/lightning/src/ln/channelmanager.rs b/lightning/src/ln/channelmanager.rs index 8cd1e61ab18..8743a3ac75a 100644 --- a/lightning/src/ln/channelmanager.rs +++ b/lightning/src/ln/channelmanager.rs @@ -1315,6 +1315,10 @@ impl MaybeReadable for EventUnblockedChannel { } #[derive(Debug)] +/// Note that these run after all *non-blocked* [`ChannelMonitorUpdate`]s have been persisted. +/// Thus, they're primarily useful for (and currently only used for) claims, where the +/// [`ChannelMonitorUpdate`] we care about is a preimage update, which bypass the monitor update +/// blocking logic entirely and can never be blocked. pub(crate) enum MonitorUpdateCompletionAction { /// Indicates that a payment ultimately destined for us was claimed and we should emit an /// [`events::Event::PaymentClaimed`] to the user if we haven't yet generated such an event for @@ -1599,6 +1603,11 @@ where /// same `temporary_channel_id` (or final `channel_id` in the case of 0conf channels or prior /// to funding appearing on-chain), the downstream `ChannelMonitor` set is required to ensure /// duplicates do not occur, so such channels should fail without a monitor update completing. + /// + /// Note that these run after all *non-blocked* [`ChannelMonitorUpdate`]s have been persisted. + /// Thus, they're primarily useful for (and currently only used for) claims, where the + /// [`ChannelMonitorUpdate`] we care about is a preimage update, which bypass the monitor + /// update blocking logic entirely and can never be blocked. monitor_update_blocked_actions: BTreeMap>, /// If another channel's [`ChannelMonitorUpdate`] needs to complete before a channel we have /// with this peer can complete an RAA [`ChannelMonitorUpdate`] (e.g. because the RAA update @@ -3537,120 +3546,132 @@ macro_rules! emit_initial_channel_ready_event { /// You should not add new direct calls to this, generally, rather rely on /// `handle_new_monitor_update` or [`ChannelManager::channel_monitor_updated`] to call it for you. /// -/// Requires that `$chan.blocked_monitor_updates_pending() == 0` and the in-flight monitor update -/// set for this channel is empty! +/// Requires that the in-flight monitor update set for this channel is empty! macro_rules! handle_monitor_update_completion { - ($self: ident, $peer_state_lock: expr, $peer_state: expr, $per_peer_state_lock: expr, $chan: expr) => { { + ($self: ident, $peer_state_lock: expr, $peer_state: expr, $per_peer_state_lock: expr, $chan: expr) => {{ let channel_id = $chan.context.channel_id(); let outbound_scid_alias = $chan.context().outbound_scid_alias(); let counterparty_node_id = $chan.context.get_counterparty_node_id(); + #[cfg(debug_assertions)] { let in_flight_updates = $peer_state.in_flight_monitor_updates.get(&channel_id); assert!(in_flight_updates.map(|(_, updates)| updates.is_empty()).unwrap_or(true)); - assert_eq!($chan.blocked_monitor_updates_pending(), 0); + assert!($chan.is_awaiting_monitor_update()); } + let logger = WithChannelContext::from(&$self.logger, &$chan.context, None); - let mut updates = $chan.monitor_updating_restored(&&logger, - &$self.node_signer, $self.chain_hash, &*$self.config.read().unwrap(), - $self.best_block.read().unwrap().height, - |htlc_id| $self.path_for_release_held_htlc(htlc_id, outbound_scid_alias, &channel_id, &counterparty_node_id)); - let channel_update = if updates.channel_ready.is_some() - && $chan.context.is_usable() - && $peer_state.is_connected - { - // We only send a channel_update in the case where we are just now sending a - // channel_ready and the channel is in a usable state. We may re-send a - // channel_update later through the announcement_signatures process for public - // channels, but there's no reason not to just inform our counterparty of our fees - // now. - if let Ok(msg) = $self.get_channel_update_for_unicast($chan) { - Some(MessageSendEvent::SendChannelUpdate { - node_id: counterparty_node_id, - msg, - }) - } else { None } - } else { None }; let update_actions = $peer_state.monitor_update_blocked_actions .remove(&channel_id).unwrap_or(Vec::new()); - let (htlc_forwards, decode_update_add_htlcs) = $self.handle_channel_resumption( - &mut $peer_state.pending_msg_events, $chan, updates.raa, - updates.commitment_update, updates.commitment_order, updates.accepted_htlcs, - updates.pending_update_adds, updates.funding_broadcastable, updates.channel_ready, - updates.announcement_sigs, updates.tx_signatures, None, updates.channel_ready_order, - ); - if let Some(upd) = channel_update { - $peer_state.pending_msg_events.push(upd); - } - - let unbroadcasted_batch_funding_txid = $chan.context.unbroadcasted_batch_funding_txid(&$chan.funding); - core::mem::drop($peer_state_lock); - core::mem::drop($per_peer_state_lock); - - // If the channel belongs to a batch funding transaction, the progress of the batch - // should be updated as we have received funding_signed and persisted the monitor. - if let Some(txid) = unbroadcasted_batch_funding_txid { - let mut funding_batch_states = $self.funding_batch_states.lock().unwrap(); - let mut batch_completed = false; - if let Some(batch_state) = funding_batch_states.get_mut(&txid) { - let channel_state = batch_state.iter_mut().find(|(chan_id, pubkey, _)| ( - *chan_id == channel_id && - *pubkey == counterparty_node_id - )); - if let Some(channel_state) = channel_state { - channel_state.2 = true; + if $chan.blocked_monitor_updates_pending() != 0 { + mem::drop($peer_state_lock); + mem::drop($per_peer_state_lock); + + log_debug!(logger, "Channel has blocked monitor updates, completing update actions but leaving channel blocked"); + $self.handle_monitor_update_completion_actions(update_actions); + } else { + log_debug!(logger, "Channel is open and awaiting update, resuming it"); + let mut updates = $chan.monitor_updating_restored(&&logger, + &$self.node_signer, $self.chain_hash, &*$self.config.read().unwrap(), + $self.best_block.read().unwrap().height, + |htlc_id| $self.path_for_release_held_htlc(htlc_id, outbound_scid_alias, &channel_id, &counterparty_node_id)); + let channel_update = if updates.channel_ready.is_some() + && $chan.context.is_usable() + && $peer_state.is_connected + { + // We only send a channel_update in the case where we are just now sending a + // channel_ready and the channel is in a usable state. We may re-send a + // channel_update later through the announcement_signatures process for public + // channels, but there's no reason not to just inform our counterparty of our fees + // now. + if let Ok(msg) = $self.get_channel_update_for_unicast($chan) { + Some(MessageSendEvent::SendChannelUpdate { + node_id: counterparty_node_id, + msg, + }) + } else { None } + } else { None }; + + let (htlc_forwards, decode_update_add_htlcs) = $self.handle_channel_resumption( + &mut $peer_state.pending_msg_events, $chan, updates.raa, + updates.commitment_update, updates.commitment_order, updates.accepted_htlcs, + updates.pending_update_adds, updates.funding_broadcastable, updates.channel_ready, + updates.announcement_sigs, updates.tx_signatures, None, updates.channel_ready_order, + ); + if let Some(upd) = channel_update { + $peer_state.pending_msg_events.push(upd); + } + + let unbroadcasted_batch_funding_txid = $chan.context.unbroadcasted_batch_funding_txid(&$chan.funding); + core::mem::drop($peer_state_lock); + core::mem::drop($per_peer_state_lock); + + + // If the channel belongs to a batch funding transaction, the progress of the batch + // should be updated as we have received funding_signed and persisted the monitor. + if let Some(txid) = unbroadcasted_batch_funding_txid { + let mut funding_batch_states = $self.funding_batch_states.lock().unwrap(); + let mut batch_completed = false; + if let Some(batch_state) = funding_batch_states.get_mut(&txid) { + let channel_state = batch_state.iter_mut().find(|(chan_id, pubkey, _)| ( + *chan_id == channel_id && + *pubkey == counterparty_node_id + )); + if let Some(channel_state) = channel_state { + channel_state.2 = true; + } else { + debug_assert!(false, "Missing channel batch state for channel which completed initial monitor update"); + } + batch_completed = batch_state.iter().all(|(_, _, completed)| *completed); } else { - debug_assert!(false, "Missing channel batch state for channel which completed initial monitor update"); + debug_assert!(false, "Missing batch state for channel which completed initial monitor update"); } - batch_completed = batch_state.iter().all(|(_, _, completed)| *completed); - } else { - debug_assert!(false, "Missing batch state for channel which completed initial monitor update"); - } - // When all channels in a batched funding transaction have become ready, it is not necessary - // to track the progress of the batch anymore and the state of the channels can be updated. - if batch_completed { - let removed_batch_state = funding_batch_states.remove(&txid).into_iter().flatten(); - let per_peer_state = $self.per_peer_state.read().unwrap(); - let mut batch_funding_tx = None; - for (channel_id, counterparty_node_id, _) in removed_batch_state { - if let Some(peer_state_mutex) = per_peer_state.get(&counterparty_node_id) { - let mut peer_state = peer_state_mutex.lock().unwrap(); - if let Some(funded_chan) = peer_state.channel_by_id - .get_mut(&channel_id) - .and_then(Channel::as_funded_mut) - { - batch_funding_tx = batch_funding_tx.or_else(|| funded_chan.context.unbroadcasted_funding(&funded_chan.funding)); - funded_chan.set_batch_ready(); - let mut pending_events = $self.pending_events.lock().unwrap(); - emit_channel_pending_event!(pending_events, funded_chan); + // When all channels in a batched funding transaction have become ready, it is not necessary + // to track the progress of the batch anymore and the state of the channels can be updated. + if batch_completed { + let removed_batch_state = funding_batch_states.remove(&txid).into_iter().flatten(); + let per_peer_state = $self.per_peer_state.read().unwrap(); + let mut batch_funding_tx = None; + for (channel_id, counterparty_node_id, _) in removed_batch_state { + if let Some(peer_state_mutex) = per_peer_state.get(&counterparty_node_id) { + let mut peer_state = peer_state_mutex.lock().unwrap(); + if let Some(funded_chan) = peer_state.channel_by_id + .get_mut(&channel_id) + .and_then(Channel::as_funded_mut) + { + batch_funding_tx = batch_funding_tx.or_else(|| funded_chan.context.unbroadcasted_funding(&funded_chan.funding)); + funded_chan.set_batch_ready(); + let mut pending_events = $self.pending_events.lock().unwrap(); + emit_channel_pending_event!(pending_events, funded_chan); + } } } - } - if let Some(tx) = batch_funding_tx { - log_info!($self.logger, "Broadcasting batch funding transaction with txid {}", tx.compute_txid()); - $self.tx_broadcaster.broadcast_transactions(&[&tx]); + if let Some(tx) = batch_funding_tx { + log_info!($self.logger, "Broadcasting batch funding transaction with txid {}", tx.compute_txid()); + $self.tx_broadcaster.broadcast_transactions(&[&tx]); + } } } - } - $self.handle_monitor_update_completion_actions(update_actions); + $self.handle_monitor_update_completion_actions(update_actions); - if let Some(forwards) = htlc_forwards { - $self.forward_htlcs(&mut [forwards][..]); - } - if let Some(decode) = decode_update_add_htlcs { - $self.push_decode_update_add_htlcs(decode); - } - $self.finalize_claims(updates.finalized_claimed_htlcs); - for failure in updates.failed_htlcs.drain(..) { - let receiver = HTLCHandlingFailureType::Forward { node_id: Some(counterparty_node_id), channel_id }; - $self.fail_htlc_backwards_internal(&failure.0, &failure.1, &failure.2, receiver, None); + if let Some(forwards) = htlc_forwards { + $self.forward_htlcs(&mut [forwards][..]); + } + if let Some(decode) = decode_update_add_htlcs { + $self.push_decode_update_add_htlcs(decode); + } + $self.finalize_claims(updates.finalized_claimed_htlcs); + for failure in updates.failed_htlcs.drain(..) { + let receiver = HTLCHandlingFailureType::Forward { node_id: Some(counterparty_node_id), channel_id }; + $self.fail_htlc_backwards_internal(&failure.0, &failure.1, &failure.2, receiver, None); + } } - } } + }}; } macro_rules! handle_new_monitor_update { @@ -3776,7 +3797,7 @@ macro_rules! handle_new_monitor_update { counterparty_node_id, in_flight_updates, idx, _internal_outer, { let _ = in_flight_updates.remove(idx); - if in_flight_updates.is_empty() && $chan.blocked_monitor_updates_pending() == 0 { + if in_flight_updates.is_empty() { handle_monitor_update_completion!($self, $peer_state_lock, $peer_state, $per_peer_state_lock, $chan); } }) @@ -9710,12 +9731,7 @@ This indicates a bug inside LDK. Please report this error at https://github.com/ .and_then(Channel::as_funded_mut) { if chan.is_awaiting_monitor_update() { - if chan.blocked_monitor_updates_pending() == 0 { - log_trace!(logger, "Channel is open and awaiting update, resuming it"); - handle_monitor_update_completion!(self, peer_state_lock, peer_state, per_peer_state, chan); - } else { - log_trace!(logger, "Channel is open and awaiting update, leaving it blocked due to a blocked monitor update"); - } + handle_monitor_update_completion!(self, peer_state_lock, peer_state, per_peer_state, chan); } else { log_trace!(logger, "Channel is open but not awaiting update"); } diff --git a/lightning/src/ln/functional_test_utils.rs b/lightning/src/ln/functional_test_utils.rs index b6db7d23484..7af632e0351 100644 --- a/lightning/src/ln/functional_test_utils.rs +++ b/lightning/src/ln/functional_test_utils.rs @@ -1109,7 +1109,7 @@ pub fn get_htlc_update_msgs(node: &Node, recipient: &PublicKey) -> msgs::Commitm assert_eq!(node_id, recipient); (*updates).clone() }, - _ => panic!("Unexpected event"), + _ => panic!("Unexpected event {events:?}"), } } @@ -3042,9 +3042,9 @@ pub fn expect_payment_sent>( bitcoin::hashes::sha256::Hash::hash(&expected_payment_preimage.0).to_byte_array(), ); if expect_per_path_claims { - assert!(events.len() > 1); + assert!(events.len() > 1, "{events:?}"); } else { - assert_eq!(events.len(), 1); + assert_eq!(events.len(), 1, "{events:?}"); } if expect_post_ev_mon_update { check_added_monitors(node, 1); @@ -3877,19 +3877,38 @@ impl<'a, 'b, 'c, 'd> ClaimAlongRouteArgs<'a, 'b, 'c, 'd> { } } +macro_rules! single_fulfill_commit_from_ev { + ($ev: expr) => { + match $ev { + &MessageSendEvent::UpdateHTLCs { + ref node_id, + ref channel_id, + updates: + msgs::CommitmentUpdate { + ref update_add_htlcs, + ref update_fulfill_htlcs, + ref update_fail_htlcs, + ref update_fail_malformed_htlcs, + ref update_fee, + ref commitment_signed, + }, + } => { + assert!(update_add_htlcs.is_empty()); + assert_eq!(update_fulfill_htlcs.len(), 1); + assert!(update_fail_htlcs.is_empty()); + assert!(update_fail_malformed_htlcs.is_empty()); + assert!(update_fee.is_none()); + assert!(commitment_signed.iter().all(|cs| cs.channel_id == *channel_id)); + ((update_fulfill_htlcs[0].clone(), commitment_signed.clone()), node_id.clone()) + }, + _ => panic!("Unexpected event"), + } + }; +} + pub fn pass_claimed_payment_along_route(args: ClaimAlongRouteArgs) -> u64 { - let ClaimAlongRouteArgs { - origin_node, - expected_paths, - expected_extra_fees, - expected_min_htlc_overpay, - skip_last, - payment_preimage: our_payment_preimage, - allow_1_msat_fee_overpay, - custom_tlvs, - } = args; - let claim_event = expected_paths[0].last().unwrap().node.get_and_clear_pending_events(); - assert_eq!(claim_event.len(), 1); + let claim_event = args.expected_paths[0].last().unwrap().node.get_and_clear_pending_events(); + assert_eq!(claim_event.len(), 1, "{claim_event:?}"); #[allow(unused)] let mut fwd_amt_msat = 0; match claim_event[0] { @@ -3904,11 +3923,11 @@ pub fn pass_claimed_payment_along_route(args: ClaimAlongRouteArgs) -> u64 { ref onion_fields, .. } => { - assert_eq!(preimage, our_payment_preimage); - assert_eq!(htlcs.len(), expected_paths.len()); // One per path. + assert_eq!(preimage, args.payment_preimage); + assert_eq!(htlcs.len(), args.expected_paths.len()); // One per path. assert_eq!(htlcs.iter().map(|h| h.value_msat).sum::(), amount_msat); - assert_eq!(onion_fields.as_ref().unwrap().custom_tlvs, custom_tlvs); - check_claimed_htlcs_match_route(origin_node, expected_paths, htlcs); + assert_eq!(onion_fields.as_ref().unwrap().custom_tlvs, args.custom_tlvs); + check_claimed_htlcs_match_route(args.origin_node, args.expected_paths, htlcs); fwd_amt_msat = amount_msat; }, Event::PaymentClaimed { @@ -3922,59 +3941,29 @@ pub fn pass_claimed_payment_along_route(args: ClaimAlongRouteArgs) -> u64 { ref onion_fields, .. } => { - assert_eq!(&payment_hash.0, &Sha256::hash(&our_payment_preimage.0)[..]); - assert_eq!(htlcs.len(), expected_paths.len()); // One per path. + assert_eq!(&payment_hash.0, &Sha256::hash(&args.payment_preimage.0)[..]); + assert_eq!(htlcs.len(), args.expected_paths.len()); // One per path. assert_eq!(htlcs.iter().map(|h| h.value_msat).sum::(), amount_msat); - assert_eq!(onion_fields.as_ref().unwrap().custom_tlvs, custom_tlvs); - check_claimed_htlcs_match_route(origin_node, expected_paths, htlcs); + assert_eq!(onion_fields.as_ref().unwrap().custom_tlvs, args.custom_tlvs); + check_claimed_htlcs_match_route(args.origin_node, args.expected_paths, htlcs); fwd_amt_msat = amount_msat; }, _ => panic!(), } - check_added_monitors!(expected_paths[0].last().unwrap(), expected_paths.len()); - - let mut expected_total_fee_msat = 0; + check_added_monitors(args.expected_paths[0].last().unwrap(), args.expected_paths.len()); - macro_rules! msgs_from_ev { - ($ev: expr) => { - match $ev { - &MessageSendEvent::UpdateHTLCs { - ref node_id, - ref channel_id, - updates: - msgs::CommitmentUpdate { - ref update_add_htlcs, - ref update_fulfill_htlcs, - ref update_fail_htlcs, - ref update_fail_malformed_htlcs, - ref update_fee, - ref commitment_signed, - }, - } => { - assert!(update_add_htlcs.is_empty()); - assert_eq!(update_fulfill_htlcs.len(), 1); - assert!(update_fail_htlcs.is_empty()); - assert!(update_fail_malformed_htlcs.is_empty()); - assert!(update_fee.is_none()); - assert!(commitment_signed.iter().all(|cs| cs.channel_id == *channel_id)); - ((update_fulfill_htlcs[0].clone(), commitment_signed.clone()), node_id.clone()) - }, - _ => panic!("Unexpected event"), - } - }; - } let mut per_path_msgs: Vec<( (msgs::UpdateFulfillHTLC, Vec), PublicKey, - )> = Vec::with_capacity(expected_paths.len()); - let mut events = expected_paths[0].last().unwrap().node.get_and_clear_pending_msg_events(); - assert_eq!(events.len(), expected_paths.len()); + )> = Vec::with_capacity(args.expected_paths.len()); + let mut events = args.expected_paths[0].last().unwrap().node.get_and_clear_pending_msg_events(); + assert_eq!(events.len(), args.expected_paths.len()); if events.len() == 1 { - per_path_msgs.push(msgs_from_ev!(&events[0])); + per_path_msgs.push(single_fulfill_commit_from_ev!(&events[0])); } else { - for expected_path in expected_paths.iter() { + for expected_path in args.expected_paths.iter() { // For MPP payments, we want the fulfill message from the payee to the penultimate hop in the // path. let penultimate_hop_node_id = expected_path @@ -3983,12 +3972,34 @@ pub fn pass_claimed_payment_along_route(args: ClaimAlongRouteArgs) -> u64 { .skip(1) .next() .map(|n| n.node.get_our_node_id()) - .unwrap_or(origin_node.node.get_our_node_id()); + .unwrap_or(args.origin_node.node.get_our_node_id()); let ev = remove_first_msg_event_to_node(&penultimate_hop_node_id, &mut events); - per_path_msgs.push(msgs_from_ev!(&ev)); + per_path_msgs.push(single_fulfill_commit_from_ev!(&ev)); } } + pass_claimed_payment_along_route_from_ev(fwd_amt_msat, per_path_msgs, args) +} + +pub fn pass_claimed_payment_along_route_from_ev( + each_htlc_claim_amt_msat: u64, + mut per_path_msgs: Vec<((msgs::UpdateFulfillHTLC, Vec), PublicKey)>, + args: ClaimAlongRouteArgs, +) -> u64 { + let ClaimAlongRouteArgs { + origin_node, + expected_paths, + expected_extra_fees, + expected_min_htlc_overpay, + skip_last, + payment_preimage: our_payment_preimage, + allow_1_msat_fee_overpay, + .. + } = args; + + let mut fwd_amt_msat = each_htlc_claim_amt_msat; + let mut expected_total_fee_msat = 0; + for (i, (expected_route, (path_msgs, next_hop))) in expected_paths.iter().zip(per_path_msgs.drain(..)).enumerate() { @@ -4069,7 +4080,7 @@ pub fn pass_claimed_payment_along_route(args: ClaimAlongRouteArgs) -> u64 { let new_next_msgs = if $new_msgs { let events = $node.node.get_and_clear_pending_msg_events(); assert_eq!(events.len(), 1); - let (res, nexthop) = msgs_from_ev!(&events[0]); + let (res, nexthop) = single_fulfill_commit_from_ev!(&events[0]); expected_next_node = nexthop; Some(res) } else { From 4619e242681ec6626a702cd36977b9b74fe46293 Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 2 Dec 2025 15:12:40 +0000 Subject: [PATCH 2/6] Apply some upstream rustfmt in `handle_monitor_update_completion` This aligns `handle_monitor_update_completion` closer with upstream after backporting 8f4a4d239d0440f2bfb1e425724ecaf4d94da7d9, making potential future backports somewhat simpler. --- lightning/src/ln/channelmanager.rs | 46 ++++++++++++++++++++---------- 1 file changed, 31 insertions(+), 15 deletions(-) diff --git a/lightning/src/ln/channelmanager.rs b/lightning/src/ln/channelmanager.rs index 8743a3ac75a..82a45dcb1a6 100644 --- a/lightning/src/ln/channelmanager.rs +++ b/lightning/src/ln/channelmanager.rs @@ -3574,10 +3574,16 @@ macro_rules! handle_monitor_update_completion { $self.handle_monitor_update_completion_actions(update_actions); } else { log_debug!(logger, "Channel is open and awaiting update, resuming it"); - let mut updates = $chan.monitor_updating_restored(&&logger, - &$self.node_signer, $self.chain_hash, &*$self.config.read().unwrap(), + let mut updates = $chan.monitor_updating_restored( + &&logger, + &$self.node_signer, + $self.chain_hash, + &*$self.config.read().unwrap(), $self.best_block.read().unwrap().height, - |htlc_id| $self.path_for_release_held_htlc(htlc_id, outbound_scid_alias, &channel_id, &counterparty_node_id)); + |htlc_id| { + $self.path_for_release_held_htlc(htlc_id, outbound_scid_alias, &channel_id, &counterparty_node_id) + }, + ); let channel_update = if updates.channel_ready.is_some() && $chan.context.is_usable() && $peer_state.is_connected @@ -3588,28 +3594,38 @@ macro_rules! handle_monitor_update_completion { // channels, but there's no reason not to just inform our counterparty of our fees // now. if let Ok(msg) = $self.get_channel_update_for_unicast($chan) { - Some(MessageSendEvent::SendChannelUpdate { - node_id: counterparty_node_id, - msg, - }) - } else { None } - } else { None }; + Some(MessageSendEvent::SendChannelUpdate { node_id: counterparty_node_id, msg }) + } else { + None + } + } else { + None + }; let (htlc_forwards, decode_update_add_htlcs) = $self.handle_channel_resumption( - &mut $peer_state.pending_msg_events, $chan, updates.raa, - updates.commitment_update, updates.commitment_order, updates.accepted_htlcs, - updates.pending_update_adds, updates.funding_broadcastable, updates.channel_ready, - updates.announcement_sigs, updates.tx_signatures, None, updates.channel_ready_order, + &mut $peer_state.pending_msg_events, + $chan, + updates.raa, + updates.commitment_update, + updates.commitment_order, + updates.accepted_htlcs, + updates.pending_update_adds, + updates.funding_broadcastable, + updates.channel_ready, + updates.announcement_sigs, + updates.tx_signatures, + None, + updates.channel_ready_order, ); if let Some(upd) = channel_update { $peer_state.pending_msg_events.push(upd); } - let unbroadcasted_batch_funding_txid = $chan.context.unbroadcasted_batch_funding_txid(&$chan.funding); + let unbroadcasted_batch_funding_txid = + $chan.context.unbroadcasted_batch_funding_txid(&$chan.funding); core::mem::drop($peer_state_lock); core::mem::drop($per_peer_state_lock); - // If the channel belongs to a batch funding transaction, the progress of the batch // should be updated as we have received funding_signed and persisted the monitor. if let Some(txid) = unbroadcasted_batch_funding_txid { From 74b84a5ec5cd5498802f43267255a30aa0b1f430 Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Mon, 1 Dec 2025 01:21:01 +0000 Subject: [PATCH 3/6] Update docs for `ChannelDetails` fields which change on splice Various fields in `ChannelDetails` refer to channel information which changes on splice, which we ensure is consistently documented here. Backport of c3cc331daaea5e1061f639df597c6740f3453331 --- lightning/src/ln/channel_state.rs | 42 ++++++++++++++++++++++++++----- 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/lightning/src/ln/channel_state.rs b/lightning/src/ln/channel_state.rs index 81a7cb4755e..d10327b259a 100644 --- a/lightning/src/ln/channel_state.rs +++ b/lightning/src/ln/channel_state.rs @@ -275,14 +275,19 @@ impl_writeable_tlv_based!(ChannelCounterparty, { /// [`ChannelMonitor::get_claimable_balances`], calculated with respect to the corresponding on-chain /// transactions. /// +/// When a channel is spliced, most fields continue to refer to the original pre-splice channel +/// state until the splice transaction reaches sufficient confirmations to be locked (and we +/// exchange `splice_locked` messages with our peer). See individual fields for details. +/// /// [`ChannelManager::list_channels`]: crate::ln::channelmanager::ChannelManager::list_channels /// [`ChannelManager::list_usable_channels`]: crate::ln::channelmanager::ChannelManager::list_usable_channels /// [`ChainMonitor::get_claimable_balances`]: crate::chain::chainmonitor::ChainMonitor::get_claimable_balances /// [`ChannelMonitor::get_claimable_balances`]: crate::chain::channelmonitor::ChannelMonitor::get_claimable_balances #[derive(Clone, Debug, PartialEq)] pub struct ChannelDetails { - /// The channel's ID (prior to funding transaction generation, this is a random 32 bytes, - /// thereafter this is the txid of the funding transaction xor the funding transaction output). + /// The channel's ID (prior to initial channel setup this is a random 32 bytes, thereafter it is + /// derived from channel funding or key material). + /// /// Note that this means this value is *not* persistent - it can change once during the /// lifetime of the channel. pub channel_id: ChannelId, @@ -290,10 +295,18 @@ pub struct ChannelDetails { pub counterparty: ChannelCounterparty, /// The Channel's funding transaction output, if we've negotiated the funding transaction with /// our counterparty already. + /// + /// When a channel is spliced, this continues to refer to the original pre-splice channel + /// state until the splice transaction reaches sufficient confirmations to be locked (and we + /// exchange `splice_locked` messages with our peer). pub funding_txo: Option, /// The features which this channel operates with. See individual features for more info. /// /// `None` until negotiation completes and the channel type is finalized. + /// + /// When a channel is spliced, this continues to refer to the original pre-splice channel + /// state until the splice transaction reaches sufficient confirmations to be locked (and we + /// exchange `splice_locked` messages with our peer). pub channel_type: Option, /// The position of the funding transaction in the chain. None if the funding transaction has /// not yet been confirmed and the channel fully opened. @@ -304,6 +317,10 @@ pub struct ChannelDetails { /// For channels with [`confirmations_required`] set to `Some(0)`, [`outbound_scid_alias`] may /// be used in place of this in outbound routes. See [`get_outbound_payment_scid`]. /// + /// When a channel is spliced, this continues to refer to the original pre-splice channel + /// state until the splice transaction reaches sufficient confirmations to be locked (and we + /// exchange `splice_locked` messages with our peer). + /// /// [`inbound_scid_alias`]: Self::inbound_scid_alias /// [`outbound_scid_alias`]: Self::outbound_scid_alias /// [`get_inbound_payment_scid`]: Self::get_inbound_payment_scid @@ -317,6 +334,10 @@ pub struct ChannelDetails { /// /// This will be `None` as long as the channel is not available for routing outbound payments. /// + /// When a channel is spliced, this continues to refer to the original pre-splice channel + /// state until the splice transaction reaches sufficient confirmations to be locked (and we + /// exchange `splice_locked` messages with our peer). + /// /// [`short_channel_id`]: Self::short_channel_id /// [`confirmations_required`]: Self::confirmations_required pub outbound_scid_alias: Option, @@ -331,6 +352,10 @@ pub struct ChannelDetails { /// [`short_channel_id`]: Self::short_channel_id pub inbound_scid_alias: Option, /// The value, in satoshis, of this channel as appears in the funding output + /// + /// When a channel is spliced, this continues to refer to the original pre-splice channel + /// state until the splice transaction reaches sufficient confirmations to be locked (and we + /// exchange `splice_locked` messages with our peer). pub channel_value_satoshis: u64, /// The value, in satoshis, that must always be held in the channel for us. This value ensures /// that if we broadcast a revoked state, our counterparty can punish us by claiming at least @@ -450,7 +475,12 @@ pub struct ChannelDetails { /// /// This field is empty for objects serialized with LDK versions prior to 0.0.122. pub pending_outbound_htlcs: Vec, - /// The witness script that is used to lock the channel's funding output to commitment transactions. + /// The witness script that is used to lock the channel's funding output to commitment + /// transactions. + /// + /// When a channel is spliced, this continues to refer to the original funding output (which + /// was spent by the splice transaction) until the splice transaction reached sufficient + /// confirmations to be locked (and we exchange `splice_locked` messages with our peer). /// /// This field will be `None` for objects serialized with LDK versions prior to 0.2.0. pub funding_redeem_script: Option, @@ -481,9 +511,9 @@ impl ChannelDetails { /// Gets the funding output for this channel, if available. /// - /// During a splice, the funding output will change and this value will be updated - /// after the splice transaction has reached sufficient confirmations and we've - /// exchanged `splice_locked` messages. + /// When a channel is spliced, this continues to refer to the original funding output (which + /// was spent by the splice transaction) until the splice transaction reaches sufficient + /// confirmations to be locked (and we exchange `splice_locked` messages with our peer). pub fn get_funding_output(&self) -> Option { match self.funding_redeem_script.as_ref() { None => None, From e8bd4792ac3fc9bcb678536771e0e20ca98764c1 Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 2 Dec 2025 15:38:39 +0000 Subject: [PATCH 4/6] Update CHANGELOG for 0.1.8 Forward-port of a0327577a25093a088252b39fee4cf93c7f17d40 from 0.1 --- CHANGELOG.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 04801919332..bfd064705e7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -180,6 +180,16 @@ TODO release stats +# 0.1.8 - Dec 2, 2025 - "Async Update Completion" + +## Bug Fixes + * In cases where an MPP payment is claimed while one channel is waiting on a + counterparty's `revoke_and_ack` message and the `revoke_and_ack` message is + received prior to the asynchronous completion of the MPP-claim + `ChannelMonitorUpdate`, the channel will no longer hang (#4236). + * Deserializing invalid `Duration`s can no longer panic (#4172). + + # 0.1.7 - Oct 21, 2025 - "Unstable Release CI" ## Bug Fixes From 66c1489e68b8d7cb2aa4ed26fcc78679020991dc Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 2 Dec 2025 15:25:10 +0000 Subject: [PATCH 5/6] Update CHANGELOG for 0.2 release --- CHANGELOG.md | 42 ++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index bfd064705e7..e14753538be 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.2 - TODO: Add release date, 2025 - "Natively Asynchronous Splicing" +# 0.2 - Dec 2, 2025 - "Natively Asynchronous Splicing" ## API Updates @@ -177,7 +177,45 @@ (#4045, #4046). * LDK now requires the `channel_type` feature in line with spec updates (#3896) -TODO release stats +In total, this release features 259 files changed, 114539 insertions, 45150 +deletions in 1628 commits since 0.1 from 36 authors, in alphabetical order: + + * Aditya Sharma + * Alec Chen + * Anonymous + * Anyitechs + * Arik Sosman + * Austin Mackillop + * Carla Kirk-Cohen + * Chuks Agbakuru + * Devrandom + * Duncan Dean + * Elias Rohrer + * Erick Cestari + * Fedeparma74 + * Fuyin + * Ian Slane + * Jeffrey Czyz + * Jesse de Wit + * Joost Jager + * Leo Nash + * Martin Saposnic + * Matt Corallo + * Matt Morehouse + * Maurice Poirrier Chuden + * Philip Kannegaard Hayes + * Prabhat Verma + * Valentine Wallace + * Vincenzo Palazzo + * Willem Van Lint + * Wilmer Paulino + * YI + * benthecarman + * elnosh + * moisesPompilio + * olegkubrakov + * optout + * shaavan # 0.1.8 - Dec 2, 2025 - "Async Update Completion" From e276b447a9747b975488ed5b84ec6f96e1d255e2 Mon Sep 17 00:00:00 2001 From: Matt Corallo Date: Tue, 2 Dec 2025 15:26:12 +0000 Subject: [PATCH 6/6] Bump crates to LDK 0.2/dns-resolver 0.3/invoice 0.34/types 0.3 --- lightning-background-processor/Cargo.toml | 16 ++++++++-------- lightning-block-sync/Cargo.toml | 6 +++--- lightning-custom-message/Cargo.toml | 4 ++-- lightning-dns-resolver/Cargo.toml | 8 ++++---- lightning-invoice/Cargo.toml | 4 ++-- lightning-liquidity/Cargo.toml | 16 ++++++++-------- lightning-macros/Cargo.toml | 2 +- lightning-net-tokio/Cargo.toml | 6 +++--- lightning-persister/Cargo.toml | 6 +++--- lightning-rapid-gossip-sync/Cargo.toml | 6 +++--- lightning-transaction-sync/Cargo.toml | 8 ++++---- lightning-types/Cargo.toml | 2 +- lightning/Cargo.toml | 12 ++++++------ 13 files changed, 48 insertions(+), 48 deletions(-) diff --git a/lightning-background-processor/Cargo.toml b/lightning-background-processor/Cargo.toml index 26278434a9b..828a8017574 100644 --- a/lightning-background-processor/Cargo.toml +++ b/lightning-background-processor/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-background-processor" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Valentine Wallace "] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning" @@ -23,17 +23,17 @@ std = ["lightning/std", "lightning-liquidity/std", "bitcoin-io/std", "bitcoin_ha bitcoin = { version = "0.32.2", default-features = false } bitcoin_hashes = { version = "0.14.0", default-features = false } bitcoin-io = { version = "0.1.2", default-features = false } -lightning = { version = "0.2.0-rc2", path = "../lightning", default-features = false } -lightning-rapid-gossip-sync = { version = "0.2.0-rc2", path = "../lightning-rapid-gossip-sync", default-features = false } -lightning-liquidity = { version = "0.2.0-rc2", path = "../lightning-liquidity", default-features = false } +lightning = { version = "0.2.0", path = "../lightning", default-features = false } +lightning-rapid-gossip-sync = { version = "0.2.0", path = "../lightning-rapid-gossip-sync", default-features = false } +lightning-liquidity = { version = "0.2.0", path = "../lightning-liquidity", default-features = false } possiblyrandom = { version = "0.2", path = "../possiblyrandom", default-features = false } [dev-dependencies] tokio = { version = "1.35", features = [ "macros", "rt", "rt-multi-thread", "sync", "time" ] } -lightning = { version = "0.2.0-rc2", path = "../lightning", features = ["_test_utils"] } -lightning-invoice = { version = "0.34.0-rc2", path = "../lightning-invoice" } -lightning-liquidity = { version = "0.2.0-rc2", path = "../lightning-liquidity", default-features = false, features = ["_test_utils"] } -lightning-persister = { version = "0.2.0-rc2", path = "../lightning-persister" } +lightning = { version = "0.2.0", path = "../lightning", features = ["_test_utils"] } +lightning-invoice = { version = "0.34.0", path = "../lightning-invoice" } +lightning-liquidity = { version = "0.2.0", path = "../lightning-liquidity", default-features = false, features = ["_test_utils"] } +lightning-persister = { version = "0.2.0", path = "../lightning-persister" } [lints] workspace = true diff --git a/lightning-block-sync/Cargo.toml b/lightning-block-sync/Cargo.toml index 5d3e3566c53..4f774fa4086 100644 --- a/lightning-block-sync/Cargo.toml +++ b/lightning-block-sync/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-block-sync" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Jeffrey Czyz", "Matt Corallo"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning" @@ -20,13 +20,13 @@ rpc-client = [ "serde_json", "chunked_transfer" ] [dependencies] bitcoin = "0.32.2" -lightning = { version = "0.2.0-rc2", path = "../lightning" } +lightning = { version = "0.2.0", path = "../lightning" } tokio = { version = "1.35", features = [ "io-util", "net", "time", "rt" ], optional = true } serde_json = { version = "1.0", optional = true } chunked_transfer = { version = "1.4", optional = true } [dev-dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", features = ["_test_utils"] } +lightning = { version = "0.2.0", path = "../lightning", features = ["_test_utils"] } tokio = { version = "1.35", features = [ "macros", "rt" ] } [lints] diff --git a/lightning-custom-message/Cargo.toml b/lightning-custom-message/Cargo.toml index 6feb206b676..96632f24bc1 100644 --- a/lightning-custom-message/Cargo.toml +++ b/lightning-custom-message/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-custom-message" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Jeffrey Czyz"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning" @@ -16,7 +16,7 @@ rustdoc-args = ["--cfg", "docsrs"] [dependencies] bitcoin = "0.32.2" -lightning = { version = "0.2.0-rc2", path = "../lightning" } +lightning = { version = "0.2.0", path = "../lightning" } [lints] workspace = true diff --git a/lightning-dns-resolver/Cargo.toml b/lightning-dns-resolver/Cargo.toml index b4a5683af5e..248cb73025a 100644 --- a/lightning-dns-resolver/Cargo.toml +++ b/lightning-dns-resolver/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-dns-resolver" -version = "0.3.0-rc2" +version = "0.3.0" authors = ["Matt Corallo"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning/" @@ -9,12 +9,12 @@ description = "A crate which implements DNSSEC resolution for lightning clients edition = "2021" [dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", default-features = false } -lightning-types = { version = "0.3.0-rc2", path = "../lightning-types", default-features = false } +lightning = { version = "0.2.0", path = "../lightning", default-features = false } +lightning-types = { version = "0.3.0", path = "../lightning-types", default-features = false } dnssec-prover = { version = "0.6", default-features = false, features = [ "std", "tokio" ] } tokio = { version = "1.0", default-features = false, features = ["rt"] } [dev-dependencies] bitcoin = { version = "0.32" } tokio = { version = "1.0", default-features = false, features = ["macros", "time"] } -lightning = { version = "0.2.0-rc2", path = "../lightning", features = ["dnssec", "_test_utils"] } +lightning = { version = "0.2.0", path = "../lightning", features = ["dnssec", "_test_utils"] } diff --git a/lightning-invoice/Cargo.toml b/lightning-invoice/Cargo.toml index 31baeb5acf5..30bbfa9a3be 100644 --- a/lightning-invoice/Cargo.toml +++ b/lightning-invoice/Cargo.toml @@ -1,7 +1,7 @@ [package] name = "lightning-invoice" description = "Data structures to parse and serialize BOLT11 lightning invoices" -version = "0.34.0-rc2" +version = "0.34.0" authors = ["Sebastian Geisler "] documentation = "https://docs.rs/lightning-invoice/" license = "MIT OR Apache-2.0" @@ -19,7 +19,7 @@ std = [] [dependencies] bech32 = { version = "0.11.0", default-features = false } -lightning-types = { version = "0.3.0-rc2", path = "../lightning-types", default-features = false } +lightning-types = { version = "0.3.0", path = "../lightning-types", default-features = false } serde = { version = "1.0", optional = true, default-features = false, features = ["alloc"] } bitcoin = { version = "0.32.2", default-features = false, features = ["secp-recovery"] } diff --git a/lightning-liquidity/Cargo.toml b/lightning-liquidity/Cargo.toml index fbec56baea3..a4855957f7a 100644 --- a/lightning-liquidity/Cargo.toml +++ b/lightning-liquidity/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-liquidity" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["John Cantrell ", "Elias Rohrer "] homepage = "https://lightningdevkit.org/" license = "MIT OR Apache-2.0" @@ -21,10 +21,10 @@ backtrace = ["dep:backtrace"] _test_utils = [] [dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", default-features = false } -lightning-types = { version = "0.3.0-rc2", path = "../lightning-types", default-features = false } -lightning-invoice = { version = "0.34.0-rc2", path = "../lightning-invoice", default-features = false, features = ["serde"] } -lightning-macros = { version = "0.2.0-rc2", path = "../lightning-macros" } +lightning = { version = "0.2.0", path = "../lightning", default-features = false } +lightning-types = { version = "0.3.0", path = "../lightning-types", default-features = false } +lightning-invoice = { version = "0.34.0", path = "../lightning-invoice", default-features = false, features = ["serde"] } +lightning-macros = { version = "0.2.0", path = "../lightning-macros" } bitcoin = { version = "0.32.2", default-features = false, features = ["serde"] } @@ -34,9 +34,9 @@ serde_json = { version = "1.0", default-features = false, features = ["alloc"] } backtrace = { version = "0.3", optional = true } [dev-dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", default-features = false, features = ["_test_utils"] } -lightning-invoice = { version = "0.34.0-rc2", path = "../lightning-invoice", default-features = false, features = ["serde", "std"] } -lightning-persister = { version = "0.2.0-rc2", path = "../lightning-persister", default-features = false } +lightning = { version = "0.2.0", path = "../lightning", default-features = false, features = ["_test_utils"] } +lightning-invoice = { version = "0.34.0", path = "../lightning-invoice", default-features = false, features = ["serde", "std"] } +lightning-persister = { version = "0.2.0", path = "../lightning-persister", default-features = false } proptest = "1.0.0" tokio = { version = "1.35", default-features = false, features = [ "rt-multi-thread", "time", "sync", "macros" ] } diff --git a/lightning-macros/Cargo.toml b/lightning-macros/Cargo.toml index c513a4972fe..546c4de5129 100644 --- a/lightning-macros/Cargo.toml +++ b/lightning-macros/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-macros" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Elias Rohrer"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning/" diff --git a/lightning-net-tokio/Cargo.toml b/lightning-net-tokio/Cargo.toml index 4900742c627..a911b3d22be 100644 --- a/lightning-net-tokio/Cargo.toml +++ b/lightning-net-tokio/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-net-tokio" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Matt Corallo"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning/" @@ -17,12 +17,12 @@ rustdoc-args = ["--cfg", "docsrs"] [dependencies] bitcoin = "0.32.2" -lightning = { version = "0.2.0-rc2", path = "../lightning" } +lightning = { version = "0.2.0", path = "../lightning" } tokio = { version = "1.35", features = [ "rt", "sync", "net", "time" ] } [dev-dependencies] tokio = { version = "1.35", features = [ "macros", "rt", "rt-multi-thread", "sync", "net", "time" ] } -lightning = { version = "0.2.0-rc2", path = "../lightning", features = ["_test_utils"] } +lightning = { version = "0.2.0", path = "../lightning", features = ["_test_utils"] } [lints] workspace = true diff --git a/lightning-persister/Cargo.toml b/lightning-persister/Cargo.toml index d4a5e683680..cdd4b3a5086 100644 --- a/lightning-persister/Cargo.toml +++ b/lightning-persister/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-persister" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Valentine Wallace", "Matt Corallo"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning" @@ -19,7 +19,7 @@ tokio = ["dep:tokio"] [dependencies] bitcoin = "0.32.2" -lightning = { version = "0.2.0-rc2", path = "../lightning" } +lightning = { version = "0.2.0", path = "../lightning" } tokio = { version = "1.35", optional = true, default-features = false, features = ["rt-multi-thread"] } [target.'cfg(windows)'.dependencies] @@ -29,7 +29,7 @@ windows-sys = { version = "0.48.0", default-features = false, features = ["Win32 criterion = { version = "0.4", optional = true, default-features = false } [dev-dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", features = ["_test_utils"] } +lightning = { version = "0.2.0", path = "../lightning", features = ["_test_utils"] } bitcoin = { version = "0.32.2", default-features = false } tokio = { version = "1.35", default-features = false, features = ["macros"] } diff --git a/lightning-rapid-gossip-sync/Cargo.toml b/lightning-rapid-gossip-sync/Cargo.toml index 015816b101e..695e41a3662 100644 --- a/lightning-rapid-gossip-sync/Cargo.toml +++ b/lightning-rapid-gossip-sync/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-rapid-gossip-sync" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Arik Sosman "] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning" @@ -14,7 +14,7 @@ default = ["std"] std = ["bitcoin-io/std", "bitcoin_hashes/std"] [dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", default-features = false } +lightning = { version = "0.2.0", path = "../lightning", default-features = false } bitcoin = { version = "0.32.2", default-features = false } bitcoin_hashes = { version = "0.14.0", default-features = false } bitcoin-io = { version = "0.1.2", default-features = false } @@ -23,7 +23,7 @@ bitcoin-io = { version = "0.1.2", default-features = false } criterion = { version = "0.4", optional = true, default-features = false } [dev-dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", features = ["_test_utils"] } +lightning = { version = "0.2.0", path = "../lightning", features = ["_test_utils"] } [lints] workspace = true diff --git a/lightning-transaction-sync/Cargo.toml b/lightning-transaction-sync/Cargo.toml index 387af475a08..2e6a99810ff 100644 --- a/lightning-transaction-sync/Cargo.toml +++ b/lightning-transaction-sync/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-transaction-sync" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Elias Rohrer"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning" @@ -32,15 +32,15 @@ electrum-rustls = ["electrum"] electrum-rustls-ring = ["_electrum", "electrum-client/use-rustls-ring"] [dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", default-features = false, features = ["std"] } -lightning-macros = { version = "0.2.0-rc2", path = "../lightning-macros", default-features = false } +lightning = { version = "0.2.0", path = "../lightning", default-features = false, features = ["std"] } +lightning-macros = { version = "0.2.0", path = "../lightning-macros", default-features = false } bitcoin = { version = "0.32.2", default-features = false } futures = { version = "0.3", optional = true } esplora-client = { version = "0.12", default-features = false, optional = true } electrum-client = { version = "0.24.0", optional = true, default-features = false, features = ["proxy"] } [dev-dependencies] -lightning = { version = "0.2.0-rc2", path = "../lightning", default-features = false, features = ["std", "_test_utils"] } +lightning = { version = "0.2.0", path = "../lightning", default-features = false, features = ["std", "_test_utils"] } tokio = { version = "1.35.0", features = ["macros"] } [target.'cfg(not(target_os = "windows"))'.dev-dependencies] diff --git a/lightning-types/Cargo.toml b/lightning-types/Cargo.toml index f6a344ed9a3..d492698eb4c 100644 --- a/lightning-types/Cargo.toml +++ b/lightning-types/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning-types" -version = "0.3.0-rc2" +version = "0.3.0" authors = ["Matt Corallo"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning/" diff --git a/lightning/Cargo.toml b/lightning/Cargo.toml index d5ac67df88e..7aa869a18bb 100644 --- a/lightning/Cargo.toml +++ b/lightning/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "lightning" -version = "0.2.0-rc2" +version = "0.2.0" authors = ["Matt Corallo"] license = "MIT OR Apache-2.0" repository = "https://github.com/lightningdevkit/rust-lightning/" @@ -33,9 +33,9 @@ grind_signatures = [] default = ["std", "grind_signatures"] [dependencies] -lightning-types = { version = "0.3.0-rc2", path = "../lightning-types", default-features = false } -lightning-invoice = { version = "0.34.0-rc2", path = "../lightning-invoice", default-features = false } -lightning-macros = { version = "0.2.0-rc2", path = "../lightning-macros" } +lightning-types = { version = "0.3.0", path = "../lightning-types", default-features = false } +lightning-invoice = { version = "0.34.0", path = "../lightning-invoice", default-features = false } +lightning-macros = { version = "0.2.0", path = "../lightning-macros" } bech32 = { version = "0.11.0", default-features = false } bitcoin = { version = "0.32.2", default-features = false, features = ["secp-recovery"] } @@ -52,8 +52,8 @@ inventory = { version = "0.3", optional = true } [dev-dependencies] regex = "1.5.6" -lightning-types = { version = "0.3.0-rc2", path = "../lightning-types", features = ["_test_utils"] } -lightning-macros = { version = "0.2.0-rc2", path = "../lightning-macros" } +lightning-types = { version = "0.3.0", path = "../lightning-types", features = ["_test_utils"] } +lightning-macros = { version = "0.2.0", path = "../lightning-macros" } parking_lot = { version = "0.12", default-features = false } [dev-dependencies.bitcoin]