From: Matt Corallo Date: Thu, 18 Nov 2021 21:54:13 +0000 (+0000) Subject: Disconect `announcement_signatures` sending from `funding_locked` X-Git-Tag: v0.0.105~24^2~2 X-Git-Url: http://git.bitcoin.ninja/index.cgi?a=commitdiff_plain;h=a265fc206260fb6909352d38c2eeef16a7f72432;p=rust-lightning Disconect `announcement_signatures` sending from `funding_locked` The spec actually requires we never send `announcement_signatures` (and, thus, `channel_announcement`s) until after six confirmations. However, we would happily have sent them prior to that as long as we exchange `funding_locked` messages with our countarparty. Thanks to re-broadcasting this issue is largely harmless, however it could have some negative interactions with less-robust peers. Much more importantly, this represents an important step towards supporting 0-conf channels, where `funding_locked` messages may be exchanged before we even have an SCID to construct the messages with. Because there is no ACK mechanism for `announcement_signatures` we rely on existing channel updates to stop rebroadcasting them - if we sent a `commitment_signed` after an `announcement_signatures` and later receive a `revoke_and_ack`, we know our counterparty also received our `announcement_signatures`. This may resolve some rare edge-cases where we send a `funding_locked` which our counterparty receives, but lose connection before the `announcement_signatures` (usually the very next message) arrives. Sadly, because the set of places where an `announcement_signatures` may now be generated more closely mirrors where `funding_locked` messages may be generated, but they are now separate, there is a substantial amount of code motion providing relevant parameters about current block information and ensuring we can return new `announcement_signatures` messages. --- diff --git a/lightning/src/ln/chanmon_update_fail_tests.rs b/lightning/src/ln/chanmon_update_fail_tests.rs index 9770638d..a6c5649c 100644 --- a/lightning/src/ln/chanmon_update_fail_tests.rs +++ b/lightning/src/ln/chanmon_update_fail_tests.rs @@ -20,6 +20,7 @@ use chain::channelmonitor::ChannelMonitor; use chain::transaction::OutPoint; use chain::{ChannelMonitorUpdateErr, Listen, Watch}; use ln::channelmanager::{ChannelManager, ChannelManagerReadArgs, RAACommitmentOrder, PaymentSendFailure}; +use ln::channel::AnnouncementSigsState; use ln::features::InitFeatures; use ln::msgs; use ln::msgs::{ChannelMessageHandler, RoutingMessageHandler}; @@ -1402,6 +1403,11 @@ fn monitor_failed_no_reestablish_response() { let node_chanmgrs = create_node_chanmgrs(2, &node_cfgs, &[None, None]); let mut nodes = create_network(2, &node_cfgs, &node_chanmgrs); let channel_id = create_announced_chan_between_nodes(&nodes, 0, 1, InitFeatures::known(), InitFeatures::known()).2; + { + let mut lock; + get_channel_ref!(nodes[0], lock, channel_id).announcement_sigs_state = AnnouncementSigsState::PeerReceived; + get_channel_ref!(nodes[1], lock, channel_id).announcement_sigs_state = AnnouncementSigsState::PeerReceived; + } // Route the payment and deliver the initial commitment_signed (with a monitor update failure // on receipt). diff --git a/lightning/src/ln/channel.rs b/lightning/src/ln/channel.rs index a923d397..e20bfc87 100644 --- a/lightning/src/ln/channel.rs +++ b/lightning/src/ln/channel.rs @@ -305,6 +305,26 @@ pub(super) enum ChannelUpdateStatus { Disabled, } +/// We track when we sent an `AnnouncementSignatures` to our peer in a few states, described here. +#[derive(PartialEq)] +pub enum AnnouncementSigsState { + /// We have not sent our peer an `AnnouncementSignatures` yet, or our peer disconnected since + /// we sent the last `AnnouncementSignatures`. + NotSent, + /// We sent an `AnnouncementSignatures` to our peer since the last time our peer disconnected. + /// This state never appears on disk - instead we write `NotSent`. + MessageSent, + /// We sent a `CommitmentSigned` after the last `AnnouncementSignatures` we sent. Because we + /// only ever have a single `CommitmentSigned` pending at once, if we sent one after sending + /// `AnnouncementSignatures` then we know the peer received our `AnnouncementSignatures` if + /// they send back a `RevokeAndACK`. + /// This state never appears on disk - instead we write `NotSent`. + Committed, + /// We received a `RevokeAndACK`, effectively ack-ing our `AnnouncementSignatures`, at this + /// point we no longer need to re-send our `AnnouncementSignatures` again on reconnect. + PeerReceived, +} + /// An enum indicating whether the local or remote side offered a given HTLC. enum HTLCInitiator { LocalOffered, @@ -399,6 +419,7 @@ pub(super) struct MonitorRestoreUpdates { pub finalized_claimed_htlcs: Vec, pub funding_broadcastable: Option, pub funding_locked: Option, + pub announcement_sigs: Option, } /// The return value of `channel_reestablish` @@ -409,6 +430,7 @@ pub(super) struct ReestablishResponses { pub order: RAACommitmentOrder, pub mon_update: Option, pub holding_cell_failed_htlcs: Vec<(HTLCSource, PaymentHash)>, + pub announcement_sigs: Option, pub shutdown_msg: Option, } @@ -466,6 +488,19 @@ pub(super) struct Channel { channel_id: [u8; 32], channel_state: u32, + + // When we reach max(6 blocks, minimum_depth), we need to send an AnnouncementSigs message to + // our peer. However, we want to make sure they received it, or else rebroadcast it when we + // next connect. + // We do so here, see `AnnouncementSigsSent` for more details on the state(s). + // Note that a number of our tests were written prior to the behavior here which retransmits + // AnnouncementSignatures until after an RAA completes, so the behavior is short-circuited in + // many tests. + #[cfg(any(test, feature = "_test_utils"))] + pub(crate) announcement_sigs_state: AnnouncementSigsState, + #[cfg(not(any(test, feature = "_test_utils")))] + announcement_sigs_state: AnnouncementSigsState, + secp_ctx: Secp256k1, channel_value_satoshis: u64, @@ -803,6 +838,7 @@ impl Channel { channel_id: keys_provider.get_secure_random_bytes(), channel_state: ChannelState::OurInitSent as u32, + announcement_sigs_state: AnnouncementSigsState::NotSent, secp_ctx, channel_value_satoshis, @@ -1101,6 +1137,7 @@ impl Channel { channel_id: msg.temporary_channel_id, channel_state: (ChannelState::OurInitSent as u32) | (ChannelState::TheirInitSent as u32), + announcement_sigs_state: AnnouncementSigsState::NotSent, secp_ctx, latest_monitor_update_id: 0, @@ -2068,7 +2105,10 @@ impl Channel { Ok((channel_monitor, self.funding_transaction.as_ref().cloned().unwrap())) } - pub fn funding_locked(&mut self, msg: &msgs::FundingLocked, logger: &L) -> Result<(), ChannelError> where L::Target: Logger { + /// Handles a funding_locked message from our peer. If we've already sent our funding_locked + /// and the channel is now usable (and public), this may generate an announcement_signatures to + /// reply with. + pub fn funding_locked(&mut self, msg: &msgs::FundingLocked, node_pk: PublicKey, genesis_block_hash: BlockHash, best_block: &BestBlock, logger: &L) -> Result, ChannelError> where L::Target: Logger { if self.channel_state & (ChannelState::PeerDisconnected as u32) == ChannelState::PeerDisconnected as u32 { self.workaround_lnd_bug_4006 = Some(msg.clone()); return Err(ChannelError::Ignore("Peer sent funding_locked when we needed a channel_reestablish. The peer is likely lnd, see https://github.com/lightningnetwork/lnd/issues/4006".to_owned())); @@ -2092,7 +2132,7 @@ impl Channel { return Err(ChannelError::Close("Peer sent a reconnect funding_locked with a different point".to_owned())); } // They probably disconnected/reconnected and re-sent the funding_locked, which is required - return Ok(()); + return Ok(None); } else { return Err(ChannelError::Close("Peer sent a funding_locked at a strange time".to_owned())); } @@ -2102,7 +2142,7 @@ impl Channel { log_info!(logger, "Received funding_locked from peer for channel {}", log_bytes!(self.channel_id())); - Ok(()) + Ok(self.get_announcement_sigs(node_pk, genesis_block_hash, best_block.height()).ok()) } /// Returns transaction if there is pending funding transaction that is yet to broadcast @@ -2992,6 +3032,10 @@ impl Channel { self.counterparty_cur_commitment_point = Some(msg.next_per_commitment_point); self.cur_counterparty_commitment_transaction_number -= 1; + if self.announcement_sigs_state == AnnouncementSigsState::Committed { + self.announcement_sigs_state = AnnouncementSigsState::PeerReceived; + } + log_trace!(logger, "Updating HTLCs on receipt of RAA in channel {}...", log_bytes!(self.channel_id())); let mut to_forward_infos = Vec::new(); let mut revoked_htlcs = Vec::new(); @@ -3268,6 +3312,11 @@ impl Channel { self.channel_state = ChannelState::ShutdownComplete as u32; return; } + + if self.announcement_sigs_state == AnnouncementSigsState::MessageSent || self.announcement_sigs_state == AnnouncementSigsState::Committed { + self.announcement_sigs_state = AnnouncementSigsState::NotSent; + } + // Upon reconnect we have to start the closing_signed dance over, but shutdown messages // will be retransmitted. self.last_sent_closing_fee = None; @@ -3344,7 +3393,7 @@ impl Channel { /// Indicates that the latest ChannelMonitor update has been committed by the client /// successfully and we should restore normal operation. Returns messages which should be sent /// to the remote side. - pub fn monitor_updating_restored(&mut self, logger: &L) -> MonitorRestoreUpdates where L::Target: Logger { + pub fn monitor_updating_restored(&mut self, logger: &L, node_pk: PublicKey, genesis_block_hash: BlockHash, best_block_height: u32) -> MonitorRestoreUpdates where L::Target: Logger { assert_eq!(self.channel_state & ChannelState::MonitorUpdateFailed as u32, ChannelState::MonitorUpdateFailed as u32); self.channel_state &= !(ChannelState::MonitorUpdateFailed as u32); @@ -3367,6 +3416,8 @@ impl Channel { }) } else { None }; + let announcement_sigs = self.get_announcement_sigs(node_pk, genesis_block_hash, best_block_height).ok(); + let mut accepted_htlcs = Vec::new(); mem::swap(&mut accepted_htlcs, &mut self.monitor_pending_forwards); let mut failed_htlcs = Vec::new(); @@ -3379,7 +3430,7 @@ impl Channel { self.monitor_pending_commitment_signed = false; return MonitorRestoreUpdates { raa: None, commitment_update: None, order: RAACommitmentOrder::RevokeAndACKFirst, - accepted_htlcs, failed_htlcs, finalized_claimed_htlcs, funding_broadcastable, funding_locked + accepted_htlcs, failed_htlcs, finalized_claimed_htlcs, funding_broadcastable, funding_locked, announcement_sigs }; } @@ -3398,7 +3449,7 @@ impl Channel { if commitment_update.is_some() { "a" } else { "no" }, if raa.is_some() { "an" } else { "no" }, match order { RAACommitmentOrder::CommitmentFirst => "commitment", RAACommitmentOrder::RevokeAndACKFirst => "RAA"}); MonitorRestoreUpdates { - raa, commitment_update, order, accepted_htlcs, failed_htlcs, finalized_claimed_htlcs, funding_broadcastable, funding_locked + raa, commitment_update, order, accepted_htlcs, failed_htlcs, finalized_claimed_htlcs, funding_broadcastable, funding_locked, announcement_sigs } } @@ -3512,7 +3563,9 @@ impl Channel { /// May panic if some calls other than message-handling calls (which will all Err immediately) /// have been called between remove_uncommitted_htlcs_and_mark_paused and this call. - pub fn channel_reestablish(&mut self, msg: &msgs::ChannelReestablish, logger: &L) -> Result where L::Target: Logger { + pub fn channel_reestablish(&mut self, msg: &msgs::ChannelReestablish, logger: &L, + node_pk: PublicKey, genesis_block_hash: BlockHash, best_block: &BestBlock) + -> Result where L::Target: Logger { if self.channel_state & (ChannelState::PeerDisconnected as u32) == 0 { // While BOLT 2 doesn't indicate explicitly we should error this channel here, it // almost certainly indicates we are going to end up out-of-sync in some way, so we @@ -3556,6 +3609,8 @@ impl Channel { }) } else { None }; + let announcement_sigs = self.get_announcement_sigs(node_pk, genesis_block_hash, best_block.height()).ok(); + if self.channel_state & (ChannelState::FundingSent as u32) == ChannelState::FundingSent as u32 { // If we're waiting on a monitor update, we shouldn't re-send any funding_locked's. if self.channel_state & (ChannelState::OurFundingLocked as u32) == 0 || @@ -3569,7 +3624,7 @@ impl Channel { raa: None, commitment_update: None, mon_update: None, order: RAACommitmentOrder::CommitmentFirst, holding_cell_failed_htlcs: Vec::new(), - shutdown_msg + shutdown_msg, announcement_sigs, }); } @@ -3583,7 +3638,7 @@ impl Channel { raa: None, commitment_update: None, mon_update: None, order: RAACommitmentOrder::CommitmentFirst, holding_cell_failed_htlcs: Vec::new(), - shutdown_msg + shutdown_msg, announcement_sigs, }); } @@ -3635,7 +3690,7 @@ impl Channel { panic!("Got non-channel-failing result from free_holding_cell_htlcs"), Ok((Some((commitment_update, monitor_update)), holding_cell_failed_htlcs)) => { Ok(ReestablishResponses { - funding_locked, shutdown_msg, + funding_locked, shutdown_msg, announcement_sigs, raa: required_revoke, commitment_update: Some(commitment_update), order: self.resend_order.clone(), @@ -3645,7 +3700,7 @@ impl Channel { }, Ok((None, holding_cell_failed_htlcs)) => { Ok(ReestablishResponses { - funding_locked, shutdown_msg, + funding_locked, shutdown_msg, announcement_sigs, raa: required_revoke, commitment_update: None, order: self.resend_order.clone(), @@ -3656,7 +3711,7 @@ impl Channel { } } else { Ok(ReestablishResponses { - funding_locked, shutdown_msg, + funding_locked, shutdown_msg, announcement_sigs, raa: required_revoke, commitment_update: None, order: self.resend_order.clone(), @@ -3674,14 +3729,14 @@ impl Channel { if self.channel_state & (ChannelState::MonitorUpdateFailed as u32) != 0 { self.monitor_pending_commitment_signed = true; Ok(ReestablishResponses { - funding_locked, shutdown_msg, + funding_locked, shutdown_msg, announcement_sigs, commitment_update: None, raa: None, mon_update: None, order: self.resend_order.clone(), holding_cell_failed_htlcs: Vec::new(), }) } else { Ok(ReestablishResponses { - funding_locked, shutdown_msg, + funding_locked, shutdown_msg, announcement_sigs, raa: required_revoke, commitment_update: Some(self.get_last_commitment_update(logger)), order: self.resend_order.clone(), @@ -4360,8 +4415,9 @@ impl Channel { /// When a transaction is confirmed, we check whether it is or spends the funding transaction /// In the first case, we store the confirmation height and calculating the short channel id. /// In the second, we simply return an Err indicating we need to be force-closed now. - pub fn transactions_confirmed(&mut self, block_hash: &BlockHash, height: u32, txdata: &TransactionData, logger: &L) - -> Result, ClosureReason> where L::Target: Logger { + pub fn transactions_confirmed(&mut self, block_hash: &BlockHash, height: u32, + txdata: &TransactionData, genesis_block_hash: BlockHash, node_pk: PublicKey, logger: &L) + -> Result<(Option, Option), ClosureReason> where L::Target: Logger { let non_shutdown_state = self.channel_state & (!MULTI_STATE_FLAGS); for &(index_in_block, tx) in txdata.iter() { if let Some(funding_txo) = self.get_funding_txo() { @@ -4408,7 +4464,8 @@ impl Channel { // may have already happened for this block). if let Some(funding_locked) = self.check_get_funding_locked(height) { log_info!(logger, "Sending a funding_locked to our peer for channel {}", log_bytes!(self.channel_id)); - return Ok(Some(funding_locked)); + let announcement_sigs = self.get_announcement_sigs(node_pk, genesis_block_hash, height).ok(); + return Ok((Some(funding_locked), announcement_sigs)); } } for inp in tx.input.iter() { @@ -4419,7 +4476,7 @@ impl Channel { } } } - Ok(None) + Ok((None, None)) } /// When a new block is connected, we check the height of the block against outbound holding @@ -4433,8 +4490,13 @@ impl Channel { /// /// May return some HTLCs (and their payment_hash) which have timed out and should be failed /// back. - pub fn best_block_updated(&mut self, height: u32, highest_header_time: u32, logger: &L) - -> Result<(Option, Vec<(HTLCSource, PaymentHash)>), ClosureReason> where L::Target: Logger { + pub fn best_block_updated(&mut self, height: u32, highest_header_time: u32, genesis_block_hash: BlockHash, node_pk: PublicKey, logger: &L) + -> Result<(Option, Vec<(HTLCSource, PaymentHash)>, Option), ClosureReason> where L::Target: Logger { + self.do_best_block_updated(height, highest_header_time, Some((genesis_block_hash, node_pk)), logger) + } + + fn do_best_block_updated(&mut self, height: u32, highest_header_time: u32, genesis_node_pk: Option<(BlockHash, PublicKey)>, logger: &L) + -> Result<(Option, Vec<(HTLCSource, PaymentHash)>, Option), ClosureReason> where L::Target: Logger { let mut timed_out_htlcs = Vec::new(); // This mirrors the check in ChannelManager::decode_update_add_htlc_onion, refusing to // forward an HTLC when our counterparty should almost certainly just fail it for expiring @@ -4455,8 +4517,11 @@ impl Channel { self.update_time_counter = cmp::max(self.update_time_counter, highest_header_time); if let Some(funding_locked) = self.check_get_funding_locked(height) { + let announcement_sigs = if let Some((genesis_block_hash, node_pk)) = genesis_node_pk { + self.get_announcement_sigs(node_pk, genesis_block_hash, height).ok() + } else { None }; log_info!(logger, "Sending a funding_locked to our peer for channel {}", log_bytes!(self.channel_id)); - return Ok((Some(funding_locked), timed_out_htlcs)); + return Ok((Some(funding_locked), timed_out_htlcs, announcement_sigs)); } let non_shutdown_state = self.channel_state & (!MULTI_STATE_FLAGS); @@ -4488,7 +4553,10 @@ impl Channel { return Err(ClosureReason::FundingTimedOut); } - Ok((None, timed_out_htlcs)) + let announcement_sigs = if let Some((genesis_block_hash, node_pk)) = genesis_node_pk { + self.get_announcement_sigs(node_pk, genesis_block_hash, height).ok() + } else { None }; + Ok((None, timed_out_htlcs, announcement_sigs)) } /// Indicates the funding transaction is no longer confirmed in the main chain. This may @@ -4503,10 +4571,11 @@ impl Channel { // larger. If we don't know that time has moved forward, we can just set it to the last // time we saw and it will be ignored. let best_time = self.update_time_counter; - match self.best_block_updated(reorg_height, best_time, logger) { - Ok((funding_locked, timed_out_htlcs)) => { + match self.do_best_block_updated(reorg_height, best_time, None, logger) { + Ok((funding_locked, timed_out_htlcs, announcement_sigs)) => { assert!(funding_locked.is_none(), "We can't generate a funding with 0 confirmations?"); assert!(timed_out_htlcs.is_empty(), "We can't have accepted HTLCs with a timeout before our funding confirmation?"); + assert!(announcement_sigs.is_none(), "We can't generate an announcement_sigs with 0 confirmations?"); Ok(()) }, Err(e) => Err(e) @@ -4690,10 +4759,27 @@ impl Channel { Ok(msg) } - pub fn get_announcement_sigs(&self, node_pk: PublicKey, genesis_block_hash: BlockHash) -> Result { + fn get_announcement_sigs(&mut self, node_pk: PublicKey, genesis_block_hash: BlockHash, best_block_height: u32) -> Result { + if self.funding_tx_confirmation_height == 0 || self.funding_tx_confirmation_height + 5 > best_block_height { + return Err(ChannelError::Ignore("Funding not yet fully confirmed".to_owned())); + } + + if !self.is_usable() { + return Err(ChannelError::Ignore("Channel not yet available for use".to_owned())); + } + + if self.channel_state & ChannelState::PeerDisconnected as u32 != 0 { + return Err(ChannelError::Ignore("Peer currently disconnected".to_owned())); + } + + if self.announcement_sigs_state != AnnouncementSigsState::NotSent { + return Err(ChannelError::Ignore("Announcement signatures already sent".to_owned())); + } + let announcement = self.get_channel_announcement(node_pk, genesis_block_hash)?; let (our_node_sig, our_bitcoin_sig) = self.holder_signer.sign_channel_announcement(&announcement, &self.secp_ctx) .map_err(|_| ChannelError::Ignore("Signer rejected channel_announcement".to_owned()))?; + self.announcement_sigs_state = AnnouncementSigsState::MessageSent; Ok(msgs::AnnouncementSignatures { channel_id: self.channel_id(), @@ -4726,7 +4812,7 @@ impl Channel { /// Processes an incoming announcement_signatures message, providing a fully-signed /// channel_announcement message which we can broadcast and storing our counterparty's /// signatures for later reconstruction/rebroadcast of the channel_announcement. - pub fn announcement_signatures(&mut self, our_node_id: PublicKey, chain_hash: BlockHash, msg: &msgs::AnnouncementSignatures) -> Result { + pub fn announcement_signatures(&mut self, our_node_id: PublicKey, chain_hash: BlockHash, best_block_height: u32, msg: &msgs::AnnouncementSignatures) -> Result { let announcement = self.get_channel_announcement(our_node_id.clone(), chain_hash)?; let msghash = hash_to_message!(&Sha256d::hash(&announcement.encode()[..])[..]); @@ -4743,13 +4829,20 @@ impl Channel { } self.announcement_sigs = Some((msg.node_signature, msg.bitcoin_signature)); + if self.funding_tx_confirmation_height == 0 || self.funding_tx_confirmation_height + 5 > best_block_height { + return Err(ChannelError::Ignore( + "Got announcement_signatures prior to the required six confirmations - we may not have received a block yet that our peer has".to_owned())); + } self.sign_channel_announcement(our_node_id, announcement) } /// Gets a signed channel_announcement for this channel, if we previously received an /// announcement_signatures from our counterparty. - pub fn get_signed_channel_announcement(&self, our_node_id: PublicKey, chain_hash: BlockHash) -> Option { + pub fn get_signed_channel_announcement(&self, our_node_id: PublicKey, chain_hash: BlockHash, best_block_height: u32) -> Option { + if self.funding_tx_confirmation_height == 0 || self.funding_tx_confirmation_height + 5 > best_block_height { + return None; + } let announcement = match self.get_channel_announcement(our_node_id.clone(), chain_hash) { Ok(res) => res, Err(_) => return None, @@ -5030,6 +5123,10 @@ impl Channel { Err(e) => return Err(e), }; + if self.announcement_sigs_state == AnnouncementSigsState::MessageSent { + self.announcement_sigs_state = AnnouncementSigsState::Committed; + } + self.latest_monitor_update_id += 1; let monitor_update = ChannelMonitorUpdate { update_id: self.latest_monitor_update_id, @@ -5289,6 +5386,29 @@ impl Readable for ChannelUpdateStatus { } } +impl Writeable for AnnouncementSigsState { + fn write(&self, writer: &mut W) -> Result<(), io::Error> { + // We only care about writing out the current state as if we had just disconnected, at + // which point we always set anything but AnnouncementSigsReceived to NotSent. + match self { + AnnouncementSigsState::NotSent => 0u8.write(writer), + AnnouncementSigsState::MessageSent => 0u8.write(writer), + AnnouncementSigsState::Committed => 0u8.write(writer), + AnnouncementSigsState::PeerReceived => 1u8.write(writer), + } + } +} + +impl Readable for AnnouncementSigsState { + fn read(reader: &mut R) -> Result { + Ok(match ::read(reader)? { + 0 => AnnouncementSigsState::NotSent, + 1 => AnnouncementSigsState::PeerReceived, + _ => return Err(DecodeError::InvalidValue), + }) + } +} + impl Writeable for Channel { fn write(&self, writer: &mut W) -> Result<(), io::Error> { // Note that we write out as if remove_uncommitted_htlcs_and_mark_paused had just been @@ -5557,6 +5677,7 @@ impl Writeable for Channel { (11, self.monitor_pending_finalized_fulfills, vec_type), (13, self.channel_creation_height, required), (15, preimages, vec_type), + (17, self.announcement_sigs_state, required), }); Ok(()) @@ -5809,6 +5930,10 @@ impl<'a, Signer: Sign, K: Deref> ReadableArgs<(&'a K, u32)> for Channel let mut channel_creation_height = Some(serialized_height); let mut preimages_opt: Option>> = None; + // If we read an old Channel, for simplicity we just treat it as "we never sent an + // AnnouncementSignatures" which implies we'll re-send it on reconnect, but that's fine. + let mut announcement_sigs_state = Some(AnnouncementSigsState::NotSent); + read_tlv_fields!(reader, { (0, announcement_sigs, option), (1, minimum_depth, option), @@ -5822,6 +5947,7 @@ impl<'a, Signer: Sign, K: Deref> ReadableArgs<(&'a K, u32)> for Channel (11, monitor_pending_finalized_fulfills, vec_type), (13, channel_creation_height, option), (15, preimages_opt, vec_type), + (17, announcement_sigs_state, option), }); if let Some(preimages) = preimages_opt { @@ -5864,6 +5990,7 @@ impl<'a, Signer: Sign, K: Deref> ReadableArgs<(&'a K, u32)> for Channel config: config.unwrap(), channel_id, channel_state, + announcement_sigs_state: announcement_sigs_state.unwrap(), secp_ctx, channel_value_satoshis, diff --git a/lightning/src/ln/channelmanager.rs b/lightning/src/ln/channelmanager.rs index c6701015..2097dfda 100644 --- a/lightning/src/ln/channelmanager.rs +++ b/lightning/src/ln/channelmanager.rs @@ -1533,7 +1533,7 @@ macro_rules! maybe_break_monitor_err { macro_rules! handle_chan_restoration_locked { ($self: ident, $channel_lock: expr, $channel_state: expr, $channel_entry: expr, $raa: expr, $commitment_update: expr, $order: expr, $chanmon_update: expr, - $pending_forwards: expr, $funding_broadcastable: expr, $funding_locked: expr) => { { + $pending_forwards: expr, $funding_broadcastable: expr, $funding_locked: expr, $announcement_sigs: expr) => { { let mut htlc_forwards = None; let counterparty_node_id = $channel_entry.get().get_counterparty_node_id(); @@ -1568,14 +1568,14 @@ macro_rules! handle_chan_restoration_locked { node_id: counterparty_node_id, msg, }); - if let Some(announcement_sigs) = $self.get_announcement_sigs($channel_entry.get()) { - $channel_state.pending_msg_events.push(events::MessageSendEvent::SendAnnouncementSignatures { - node_id: counterparty_node_id, - msg: announcement_sigs, - }); - } $channel_state.short_to_id.insert($channel_entry.get().get_short_channel_id().unwrap(), $channel_entry.get().channel_id()); } + if let Some(msg) = $announcement_sigs { + $channel_state.pending_msg_events.push(events::MessageSendEvent::SendAnnouncementSignatures { + node_id: counterparty_node_id, + msg, + }); + } let funding_broadcastable: Option = $funding_broadcastable; // Force type-checking to resolve if let Some(monitor_update) = chanmon_update { @@ -2891,14 +2891,6 @@ impl ChannelMana }) } - fn get_announcement_sigs(&self, chan: &Channel) -> Option { - if !chan.should_announce() { - log_trace!(self.logger, "Can't send announcement_signatures for private channel {}", log_bytes!(chan.channel_id())); - return None - } - chan.get_announcement_sigs(self.get_our_node_id(), self.genesis_hash.clone()).ok() - } - #[allow(dead_code)] // Messages of up to 64KB should never end up more than half full with addresses, as that would // be absurd. We ensure this by checking that at least 500 (our stated public contract on when @@ -2956,7 +2948,7 @@ impl ChannelMana let mut announced_chans = false; for (_, chan) in channel_state.by_id.iter() { - if let Some(msg) = chan.get_signed_channel_announcement(self.get_our_node_id(), self.genesis_hash.clone()) { + if let Some(msg) = chan.get_signed_channel_announcement(self.get_our_node_id(), self.genesis_hash.clone(), self.best_block.read().unwrap().height()) { channel_state.pending_msg_events.push(events::MessageSendEvent::BroadcastChannelAnnouncement { msg, update_msg: match self.get_channel_update_for_broadcast(chan) { @@ -4064,18 +4056,19 @@ impl ChannelMana return; } - let updates = channel.get_mut().monitor_updating_restored(&self.logger); - let channel_update = if updates.funding_locked.is_some() && channel.get().is_usable() && !channel.get().should_announce() { + let updates = channel.get_mut().monitor_updating_restored(&self.logger, self.get_our_node_id(), self.genesis_hash, self.best_block.read().unwrap().height()); + let channel_update = if updates.funding_locked.is_some() && channel.get().is_usable() { // We only send a channel_update in the case where we are just now sending a - // funding_locked and the channel is in a usable state. Further, we rely on the - // normal announcement_signatures process to send a channel_update for public - // channels, only generating a unicast channel_update if this is a private channel. + // funding_locked 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. Some(events::MessageSendEvent::SendChannelUpdate { node_id: channel.get().get_counterparty_node_id(), msg: self.get_channel_update_for_unicast(channel.get()).unwrap(), }) } else { None }; - chan_restoration_res = handle_chan_restoration_locked!(self, channel_lock, channel_state, channel, updates.raa, updates.commitment_update, updates.order, None, updates.accepted_htlcs, updates.funding_broadcastable, updates.funding_locked); + chan_restoration_res = handle_chan_restoration_locked!(self, channel_lock, channel_state, channel, updates.raa, updates.commitment_update, updates.order, None, updates.accepted_htlcs, updates.funding_broadcastable, updates.funding_locked, updates.announcement_sigs); if let Some(upd) = channel_update { channel_state.pending_msg_events.push(upd); } @@ -4241,23 +4234,21 @@ impl ChannelMana if chan.get().get_counterparty_node_id() != *counterparty_node_id { return Err(MsgHandleErrInternal::send_err_msg_no_close("Got a message for a channel from the wrong node!".to_owned(), msg.channel_id)); } - try_chan_entry!(self, chan.get_mut().funding_locked(&msg, &self.logger), channel_state, chan); - if let Some(announcement_sigs) = self.get_announcement_sigs(chan.get()) { - log_trace!(self.logger, "Sending announcement_signatures for {} in response to funding_locked", log_bytes!(chan.get().channel_id())); - // If we see locking block before receiving remote funding_locked, we broadcast our - // announcement_sigs at remote funding_locked reception. If we receive remote - // funding_locked before seeing locking block, we broadcast our announcement_sigs at locking - // block connection. We should guanrantee to broadcast announcement_sigs to our peer whatever - // the order of the events but our peer may not receive it due to disconnection. The specs - // lacking an acknowledgement for announcement_sigs we may have to re-send them at peer - // connection in the future if simultaneous misses by both peers due to network/hardware - // failures is an issue. Note, to achieve its goal, only one of the announcement_sigs needs - // to be received, from then sigs are going to be flood to the whole network. + let announcement_sigs_opt = try_chan_entry!(self, chan.get_mut().funding_locked(&msg, self.get_our_node_id(), + self.genesis_hash.clone(), &self.best_block.read().unwrap(), &self.logger), channel_state, chan); + if let Some(announcement_sigs) = announcement_sigs_opt { + log_trace!(self.logger, "Sending announcement_signatures for channel {}", log_bytes!(chan.get().channel_id())); channel_state.pending_msg_events.push(events::MessageSendEvent::SendAnnouncementSignatures { node_id: counterparty_node_id.clone(), msg: announcement_sigs, }); } else if chan.get().is_usable() { + // If we're sending an announcement_signatures, we'll send the (public) + // channel_update after sending a channel_announcement when we receive our + // counterparty's announcement_signatures. Thus, we only bother to send a + // channel_update here if the channel is not public, i.e. we're not sending an + // announcement_signatures. + log_trace!(self.logger, "Sending private initial channel_update for our counterparty on channel {}", log_bytes!(chan.get().channel_id())); channel_state.pending_msg_events.push(events::MessageSendEvent::SendChannelUpdate { node_id: counterparty_node_id.clone(), msg: self.get_channel_update_for_unicast(chan.get()).unwrap(), @@ -4661,7 +4652,8 @@ impl ChannelMana } channel_state.pending_msg_events.push(events::MessageSendEvent::BroadcastChannelAnnouncement { - msg: try_chan_entry!(self, chan.get_mut().announcement_signatures(self.get_our_node_id(), self.genesis_hash.clone(), msg), channel_state, chan), + msg: try_chan_entry!(self, chan.get_mut().announcement_signatures( + self.get_our_node_id(), self.genesis_hash.clone(), self.best_block.read().unwrap().height(), msg), channel_state, chan), // Note that announcement_signatures fails if the channel cannot be announced, // so get_channel_update_for_broadcast will never fail by the time we get here. update_msg: self.get_channel_update_for_broadcast(chan.get()).unwrap(), @@ -4722,7 +4714,9 @@ impl ChannelMana // disconnect, so Channel's reestablish will never hand us any holding cell // freed HTLCs to fail backwards. If in the future we no longer drop pending // add-HTLCs on disconnect, we may be handed HTLCs to fail backwards here. - let responses = try_chan_entry!(self, chan.get_mut().channel_reestablish(msg, &self.logger), channel_state, chan); + let responses = try_chan_entry!(self, chan.get_mut().channel_reestablish( + msg, &self.logger, self.our_network_pubkey.clone(), self.genesis_hash, + &*self.best_block.read().unwrap()), channel_state, chan); let mut channel_update = None; if let Some(msg) = responses.shutdown_msg { channel_state.pending_msg_events.push(events::MessageSendEvent::SendShutdown { @@ -4741,7 +4735,7 @@ impl ChannelMana let need_lnd_workaround = chan.get_mut().workaround_lnd_bug_4006.take(); chan_restoration_res = handle_chan_restoration_locked!( self, channel_state_lock, channel_state, chan, responses.raa, responses.commitment_update, responses.order, - responses.mon_update, Vec::new(), None, responses.funding_locked); + responses.mon_update, Vec::new(), None, responses.funding_locked, responses.announcement_sigs); if let Some(upd) = channel_update { channel_state.pending_msg_events.push(upd); } @@ -5248,7 +5242,7 @@ where *best_block = BestBlock::new(header.prev_blockhash, new_height) } - self.do_chain_event(Some(new_height), |channel| channel.best_block_updated(new_height, header.time, &self.logger)); + self.do_chain_event(Some(new_height), |channel| channel.best_block_updated(new_height, header.time, self.genesis_hash.clone(), self.get_our_node_id(), &self.logger)); } } @@ -5269,7 +5263,8 @@ where log_trace!(self.logger, "{} transactions included in block {} at height {} provided", txdata.len(), block_hash, height); let _persistence_guard = PersistenceNotifierGuard::notify_on_drop(&self.total_consistency_lock, &self.persistence_notifier); - self.do_chain_event(Some(height), |channel| channel.transactions_confirmed(&block_hash, height, txdata, &self.logger).map(|a| (a, Vec::new()))); + self.do_chain_event(Some(height), |channel| channel.transactions_confirmed(&block_hash, height, txdata, self.genesis_hash.clone(), self.get_our_node_id(), &self.logger) + .map(|(a, b)| (a, Vec::new(), b))); } fn best_block_updated(&self, header: &BlockHeader, height: u32) { @@ -5284,7 +5279,7 @@ where *self.best_block.write().unwrap() = BestBlock::new(block_hash, height); - self.do_chain_event(Some(height), |channel| channel.best_block_updated(height, header.time, &self.logger)); + self.do_chain_event(Some(height), |channel| channel.best_block_updated(height, header.time, self.genesis_hash.clone(), self.get_our_node_id(), &self.logger)); macro_rules! max_time { ($timestamp: expr) => { @@ -5341,9 +5336,9 @@ where self.do_chain_event(None, |channel| { if let Some(funding_txo) = channel.get_funding_txo() { if funding_txo.txid == *txid { - channel.funding_transaction_unconfirmed(&self.logger).map(|_| (None, Vec::new())) - } else { Ok((None, Vec::new())) } - } else { Ok((None, Vec::new())) } + channel.funding_transaction_unconfirmed(&self.logger).map(|()| (None, Vec::new(), None)) + } else { Ok((None, Vec::new(), None)) } + } else { Ok((None, Vec::new(), None)) } }); } } @@ -5359,7 +5354,7 @@ where /// Calls a function which handles an on-chain event (blocks dis/connected, transactions /// un/confirmed, etc) on each channel, handling any resulting errors or messages generated by /// the function. - fn do_chain_event) -> Result<(Option, Vec<(HTLCSource, PaymentHash)>), ClosureReason>> + fn do_chain_event) -> Result<(Option, Vec<(HTLCSource, PaymentHash)>, Option), ClosureReason>> (&self, height_opt: Option, f: FN) { // Note that we MUST NOT end up calling methods on self.chain_monitor here - we're called // during initialization prior to the chain_monitor being fully configured in some cases. @@ -5374,7 +5369,7 @@ where let pending_msg_events = &mut channel_state.pending_msg_events; channel_state.by_id.retain(|_, channel| { let res = f(channel); - if let Ok((chan_res, mut timed_out_pending_htlcs)) = res { + if let Ok((funding_locked_opt, mut timed_out_pending_htlcs, announcement_sigs)) = res { for (source, payment_hash) in timed_out_pending_htlcs.drain(..) { let chan_update = self.get_channel_update_for_unicast(&channel).map(|u| u.encode_with_len()).unwrap(); // Cannot add/recv HTLCs before we have a short_id so unwrap is safe timed_out_htlcs.push((source, payment_hash, HTLCFailReason::Reason { @@ -5382,28 +5377,39 @@ where data: chan_update, })); } - if let Some(funding_locked) = chan_res { + if let Some(funding_locked) = funding_locked_opt { pending_msg_events.push(events::MessageSendEvent::SendFundingLocked { node_id: channel.get_counterparty_node_id(), msg: funding_locked, }); - if let Some(announcement_sigs) = self.get_announcement_sigs(channel) { - log_trace!(self.logger, "Sending funding_locked and announcement_signatures for {}", log_bytes!(channel.channel_id())); - pending_msg_events.push(events::MessageSendEvent::SendAnnouncementSignatures { - node_id: channel.get_counterparty_node_id(), - msg: announcement_sigs, - }); - } else if channel.is_usable() { - log_trace!(self.logger, "Sending funding_locked WITHOUT announcement_signatures but with private channel_update for our counterparty on channel {}", log_bytes!(channel.channel_id())); + if channel.is_usable() { + log_trace!(self.logger, "Sending funding_locked with private initial channel_update for our counterparty on channel {}", log_bytes!(channel.channel_id())); pending_msg_events.push(events::MessageSendEvent::SendChannelUpdate { node_id: channel.get_counterparty_node_id(), msg: self.get_channel_update_for_unicast(channel).unwrap(), }); } else { - log_trace!(self.logger, "Sending funding_locked WITHOUT announcement_signatures for {}", log_bytes!(channel.channel_id())); + log_trace!(self.logger, "Sending funding_locked WITHOUT channel_update for {}", log_bytes!(channel.channel_id())); } short_to_id.insert(channel.get_short_channel_id().unwrap(), channel.channel_id()); } + if let Some(announcement_sigs) = announcement_sigs { + log_trace!(self.logger, "Sending announcement_signatures for channel {}", log_bytes!(channel.channel_id())); + pending_msg_events.push(events::MessageSendEvent::SendAnnouncementSignatures { + node_id: channel.get_counterparty_node_id(), + msg: announcement_sigs, + }); + if let Some(height) = height_opt { + if let Some(announcement) = channel.get_signed_channel_announcement(self.get_our_node_id(), self.genesis_hash, height) { + pending_msg_events.push(events::MessageSendEvent::BroadcastChannelAnnouncement { + msg: announcement, + // Note that announcement_signatures fails if the channel cannot be announced, + // so get_channel_update_for_broadcast will never fail by the time we get here. + update_msg: self.get_channel_update_for_broadcast(channel).unwrap(), + }); + } + } + } } else if let Err(reason) = res { if let Some(short_id) = channel.get_short_channel_id() { short_to_id.remove(&short_id); diff --git a/lightning/src/ln/functional_test_utils.rs b/lightning/src/ln/functional_test_utils.rs index 5ec16f90..b7d7011c 100644 --- a/lightning/src/ln/functional_test_utils.rs +++ b/lightning/src/ln/functional_test_utils.rs @@ -600,7 +600,14 @@ pub fn create_chan_between_nodes_with_value_confirm_first<'a, 'b, 'c, 'd>(node_r pub fn create_chan_between_nodes_with_value_confirm_second<'a, 'b, 'c>(node_recv: &Node<'a, 'b, 'c>, node_conf: &Node<'a, 'b, 'c>) -> ((msgs::FundingLocked, msgs::AnnouncementSignatures), [u8; 32]) { let channel_id; let events_6 = node_conf.node.get_and_clear_pending_msg_events(); - assert_eq!(events_6.len(), 2); + assert_eq!(events_6.len(), 3); + let announcement_sigs_idx = if let MessageSendEvent::SendChannelUpdate { ref node_id, msg: _ } = events_6[1] { + assert_eq!(*node_id, node_recv.node.get_our_node_id()); + 2 + } else if let MessageSendEvent::SendChannelUpdate { ref node_id, msg: _ } = events_6[2] { + assert_eq!(*node_id, node_recv.node.get_our_node_id()); + 1 + } else { panic!("Unexpected event: {:?}", events_6[1]); }; ((match events_6[0] { MessageSendEvent::SendFundingLocked { ref node_id, ref msg } => { channel_id = msg.channel_id.clone(); @@ -608,7 +615,7 @@ pub fn create_chan_between_nodes_with_value_confirm_second<'a, 'b, 'c>(node_recv msg.clone() }, _ => panic!("Unexpected event"), - }, match events_6[1] { + }, match events_6[announcement_sigs_idx] { MessageSendEvent::SendAnnouncementSignatures { ref node_id, ref msg } => { assert_eq!(*node_id, node_recv.node.get_our_node_id()); msg.clone() @@ -2002,8 +2009,7 @@ macro_rules! handle_chan_reestablish_msgs { idx += 1; RAACommitmentOrder::CommitmentFirst }, - &MessageSendEvent::SendChannelUpdate { .. } => RAACommitmentOrder::CommitmentFirst, - _ => panic!("Unexpected event"), + _ => RAACommitmentOrder::CommitmentFirst, } } else { RAACommitmentOrder::CommitmentFirst @@ -2023,16 +2029,18 @@ macro_rules! handle_chan_reestablish_msgs { commitment_update = Some(updates.clone()); idx += 1; }, - &MessageSendEvent::SendChannelUpdate { .. } => {}, - _ => panic!("Unexpected event"), + _ => {}, } } if let Some(&MessageSendEvent::SendChannelUpdate { ref node_id, ref msg }) = msg_events.get(idx) { assert_eq!(*node_id, $dst_node.node.get_our_node_id()); + idx += 1; assert_eq!(msg.contents.flags & 2, 0); // "disabled" flag must not be set as we just reconnected. } + assert_eq!(msg_events.len(), idx); + (funding_locked, revoke_and_ack, commitment_update, order) } } @@ -2105,9 +2113,9 @@ pub fn reconnect_nodes<'a, 'b, 'c>(node_a: &Node<'a, 'b, 'c>, node_b: &Node<'a, let announcement_event = node_a.node.get_and_clear_pending_msg_events(); if !announcement_event.is_empty() { assert_eq!(announcement_event.len(), 1); - if let MessageSendEvent::SendAnnouncementSignatures { .. } = announcement_event[0] { + if let MessageSendEvent::SendChannelUpdate { .. } = announcement_event[0] { //TODO: Test announcement_sigs re-sending - } else { panic!("Unexpected event!"); } + } else { panic!("Unexpected event! {:?}", announcement_event[0]); } } } else { assert!(chan_msgs.0.is_none()); @@ -2162,9 +2170,11 @@ pub fn reconnect_nodes<'a, 'b, 'c>(node_a: &Node<'a, 'b, 'c>, node_b: &Node<'a, let announcement_event = node_b.node.get_and_clear_pending_msg_events(); if !announcement_event.is_empty() { assert_eq!(announcement_event.len(), 1); - if let MessageSendEvent::SendAnnouncementSignatures { .. } = announcement_event[0] { - //TODO: Test announcement_sigs re-sending - } else { panic!("Unexpected event!"); } + match announcement_event[0] { + MessageSendEvent::SendChannelUpdate { .. } => {}, + MessageSendEvent::SendAnnouncementSignatures { .. } => {}, + _ => panic!("Unexpected event {:?}!", announcement_event[0]), + } } } else { assert!(chan_msgs.0.is_none()); diff --git a/lightning/src/ln/functional_tests.rs b/lightning/src/ln/functional_tests.rs index 44696378..6f6e51a6 100644 --- a/lightning/src/ln/functional_tests.rs +++ b/lightning/src/ln/functional_tests.rs @@ -483,10 +483,54 @@ fn do_test_1_conf_open(connect_style: ConnectStyle) { let tx = create_chan_between_nodes_with_value_init(&nodes[0], &nodes[1], 100000, 10001, InitFeatures::known(), InitFeatures::known()); mine_transaction(&nodes[1], &tx); nodes[0].node.handle_funding_locked(&nodes[1].node.get_our_node_id(), &get_event_msg!(nodes[1], MessageSendEvent::SendFundingLocked, nodes[0].node.get_our_node_id())); + assert!(nodes[0].node.get_and_clear_pending_msg_events().is_empty()); mine_transaction(&nodes[0], &tx); - let (funding_locked, _) = create_chan_between_nodes_with_value_confirm_second(&nodes[1], &nodes[0]); - let (announcement, as_update, bs_update) = create_chan_between_nodes_with_value_b(&nodes[0], &nodes[1], &funding_locked); + let as_msg_events = nodes[0].node.get_and_clear_pending_msg_events(); + assert_eq!(as_msg_events.len(), 2); + let as_funding_locked = if let MessageSendEvent::SendFundingLocked { ref node_id, ref msg } = as_msg_events[0] { + assert_eq!(*node_id, nodes[1].node.get_our_node_id()); + msg.clone() + } else { panic!("Unexpected event"); }; + if let MessageSendEvent::SendChannelUpdate { ref node_id, msg: _ } = as_msg_events[1] { + assert_eq!(*node_id, nodes[1].node.get_our_node_id()); + } else { panic!("Unexpected event"); } + + nodes[1].node.handle_funding_locked(&nodes[0].node.get_our_node_id(), &as_funding_locked); + let bs_msg_events = nodes[1].node.get_and_clear_pending_msg_events(); + assert_eq!(bs_msg_events.len(), 1); + if let MessageSendEvent::SendChannelUpdate { ref node_id, msg: _ } = bs_msg_events[0] { + assert_eq!(*node_id, nodes[0].node.get_our_node_id()); + } else { panic!("Unexpected event"); } + + send_payment(&nodes[0], &[&nodes[1]], 100_000); + + // After 6 confirmations, as required by the spec, we'll send announcement_signatures and + // broadcast the channel_announcement (but not before exactly 6 confirmations). + connect_blocks(&nodes[0], 4); + assert!(nodes[0].node.get_and_clear_pending_msg_events().is_empty()); + connect_blocks(&nodes[0], 1); + nodes[1].node.handle_announcement_signatures(&nodes[0].node.get_our_node_id(), &get_event_msg!(nodes[0], MessageSendEvent::SendAnnouncementSignatures, nodes[1].node.get_our_node_id())); + assert!(nodes[1].node.get_and_clear_pending_msg_events().is_empty()); + + connect_blocks(&nodes[1], 5); + let bs_announce_events = nodes[1].node.get_and_clear_pending_msg_events(); + assert_eq!(bs_announce_events.len(), 2); + let bs_announcement_sigs = if let MessageSendEvent::SendAnnouncementSignatures { ref node_id, ref msg } = bs_announce_events[0] { + assert_eq!(*node_id, nodes[0].node.get_our_node_id()); + msg.clone() + } else { panic!("Unexpected event"); }; + let (bs_announcement, bs_update) = if let MessageSendEvent::BroadcastChannelAnnouncement { ref msg, ref update_msg } = bs_announce_events[1] { + (msg.clone(), update_msg.clone()) + } else { panic!("Unexpected event"); }; + + nodes[0].node.handle_announcement_signatures(&nodes[1].node.get_our_node_id(), &bs_announcement_sigs); + let as_announce_events = nodes[0].node.get_and_clear_pending_msg_events(); + assert_eq!(as_announce_events.len(), 1); + let (announcement, as_update) = if let MessageSendEvent::BroadcastChannelAnnouncement { ref msg, ref update_msg } = as_announce_events[0] { + (msg.clone(), update_msg.clone()) + } else { panic!("Unexpected event"); }; + assert_eq!(announcement, bs_announcement); for node in nodes { assert!(node.net_graph_msg_handler.handle_channel_announcement(&announcement).unwrap()); @@ -3864,14 +3908,13 @@ fn test_funding_peer_disconnect() { } // Re-deliver nodes[0]'s funding_locked, which nodes[1] can safely ignore. It currently - // generates a duplicative announcement_signatures + // generates a duplicative private channel_update nodes[1].node.handle_funding_locked(&nodes[0].node.get_our_node_id(), &as_funding_locked); let events_5 = nodes[1].node.get_and_clear_pending_msg_events(); assert_eq!(events_5.len(), 1); match events_5[0] { - MessageSendEvent::SendAnnouncementSignatures { ref node_id, ref msg } => { + MessageSendEvent::SendChannelUpdate { ref node_id, msg: _ } => { assert_eq!(*node_id, nodes[0].node.get_our_node_id()); - assert_eq!(*msg, bs_announcement_sigs); }, _ => panic!("Unexpected event {:?}", events_5[0]), };