Merge pull request #3638 from wpaulino/bye-bye-outpoint-to-peer

Require counterparty_node_id TLV for ChannelMonitor
This commit is contained in:
Matt Corallo 2025-03-06 22:59:23 +00:00 committed by GitHub
commit b7328657e2
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
10 changed files with 225 additions and 488 deletions

View file

@ -282,7 +282,7 @@ impl chain::Watch<TestChannelSigner> for TestChainMonitor {
fn release_pending_monitor_events(
&self,
) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, Option<PublicKey>)> {
) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, PublicKey)> {
return self.chain_monitor.release_pending_monitor_events();
}
}

View file

@ -246,7 +246,7 @@ pub struct ChainMonitor<ChannelSigner: EcdsaChannelSigner, C: Deref, T: Deref, F
persister: P,
/// "User-provided" (ie persistence-completion/-failed) [`MonitorEvent`]s. These came directly
/// from the user and not from a [`ChannelMonitor`].
pending_monitor_events: Mutex<Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, Option<PublicKey>)>>,
pending_monitor_events: Mutex<Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, PublicKey)>>,
/// The best block height seen, used as a proxy for the passage of time.
highest_chain_height: AtomicUsize,
@ -804,7 +804,7 @@ where C::Target: chain::Filter,
let monitors = self.monitors.read().unwrap();
match monitors.get(&channel_id) {
None => {
let logger = WithContext::from(&self.logger, update.counterparty_node_id, Some(channel_id), None);
let logger = WithContext::from(&self.logger, None, Some(channel_id), None);
log_error!(logger, "Failed to update channel monitor: no such monitor registered");
// We should never ever trigger this from within ChannelManager. Technically a
@ -874,7 +874,7 @@ where C::Target: chain::Filter,
}
}
fn release_pending_monitor_events(&self) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, Option<PublicKey>)> {
fn release_pending_monitor_events(&self) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, PublicKey)> {
let mut pending_monitor_events = self.pending_monitor_events.lock().unwrap().split_off(0);
for monitor_state in self.monitors.read().unwrap().values() {
let monitor_events = monitor_state.monitor.get_and_clear_pending_monitor_events();

View file

@ -74,15 +74,6 @@ use crate::sync::{Mutex, LockTestExt};
#[must_use]
pub struct ChannelMonitorUpdate {
pub(crate) updates: Vec<ChannelMonitorUpdateStep>,
/// Historically, [`ChannelMonitor`]s didn't know their counterparty node id. However,
/// `ChannelManager` really wants to know it so that it can easily look up the corresponding
/// channel. For now, this results in a temporary map in `ChannelManager` to look up channels
/// by only the funding outpoint.
///
/// To eventually remove that, we repeat the counterparty node id here so that we can upgrade
/// `ChannelMonitor`s to become aware of the counterparty node id if they were generated prior
/// to when it was stored directly in them.
pub(crate) counterparty_node_id: Option<PublicKey>,
/// The sequence number of this update. Updates *must* be replayed in-order according to this
/// sequence number (and updates may panic if they are not). The update_id values are strictly
/// increasing and increase by one for each new update, with two exceptions specified below.
@ -117,7 +108,7 @@ impl Writeable for ChannelMonitorUpdate {
update_step.write(w)?;
}
write_tlv_fields!(w, {
(1, self.counterparty_node_id, option),
// 1 was previously used to store `counterparty_node_id`
(3, self.channel_id, option),
});
Ok(())
@ -134,13 +125,12 @@ impl Readable for ChannelMonitorUpdate {
updates.push(upd);
}
}
let mut counterparty_node_id = None;
let mut channel_id = None;
read_tlv_fields!(r, {
(1, counterparty_node_id, option),
// 1 was previously used to store `counterparty_node_id`
(3, channel_id, option),
});
Ok(Self { update_id, counterparty_node_id, updates, channel_id })
Ok(Self { update_id, updates, channel_id })
}
}
@ -1020,7 +1010,7 @@ pub(crate) struct ChannelMonitorImpl<Signer: EcdsaChannelSigner> {
best_block: BestBlock,
/// The node_id of our counterparty
counterparty_node_id: Option<PublicKey>,
counterparty_node_id: PublicKey,
/// Initial counterparty commmitment data needed to recreate the commitment tx
/// in the persistence pipeline for third-party watchtowers. This will only be present on
@ -1242,7 +1232,7 @@ impl<Signer: EcdsaChannelSigner> Writeable for ChannelMonitorImpl<Signer> {
(3, self.htlcs_resolved_on_chain, required_vec),
(5, pending_monitor_events, required_vec),
(7, self.funding_spend_seen, required),
(9, self.counterparty_node_id, option),
(9, self.counterparty_node_id, required),
(11, self.confirmed_commitment_tx_counterparty_output, option),
(13, self.spendable_txids_confirmed, required_vec),
(15, self.counterparty_fulfilled_htlcs, required),
@ -1338,7 +1328,7 @@ impl<'a, L: Deref> WithChannelMonitor<'a, L> where L::Target: Logger {
}
pub(crate) fn from_impl<S: EcdsaChannelSigner>(logger: &'a L, monitor_impl: &ChannelMonitorImpl<S>, payment_hash: Option<PaymentHash>) -> Self {
let peer_id = monitor_impl.counterparty_node_id;
let peer_id = Some(monitor_impl.counterparty_node_id);
let channel_id = Some(monitor_impl.channel_id());
WithChannelMonitor {
logger, peer_id, channel_id, payment_hash,
@ -1462,7 +1452,7 @@ impl<Signer: EcdsaChannelSigner> ChannelMonitor<Signer> {
spendable_txids_confirmed: Vec::new(),
best_block,
counterparty_node_id: Some(counterparty_node_id),
counterparty_node_id: counterparty_node_id,
initial_counterparty_commitment_info: None,
balances_empty_height: None,
@ -1788,10 +1778,7 @@ impl<Signer: EcdsaChannelSigner> ChannelMonitor<Signer> {
}
/// Gets the `node_id` of the counterparty for this channel.
///
/// Will be `None` for channels constructed on LDK versions prior to 0.0.110 and always `Some`
/// otherwise.
pub fn get_counterparty_node_id(&self) -> Option<PublicKey> {
pub fn get_counterparty_node_id(&self) -> PublicKey {
self.inner.lock().unwrap().counterparty_node_id
}
@ -3200,14 +3187,6 @@ impl<Signer: EcdsaChannelSigner> ChannelMonitorImpl<Signer> {
log_funding_info!(self), self.latest_update_id, updates.update_id, updates.updates.len());
}
if updates.counterparty_node_id.is_some() {
if self.counterparty_node_id.is_none() {
self.counterparty_node_id = updates.counterparty_node_id;
} else {
debug_assert_eq!(self.counterparty_node_id, updates.counterparty_node_id);
}
}
// ChannelMonitor updates may be applied after force close if we receive a preimage for a
// broadcasted commitment transaction HTLC output that we'd like to claim on-chain. If this
// is the case, we no longer have guaranteed access to the monitor's update ID, so we use a
@ -3376,10 +3355,7 @@ impl<Signer: EcdsaChannelSigner> ChannelMonitorImpl<Signer> {
package_target_feerate_sat_per_1000_weight, commitment_tx, anchor_output_idx,
} => {
let channel_id = self.channel_id;
// unwrap safety: `ClaimEvent`s are only available for Anchor channels,
// introduced with v0.0.116. counterparty_node_id is guaranteed to be `Some`
// since v0.0.110.
let counterparty_node_id = self.counterparty_node_id.unwrap();
let counterparty_node_id = self.counterparty_node_id;
let commitment_txid = commitment_tx.compute_txid();
debug_assert_eq!(self.current_holder_commitment_tx.txid, commitment_txid);
let pending_htlcs = self.current_holder_commitment_tx.non_dust_htlcs();
@ -3410,10 +3386,7 @@ impl<Signer: EcdsaChannelSigner> ChannelMonitorImpl<Signer> {
target_feerate_sat_per_1000_weight, htlcs, tx_lock_time,
} => {
let channel_id = self.channel_id;
// unwrap safety: `ClaimEvent`s are only available for Anchor channels,
// introduced with v0.0.116. counterparty_node_id is guaranteed to be `Some`
// since v0.0.110.
let counterparty_node_id = self.counterparty_node_id.unwrap();
let counterparty_node_id = self.counterparty_node_id;
let mut htlc_descriptors = Vec::with_capacity(htlcs.len());
for htlc in htlcs {
htlc_descriptors.push(HTLCDescriptor {
@ -5129,6 +5102,13 @@ impl<'a, 'b, ES: EntropySource, SP: SignerProvider> ReadableArgs<(&'a ES, &'b SP
chan_utils::get_to_countersignatory_with_anchors_redeemscript(&payment_point).to_p2wsh();
}
let channel_id = channel_id.unwrap_or(ChannelId::v1_from_funding_outpoint(outpoint));
if counterparty_node_id.is_none() {
panic!("Found monitor for channel {} with no updates since v0.0.118.\
These monitors are no longer supported.\
To continue, run a v0.1 release, send/route a payment over the channel or close it.", channel_id);
}
Ok((best_block.block_hash, ChannelMonitor::from_impl(ChannelMonitorImpl {
latest_update_id,
commitment_transaction_number_obscure_factor,
@ -5140,7 +5120,7 @@ impl<'a, 'b, ES: EntropySource, SP: SignerProvider> ReadableArgs<(&'a ES, &'b SP
channel_keys_id,
holder_revocation_basepoint,
channel_id: channel_id.unwrap_or(ChannelId::v1_from_funding_outpoint(outpoint)),
channel_id,
funding_info,
first_confirmed_funding_txo: first_confirmed_funding_txo.0.unwrap(),
current_counterparty_commitment_txid,
@ -5184,7 +5164,7 @@ impl<'a, 'b, ES: EntropySource, SP: SignerProvider> ReadableArgs<(&'a ES, &'b SP
spendable_txids_confirmed: spendable_txids_confirmed.unwrap(),
best_block,
counterparty_node_id,
counterparty_node_id: counterparty_node_id.unwrap(),
initial_counterparty_commitment_info,
balances_empty_height,
failed_back_htlc_ids: new_hash_set(),

View file

@ -304,7 +304,7 @@ pub trait Watch<ChannelSigner: EcdsaChannelSigner> {
///
/// For details on asynchronous [`ChannelMonitor`] updating and returning
/// [`MonitorEvent::Completed`] here, see [`ChannelMonitorUpdateStatus::InProgress`].
fn release_pending_monitor_events(&self) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, Option<PublicKey>)>;
fn release_pending_monitor_events(&self) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, PublicKey)>;
}
/// The `Filter` trait defines behavior for indicating chain activity of interest pertaining to

View file

@ -18,7 +18,6 @@ use bitcoin::transaction::Version;
use crate::chain::channelmonitor::LATENCY_GRACE_PERIOD_BLOCKS;
use crate::chain::ChannelMonitorUpdateStatus;
use crate::chain::transaction::OutPoint;
use crate::events::bump_transaction::WalletSource;
use crate::events::{ClosureReason, Event};
use crate::ln::chan_utils::ClosingTransaction;
@ -1091,9 +1090,4 @@ fn do_test_closing_signed(extra_closing_signed: bool, reconnect: bool) {
assert!(nodes[1].node.list_channels().is_empty());
check_closed_event!(nodes[0], 1, ClosureReason::LocallyInitiatedCooperativeClosure, [nodes[1].node.get_our_node_id()], 100000);
check_closed_event!(nodes[1], 1, ClosureReason::CounterpartyInitiatedCooperativeClosure, [nodes[0].node.get_our_node_id()], 100000);
// Check that our maps have been updated after async signing channel closure.
let funding_outpoint = OutPoint { txid: funding_tx.compute_txid(), index: 0 };
assert!(nodes[0].node().outpoint_to_peer.lock().unwrap().get(&funding_outpoint).is_none());
assert!(nodes[1].node().outpoint_to_peer.lock().unwrap().get(&funding_outpoint).is_none());
}

View file

@ -3457,6 +3457,18 @@ impl<SP: Deref> ChannelContext<SP> where SP::Target: SignerProvider {
!matches!(self.channel_state, ChannelState::AwaitingChannelReady(flags) if flags.is_set(AwaitingChannelReadyFlags::WAITING_FOR_BATCH))
}
fn unset_funding_info(&mut self, funding: &mut FundingScope) {
debug_assert!(
matches!(self.channel_state, ChannelState::FundingNegotiated)
|| matches!(self.channel_state, ChannelState::AwaitingChannelReady(_))
);
funding.channel_transaction_parameters.funding_outpoint = None;
self.channel_id = self.temporary_channel_id.expect(
"temporary_channel_id should be set since unset_funding_info is only called on funded \
channels that were unfunded immediately beforehand"
);
}
fn validate_commitment_signed<L: Deref>(
&self, funding: &FundingScope, holder_commitment_point: &HolderCommitmentPoint,
msg: &msgs::CommitmentSigned, logger: &L,
@ -4516,7 +4528,6 @@ impl<SP: Deref> ChannelContext<SP> where SP::Target: SignerProvider {
self.latest_monitor_update_id += 1;
Some((self.get_counterparty_node_id(), funding_txo, self.channel_id(), ChannelMonitorUpdate {
update_id: self.latest_monitor_update_id,
counterparty_node_id: Some(self.counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::ChannelForceClosed { should_broadcast }],
channel_id: Some(self.channel_id()),
}))
@ -5095,7 +5106,6 @@ impl<SP: Deref> FundedChannel<SP> where
self.context.latest_monitor_update_id += 1;
let monitor_update = ChannelMonitorUpdate {
update_id: self.context.latest_monitor_update_id,
counterparty_node_id: Some(self.context.counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::PaymentPreimage {
payment_preimage: payment_preimage_arg.clone(),
payment_info,
@ -5310,14 +5320,7 @@ impl<SP: Deref> FundedChannel<SP> where
/// Further, the channel must be immediately shut down after this with a call to
/// [`ChannelContext::force_shutdown`].
pub fn unset_funding_info(&mut self) {
debug_assert!(matches!(
self.context.channel_state, ChannelState::AwaitingChannelReady(_)
));
self.funding.channel_transaction_parameters.funding_outpoint = None;
self.context.channel_id = self.context.temporary_channel_id.expect(
"temporary_channel_id should be set since unset_funding_info is only called on funded \
channels that were unfunded immediately beforehand"
);
self.context.unset_funding_info(&mut self.funding);
}
/// Handles a channel_ready message from our peer. If we've already sent our channel_ready
@ -5710,7 +5713,6 @@ impl<SP: Deref> FundedChannel<SP> where
self.context.latest_monitor_update_id += 1;
let mut monitor_update = ChannelMonitorUpdate {
update_id: self.context.latest_monitor_update_id,
counterparty_node_id: Some(self.context.counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::LatestHolderCommitmentTXInfo {
commitment_tx,
htlc_outputs,
@ -5792,7 +5794,6 @@ impl<SP: Deref> FundedChannel<SP> where
let mut monitor_update = ChannelMonitorUpdate {
update_id: self.context.latest_monitor_update_id + 1, // We don't increment this yet!
counterparty_node_id: Some(self.context.counterparty_node_id),
updates: Vec::new(),
channel_id: Some(self.context.channel_id()),
};
@ -5985,7 +5986,6 @@ impl<SP: Deref> FundedChannel<SP> where
self.context.latest_monitor_update_id += 1;
let mut monitor_update = ChannelMonitorUpdate {
update_id: self.context.latest_monitor_update_id,
counterparty_node_id: Some(self.context.counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::CommitmentSecret {
idx: self.context.cur_counterparty_commitment_transaction_number + 1,
secret: msg.per_commitment_secret,
@ -7257,7 +7257,6 @@ impl<SP: Deref> FundedChannel<SP> where
self.context.latest_monitor_update_id += 1;
let monitor_update = ChannelMonitorUpdate {
update_id: self.context.latest_monitor_update_id,
counterparty_node_id: Some(self.context.counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::ShutdownScript {
scriptpubkey: self.get_closing_scriptpubkey(),
}],
@ -8543,7 +8542,6 @@ impl<SP: Deref> FundedChannel<SP> where
self.context.latest_monitor_update_id += 1;
let monitor_update = ChannelMonitorUpdate {
update_id: self.context.latest_monitor_update_id,
counterparty_node_id: Some(self.context.counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::LatestCounterpartyCommitmentTXInfo {
commitment_txid: counterparty_commitment_txid,
htlc_outputs: htlcs.clone(),
@ -8755,7 +8753,6 @@ impl<SP: Deref> FundedChannel<SP> where
self.context.latest_monitor_update_id += 1;
let monitor_update = ChannelMonitorUpdate {
update_id: self.context.latest_monitor_update_id,
counterparty_node_id: Some(self.context.counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::ShutdownScript {
scriptpubkey: self.get_closing_scriptpubkey(),
}],
@ -9315,6 +9312,14 @@ impl<SP: Deref> OutboundV1Channel<SP> where SP::Target: SignerProvider {
} else { None };
(open_channel, funding_created)
}
/// Unsets the existing funding information.
///
/// The channel must be immediately shut down after this with a call to
/// [`ChannelContext::force_shutdown`].
pub fn unset_funding_info(&mut self) {
self.context.unset_funding_info(&mut self.funding);
}
}
/// A not-yet-funded inbound (from counterparty) channel using V1 channel establishment.

View file

@ -1060,22 +1060,9 @@ impl ClaimablePayments {
/// [`ChannelMonitorUpdate`]s are applied.
#[derive(Debug)]
enum BackgroundEvent {
/// Handle a ChannelMonitorUpdate which closes the channel or for an already-closed channel.
/// This is only separated from [`Self::MonitorUpdateRegeneratedOnStartup`] as for truly
/// ancient [`ChannelMonitor`]s that haven't seen an update since LDK 0.0.118 we may not have
/// the counterparty node ID available.
///
/// Note that any such events are lost on shutdown, so in general they must be updates which
/// are regenerated on startup.
ClosedMonitorUpdateRegeneratedOnStartup((OutPoint, ChannelId, ChannelMonitorUpdate)),
/// Handle a ChannelMonitorUpdate which may or may not close the channel and may unblock the
/// channel to continue normal operation.
///
/// In general this should be used rather than
/// [`Self::ClosedMonitorUpdateRegeneratedOnStartup`], however in cases where the
/// `counterparty_node_id` is not available as the channel has closed from a [`ChannelMonitor`]
/// error the other variant is acceptable.
///
/// Any such events that exist in [`ChannelManager::pending_background_events`] will *also* be
/// tracked in [`PeerState::in_flight_monitor_updates`].
///
@ -2423,8 +2410,6 @@ where
// |
// |__`peer_state`
// |
// |__`outpoint_to_peer`
// |
// |__`short_to_chan_info`
// |
// |__`outbound_scid_aliases`
@ -2523,29 +2508,6 @@ where
/// See `ChannelManager` struct-level documentation for lock order requirements.
outbound_scid_aliases: Mutex<HashSet<u64>>,
/// Channel funding outpoint -> `counterparty_node_id`.
///
/// Note that this map should only be used for `MonitorEvent` handling, to be able to access
/// the corresponding channel for the event, as we only have access to the `channel_id` during
/// the handling of the events.
///
/// Note that no consistency guarantees are made about the existence of a peer with the
/// `counterparty_node_id` in our other maps.
///
/// TODO:
/// The `counterparty_node_id` isn't passed with `MonitorEvent`s currently. To pass it, we need
/// to make `counterparty_node_id`'s a required field in `ChannelMonitor`s, which unfortunately
/// would break backwards compatability.
/// We should add `counterparty_node_id`s to `MonitorEvent`s, and eventually rely on it in the
/// future. That would make this map redundant, as only the `ChannelManager::per_peer_state` is
/// required to access the channel with the `counterparty_node_id`.
///
/// See `ChannelManager` struct-level documentation for lock order requirements.
#[cfg(any(test, feature = "_test_utils"))]
pub(crate) outpoint_to_peer: Mutex<HashMap<OutPoint, PublicKey>>,
#[cfg(not(any(test, feature = "_test_utils")))]
outpoint_to_peer: Mutex<HashMap<OutPoint, PublicKey>>,
/// SCIDs (and outbound SCID aliases) -> `counterparty_node_id`s and `channel_id`s.
///
/// Outbound SCID aliases are added here once the channel is available for normal use, with
@ -3051,7 +3013,7 @@ macro_rules! handle_error {
/// Note that this step can be skipped if the channel was never opened (through the creation of a
/// [`ChannelMonitor`]/channel funding transaction) to begin with.
macro_rules! locked_close_channel {
($self: ident, $peer_state: expr, $channel_context: expr, $channel_funding: expr, $shutdown_res_mut: expr) => {{
($self: ident, $peer_state: expr, $channel_context: expr, $shutdown_res_mut: expr) => {{
if let Some((_, funding_txo, _, update)) = $shutdown_res_mut.monitor_update.take() {
handle_new_monitor_update!($self, funding_txo, update, $peer_state,
$channel_context, REMAIN_LOCKED_UPDATE_ACTIONS_PROCESSED_LATER);
@ -3065,9 +3027,6 @@ macro_rules! locked_close_channel {
let chan_id = $channel_context.channel_id();
$peer_state.closed_channel_monitor_update_ids.insert(chan_id, update_id);
}
if let Some(outpoint) = $channel_funding.get_funding_txo() {
$self.outpoint_to_peer.lock().unwrap().remove(&outpoint);
}
let mut short_to_chan_info = $self.short_to_chan_info.write().unwrap();
if let Some(short_id) = $channel_context.get_short_channel_id() {
short_to_chan_info.remove(&short_id);
@ -3102,7 +3061,7 @@ macro_rules! convert_channel_err {
let logger = WithChannelContext::from(&$self.logger, &$context, None);
log_error!(logger, "Closing channel {} due to close-required error: {}", $channel_id, msg);
let mut shutdown_res = $context.force_shutdown($funding, true, reason);
locked_close_channel!($self, $peer_state, $context, $funding, &mut shutdown_res);
locked_close_channel!($self, $peer_state, $context, &mut shutdown_res);
let err =
MsgHandleErrInternal::from_finish_shutdown(msg, *$channel_id, shutdown_res, $channel_update);
(true, err)
@ -3167,7 +3126,7 @@ macro_rules! remove_channel_entry {
($self: ident, $peer_state: expr, $entry: expr, $shutdown_res_mut: expr) => {
{
let channel = $entry.remove_entry().1;
locked_close_channel!($self, $peer_state, &channel.context(), channel.funding(), $shutdown_res_mut);
locked_close_channel!($self, $peer_state, &channel.context(), $shutdown_res_mut);
channel
}
}
@ -3600,7 +3559,6 @@ where
decode_update_add_htlcs: Mutex::new(new_hash_map()),
claimable_payments: Mutex::new(ClaimablePayments { claimable_payments: new_hash_map(), pending_claiming_payments: new_hash_map() }),
pending_intercepted_htlcs: Mutex::new(new_hash_map()),
outpoint_to_peer: Mutex::new(new_hash_map()),
short_to_chan_info: FairRwLock::new(new_hash_map()),
our_network_pubkey: node_signer.get_node_id(Recipient::Node).unwrap(),
@ -3772,8 +3730,7 @@ where
fn list_funded_channels_with_filter<Fn: FnMut(&(&ChannelId, &FundedChannel<SP>)) -> bool + Copy>(&self, f: Fn) -> Vec<ChannelDetails> {
// Allocate our best estimate of the number of channels we have in the `res`
// Vec. Sadly the `short_to_chan_info` map doesn't cover channels without
// a scid or a scid alias, and the `outpoint_to_peer` shouldn't be used outside
// of the ChannelMonitor handling. Therefore reallocations may still occur, but is
// a scid or a scid alias. Therefore reallocations may still occur, but is
// unlikely as the `short_to_chan_info` map often contains 2 entries for
// the same channel.
let mut res = Vec::with_capacity(self.short_to_chan_info.read().unwrap().len());
@ -3802,8 +3759,7 @@ where
pub fn list_channels(&self) -> Vec<ChannelDetails> {
// Allocate our best estimate of the number of channels we have in the `res`
// Vec. Sadly the `short_to_chan_info` map doesn't cover channels without
// a scid or a scid alias, and the `outpoint_to_peer` shouldn't be used outside
// of the ChannelMonitor handling. Therefore reallocations may still occur, but is
// a scid or a scid alias. Therefore reallocations may still occur, but is
// unlikely as the `short_to_chan_info` map often contains 2 entries for
// the same channel.
let mut res = Vec::with_capacity(self.short_to_chan_info.read().unwrap().len());
@ -4119,7 +4075,7 @@ where
let mut peer_state = peer_state_mutex.lock().unwrap();
if let Some(mut chan) = peer_state.channel_by_id.remove(&channel_id) {
let mut close_res = chan.force_shutdown(false, ClosureReason::FundingBatchClosure);
locked_close_channel!(self, &mut *peer_state, chan.context(), chan.funding(), close_res);
locked_close_channel!(self, &mut *peer_state, chan.context(), close_res);
shutdown_results.push(close_res);
}
}
@ -5151,25 +5107,27 @@ where
let mut peer_state_lock = peer_state_mutex.lock().unwrap();
let peer_state = &mut *peer_state_lock;
macro_rules! close_chan { ($err: expr, $api_err: expr, $chan: expr) => { {
let counterparty;
let err = if let ChannelError::Close((msg, reason)) = $err {
let channel_id = $chan.context.channel_id();
counterparty = $chan.context.get_counterparty_node_id();
let shutdown_res = $chan.context.force_shutdown(&$chan.funding, false, reason);
MsgHandleErrInternal::from_finish_shutdown(msg, channel_id, shutdown_res, None)
} else { unreachable!(); };
mem::drop(peer_state_lock);
mem::drop(per_peer_state);
let _: Result<(), _> = handle_error!(self, Err(err), counterparty);
Err($api_err)
} } }
let funding_txo;
let (mut chan, msg_opt) = match peer_state.channel_by_id.remove(&temporary_channel_id)
.map(Channel::into_unfunded_outbound_v1)
{
Some(Ok(mut chan)) => {
macro_rules! close_chan { ($err: expr, $api_err: expr, $chan: expr) => { {
let counterparty;
let err = if let ChannelError::Close((msg, reason)) = $err {
let channel_id = $chan.context.channel_id();
counterparty = $chan.context.get_counterparty_node_id();
let shutdown_res = $chan.context.force_shutdown(&$chan.funding, false, reason);
MsgHandleErrInternal::from_finish_shutdown(msg, channel_id, shutdown_res, None)
} else { unreachable!(); };
mem::drop(peer_state_lock);
mem::drop(per_peer_state);
let _: Result<(), _> = handle_error!(self, Err(err), counterparty);
Err($api_err)
} } }
match find_funding_output(&chan) {
Ok(found_funding_txo) => funding_txo = found_funding_txo,
Err(err) => {
@ -5203,40 +5161,34 @@ where
}),
};
if let Some(msg) = msg_opt {
peer_state.pending_msg_events.push(MessageSendEvent::SendFundingCreated {
node_id: chan.context.get_counterparty_node_id(),
msg,
});
}
if is_manual_broadcast {
chan.context.set_manual_broadcast();
}
match peer_state.channel_by_id.entry(chan.context.channel_id()) {
hash_map::Entry::Occupied(_) => {
panic!("Generated duplicate funding txid?");
// We need to `unset_funding_info` to make sure we don't close the already open
// channel and instead close the one pending.
let err = format!(
"An existing channel using ID {} is open with peer {}",
chan.context.channel_id(), chan.context.get_counterparty_node_id(),
);
let chan_err = ChannelError::close(err.to_owned());
let api_err = APIError::APIMisuseError { err: err.to_owned() };
chan.unset_funding_info();
return close_chan!(chan_err, api_err, chan);
},
hash_map::Entry::Vacant(e) => {
let mut outpoint_to_peer = self.outpoint_to_peer.lock().unwrap();
match outpoint_to_peer.entry(funding_txo) {
hash_map::Entry::Vacant(e) => { e.insert(chan.context.get_counterparty_node_id()); },
hash_map::Entry::Occupied(o) => {
let err = format!(
"An existing channel using outpoint {} is open with peer {}",
funding_txo, o.get()
);
mem::drop(outpoint_to_peer);
mem::drop(peer_state_lock);
mem::drop(per_peer_state);
let reason = ClosureReason::ProcessingError { err: err.clone() };
self.finish_close_channel(chan.context.force_shutdown(&chan.funding, true, reason));
return Err(APIError::ChannelUnavailable { err });
}
if let Some(msg) = msg_opt {
peer_state.pending_msg_events.push(MessageSendEvent::SendFundingCreated {
node_id: chan.context.get_counterparty_node_id(),
msg,
});
}
if is_manual_broadcast {
chan.context.set_manual_broadcast();
}
e.insert(Channel::from(chan));
Ok(())
}
}
Ok(())
}
#[cfg(any(test, feature = "_externalize_tests"))]
@ -5443,7 +5395,7 @@ where
.map(|(mut chan, mut peer_state)| {
let closure_reason = ClosureReason::ProcessingError { err: e.clone() };
let mut close_res = chan.force_shutdown(false, closure_reason);
locked_close_channel!(self, peer_state, chan.context(), chan.funding(), close_res);
locked_close_channel!(self, peer_state, chan.context(), close_res);
shutdown_results.push(close_res);
peer_state.pending_msg_events.push(MessageSendEvent::HandleError {
node_id: counterparty_node_id,
@ -6475,11 +6427,6 @@ where
for event in background_events.drain(..) {
match event {
BackgroundEvent::ClosedMonitorUpdateRegeneratedOnStartup((_funding_txo, channel_id, update)) => {
// The channel has already been closed, so no use bothering to care about the
// monitor updating completing.
let _ = self.chain_monitor.update_channel(channel_id, &update);
},
BackgroundEvent::MonitorUpdateRegeneratedOnStartup { counterparty_node_id, funding_txo, channel_id, update } => {
self.apply_post_close_monitor_update(counterparty_node_id, channel_id, funding_txo, update);
},
@ -6696,8 +6643,8 @@ where
"Force-closing pending channel with ID {} for not establishing in a timely manner",
context.channel_id());
let mut close_res = chan.force_shutdown(false, ClosureReason::HolderForceClosed { broadcasted_latest_txn: Some(false) });
let (funding, context) = chan.funding_and_context_mut();
locked_close_channel!(self, peer_state, context, funding, close_res);
let context = chan.context_mut();
locked_close_channel!(self, peer_state, context, close_res);
shutdown_channels.push(close_res);
pending_msg_events.push(MessageSendEvent::HandleError {
node_id: context.get_counterparty_node_id(),
@ -7363,7 +7310,6 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
let preimage_update = ChannelMonitorUpdate {
update_id,
counterparty_node_id: Some(counterparty_node_id),
updates: vec![ChannelMonitorUpdateStep::PaymentPreimage {
payment_preimage,
payment_info,
@ -7408,16 +7354,14 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
fn claim_funds_internal(&self, source: HTLCSource, payment_preimage: PaymentPreimage,
forwarded_htlc_value_msat: Option<u64>, skimmed_fee_msat: Option<u64>, from_onchain: bool,
startup_replay: bool, next_channel_counterparty_node_id: Option<PublicKey>,
startup_replay: bool, next_channel_counterparty_node_id: PublicKey,
next_channel_outpoint: OutPoint, next_channel_id: ChannelId, next_user_channel_id: Option<u128>,
) {
match source {
HTLCSource::OutboundRoute { session_priv, payment_id, path, .. } => {
debug_assert!(self.background_events_processed_since_startup.load(Ordering::Acquire),
"We don't support claim_htlc claims during startup - monitors may not be available yet");
if let Some(pubkey) = next_channel_counterparty_node_id {
debug_assert_eq!(pubkey, path.hops[0].pubkey);
}
debug_assert_eq!(next_channel_counterparty_node_id, path.hops[0].pubkey);
let ev_completion_action = EventCompletionAction::ReleaseRAAChannelMonitorUpdate {
channel_funding_outpoint: next_channel_outpoint, channel_id: next_channel_id,
counterparty_node_id: path.hops[0].pubkey,
@ -7433,22 +7377,12 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
let completed_blocker = RAAMonitorUpdateBlockingAction::from_prev_hop_data(&hop_data);
self.claim_funds_from_hop(hop_data, payment_preimage, None,
|htlc_claim_value_msat, definitely_duplicate| {
let chan_to_release =
if let Some(node_id) = next_channel_counterparty_node_id {
Some(EventUnblockedChannel {
counterparty_node_id: node_id,
funding_txo: next_channel_outpoint,
channel_id: next_channel_id,
blocking_action: completed_blocker
})
} else {
// We can only get `None` here if we are processing a
// `ChannelMonitor`-originated event, in which case we
// don't care about ensuring we wake the downstream
// channel's monitor updating - the channel is already
// closed.
None
};
let chan_to_release = Some(EventUnblockedChannel {
counterparty_node_id: next_channel_counterparty_node_id,
funding_txo: next_channel_outpoint,
channel_id: next_channel_id,
blocking_action: completed_blocker
});
if definitely_duplicate && startup_replay {
// On startup we may get redundant claims which are related to
@ -7480,7 +7414,7 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
prev_user_channel_id,
next_user_channel_id,
prev_node_id,
next_node_id: next_channel_counterparty_node_id,
next_node_id: Some(next_channel_counterparty_node_id),
total_fee_earned_msat,
skimmed_fee_msat,
claim_from_onchain_tx: from_onchain,
@ -7721,24 +7655,12 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
(htlc_forwards, decode_update_add_htlcs)
}
fn channel_monitor_updated(&self, funding_txo: &OutPoint, channel_id: &ChannelId, highest_applied_update_id: u64, counterparty_node_id: Option<&PublicKey>) {
fn channel_monitor_updated(&self, channel_id: &ChannelId, highest_applied_update_id: u64, counterparty_node_id: &PublicKey) {
debug_assert!(self.total_consistency_lock.try_write().is_err()); // Caller holds read lock
let counterparty_node_id = match counterparty_node_id {
Some(cp_id) => cp_id.clone(),
None => {
// TODO: Once we can rely on the counterparty_node_id from the
// monitor event, this and the outpoint_to_peer map should be removed.
let outpoint_to_peer = self.outpoint_to_peer.lock().unwrap();
match outpoint_to_peer.get(funding_txo) {
Some(cp_id) => cp_id.clone(),
None => return,
}
}
};
let per_peer_state = self.per_peer_state.read().unwrap();
let mut peer_state_lock;
let peer_state_mutex_opt = per_peer_state.get(&counterparty_node_id);
let peer_state_mutex_opt = per_peer_state.get(counterparty_node_id);
if peer_state_mutex_opt.is_none() { return }
peer_state_lock = peer_state_mutex_opt.unwrap().lock().unwrap();
let peer_state = &mut *peer_state_lock;
@ -7749,7 +7671,7 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
pending.len()
} else { 0 };
let logger = WithContext::from(&self.logger, Some(counterparty_node_id), Some(*channel_id), None);
let logger = WithContext::from(&self.logger, Some(*counterparty_node_id), Some(*channel_id), None);
log_trace!(logger, "ChannelMonitor updated to {}. {} pending in-flight updates.",
highest_applied_update_id, remaining_in_flight);
@ -8283,41 +8205,30 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
fail_chan!("Already had channel with the new channel_id");
},
hash_map::Entry::Vacant(e) => {
let mut outpoint_to_peer_lock = self.outpoint_to_peer.lock().unwrap();
match outpoint_to_peer_lock.entry(monitor.get_funding_txo()) {
hash_map::Entry::Occupied(_) => {
fail_chan!("The funding_created message had the same funding_txid as an existing channel - funding is not possible");
},
hash_map::Entry::Vacant(i_e) => {
let monitor_res = self.chain_monitor.watch_channel(monitor.channel_id(), monitor);
if let Ok(persist_state) = monitor_res {
i_e.insert(chan.context.get_counterparty_node_id());
mem::drop(outpoint_to_peer_lock);
// There's no problem signing a counterparty's funding transaction if our monitor
// hasn't persisted to disk yet - we can't lose money on a transaction that we haven't
// accepted payment from yet. We do, however, need to wait to send our channel_ready
// until we have persisted our monitor.
if let Some(msg) = funding_msg_opt {
peer_state.pending_msg_events.push(MessageSendEvent::SendFundingSigned {
node_id: counterparty_node_id.clone(),
msg,
});
}
if let Some(funded_chan) = e.insert(Channel::from(chan)).as_funded_mut() {
handle_new_monitor_update!(self, persist_state, peer_state_lock, peer_state,
per_peer_state, funded_chan, INITIAL_MONITOR);
} else {
unreachable!("This must be a funded channel as we just inserted it.");
}
Ok(())
} else {
let logger = WithChannelContext::from(&self.logger, &chan.context, None);
log_error!(logger, "Persisting initial ChannelMonitor failed, implying the channel ID was duplicated");
fail_chan!("Duplicate channel ID");
}
let monitor_res = self.chain_monitor.watch_channel(monitor.channel_id(), monitor);
if let Ok(persist_state) = monitor_res {
// There's no problem signing a counterparty's funding transaction if our monitor
// hasn't persisted to disk yet - we can't lose money on a transaction that we haven't
// accepted payment from yet. We do, however, need to wait to send our channel_ready
// until we have persisted our monitor.
if let Some(msg) = funding_msg_opt {
peer_state.pending_msg_events.push(MessageSendEvent::SendFundingSigned {
node_id: *counterparty_node_id,
msg,
});
}
if let Some(funded_chan) = e.insert(Channel::from(chan)).as_funded_mut() {
handle_new_monitor_update!(self, persist_state, peer_state_lock, peer_state,
per_peer_state, funded_chan, INITIAL_MONITOR);
} else {
unreachable!("This must be a funded channel as we just inserted it.");
}
Ok(())
} else {
let logger = WithChannelContext::from(&self.logger, &chan.context, None);
log_error!(logger, "Persisting initial ChannelMonitor failed, implying the channel ID was duplicated");
fail_chan!("Duplicate channel ID");
}
}
}
@ -8912,7 +8823,7 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
}
};
self.claim_funds_internal(htlc_source, msg.payment_preimage.clone(),
Some(forwarded_htlc_value), skimmed_fee_msat, false, false, Some(*counterparty_node_id),
Some(forwarded_htlc_value), skimmed_fee_msat, false, false, *counterparty_node_id,
funding_txo, msg.channel_id, Some(next_user_channel_id),
);
@ -9501,67 +9412,58 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
for monitor_event in monitor_events.drain(..) {
match monitor_event {
MonitorEvent::HTLCEvent(htlc_update) => {
let logger = WithContext::from(&self.logger, counterparty_node_id, Some(channel_id), Some(htlc_update.payment_hash));
let logger = WithContext::from(&self.logger, Some(counterparty_node_id), Some(channel_id), Some(htlc_update.payment_hash));
if let Some(preimage) = htlc_update.payment_preimage {
log_trace!(logger, "Claiming HTLC with preimage {} from our monitor", preimage);
self.claim_funds_internal(htlc_update.source, preimage,
self.claim_funds_internal(
htlc_update.source, preimage,
htlc_update.htlc_value_satoshis.map(|v| v * 1000), None, true,
false, counterparty_node_id, funding_outpoint, channel_id, None);
false, counterparty_node_id, funding_outpoint, channel_id, None,
);
} else {
log_trace!(logger, "Failing HTLC with hash {} from our monitor", &htlc_update.payment_hash);
let receiver = HTLCDestination::NextHopChannel { node_id: counterparty_node_id, channel_id };
let receiver = HTLCDestination::NextHopChannel { node_id: Some(counterparty_node_id), channel_id };
let reason = HTLCFailReason::from_failure_code(0x4000 | 8);
self.fail_htlc_backwards_internal(&htlc_update.source, &htlc_update.payment_hash, &reason, receiver);
}
},
MonitorEvent::HolderForceClosed(_) | MonitorEvent::HolderForceClosedWithInfo { .. } => {
let counterparty_node_id_opt = match counterparty_node_id {
Some(cp_id) => Some(cp_id),
None => {
// TODO: Once we can rely on the counterparty_node_id from the
// monitor event, this and the outpoint_to_peer map should be removed.
let outpoint_to_peer = self.outpoint_to_peer.lock().unwrap();
outpoint_to_peer.get(&funding_outpoint).cloned()
}
};
if let Some(counterparty_node_id) = counterparty_node_id_opt {
let per_peer_state = self.per_peer_state.read().unwrap();
if let Some(peer_state_mutex) = per_peer_state.get(&counterparty_node_id) {
let mut peer_state_lock = peer_state_mutex.lock().unwrap();
let peer_state = &mut *peer_state_lock;
let pending_msg_events = &mut peer_state.pending_msg_events;
if let hash_map::Entry::Occupied(mut chan_entry) = peer_state.channel_by_id.entry(channel_id) {
let reason = if let MonitorEvent::HolderForceClosedWithInfo { reason, .. } = monitor_event {
reason
} else {
ClosureReason::HolderForceClosed { broadcasted_latest_txn: Some(true) }
};
let mut shutdown_res = chan_entry.get_mut().force_shutdown(false, reason.clone());
let chan = remove_channel_entry!(self, peer_state, chan_entry, shutdown_res);
failed_channels.push(shutdown_res);
if let Some(funded_chan) = chan.as_funded() {
if let Ok(update) = self.get_channel_update_for_broadcast(funded_chan) {
let mut pending_broadcast_messages = self.pending_broadcast_messages.lock().unwrap();
pending_broadcast_messages.push(MessageSendEvent::BroadcastChannelUpdate {
msg: update
});
}
pending_msg_events.push(MessageSendEvent::HandleError {
node_id: funded_chan.context.get_counterparty_node_id(),
action: msgs::ErrorAction::DisconnectPeer {
msg: Some(msgs::ErrorMessage {
channel_id: funded_chan.context.channel_id(),
data: reason.to_string()
})
},
let per_peer_state = self.per_peer_state.read().unwrap();
if let Some(peer_state_mutex) = per_peer_state.get(&counterparty_node_id) {
let mut peer_state_lock = peer_state_mutex.lock().unwrap();
let peer_state = &mut *peer_state_lock;
let pending_msg_events = &mut peer_state.pending_msg_events;
if let hash_map::Entry::Occupied(mut chan_entry) = peer_state.channel_by_id.entry(channel_id) {
let reason = if let MonitorEvent::HolderForceClosedWithInfo { reason, .. } = monitor_event {
reason
} else {
ClosureReason::HolderForceClosed { broadcasted_latest_txn: Some(true) }
};
let mut shutdown_res = chan_entry.get_mut().force_shutdown(false, reason.clone());
let chan = remove_channel_entry!(self, peer_state, chan_entry, shutdown_res);
failed_channels.push(shutdown_res);
if let Some(funded_chan) = chan.as_funded() {
if let Ok(update) = self.get_channel_update_for_broadcast(funded_chan) {
let mut pending_broadcast_messages = self.pending_broadcast_messages.lock().unwrap();
pending_broadcast_messages.push(MessageSendEvent::BroadcastChannelUpdate {
msg: update
});
}
pending_msg_events.push(MessageSendEvent::HandleError {
node_id: counterparty_node_id,
action: msgs::ErrorAction::DisconnectPeer {
msg: Some(msgs::ErrorMessage {
channel_id: funded_chan.context.channel_id(),
data: reason.to_string()
})
},
});
}
}
}
},
MonitorEvent::Completed { funding_txo, channel_id, monitor_update_id } => {
self.channel_monitor_updated(&funding_txo, &channel_id, monitor_update_id, counterparty_node_id.as_ref());
MonitorEvent::Completed { channel_id, monitor_update_id, .. } => {
self.channel_monitor_updated(&channel_id, monitor_update_id, &counterparty_node_id);
},
}
}
@ -9742,10 +9644,9 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
};
if let Some(mut shutdown_result) = shutdown_result {
let context = &chan.context();
let funding = chan.funding();
let logger = WithChannelContext::from(&self.logger, context, None);
log_trace!(logger, "Removing channel {} now that the signer is unblocked", context.channel_id());
locked_close_channel!(self, peer_state, context, funding, shutdown_result);
locked_close_channel!(self, peer_state, context, shutdown_result);
shutdown_results.push(shutdown_result);
false
} else {
@ -9787,7 +9688,7 @@ This indicates a bug inside LDK. Please report this error at https://github.com/
}
debug_assert_eq!(shutdown_result_opt.is_some(), funded_chan.is_shutdown());
if let Some(mut shutdown_result) = shutdown_result_opt {
locked_close_channel!(self, peer_state, &funded_chan.context, &funded_chan.funding, shutdown_result);
locked_close_channel!(self, peer_state, &funded_chan.context, shutdown_result);
shutdown_results.push(shutdown_result);
}
if let Some(tx) = tx_opt {
@ -11129,8 +11030,8 @@ where
}
// Clean up for removal.
let mut close_res = chan.force_shutdown(false, ClosureReason::DisconnectedPeer);
let (funding, context) = chan.funding_and_context_mut();
locked_close_channel!(self, peer_state, &context, funding, close_res);
let context = chan.context_mut();
locked_close_channel!(self, peer_state, &context, close_res);
failed_channels.push(close_res);
false
});
@ -11700,7 +11601,7 @@ where
// reorged out of the main chain. Close the channel.
let reason_message = format!("{}", reason);
let mut close_res = funded_channel.context.force_shutdown(&funded_channel.funding, true, reason);
locked_close_channel!(self, peer_state, &funded_channel.context, &funded_channel.funding, close_res);
locked_close_channel!(self, peer_state, &funded_channel.context, close_res);
failed_channels.push(close_res);
if let Ok(update) = self.get_channel_update_for_broadcast(&funded_channel) {
let mut pending_broadcast_messages = self.pending_broadcast_messages.lock().unwrap();
@ -13663,7 +13564,6 @@ where
let channel_count: u64 = Readable::read(reader)?;
let mut channel_id_set = hash_set_with_capacity(cmp::min(channel_count as usize, 128));
let mut per_peer_state = hash_map_with_capacity(cmp::min(channel_count as usize, MAX_ALLOC_SIZE/mem::size_of::<(PublicKey, Mutex<PeerState<SP>>)>()));
let mut outpoint_to_peer = hash_map_with_capacity(cmp::min(channel_count as usize, 128));
let mut short_to_chan_info = hash_map_with_capacity(cmp::min(channel_count as usize, 128));
let mut channel_closures = VecDeque::new();
let mut close_background_events = Vec::new();
@ -13674,7 +13574,6 @@ where
let logger = WithChannelContext::from(&args.logger, &channel.context, None);
let channel_id = channel.context.channel_id();
channel_id_set.insert(channel_id);
let funding_txo = channel.funding.get_funding_txo().ok_or(DecodeError::InvalidValue)?;
if let Some(ref mut monitor) = args.channel_monitors.get_mut(&channel_id) {
if channel.get_cur_holder_commitment_transaction_number() > monitor.get_cur_holder_commitment_number() ||
channel.get_revoked_counterparty_commitment_transaction_number() > monitor.get_min_seen_secret() ||
@ -13758,7 +13657,6 @@ where
if let Some(short_channel_id) = channel.context.get_short_channel_id() {
short_to_chan_info.insert(short_channel_id, (channel.context.get_counterparty_node_id(), channel.context.channel_id()));
}
outpoint_to_peer.insert(funding_txo, channel.context.get_counterparty_node_id());
per_peer_state.entry(channel.context.get_counterparty_node_id())
.or_insert_with(|| Mutex::new(empty_peer_state()))
.get_mut().unwrap()
@ -13791,26 +13689,26 @@ where
for (channel_id, monitor) in args.channel_monitors.iter() {
if !channel_id_set.contains(channel_id) {
let mut should_queue_fc_update = false;
if let Some(counterparty_node_id) = monitor.get_counterparty_node_id() {
// If the ChannelMonitor had any updates, we may need to update it further and
// thus track it in `closed_channel_monitor_update_ids`. If the channel never
// had any updates at all, there can't be any HTLCs pending which we need to
// claim.
// Note that a `ChannelMonitor` is created with `update_id` 0 and after we
// provide it with a closure update its `update_id` will be at 1.
if !monitor.no_further_updates_allowed() || monitor.get_latest_update_id() > 1 {
should_queue_fc_update = !monitor.no_further_updates_allowed();
let mut latest_update_id = monitor.get_latest_update_id();
if should_queue_fc_update {
latest_update_id += 1;
}
per_peer_state.entry(counterparty_node_id)
.or_insert_with(|| Mutex::new(empty_peer_state()))
.lock().unwrap()
.closed_channel_monitor_update_ids.entry(monitor.channel_id())
.and_modify(|v| *v = cmp::max(latest_update_id, *v))
.or_insert(latest_update_id);
let counterparty_node_id = monitor.get_counterparty_node_id();
// If the ChannelMonitor had any updates, we may need to update it further and
// thus track it in `closed_channel_monitor_update_ids`. If the channel never
// had any updates at all, there can't be any HTLCs pending which we need to
// claim.
// Note that a `ChannelMonitor` is created with `update_id` 0 and after we
// provide it with a closure update its `update_id` will be at 1.
if !monitor.no_further_updates_allowed() || monitor.get_latest_update_id() > 1 {
should_queue_fc_update = !monitor.no_further_updates_allowed();
let mut latest_update_id = monitor.get_latest_update_id();
if should_queue_fc_update {
latest_update_id += 1;
}
per_peer_state.entry(counterparty_node_id)
.or_insert_with(|| Mutex::new(empty_peer_state()))
.lock().unwrap()
.closed_channel_monitor_update_ids.entry(monitor.channel_id())
.and_modify(|v| *v = cmp::max(latest_update_id, *v))
.or_insert(latest_update_id);
}
if !should_queue_fc_update {
@ -13821,31 +13719,19 @@ where
let channel_id = monitor.channel_id();
log_info!(logger, "Queueing monitor update to ensure missing channel {} is force closed",
&channel_id);
let mut monitor_update = ChannelMonitorUpdate {
let monitor_update = ChannelMonitorUpdate {
update_id: monitor.get_latest_update_id().saturating_add(1),
counterparty_node_id: None,
updates: vec![ChannelMonitorUpdateStep::ChannelForceClosed { should_broadcast: true }],
channel_id: Some(monitor.channel_id()),
};
let funding_txo = monitor.get_funding_txo();
if let Some(counterparty_node_id) = monitor.get_counterparty_node_id() {
let update = BackgroundEvent::MonitorUpdateRegeneratedOnStartup {
counterparty_node_id,
funding_txo,
channel_id,
update: monitor_update,
};
close_background_events.push(update);
} else {
// This is a fairly old `ChannelMonitor` that hasn't seen an update to its
// off-chain state since LDK 0.0.118 (as in LDK 0.0.119 any off-chain
// `ChannelMonitorUpdate` will set the counterparty ID).
// Thus, we assume that it has no pending HTLCs and we will not need to
// generate a `ChannelMonitorUpdate` for it aside from this
// `ChannelForceClosed` one.
monitor_update.update_id = u64::MAX;
close_background_events.push(BackgroundEvent::ClosedMonitorUpdateRegeneratedOnStartup((funding_txo, channel_id, monitor_update)));
}
let update = BackgroundEvent::MonitorUpdateRegeneratedOnStartup {
counterparty_node_id,
funding_txo,
channel_id,
update: monitor_update,
};
close_background_events.push(update);
}
}
@ -14211,9 +14097,16 @@ where
// payments which are still in-flight via their on-chain state.
// We only rebuild the pending payments map if we were most recently serialized by
// 0.0.102+
for (_, monitor) in args.channel_monitors.iter() {
let counterparty_opt = outpoint_to_peer.get(&monitor.get_funding_txo());
if counterparty_opt.is_none() {
for (channel_id, monitor) in args.channel_monitors.iter() {
let mut is_channel_closed = false;
let counterparty_node_id = monitor.get_counterparty_node_id();
if let Some(peer_state_mtx) = per_peer_state.get(&counterparty_node_id) {
let mut peer_state_lock = peer_state_mtx.lock().unwrap();
let peer_state = &mut *peer_state_lock;
is_channel_closed = !peer_state.channel_by_id.contains_key(channel_id);
}
if is_channel_closed {
for (htlc_source, (htlc, _)) in monitor.get_pending_or_resolved_outbound_htlcs() {
let logger = WithChannelMonitor::from(&args.logger, monitor, Some(htlc.payment_hash));
if let HTLCSource::OutboundRoute { payment_id, session_priv, path, .. } = htlc_source {
@ -14400,11 +14293,7 @@ where
}
Some((htlc_source, payment_preimage, htlc.amount_msat,
// Check if `counterparty_opt.is_none()` to see if the
// downstream chan is closed (because we don't have a
// channel_id -> peer map entry).
counterparty_opt.is_none(),
counterparty_opt.cloned().or(monitor.get_counterparty_node_id()),
is_channel_closed, monitor.get_counterparty_node_id(),
monitor.get_funding_txo(), monitor.channel_id()))
} else { None }
} else {
@ -14636,7 +14525,6 @@ where
decode_update_add_htlcs: Mutex::new(decode_update_add_htlcs),
claimable_payments: Mutex::new(ClaimablePayments { claimable_payments, pending_claiming_payments: pending_claiming_payments.unwrap() }),
outbound_scid_aliases: Mutex::new(outbound_scid_aliases),
outpoint_to_peer: Mutex::new(outpoint_to_peer),
short_to_chan_info: FairRwLock::new(short_to_chan_info),
fake_scid_rand_bytes: fake_scid_rand_bytes.unwrap(),
@ -14786,9 +14674,8 @@ where
// without the new monitor persisted - we'll end up right back here on
// restart.
let previous_channel_id = claimable_htlc.prev_hop.channel_id;
let peer_node_id_opt = channel_manager.outpoint_to_peer.lock().unwrap()
.get(&claimable_htlc.prev_hop.outpoint).cloned();
if let Some(peer_node_id) = peer_node_id_opt {
let peer_node_id = monitor.get_counterparty_node_id();
{
let peer_state_mutex = per_peer_state.get(&peer_node_id).unwrap();
let mut peer_state_lock = peer_state_mutex.lock().unwrap();
let peer_state = &mut *peer_state_lock;
@ -14864,7 +14751,6 @@ where
#[cfg(test)]
mod tests {
use bitcoin::hashes::Hash;
use bitcoin::secp256k1::{PublicKey, Secp256k1, SecretKey};
use bitcoin::secp256k1::ecdh::SharedSecret;
use core::sync::atomic::Ordering;
@ -15523,125 +15409,6 @@ mod tests {
assert!(inbound_payment::verify(payment_hash, &payment_data, nodes[0].node.highest_seen_timestamp.load(Ordering::Acquire) as u64, &nodes[0].node.inbound_payment_key, &nodes[0].logger).is_ok());
}
#[test]
fn test_outpoint_to_peer_coverage() {
// Test that the `ChannelManager:outpoint_to_peer` contains channels which have been assigned
// a `channel_id` (i.e. have had the funding tx created), and that they are removed once
// the channel is successfully closed.
let chanmon_cfgs = create_chanmon_cfgs(2);
let node_cfgs = create_node_cfgs(2, &chanmon_cfgs);
let node_chanmgrs = create_node_chanmgrs(2, &node_cfgs, &[None, None]);
let nodes = create_network(2, &node_cfgs, &node_chanmgrs);
nodes[0].node.create_channel(nodes[1].node.get_our_node_id(), 1_000_000, 500_000_000, 42, None, None).unwrap();
let open_channel = get_event_msg!(nodes[0], MessageSendEvent::SendOpenChannel, nodes[1].node.get_our_node_id());
nodes[1].node.handle_open_channel(nodes[0].node.get_our_node_id(), &open_channel);
let accept_channel = get_event_msg!(nodes[1], MessageSendEvent::SendAcceptChannel, nodes[0].node.get_our_node_id());
nodes[0].node.handle_accept_channel(nodes[1].node.get_our_node_id(), &accept_channel);
let (temporary_channel_id, tx, funding_output) = create_funding_transaction(&nodes[0], &nodes[1].node.get_our_node_id(), 1_000_000, 42);
let channel_id = ChannelId::from_bytes(tx.compute_txid().to_byte_array());
{
// Ensure that the `outpoint_to_peer` map is empty until either party has received the
// funding transaction, and have the real `channel_id`.
assert_eq!(nodes[0].node.outpoint_to_peer.lock().unwrap().len(), 0);
assert_eq!(nodes[1].node.outpoint_to_peer.lock().unwrap().len(), 0);
}
nodes[0].node.funding_transaction_generated(temporary_channel_id, nodes[1].node.get_our_node_id(), tx.clone()).unwrap();
{
// Assert that `nodes[0]`'s `outpoint_to_peer` map is populated with the channel as soon as
// as it has the funding transaction.
let nodes_0_lock = nodes[0].node.outpoint_to_peer.lock().unwrap();
assert_eq!(nodes_0_lock.len(), 1);
assert!(nodes_0_lock.contains_key(&funding_output));
}
assert_eq!(nodes[1].node.outpoint_to_peer.lock().unwrap().len(), 0);
let funding_created_msg = get_event_msg!(nodes[0], MessageSendEvent::SendFundingCreated, nodes[1].node.get_our_node_id());
nodes[1].node.handle_funding_created(nodes[0].node.get_our_node_id(), &funding_created_msg);
{
let nodes_0_lock = nodes[0].node.outpoint_to_peer.lock().unwrap();
assert_eq!(nodes_0_lock.len(), 1);
assert!(nodes_0_lock.contains_key(&funding_output));
}
expect_channel_pending_event(&nodes[1], &nodes[0].node.get_our_node_id());
{
// Assert that `nodes[1]`'s `outpoint_to_peer` map is populated with the channel as
// soon as it has the funding transaction.
let nodes_1_lock = nodes[1].node.outpoint_to_peer.lock().unwrap();
assert_eq!(nodes_1_lock.len(), 1);
assert!(nodes_1_lock.contains_key(&funding_output));
}
check_added_monitors!(nodes[1], 1);
let funding_signed = get_event_msg!(nodes[1], MessageSendEvent::SendFundingSigned, nodes[0].node.get_our_node_id());
nodes[0].node.handle_funding_signed(nodes[1].node.get_our_node_id(), &funding_signed);
check_added_monitors!(nodes[0], 1);
expect_channel_pending_event(&nodes[0], &nodes[1].node.get_our_node_id());
let (channel_ready, _) = create_chan_between_nodes_with_value_confirm(&nodes[0], &nodes[1], &tx);
let (announcement, nodes_0_update, nodes_1_update) = create_chan_between_nodes_with_value_b(&nodes[0], &nodes[1], &channel_ready);
update_nodes_with_chan_announce(&nodes, 0, 1, &announcement, &nodes_0_update, &nodes_1_update);
nodes[0].node.close_channel(&channel_id, &nodes[1].node.get_our_node_id()).unwrap();
nodes[1].node.handle_shutdown(nodes[0].node.get_our_node_id(), &get_event_msg!(nodes[0], MessageSendEvent::SendShutdown, nodes[1].node.get_our_node_id()));
let nodes_1_shutdown = get_event_msg!(nodes[1], MessageSendEvent::SendShutdown, nodes[0].node.get_our_node_id());
nodes[0].node.handle_shutdown(nodes[1].node.get_our_node_id(), &nodes_1_shutdown);
let closing_signed_node_0 = get_event_msg!(nodes[0], MessageSendEvent::SendClosingSigned, nodes[1].node.get_our_node_id());
nodes[1].node.handle_closing_signed(nodes[0].node.get_our_node_id(), &closing_signed_node_0);
{
// Assert that the channel is kept in the `outpoint_to_peer` map for both nodes until the
// channel can be fully closed by both parties (i.e. no outstanding htlcs exists, the
// fee for the closing transaction has been negotiated and the parties has the other
// party's signature for the fee negotiated closing transaction.)
let nodes_0_lock = nodes[0].node.outpoint_to_peer.lock().unwrap();
assert_eq!(nodes_0_lock.len(), 1);
assert!(nodes_0_lock.contains_key(&funding_output));
}
{
// At this stage, `nodes[1]` has proposed a fee for the closing transaction in the
// `handle_closing_signed` call above. As `nodes[1]` has not yet received the signature
// from `nodes[0]` for the closing transaction with the proposed fee, the channel is
// kept in the `nodes[1]`'s `outpoint_to_peer` map.
let nodes_1_lock = nodes[1].node.outpoint_to_peer.lock().unwrap();
assert_eq!(nodes_1_lock.len(), 1);
assert!(nodes_1_lock.contains_key(&funding_output));
}
nodes[0].node.handle_closing_signed(nodes[1].node.get_our_node_id(), &get_event_msg!(nodes[1], MessageSendEvent::SendClosingSigned, nodes[0].node.get_our_node_id()));
{
// `nodes[0]` accepts `nodes[1]`'s proposed fee for the closing transaction, and
// therefore has all it needs to fully close the channel (both signatures for the
// closing transaction).
// Assert that the channel is removed from `nodes[0]`'s `outpoint_to_peer` map as it can be
// fully closed by `nodes[0]`.
assert_eq!(nodes[0].node.outpoint_to_peer.lock().unwrap().len(), 0);
// Assert that the channel is still in `nodes[1]`'s `outpoint_to_peer` map, as `nodes[1]`
// doesn't have `nodes[0]`'s signature for the closing transaction yet.
let nodes_1_lock = nodes[1].node.outpoint_to_peer.lock().unwrap();
assert_eq!(nodes_1_lock.len(), 1);
assert!(nodes_1_lock.contains_key(&funding_output));
}
let (_nodes_0_update, closing_signed_node_0) = get_closing_signed_broadcast!(nodes[0].node, nodes[1].node.get_our_node_id());
nodes[1].node.handle_closing_signed(nodes[0].node.get_our_node_id(), &closing_signed_node_0.unwrap());
{
// Assert that the channel has now been removed from both parties `outpoint_to_peer` map once
// they both have everything required to fully close the channel.
assert_eq!(nodes[1].node.outpoint_to_peer.lock().unwrap().len(), 0);
}
let (_nodes_1_update, _none) = get_closing_signed_broadcast!(nodes[1].node, nodes[0].node.get_our_node_id());
check_closed_event!(nodes[0], 1, ClosureReason::LocallyInitiatedCooperativeClosure, [nodes[1].node.get_our_node_id()], 1000000);
check_closed_event!(nodes[1], 1, ClosureReason::CounterpartyInitiatedCooperativeClosure, [nodes[0].node.get_our_node_id()], 1000000);
}
fn check_not_connected_to_peer_error<T>(res_err: Result<T, APIError>, expected_public_key: PublicKey) {
let expected_message = format!("Not connected to node: {}", expected_public_key);
check_api_error_message(expected_message, res_err)

View file

@ -9478,41 +9478,32 @@ pub fn test_peer_funding_sidechannel() {
let nodes = create_network(3, &node_cfgs, &node_chanmgrs);
let temp_chan_id_ab = exchange_open_accept_chan(&nodes[0], &nodes[1], 1_000_000, 0);
let temp_chan_id_ca = exchange_open_accept_chan(&nodes[2], &nodes[0], 1_000_000, 0);
let temp_chan_id_ca = exchange_open_accept_chan(&nodes[1], &nodes[0], 1_000_000, 0);
let (_, tx, funding_output) =
create_funding_transaction(&nodes[0], &nodes[1].node.get_our_node_id(), 1_000_000, 42);
let cs_funding_events = nodes[2].node.get_and_clear_pending_events();
let cs_funding_events = nodes[1].node.get_and_clear_pending_events();
assert_eq!(cs_funding_events.len(), 1);
match cs_funding_events[0] {
Event::FundingGenerationReady { .. } => {}
_ => panic!("Unexpected event {:?}", cs_funding_events),
}
nodes[2].node.funding_transaction_generated_unchecked(temp_chan_id_ca, nodes[0].node.get_our_node_id(), tx.clone(), funding_output.index).unwrap();
let funding_created_msg = get_event_msg!(nodes[2], MessageSendEvent::SendFundingCreated, nodes[0].node.get_our_node_id());
nodes[0].node.handle_funding_created(nodes[2].node.get_our_node_id(), &funding_created_msg);
get_event_msg!(nodes[0], MessageSendEvent::SendFundingSigned, nodes[2].node.get_our_node_id());
expect_channel_pending_event(&nodes[0], &nodes[2].node.get_our_node_id());
nodes[1].node.funding_transaction_generated_unchecked(temp_chan_id_ca, nodes[0].node.get_our_node_id(), tx.clone(), funding_output.index).unwrap();
let funding_created_msg = get_event_msg!(nodes[1], MessageSendEvent::SendFundingCreated, nodes[0].node.get_our_node_id());
nodes[0].node.handle_funding_created(nodes[1].node.get_our_node_id(), &funding_created_msg);
get_event_msg!(nodes[0], MessageSendEvent::SendFundingSigned, nodes[1].node.get_our_node_id());
expect_channel_pending_event(&nodes[0], &nodes[1].node.get_our_node_id());
check_added_monitors!(nodes[0], 1);
let res = nodes[0].node.funding_transaction_generated(temp_chan_id_ab, nodes[1].node.get_our_node_id(), tx.clone());
let err_msg = format!("{:?}", res.unwrap_err());
assert!(err_msg.contains("An existing channel using outpoint "));
assert!(err_msg.contains(" is open with peer"));
// Even though the last funding_transaction_generated errored, it still generated a
// SendFundingCreated. However, when the peer responds with a funding_signed it will send the
// appropriate error message.
let as_funding_created = get_event_msg!(nodes[0], MessageSendEvent::SendFundingCreated, nodes[1].node.get_our_node_id());
nodes[1].node.handle_funding_created(nodes[0].node.get_our_node_id(), &as_funding_created);
check_added_monitors!(nodes[1], 1);
expect_channel_pending_event(&nodes[1], &nodes[0].node.get_our_node_id());
let reason = ClosureReason::ProcessingError { err: format!("An existing channel using outpoint {} is open with peer {}", funding_output, nodes[2].node.get_our_node_id()), };
check_closed_events(&nodes[0], &[ExpectedCloseEvent::from_id_reason(ChannelId::v1_from_funding_outpoint(funding_output), true, reason)]);
let funding_signed = get_event_msg!(nodes[1], MessageSendEvent::SendFundingSigned, nodes[0].node.get_our_node_id());
nodes[0].node.handle_funding_signed(nodes[1].node.get_our_node_id(), &funding_signed);
assert!(err_msg.contains("An existing channel using ID"));
assert!(err_msg.contains("is open with peer"));
let channel_id = ChannelId::v1_from_funding_outpoint(funding_output);
let reason = ClosureReason::ProcessingError { err: format!("An existing channel using ID {} is open with peer {}", channel_id, nodes[1].node.get_our_node_id()), };
check_closed_events(&nodes[0], &[ExpectedCloseEvent::from_id_reason(temp_chan_id_ab, true, reason)]);
get_err_msg(&nodes[0], &nodes[1].node.get_our_node_id());
}
@ -9598,11 +9589,6 @@ pub fn test_duplicate_funding_err_in_funding() {
let reason = ClosureReason::ProcessingError { err };
let expected_closing = ExpectedCloseEvent::from_id_reason(real_channel_id, false, reason);
check_closed_events(&nodes[1], &[expected_closing]);
assert_eq!(
*nodes[1].node.outpoint_to_peer.lock().unwrap().get(&real_chan_funding_txo).unwrap(),
nodes[0].node.get_our_node_id()
);
}
#[xtest(feature = "_externalize_tests")]

View file

@ -506,7 +506,7 @@ impl<'a> chain::Watch<TestChannelSigner> for TestChainMonitor<'a> {
fn release_pending_monitor_events(
&self,
) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, Option<PublicKey>)> {
) -> Vec<(OutPoint, ChannelId, Vec<MonitorEvent>, PublicKey)> {
return self.chain_monitor.release_pending_monitor_events();
}
}

View file

@ -0,0 +1,5 @@
## API Updates (0.2)
* Upgrading to v0.2.0 is not allowed when a `ChannelMonitor` that does not track the channel's
`counterparty_node_id` is loaded. Upgrade to a v0.1.* release first and either send/route a
payment over the channel, or close it, before upgrading to v0.2.0.