claim_payment_along_route(&nodes[0], &[&[&nodes[1], &nodes[3]], &[&nodes[2], &nodes[3]]], false, payment_preimage);
}
+#[test]
+fn test_pending_update_fee_ack_on_reconnect() {
+ // In early versions of our automated fee update patch, nodes did not correctly use the
+ // previous channel feerate after sending an undelivered revoke_and_ack when re-sending an
+ // undelivered commitment_signed.
+ //
+ // B sends A new HTLC + CS, not delivered
+ // A sends B update_fee + CS
+ // B receives the CS and sends RAA, previously causing B to lock in the new feerate
+ // reconnect
+ // B resends initial CS, using the original fee
+
+ 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 mut nodes = create_network(2, &node_cfgs, &node_chanmgrs);
+
+ create_announced_chan_between_nodes(&nodes, 0, 1, InitFeatures::known(), InitFeatures::known());
+ send_payment(&nodes[0], &[&nodes[1]], 100_000_00);
+
+ let (payment_preimage, payment_hash, payment_secret) = get_payment_preimage_hash!(&nodes[0]);
+ let route = get_route(&nodes[1].node.get_our_node_id(), &nodes[1].net_graph_msg_handler.network_graph.read().unwrap(),
+ &nodes[0].node.get_our_node_id(), Some(InvoiceFeatures::known()), None, &Vec::new(), 1_000_000, TEST_FINAL_CLTV, nodes[1].logger).unwrap();
+ nodes[1].node.send_payment(&route, payment_hash, &Some(payment_secret)).unwrap();
+ check_added_monitors!(nodes[1], 1);
+ let bs_initial_send_msgs = get_htlc_update_msgs!(nodes[1], nodes[0].node.get_our_node_id());
+ // bs_initial_send_msgs are not delivered until they are re-generated after reconnect
+
+ {
+ let mut feerate_lock = chanmon_cfgs[0].fee_estimator.sat_per_kw.lock().unwrap();
+ *feerate_lock *= 2;
+ }
+ nodes[0].node.timer_tick_occurred();
+ check_added_monitors!(nodes[0], 1);
+ let as_update_fee_msgs = get_htlc_update_msgs!(nodes[0], nodes[1].node.get_our_node_id());
+ assert!(as_update_fee_msgs.update_fee.is_some());
+
+ nodes[1].node.handle_update_fee(&nodes[0].node.get_our_node_id(), as_update_fee_msgs.update_fee.as_ref().unwrap());
+ nodes[1].node.handle_commitment_signed(&nodes[0].node.get_our_node_id(), &as_update_fee_msgs.commitment_signed);
+ check_added_monitors!(nodes[1], 1);
+ let bs_first_raa = get_event_msg!(nodes[1], MessageSendEvent::SendRevokeAndACK, nodes[0].node.get_our_node_id());
+ // bs_first_raa is not delivered until it is re-generated after reconnect
+
+ nodes[0].node.peer_disconnected(&nodes[1].node.get_our_node_id(), false);
+ nodes[1].node.peer_disconnected(&nodes[0].node.get_our_node_id(), false);
+
+ nodes[0].node.peer_connected(&nodes[1].node.get_our_node_id(), &msgs::Init { features: InitFeatures::known() });
+ let as_connect_msg = get_event_msg!(nodes[0], MessageSendEvent::SendChannelReestablish, nodes[1].node.get_our_node_id());
+ nodes[1].node.peer_connected(&nodes[0].node.get_our_node_id(), &msgs::Init { features: InitFeatures::known() });
+ let bs_connect_msg = get_event_msg!(nodes[1], MessageSendEvent::SendChannelReestablish, nodes[0].node.get_our_node_id());
+
+ nodes[1].node.handle_channel_reestablish(&nodes[0].node.get_our_node_id(), &as_connect_msg);
+ let bs_resend_msgs = nodes[1].node.get_and_clear_pending_msg_events();
+ assert_eq!(bs_resend_msgs.len(), 3);
+ if let MessageSendEvent::UpdateHTLCs { ref updates, .. } = bs_resend_msgs[0] {
+ assert_eq!(*updates, bs_initial_send_msgs);
+ } else { panic!(); }
+ if let MessageSendEvent::SendRevokeAndACK { ref msg, .. } = bs_resend_msgs[1] {
+ assert_eq!(*msg, bs_first_raa);
+ } else { panic!(); }
+ if let MessageSendEvent::SendChannelUpdate { .. } = bs_resend_msgs[2] { } else { panic!(); }
+
+ nodes[0].node.handle_channel_reestablish(&nodes[1].node.get_our_node_id(), &bs_connect_msg);
+ get_event_msg!(nodes[0], MessageSendEvent::SendChannelUpdate, nodes[1].node.get_our_node_id());
+
+ nodes[0].node.handle_update_add_htlc(&nodes[1].node.get_our_node_id(), &bs_initial_send_msgs.update_add_htlcs[0]);
+ nodes[0].node.handle_commitment_signed(&nodes[1].node.get_our_node_id(), &bs_initial_send_msgs.commitment_signed);
+ check_added_monitors!(nodes[0], 1);
+ nodes[1].node.handle_revoke_and_ack(&nodes[0].node.get_our_node_id(), &get_event_msg!(nodes[0], MessageSendEvent::SendRevokeAndACK, nodes[1].node.get_our_node_id()));
+ check_added_monitors!(nodes[1], 1);
+ let bs_second_cs = get_htlc_update_msgs!(nodes[1], nodes[0].node.get_our_node_id()).commitment_signed;
+
+ nodes[0].node.handle_revoke_and_ack(&nodes[1].node.get_our_node_id(), &bs_first_raa);
+ check_added_monitors!(nodes[0], 1);
+ nodes[1].node.handle_commitment_signed(&nodes[0].node.get_our_node_id(), &get_htlc_update_msgs!(nodes[0], nodes[1].node.get_our_node_id()).commitment_signed);
+ check_added_monitors!(nodes[1], 1);
+ let bs_third_raa = get_event_msg!(nodes[1], MessageSendEvent::SendRevokeAndACK, nodes[0].node.get_our_node_id());
+
+ nodes[0].node.handle_commitment_signed(&nodes[1].node.get_our_node_id(), &bs_second_cs);
+ check_added_monitors!(nodes[0], 1);
+ nodes[0].node.handle_revoke_and_ack(&nodes[1].node.get_our_node_id(), &bs_third_raa);
+ check_added_monitors!(nodes[0], 1);
+
+ nodes[1].node.handle_revoke_and_ack(&nodes[0].node.get_our_node_id(), &get_event_msg!(nodes[0], MessageSendEvent::SendRevokeAndACK, nodes[1].node.get_our_node_id()));
+ check_added_monitors!(nodes[1], 1);
+
+ expect_pending_htlcs_forwardable!(nodes[0]);
+ expect_payment_received!(nodes[0], payment_hash, payment_secret, 1_000_000);
+
+ claim_payment(&nodes[1], &[&nodes[0]], payment_preimage);
+}
+
fn do_update_fee_resend_test(deliver_update: bool, parallel_updates: bool) {
// In early versions we did not handle resending of update_fee on reconnect correctly. The
// chanmon_consistency fuzz target, of course, immediately found it, but we test a few cases
let bs_connect_msg = get_event_msg!(nodes[1], MessageSendEvent::SendChannelReestablish, nodes[0].node.get_our_node_id());
nodes[1].node.handle_channel_reestablish(&nodes[0].node.get_our_node_id(), &as_connect_msg);
+ get_event_msg!(nodes[1], MessageSendEvent::SendChannelUpdate, nodes[0].node.get_our_node_id());
assert!(nodes[1].node.get_and_clear_pending_msg_events().is_empty());
nodes[0].node.handle_channel_reestablish(&nodes[1].node.get_our_node_id(), &bs_connect_msg);
- let update_msgs = get_htlc_update_msgs!(nodes[0], nodes[1].node.get_our_node_id());
+ let mut as_reconnect_msgs = nodes[0].node.get_and_clear_pending_msg_events();
+ assert_eq!(as_reconnect_msgs.len(), 2);
+ if let MessageSendEvent::SendChannelUpdate { .. } = as_reconnect_msgs.pop().unwrap() {} else { panic!(); }
+ let update_msgs = if let MessageSendEvent::UpdateHTLCs { updates, .. } = as_reconnect_msgs.pop().unwrap()
+ { updates } else { panic!(); };
assert!(update_msgs.update_fee.is_some());
nodes[1].node.handle_update_fee(&nodes[0].node.get_our_node_id(), update_msgs.update_fee.as_ref().unwrap());
if parallel_updates {
pub counterparty_dust_limit_msat: u64,
}
+#[derive(Clone, Copy, PartialEq)]
+enum FeeUpdateState {
+ // Inbound states mirroring InboundHTLCState
+ RemoteAnnounced,
+ AwaitingRemoteRevokeToAnnounce,
+ // Note that we do not have a AwaitingAnnouncedRemoteRevoke variant here as it is universally
+ // handled the same as `Committed`, with the only exception in `InboundHTLCState` being the
+ // distinction of when we allow ourselves to forward the HTLC. Because we aren't "forwarding"
+ // the fee update anywhere, we can simply consider the fee update `Committed` immediately
+ // instead of setting it to AwaitingAnnouncedRemoteRevoke.
+
+ // Outbound state can only be `LocalAnnounced` or `Committed`
+ Outbound,
+}
+
enum InboundHTLCRemovalReason {
FailRelay(msgs::OnionErrorPacket),
FailMalformed(([u8; 32], u16)),
// revoke_and_ack is received and new commitment_signed is generated to be
// sent to the funder. Otherwise, the pending value is removed when receiving
// commitment_signed.
- pending_update_fee: Option<u32>,
+ pending_update_fee: Option<(u32, FeeUpdateState)>,
// update_fee() during ChannelState::AwaitingRemoteRevoke is hold in
// holdina_cell_update_fee then moved to pending_udpate_fee when revoke_and_ack
// is received. holding_cell_update_fee is updated when there are additional
/// which peer generated this transaction and "to whom" this transaction flows.
/// Returns (the transaction info, the number of HTLC outputs which were present in the
/// transaction, the list of HTLCs which were not ignored when building the transaction).
- /// Note that below-dust HTLCs are included in the third return value, but not the second, and
- /// sources are provided only for outbound HTLCs in the third return value.
+ /// Note that below-dust HTLCs are included in the fourth return value, but not the third, and
+ /// sources are provided only for outbound HTLCs in the fourth return value.
#[inline]
- fn build_commitment_transaction<L: Deref>(&self, commitment_number: u64, keys: &TxCreationKeys, local: bool, generated_by_local: bool, feerate_per_kw: u32, logger: &L) -> (CommitmentTransaction, usize, Vec<(HTLCOutputInCommitment, Option<&HTLCSource>)>) where L::Target: Logger {
+ fn build_commitment_transaction<L: Deref>(&self, commitment_number: u64, keys: &TxCreationKeys, local: bool, generated_by_local: bool, logger: &L) -> (CommitmentTransaction, u32, usize, Vec<(HTLCOutputInCommitment, Option<&HTLCSource>)>) where L::Target: Logger {
let mut included_dust_htlcs: Vec<(HTLCOutputInCommitment, Option<&HTLCSource>)> = Vec::new();
let num_htlcs = self.pending_inbound_htlcs.len() + self.pending_outbound_htlcs.len();
let mut included_non_dust_htlcs: Vec<(HTLCOutputInCommitment, Option<&HTLCSource>)> = Vec::with_capacity(num_htlcs);
let mut local_htlc_total_msat = 0;
let mut value_to_self_msat_offset = 0;
+ let mut feerate_per_kw = self.feerate_per_kw;
+ if let Some((feerate, update_state)) = self.pending_update_fee {
+ if match update_state {
+ // Note that these match the inclusion criteria when scanning
+ // pending_inbound_htlcs below.
+ FeeUpdateState::RemoteAnnounced => { debug_assert!(!self.is_outbound()); !generated_by_local },
+ FeeUpdateState::AwaitingRemoteRevokeToAnnounce => { debug_assert!(!self.is_outbound()); !generated_by_local },
+ FeeUpdateState::Outbound => { assert!(self.is_outbound()); generated_by_local },
+ } {
+ feerate_per_kw = feerate;
+ }
+ }
+
log_trace!(logger, "Building commitment transaction number {} (really {} xor {}) for channel {} for {}, generated by {} with fee {}...",
commitment_number, (INITIAL_COMMITMENT_NUMBER - commitment_number),
get_commitment_transaction_number_obscure_factor(&self.get_holder_pubkeys().payment_point, &self.get_counterparty_pubkeys().payment_point, self.is_outbound()),
htlcs_included.sort_unstable_by_key(|h| h.0.transaction_output_index.unwrap());
htlcs_included.append(&mut included_dust_htlcs);
- (tx, num_nondust_htlcs, htlcs_included)
+ (tx, feerate_per_kw, num_nondust_htlcs, htlcs_included)
}
#[inline]
assert!(self.pending_inbound_htlcs.is_empty());
assert!(self.pending_outbound_htlcs.is_empty());
+ assert!(self.pending_update_fee.is_none());
let mut txouts: Vec<(TxOut, ())> = Vec::new();
let mut total_fee_satoshis = proposed_total_fee_satoshis;
let funding_script = self.get_funding_redeemscript();
let keys = self.build_holder_transaction_keys(self.cur_holder_commitment_transaction_number)?;
- let initial_commitment_tx = self.build_commitment_transaction(self.cur_holder_commitment_transaction_number, &keys, true, false, self.feerate_per_kw, logger).0;
+ let initial_commitment_tx = self.build_commitment_transaction(self.cur_holder_commitment_transaction_number, &keys, true, false, logger).0;
{
let trusted_tx = initial_commitment_tx.trust();
let initial_commitment_bitcoin_tx = trusted_tx.built_transaction();
}
let counterparty_keys = self.build_remote_transaction_keys()?;
- let counterparty_initial_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, self.feerate_per_kw, logger).0;
+ let counterparty_initial_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, logger).0;
let counterparty_trusted_tx = counterparty_initial_commitment_tx.trust();
let counterparty_initial_bitcoin_tx = counterparty_trusted_tx.built_transaction();
let funding_script = self.get_funding_redeemscript();
let counterparty_keys = self.build_remote_transaction_keys()?;
- let counterparty_initial_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, self.feerate_per_kw, logger).0;
+ let counterparty_initial_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, logger).0;
let counterparty_trusted_tx = counterparty_initial_commitment_tx.trust();
let counterparty_initial_bitcoin_tx = counterparty_trusted_tx.built_transaction();
log_bytes!(self.channel_id()), counterparty_initial_bitcoin_tx.txid, encode::serialize_hex(&counterparty_initial_bitcoin_tx.transaction));
let holder_signer = self.build_holder_transaction_keys(self.cur_holder_commitment_transaction_number)?;
- let initial_commitment_tx = self.build_commitment_transaction(self.cur_holder_commitment_transaction_number, &holder_signer, true, false, self.feerate_per_kw, logger).0;
+ let initial_commitment_tx = self.build_commitment_transaction(self.cur_holder_commitment_transaction_number, &holder_signer, true, false, logger).0;
{
let trusted_tx = initial_commitment_tx.trust();
let initial_commitment_bitcoin_tx = trusted_tx.built_transaction();
let keys = self.build_holder_transaction_keys(self.cur_holder_commitment_transaction_number).map_err(|e| (None, e))?;
- let mut update_fee = false;
- let feerate_per_kw = if !self.is_outbound() && self.pending_update_fee.is_some() {
- update_fee = true;
- self.pending_update_fee.unwrap()
- } else {
- self.feerate_per_kw
- };
-
- let (num_htlcs, mut htlcs_cloned, commitment_tx, commitment_txid) = {
- let commitment_tx = self.build_commitment_transaction(self.cur_holder_commitment_transaction_number, &keys, true, false, feerate_per_kw, logger);
+ let (num_htlcs, mut htlcs_cloned, commitment_tx, commitment_txid, feerate_per_kw) = {
+ let commitment_tx = self.build_commitment_transaction(self.cur_holder_commitment_transaction_number, &keys, true, false, logger);
let commitment_txid = {
let trusted_tx = commitment_tx.0.trust();
let bitcoin_tx = trusted_tx.built_transaction();
}
bitcoin_tx.txid
};
- let htlcs_cloned: Vec<_> = commitment_tx.2.iter().map(|htlc| (htlc.0.clone(), htlc.1.map(|h| h.clone()))).collect();
- (commitment_tx.1, htlcs_cloned, commitment_tx.0, commitment_txid)
+ let htlcs_cloned: Vec<_> = commitment_tx.3.iter().map(|htlc| (htlc.0.clone(), htlc.1.map(|h| h.clone()))).collect();
+ (commitment_tx.2, htlcs_cloned, commitment_tx.0, commitment_txid, commitment_tx.1)
};
+ // If our counterparty updated the channel fee in this commitment transaction, check that
+ // they can actually afford the new fee now.
+ let update_fee = if let Some((_, update_state)) = self.pending_update_fee {
+ update_state == FeeUpdateState::RemoteAnnounced
+ } else { false };
+ if update_fee { debug_assert!(!self.is_outbound()); }
let total_fee = feerate_per_kw as u64 * (COMMITMENT_TX_BASE_WEIGHT + (num_htlcs as u64) * COMMITMENT_TX_WEIGHT_PER_HTLC) / 1000;
- //If channel fee was updated by funder confirm funder can afford the new fee rate when applied to the current local commitment transaction
if update_fee {
let counterparty_reserve_we_require = Channel::<Signer>::get_holder_selected_channel_reserve_satoshis(self.channel_value_satoshis);
if self.channel_value_satoshis - self.value_to_self_msat / 1000 < total_fee + counterparty_reserve_we_require {
// Update state now that we've passed all the can-fail calls...
let mut need_commitment = false;
- if !self.is_outbound() {
- if let Some(fee_update) = self.pending_update_fee {
- self.feerate_per_kw = fee_update;
- // We later use the presence of pending_update_fee to indicate we should generate a
- // commitment_signed upon receipt of revoke_and_ack, so we can only set it to None
- // if we're not awaiting a revoke (ie will send a commitment_signed now).
- if (self.channel_state & ChannelState::AwaitingRemoteRevoke as u32) == 0 {
- need_commitment = true;
- self.pending_update_fee = None;
- }
+ if let &mut Some((_, ref mut update_state)) = &mut self.pending_update_fee {
+ if *update_state == FeeUpdateState::RemoteAnnounced {
+ *update_state = FeeUpdateState::AwaitingRemoteRevokeToAnnounce;
+ need_commitment = true;
}
}
if update_add_htlcs.is_empty() && update_fulfill_htlcs.is_empty() && update_fail_htlcs.is_empty() && self.holding_cell_update_fee.is_none() {
return Ok((None, htlcs_to_fail));
}
- let update_fee = if let Some(feerate) = self.holding_cell_update_fee {
- self.pending_update_fee = self.holding_cell_update_fee.take();
+ let update_fee = if let Some(feerate) = self.holding_cell_update_fee.take() {
+ assert!(self.is_outbound());
+ self.pending_update_fee = Some((feerate, FeeUpdateState::Outbound));
Some(msgs::UpdateFee {
channel_id: self.channel_id,
feerate_per_kw: feerate as u32,
}
self.value_to_self_msat = (self.value_to_self_msat as i64 + value_to_self_msat_diff) as u64;
- if self.is_outbound() {
- if let Some(feerate) = self.pending_update_fee.take() {
- self.feerate_per_kw = feerate;
- }
- } else {
- if let Some(feerate) = self.pending_update_fee {
- // Because a node cannot send two commitment_signeds in a row without getting a
- // revoke_and_ack from us (as it would otherwise not know the per_commitment_point
- // it should use to create keys with) and because a node can't send a
- // commitment_signed without changes, checking if the feerate is equal to the
- // pending feerate update is sufficient to detect require_commitment.
- if feerate == self.feerate_per_kw {
+ if let Some((feerate, update_state)) = self.pending_update_fee {
+ match update_state {
+ FeeUpdateState::Outbound => {
+ debug_assert!(self.is_outbound());
+ log_trace!(logger, " ...promoting outbound fee update {} to Committed", feerate);
+ self.feerate_per_kw = feerate;
+ self.pending_update_fee = None;
+ },
+ FeeUpdateState::RemoteAnnounced => { debug_assert!(!self.is_outbound()); },
+ FeeUpdateState::AwaitingRemoteRevokeToAnnounce => {
+ debug_assert!(!self.is_outbound());
+ log_trace!(logger, " ...promoting inbound AwaitingRemoteRevokeToAnnounce fee update {} to Committed", feerate);
require_commitment = true;
+ self.feerate_per_kw = feerate;
self.pending_update_fee = None;
- }
+ },
}
}
}
debug_assert!(self.pending_update_fee.is_none());
- self.pending_update_fee = Some(feerate_per_kw);
+ self.pending_update_fee = Some((feerate_per_kw, FeeUpdateState::Outbound));
Some(msgs::UpdateFee {
channel_id: self.channel_id,
});
self.next_counterparty_htlc_id -= inbound_drop_count;
+ if let Some((_, update_state)) = self.pending_update_fee {
+ if update_state == FeeUpdateState::RemoteAnnounced {
+ debug_assert!(!self.is_outbound());
+ self.pending_update_fee = None;
+ }
+ }
+
for htlc in self.pending_outbound_htlcs.iter_mut() {
if let OutboundHTLCState::RemoteRemoved(_) = htlc.state {
// They sent us an update to remove this but haven't yet sent the corresponding
return Err(ChannelError::Close("Peer sent update_fee when we needed a channel_reestablish".to_owned()));
}
Channel::<Signer>::check_remote_fee(fee_estimator, msg.feerate_per_kw)?;
- self.pending_update_fee = Some(msg.feerate_per_kw);
+ self.pending_update_fee = Some((msg.feerate_per_kw, FeeUpdateState::RemoteAnnounced));
self.update_time_counter += 1;
Ok(())
}
let update_fee = if self.is_outbound() && self.pending_update_fee.is_some() {
Some(msgs::UpdateFee {
channel_id: self.channel_id(),
- feerate_per_kw: self.pending_update_fee.unwrap(),
+ feerate_per_kw: self.pending_update_fee.unwrap().0,
})
} else { None };
/// If an Err is returned, it is a ChannelError::Close (for get_outbound_funding_created)
fn get_outbound_funding_created_signature<L: Deref>(&mut self, logger: &L) -> Result<Signature, ChannelError> where L::Target: Logger {
let counterparty_keys = self.build_remote_transaction_keys()?;
- let counterparty_initial_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, self.feerate_per_kw, logger).0;
+ let counterparty_initial_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, logger).0;
Ok(self.holder_signer.sign_counterparty_commitment(&counterparty_initial_commitment_tx, &self.secp_ctx)
.map_err(|_| ChannelError::Close("Failed to get signatures for new commitment_signed".to_owned()))?.0)
}
if (self.channel_state & (ChannelState::MonitorUpdateFailed as u32)) == (ChannelState::MonitorUpdateFailed as u32) {
panic!("Cannot create commitment tx while awaiting monitor update unfreeze, as send_htlc will have returned an Err so a send_commitment precondition has been violated");
}
- let mut have_updates = self.pending_update_fee.is_some();
+ let mut have_updates = self.is_outbound() && self.pending_update_fee.is_some();
for htlc in self.pending_outbound_htlcs.iter() {
if let OutboundHTLCState::LocalAnnounced(_) = htlc.state {
have_updates = true;
htlc.state = OutboundHTLCState::AwaitingRemovedRemoteRevoke(fail_reason);
}
}
+ if let Some((feerate, update_state)) = self.pending_update_fee {
+ if update_state == FeeUpdateState::AwaitingRemoteRevokeToAnnounce {
+ debug_assert!(!self.is_outbound());
+ self.feerate_per_kw = feerate;
+ self.pending_update_fee = None;
+ }
+ }
self.resend_order = RAACommitmentOrder::RevokeAndACKFirst;
let (res, counterparty_commitment_txid, htlcs) = match self.send_commitment_no_state_update(logger) {
/// Only fails in case of bad keys. Used for channel_reestablish commitment_signed generation
/// when we shouldn't change HTLC/channel state.
fn send_commitment_no_state_update<L: Deref>(&self, logger: &L) -> Result<(msgs::CommitmentSigned, (Txid, Vec<(HTLCOutputInCommitment, Option<&HTLCSource>)>)), ChannelError> where L::Target: Logger {
- let mut feerate_per_kw = self.feerate_per_kw;
- if let Some(feerate) = self.pending_update_fee {
- if self.is_outbound() {
- feerate_per_kw = feerate;
- }
- }
-
let counterparty_keys = self.build_remote_transaction_keys()?;
- let counterparty_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, true, feerate_per_kw, logger);
+ let counterparty_commitment_tx = self.build_commitment_transaction(self.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, true, logger);
+ let feerate_per_kw = counterparty_commitment_tx.1;
let counterparty_commitment_txid = counterparty_commitment_tx.0.trust().txid();
let (signature, htlc_signatures);
&& info.next_holder_htlc_id == self.next_holder_htlc_id
&& info.next_counterparty_htlc_id == self.next_counterparty_htlc_id
&& info.feerate == self.feerate_per_kw {
- let actual_fee = self.commit_tx_fee_msat(counterparty_commitment_tx.1);
+ let actual_fee = self.commit_tx_fee_msat(counterparty_commitment_tx.2);
assert_eq!(actual_fee, info.fee);
}
}
}
{
- let mut htlcs = Vec::with_capacity(counterparty_commitment_tx.2.len());
- for &(ref htlc, _) in counterparty_commitment_tx.2.iter() {
+ let mut htlcs = Vec::with_capacity(counterparty_commitment_tx.3.len());
+ for &(ref htlc, _) in counterparty_commitment_tx.3.iter() {
htlcs.push(htlc);
}
channel_id: self.channel_id,
signature,
htlc_signatures,
- }, (counterparty_commitment_txid, counterparty_commitment_tx.2)))
+ }, (counterparty_commitment_txid, counterparty_commitment_tx.3)))
}
/// Adds a pending outbound HTLC to this channel, and creates a signed commitment transaction
fail_reason.write(writer)?;
}
- self.pending_update_fee.write(writer)?;
+ if self.is_outbound() {
+ self.pending_update_fee.map(|(a, _)| a).write(writer)?;
+ } else if let Some((feerate, FeeUpdateState::AwaitingRemoteRevokeToAnnounce)) = self.pending_update_fee {
+ // As for inbound HTLCs, if the update was only announced and never committed, drop it.
+ Some(feerate).write(writer)?;
+ } else {
+ None::<u32>.write(writer)?;
+ }
self.holding_cell_update_fee.write(writer)?;
self.next_holder_htlc_id.write(writer)?;
monitor_pending_failures.push((Readable::read(reader)?, Readable::read(reader)?, Readable::read(reader)?));
}
- let pending_update_fee = Readable::read(reader)?;
+ let pending_update_fee_value: Option<u32> = Readable::read(reader)?;
+
let holding_cell_update_fee = Readable::read(reader)?;
let next_holder_htlc_id = Readable::read(reader)?;
_ => return Err(DecodeError::InvalidValue),
};
- let channel_parameters = Readable::read(reader)?;
+ let channel_parameters: ChannelTransactionParameters = Readable::read(reader)?;
let funding_transaction = Readable::read(reader)?;
let counterparty_cur_commitment_point = Readable::read(reader)?;
}
}
+ let pending_update_fee = if let Some(feerate) = pending_update_fee_value {
+ Some((feerate, if channel_parameters.is_outbound_from_holder {
+ FeeUpdateState::Outbound
+ } else {
+ FeeUpdateState::AwaitingRemoteRevokeToAnnounce
+ }))
+ } else {
+ None
+ };
+
let mut announcement_sigs = None;
read_tlv_fields!(reader, {
(0, announcement_sigs, option),
$( { $htlc_idx: expr, $counterparty_htlc_sig_hex: expr, $htlc_sig_hex: expr, $htlc_tx_hex: expr } ), *
} ) => { {
let (commitment_tx, htlcs): (_, Vec<HTLCOutputInCommitment>) = {
- let mut res = chan.build_commitment_transaction(0xffffffffffff - 42, &keys, true, false, chan.feerate_per_kw, &logger);
+ let mut res = chan.build_commitment_transaction(0xffffffffffff - 42, &keys, true, false, &logger);
- let htlcs = res.2.drain(..)
+ let htlcs = res.3.drain(..)
.filter_map(|(htlc, _)| if htlc.transaction_output_index.is_some() { Some(htlc) } else { None })
.collect();
(res.0, htlcs)