Move to Funded after funding_signed rather than on funding

Previously, channels were stored in different maps in `PeerState`
based on whether the funding had been set, keeping the keys across
the maps consistent (pre-funding temporary_channel_ids vs
funding-outpoint-based channel_ids). However, channels are now
stored in a single `channel_by_id` map, making that point moot.

Instead, here, we convert the `ChannelPhase` state transition
boundary to "once we have a `ChannelMonitor`", which makes more
sense now, and was actually the original proposed boundary.

This also requires calling `signer_maybe_unblocked` on a pre-funded
outbound channel, but that nicely also lets us limit the scope of
`FundingCreated` message generation, which we do in the next
commit.
This commit is contained in:
Matt Corallo 2023-11-30 00:36:16 +00:00
parent 2d266794c2
commit 6ab56a40be
3 changed files with 193 additions and 166 deletions

View file

@ -793,7 +793,6 @@ pub(super) struct MonitorRestoreUpdates {
pub(super) struct SignerResumeUpdates {
pub commitment_update: Option<msgs::CommitmentUpdate>,
pub funding_signed: Option<msgs::FundingSigned>,
pub funding_created: Option<msgs::FundingCreated>,
pub channel_ready: Option<msgs::ChannelReady>,
}
@ -2942,99 +2941,6 @@ impl<SP: Deref> Channel<SP> where
}
// Message handlers:
/// Handles a funding_signed message from the remote end.
/// If this call is successful, broadcast the funding transaction (and not before!)
pub fn funding_signed<L: Deref>(
&mut self, msg: &msgs::FundingSigned, best_block: BestBlock, signer_provider: &SP, logger: &L
) -> Result<ChannelMonitor<<SP::Target as SignerProvider>::EcdsaSigner>, ChannelError>
where
L::Target: Logger
{
if !self.context.is_outbound() {
return Err(ChannelError::Close("Received funding_signed for an inbound channel?".to_owned()));
}
if !matches!(self.context.channel_state, ChannelState::FundingNegotiated) {
return Err(ChannelError::Close("Received funding_signed in strange state!".to_owned()));
}
if self.context.commitment_secrets.get_min_seen_secret() != (1 << 48) ||
self.context.cur_counterparty_commitment_transaction_number != INITIAL_COMMITMENT_NUMBER ||
self.context.cur_holder_commitment_transaction_number != INITIAL_COMMITMENT_NUMBER {
panic!("Should not have advanced channel commitment tx numbers prior to funding_created");
}
let funding_script = self.context.get_funding_redeemscript();
let counterparty_keys = self.context.build_remote_transaction_keys();
let counterparty_initial_commitment_tx = self.context.build_commitment_transaction(self.context.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, logger).tx;
let counterparty_trusted_tx = counterparty_initial_commitment_tx.trust();
let counterparty_initial_bitcoin_tx = counterparty_trusted_tx.built_transaction();
log_trace!(logger, "Initial counterparty tx for channel {} is: txid {} tx {}",
&self.context.channel_id(), counterparty_initial_bitcoin_tx.txid, encode::serialize_hex(&counterparty_initial_bitcoin_tx.transaction));
let holder_signer = self.context.build_holder_transaction_keys(self.context.cur_holder_commitment_transaction_number);
let initial_commitment_tx = self.context.build_commitment_transaction(self.context.cur_holder_commitment_transaction_number, &holder_signer, true, false, logger).tx;
{
let trusted_tx = initial_commitment_tx.trust();
let initial_commitment_bitcoin_tx = trusted_tx.built_transaction();
let sighash = initial_commitment_bitcoin_tx.get_sighash_all(&funding_script, self.context.channel_value_satoshis);
// They sign our commitment transaction, allowing us to broadcast the tx if we wish.
if let Err(_) = self.context.secp_ctx.verify_ecdsa(&sighash, &msg.signature, &self.context.get_counterparty_pubkeys().funding_pubkey) {
return Err(ChannelError::Close("Invalid funding_signed signature from peer".to_owned()));
}
}
let holder_commitment_tx = HolderCommitmentTransaction::new(
initial_commitment_tx,
msg.signature,
Vec::new(),
&self.context.get_holder_pubkeys().funding_pubkey,
self.context.counterparty_funding_pubkey()
);
self.context.holder_signer.as_ref().validate_holder_commitment(&holder_commitment_tx, Vec::new())
.map_err(|_| ChannelError::Close("Failed to validate our commitment".to_owned()))?;
let funding_redeemscript = self.context.get_funding_redeemscript();
let funding_txo = self.context.get_funding_txo().unwrap();
let funding_txo_script = funding_redeemscript.to_v0_p2wsh();
let obscure_factor = get_commitment_transaction_number_obscure_factor(&self.context.get_holder_pubkeys().payment_point, &self.context.get_counterparty_pubkeys().payment_point, self.context.is_outbound());
let shutdown_script = self.context.shutdown_scriptpubkey.clone().map(|script| script.into_inner());
let mut monitor_signer = signer_provider.derive_channel_signer(self.context.channel_value_satoshis, self.context.channel_keys_id);
monitor_signer.provide_channel_parameters(&self.context.channel_transaction_parameters);
let channel_monitor = ChannelMonitor::new(self.context.secp_ctx.clone(), monitor_signer,
shutdown_script, self.context.get_holder_selected_contest_delay(),
&self.context.destination_script, (funding_txo, funding_txo_script),
&self.context.channel_transaction_parameters,
funding_redeemscript.clone(), self.context.channel_value_satoshis,
obscure_factor,
holder_commitment_tx, best_block, self.context.counterparty_node_id);
channel_monitor.provide_initial_counterparty_commitment_tx(
counterparty_initial_bitcoin_tx.txid, Vec::new(),
self.context.cur_counterparty_commitment_transaction_number,
self.context.counterparty_cur_commitment_point.unwrap(),
counterparty_initial_commitment_tx.feerate_per_kw(),
counterparty_initial_commitment_tx.to_broadcaster_value_sat(),
counterparty_initial_commitment_tx.to_countersignatory_value_sat(), logger);
assert!(!self.context.channel_state.is_monitor_update_in_progress()); // We have no had any monitor(s) yet to fail update!
if self.context.is_batch_funding() {
self.context.channel_state = ChannelState::AwaitingChannelReady(AwaitingChannelReadyFlags::WAITING_FOR_BATCH);
} else {
self.context.channel_state = ChannelState::AwaitingChannelReady(AwaitingChannelReadyFlags::new());
}
self.context.cur_holder_commitment_transaction_number -= 1;
self.context.cur_counterparty_commitment_transaction_number -= 1;
log_info!(logger, "Received funding_signed from peer for channel {}", &self.context.channel_id());
let need_channel_ready = self.check_get_channel_ready(0).is_some();
self.monitor_updating_paused(false, false, need_channel_ready, Vec::new(), Vec::new(), Vec::new());
Ok(channel_monitor)
}
/// Updates the state of the channel to indicate that all channels in the batch have received
/// funding_signed and persisted their monitors.
/// The funding transaction is consequently allowed to be broadcast, and the channel can be
@ -4308,20 +4214,15 @@ impl<SP: Deref> Channel<SP> where
let channel_ready = if funding_signed.is_some() {
self.check_get_channel_ready(0)
} else { None };
let funding_created = if self.context.signer_pending_funding && self.context.is_outbound() {
self.context.get_funding_created_msg(logger)
} else { None };
log_trace!(logger, "Signer unblocked with {} commitment_update, {} funding_signed, {} funding_created, and {} channel_ready",
log_trace!(logger, "Signer unblocked with {} commitment_update, {} funding_signed and {} channel_ready",
if commitment_update.is_some() { "a" } else { "no" },
if funding_signed.is_some() { "a" } else { "no" },
if funding_created.is_some() { "a" } else { "no" },
if channel_ready.is_some() { "a" } else { "no" });
SignerResumeUpdates {
commitment_update,
funding_signed,
funding_created,
channel_ready,
}
}
@ -6445,8 +6346,8 @@ impl<SP: Deref> OutboundV1Channel<SP> where SP::Target: SignerProvider {
/// Note that channel_id changes during this call!
/// Do NOT broadcast the funding transaction until after a successful funding_signed call!
/// If an Err is returned, it is a ChannelError::Close.
pub fn get_funding_created<L: Deref>(mut self, funding_transaction: Transaction, funding_txo: OutPoint, is_batch_funding: bool, logger: &L)
-> Result<(Channel<SP>, Option<msgs::FundingCreated>), (Self, ChannelError)> where L::Target: Logger {
pub fn get_funding_created<L: Deref>(&mut self, funding_transaction: Transaction, funding_txo: OutPoint, is_batch_funding: bool, logger: &L)
-> Result<Option<msgs::FundingCreated>, (Self, ChannelError)> where L::Target: Logger {
if !self.context.is_outbound() {
panic!("Tried to create outbound funding_created message on an inbound channel!");
}
@ -6489,11 +6390,7 @@ impl<SP: Deref> OutboundV1Channel<SP> where SP::Target: SignerProvider {
}
}
let channel = Channel {
context: self.context,
};
Ok((channel, funding_created))
Ok(funding_created)
}
fn get_initial_channel_type(config: &UserConfig, their_features: &InitFeatures) -> ChannelTypeFeatures {
@ -6736,6 +6633,112 @@ impl<SP: Deref> OutboundV1Channel<SP> where SP::Target: SignerProvider {
Ok(())
}
/// Handles a funding_signed message from the remote end.
/// If this call is successful, broadcast the funding transaction (and not before!)
pub fn funding_signed<L: Deref>(
mut self, msg: &msgs::FundingSigned, best_block: BestBlock, signer_provider: &SP, logger: &L
) -> Result<(Channel<SP>, ChannelMonitor<<SP::Target as SignerProvider>::EcdsaSigner>), (OutboundV1Channel<SP>, ChannelError)>
where
L::Target: Logger
{
if !self.context.is_outbound() {
return Err((self, ChannelError::Close("Received funding_signed for an inbound channel?".to_owned())));
}
if !matches!(self.context.channel_state, ChannelState::FundingNegotiated) {
return Err((self, ChannelError::Close("Received funding_signed in strange state!".to_owned())));
}
if self.context.commitment_secrets.get_min_seen_secret() != (1 << 48) ||
self.context.cur_counterparty_commitment_transaction_number != INITIAL_COMMITMENT_NUMBER ||
self.context.cur_holder_commitment_transaction_number != INITIAL_COMMITMENT_NUMBER {
panic!("Should not have advanced channel commitment tx numbers prior to funding_created");
}
let funding_script = self.context.get_funding_redeemscript();
let counterparty_keys = self.context.build_remote_transaction_keys();
let counterparty_initial_commitment_tx = self.context.build_commitment_transaction(self.context.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, logger).tx;
let counterparty_trusted_tx = counterparty_initial_commitment_tx.trust();
let counterparty_initial_bitcoin_tx = counterparty_trusted_tx.built_transaction();
log_trace!(logger, "Initial counterparty tx for channel {} is: txid {} tx {}",
&self.context.channel_id(), counterparty_initial_bitcoin_tx.txid, encode::serialize_hex(&counterparty_initial_bitcoin_tx.transaction));
let holder_signer = self.context.build_holder_transaction_keys(self.context.cur_holder_commitment_transaction_number);
let initial_commitment_tx = self.context.build_commitment_transaction(self.context.cur_holder_commitment_transaction_number, &holder_signer, true, false, logger).tx;
{
let trusted_tx = initial_commitment_tx.trust();
let initial_commitment_bitcoin_tx = trusted_tx.built_transaction();
let sighash = initial_commitment_bitcoin_tx.get_sighash_all(&funding_script, self.context.channel_value_satoshis);
// They sign our commitment transaction, allowing us to broadcast the tx if we wish.
if let Err(_) = self.context.secp_ctx.verify_ecdsa(&sighash, &msg.signature, &self.context.get_counterparty_pubkeys().funding_pubkey) {
return Err((self, ChannelError::Close("Invalid funding_signed signature from peer".to_owned())));
}
}
let holder_commitment_tx = HolderCommitmentTransaction::new(
initial_commitment_tx,
msg.signature,
Vec::new(),
&self.context.get_holder_pubkeys().funding_pubkey,
self.context.counterparty_funding_pubkey()
);
let validated =
self.context.holder_signer.as_ref().validate_holder_commitment(&holder_commitment_tx, Vec::new());
if validated.is_err() {
return Err((self, ChannelError::Close("Failed to validate our commitment".to_owned())));
}
let funding_redeemscript = self.context.get_funding_redeemscript();
let funding_txo = self.context.get_funding_txo().unwrap();
let funding_txo_script = funding_redeemscript.to_v0_p2wsh();
let obscure_factor = get_commitment_transaction_number_obscure_factor(&self.context.get_holder_pubkeys().payment_point, &self.context.get_counterparty_pubkeys().payment_point, self.context.is_outbound());
let shutdown_script = self.context.shutdown_scriptpubkey.clone().map(|script| script.into_inner());
let mut monitor_signer = signer_provider.derive_channel_signer(self.context.channel_value_satoshis, self.context.channel_keys_id);
monitor_signer.provide_channel_parameters(&self.context.channel_transaction_parameters);
let channel_monitor = ChannelMonitor::new(self.context.secp_ctx.clone(), monitor_signer,
shutdown_script, self.context.get_holder_selected_contest_delay(),
&self.context.destination_script, (funding_txo, funding_txo_script),
&self.context.channel_transaction_parameters,
funding_redeemscript.clone(), self.context.channel_value_satoshis,
obscure_factor,
holder_commitment_tx, best_block, self.context.counterparty_node_id);
channel_monitor.provide_initial_counterparty_commitment_tx(
counterparty_initial_bitcoin_tx.txid, Vec::new(),
self.context.cur_counterparty_commitment_transaction_number,
self.context.counterparty_cur_commitment_point.unwrap(),
counterparty_initial_commitment_tx.feerate_per_kw(),
counterparty_initial_commitment_tx.to_broadcaster_value_sat(),
counterparty_initial_commitment_tx.to_countersignatory_value_sat(), logger);
assert!(!self.context.channel_state.is_monitor_update_in_progress()); // We have no had any monitor(s) yet to fail update!
if self.context.is_batch_funding() {
self.context.channel_state = ChannelState::AwaitingChannelReady(AwaitingChannelReadyFlags::WAITING_FOR_BATCH);
} else {
self.context.channel_state = ChannelState::AwaitingChannelReady(AwaitingChannelReadyFlags::new());
}
self.context.cur_holder_commitment_transaction_number -= 1;
self.context.cur_counterparty_commitment_transaction_number -= 1;
log_info!(logger, "Received funding_signed from peer for channel {}", &self.context.channel_id());
let mut channel = Channel { context: self.context };
let need_channel_ready = channel.check_get_channel_ready(0).is_some();
channel.monitor_updating_paused(false, false, need_channel_ready, Vec::new(), Vec::new(), Vec::new());
Ok((channel, channel_monitor))
}
/// Indicates that the signer may have some signatures for us, so we should retry if we're
/// blocked.
#[allow(unused)]
pub fn signer_maybe_unblocked<L: Deref>(&mut self, logger: &L) -> Option<msgs::FundingCreated> where L::Target: Logger {
if self.context.signer_pending_funding && self.context.is_outbound() {
log_trace!(logger, "Signer unblocked a funding_created");
self.context.get_funding_created_msg(logger)
} else { None }
}
}
/// A not-yet-funded inbound (from counterparty) channel using V1 channel establishment.
@ -8364,11 +8367,12 @@ mod tests {
value: 10000000, script_pubkey: output_script.clone(),
}]};
let funding_outpoint = OutPoint{ txid: tx.txid(), index: 0 };
let (mut node_a_chan, funding_created_msg) = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, false, &&logger).map_err(|_| ()).unwrap();
let funding_created_msg = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, false, &&logger).map_err(|_| ()).unwrap();
let (_, funding_signed_msg, _) = node_b_chan.funding_created(&funding_created_msg.unwrap(), best_block, &&keys_provider, &&logger).map_err(|_| ()).unwrap();
// Node B --> Node A: funding signed
let _ = node_a_chan.funding_signed(&funding_signed_msg.unwrap(), best_block, &&keys_provider, &&logger).unwrap();
let res = node_a_chan.funding_signed(&funding_signed_msg.unwrap(), best_block, &&keys_provider, &&logger);
let (mut node_a_chan, _) = if let Ok(res) = res { res } else { panic!(); };
// Put some inbound and outbound HTLCs in A's channel.
let htlc_amount_msat = 11_092_000; // put an amount below A's effective dust limit but above B's.
@ -8492,11 +8496,12 @@ mod tests {
value: 10000000, script_pubkey: output_script.clone(),
}]};
let funding_outpoint = OutPoint{ txid: tx.txid(), index: 0 };
let (mut node_a_chan, funding_created_msg) = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, false, &&logger).map_err(|_| ()).unwrap();
let funding_created_msg = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, false, &&logger).map_err(|_| ()).unwrap();
let (mut node_b_chan, funding_signed_msg, _) = node_b_chan.funding_created(&funding_created_msg.unwrap(), best_block, &&keys_provider, &&logger).map_err(|_| ()).unwrap();
// Node B --> Node A: funding signed
let _ = node_a_chan.funding_signed(&funding_signed_msg.unwrap(), best_block, &&keys_provider, &&logger).unwrap();
let res = node_a_chan.funding_signed(&funding_signed_msg.unwrap(), best_block, &&keys_provider, &&logger);
let (mut node_a_chan, _) = if let Ok(res) = res { res } else { panic!(); };
// Now disconnect the two nodes and check that the commitment point in
// Node B's channel_reestablish message is sane.
@ -8680,11 +8685,12 @@ mod tests {
value: 10000000, script_pubkey: output_script.clone(),
}]};
let funding_outpoint = OutPoint{ txid: tx.txid(), index: 0 };
let (mut node_a_chan, funding_created_msg) = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, false, &&logger).map_err(|_| ()).unwrap();
let funding_created_msg = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, false, &&logger).map_err(|_| ()).unwrap();
let (_, funding_signed_msg, _) = node_b_chan.funding_created(&funding_created_msg.unwrap(), best_block, &&keys_provider, &&logger).map_err(|_| ()).unwrap();
// Node B --> Node A: funding signed
let _ = node_a_chan.funding_signed(&funding_signed_msg.unwrap(), best_block, &&keys_provider, &&logger).unwrap();
let res = node_a_chan.funding_signed(&funding_signed_msg.unwrap(), best_block, &&keys_provider, &&logger);
let (mut node_a_chan, _) = if let Ok(res) = res { res } else { panic!(); };
// Make sure that receiving a channel update will update the Channel as expected.
let update = ChannelUpdate {
@ -9850,11 +9856,8 @@ mod tests {
},
]};
let funding_outpoint = OutPoint{ txid: tx.txid(), index: 0 };
let (mut node_a_chan, funding_created_msg) = node_a_chan.get_funding_created(
tx.clone(),
funding_outpoint,
true,
&&logger,
let funding_created_msg = node_a_chan.get_funding_created(
tx.clone(), funding_outpoint, true, &&logger,
).map_err(|_| ()).unwrap();
let (mut node_b_chan, funding_signed_msg, _) = node_b_chan.funding_created(
&funding_created_msg.unwrap(),
@ -9872,12 +9875,10 @@ mod tests {
// Receive funding_signed, but the channel will be configured to hold sending channel_ready and
// broadcasting the funding transaction until the batch is ready.
let _ = node_a_chan.funding_signed(
&funding_signed_msg.unwrap(),
best_block,
&&keys_provider,
&&logger,
).unwrap();
let res = node_a_chan.funding_signed(
&funding_signed_msg.unwrap(), best_block, &&keys_provider, &&logger,
);
let (mut node_a_chan, _) = if let Ok(res) = res { res } else { panic!(); };
let node_a_updates = node_a_chan.monitor_updating_restored(
&&logger,
&&keys_provider,

View file

@ -3730,7 +3730,7 @@ where
let mut peer_state_lock = peer_state_mutex.lock().unwrap();
let peer_state = &mut *peer_state_lock;
let (chan, msg_opt) = match peer_state.channel_by_id.remove(temporary_channel_id) {
Some(ChannelPhase::UnfundedOutboundV1(chan)) => {
Some(ChannelPhase::UnfundedOutboundV1(mut chan)) => {
let funding_txo = find_funding_output(&chan, &funding_transaction)?;
let logger = WithChannelContext::from(&self.logger, &chan.context);
@ -3743,7 +3743,7 @@ where
(chan, MsgHandleErrInternal::from_finish_shutdown(msg, channel_id, user_id, shutdown_res, None, channel_capacity))
} else { unreachable!(); });
match funding_res {
Ok((chan, funding_msg)) => (chan, funding_msg),
Ok(funding_msg) => (chan, funding_msg),
Err((chan, err)) => {
mem::drop(peer_state_lock);
mem::drop(per_peer_state);
@ -3783,7 +3783,7 @@ where
if id_to_peer.insert(chan.context.channel_id(), chan.context.get_counterparty_node_id()).is_some() {
panic!("id_to_peer map already contained funding txid, which shouldn't be possible");
}
e.insert(ChannelPhase::Funded(chan));
e.insert(ChannelPhase::UnfundedOutboundV1(chan));
}
}
Ok(())
@ -5429,7 +5429,7 @@ where
// We got a temporary failure updating monitor, but will claim the
// HTLC when the monitor updating is restored (or on chain).
let logger = WithContext::from(&self.logger, None, Some(prev_hop_chan_id));
log_error!(self.logger, "Temporary failure claiming HTLC, treating as success: {}", err.err.err);
log_error!(logger, "Temporary failure claiming HTLC, treating as success: {}", err.err.err);
} else { errs.push((pk, err)); }
}
}
@ -6310,22 +6310,43 @@ where
let mut peer_state_lock = peer_state_mutex.lock().unwrap();
let peer_state = &mut *peer_state_lock;
match peer_state.channel_by_id.entry(msg.channel_id) {
hash_map::Entry::Occupied(mut chan_phase_entry) => {
match chan_phase_entry.get_mut() {
ChannelPhase::Funded(ref mut chan) => {
let logger = WithChannelContext::from(&self.logger, &chan.context);
let monitor = try_chan_phase_entry!(self,
chan.funding_signed(&msg, best_block, &self.signer_provider, &&logger), chan_phase_entry);
if let Ok(persist_status) = self.chain_monitor.watch_channel(chan.context.get_funding_txo().unwrap(), monitor) {
handle_new_monitor_update!(self, persist_status, peer_state_lock, peer_state, per_peer_state, chan, INITIAL_MONITOR);
Ok(())
} else {
try_chan_phase_entry!(self, Err(ChannelError::Close("Channel funding outpoint was a duplicate".to_owned())), chan_phase_entry)
hash_map::Entry::Occupied(chan_phase_entry) => {
if matches!(chan_phase_entry.get(), ChannelPhase::UnfundedOutboundV1(_)) {
let chan = if let ChannelPhase::UnfundedOutboundV1(chan) = chan_phase_entry.remove() { chan } else { unreachable!() };
let logger = WithContext::from(
&self.logger,
Some(chan.context.get_counterparty_node_id()),
Some(chan.context.channel_id())
);
let res =
chan.funding_signed(&msg, best_block, &self.signer_provider, &&logger);
match res {
Ok((chan, monitor)) => {
if let Ok(persist_status) = self.chain_monitor.watch_channel(chan.context.get_funding_txo().unwrap(), monitor) {
// We really should be able to insert here without doing a second
// lookup, but sadly rust stdlib doesn't currently allow keeping
// the original Entry around with the value removed.
let mut chan = peer_state.channel_by_id.entry(msg.channel_id).or_insert(ChannelPhase::Funded(chan));
if let ChannelPhase::Funded(ref mut chan) = &mut chan {
handle_new_monitor_update!(self, persist_status, peer_state_lock, peer_state, per_peer_state, chan, INITIAL_MONITOR);
} else { unreachable!(); }
Ok(())
} else {
let e = ChannelError::Close("Channel funding outpoint was a duplicate".to_owned());
return Err(convert_chan_phase_err!(self, e, &mut ChannelPhase::Funded(chan), &msg.channel_id).1);
}
},
Err((chan, e)) => {
debug_assert!(matches!(e, ChannelError::Close(_)),
"We don't have a channel anymore, so the error better have expected close");
// We've already removed this outbound channel from the map in
// `PeerState` above so at this point we just need to clean up any
// lingering entries concerning this channel as it is safe to do so.
return Err(convert_chan_phase_err!(self, e, &mut ChannelPhase::UnfundedOutboundV1(chan), &msg.channel_id).1);
}
},
_ => {
return Err(MsgHandleErrInternal::send_err_msg_no_close("Failed to find corresponding channel".to_owned(), msg.channel_id));
},
}
} else {
return Err(MsgHandleErrInternal::send_err_msg_no_close("Failed to find corresponding channel".to_owned(), msg.channel_id));
}
},
hash_map::Entry::Vacant(_) => return Err(MsgHandleErrInternal::send_err_msg_no_close("Failed to find corresponding channel".to_owned(), msg.channel_id))
@ -7251,29 +7272,34 @@ where
let unblock_chan = |phase: &mut ChannelPhase<SP>, pending_msg_events: &mut Vec<MessageSendEvent>| {
let node_id = phase.context().get_counterparty_node_id();
if let ChannelPhase::Funded(chan) = phase {
let msgs = chan.signer_maybe_unblocked(&self.logger);
if let Some(updates) = msgs.commitment_update {
pending_msg_events.push(events::MessageSendEvent::UpdateHTLCs {
node_id,
updates,
});
match phase {
ChannelPhase::Funded(chan) => {
let msgs = chan.signer_maybe_unblocked(&self.logger);
if let Some(updates) = msgs.commitment_update {
pending_msg_events.push(events::MessageSendEvent::UpdateHTLCs {
node_id,
updates,
});
}
if let Some(msg) = msgs.funding_signed {
pending_msg_events.push(events::MessageSendEvent::SendFundingSigned {
node_id,
msg,
});
}
if let Some(msg) = msgs.channel_ready {
send_channel_ready!(self, pending_msg_events, chan, msg);
}
}
if let Some(msg) = msgs.funding_signed {
pending_msg_events.push(events::MessageSendEvent::SendFundingSigned {
node_id,
msg,
});
}
if let Some(msg) = msgs.funding_created {
pending_msg_events.push(events::MessageSendEvent::SendFundingCreated {
node_id,
msg,
});
}
if let Some(msg) = msgs.channel_ready {
send_channel_ready!(self, pending_msg_events, chan, msg);
ChannelPhase::UnfundedOutboundV1(chan) => {
if let Some(msg) = chan.signer_maybe_unblocked(&self.logger) {
pending_msg_events.push(events::MessageSendEvent::SendFundingCreated {
node_id,
msg,
});
}
}
ChannelPhase::UnfundedInboundV1(_) => {},
}
};

View file

@ -9059,7 +9059,7 @@ fn test_duplicate_chan_id() {
nodes[0].node.handle_accept_channel(&nodes[1].node.get_our_node_id(), &get_event_msg!(nodes[1], MessageSendEvent::SendAcceptChannel, nodes[0].node.get_our_node_id()));
create_funding_transaction(&nodes[0], &nodes[1].node.get_our_node_id(), 100000, 42); // Get and check the FundingGenerationReady event
let (_, funding_created) = {
let funding_created = {
let per_peer_state = nodes[0].node.per_peer_state.read().unwrap();
let mut a_peer_state = per_peer_state.get(&nodes[1].node.get_our_node_id()).unwrap().lock().unwrap();
// Once we call `get_funding_created` the channel has a duplicate channel_id as
@ -9067,7 +9067,7 @@ fn test_duplicate_chan_id() {
// try to create another channel. Instead, we drop the channel entirely here (leaving the
// channelmanager in a possibly nonsense state instead).
match a_peer_state.channel_by_id.remove(&open_chan_2_msg.temporary_channel_id).unwrap() {
ChannelPhase::UnfundedOutboundV1(chan) => {
ChannelPhase::UnfundedOutboundV1(mut chan) => {
let logger = test_utils::TestLogger::new();
chan.get_funding_created(tx.clone(), funding_outpoint, false, &&logger).map_err(|_| ()).unwrap()
},