Skip to content

Commit c4d0560

Browse files
authored
Merge pull request #3633 from jkczyz/2025-02-split-commitment-signed
Split `commitment_signed` handling by check-accept
2 parents 43de15e + 2be03ce commit c4d0560

File tree

2 files changed

+154
-124
lines changed

2 files changed

+154
-124
lines changed

lightning/src/chain/channelmonitor.rs

+1
Original file line numberDiff line numberDiff line change
@@ -535,6 +535,7 @@ impl_writeable_tlv_based_enum_upgradable!(OnchainEvent,
535535

536536
#[derive(Clone, Debug, PartialEq, Eq)]
537537
pub(crate) enum ChannelMonitorUpdateStep {
538+
// Update LatestHolderCommitmentTXInfo in channel.rs if adding new fields to this variant.
538539
LatestHolderCommitmentTXInfo {
539540
commitment_tx: HolderCommitmentTransaction,
540541
/// Note that LDK after 0.0.115 supports this only containing dust HTLCs (implying the

lightning/src/ln/channel.rs

+153-124
Original file line numberDiff line numberDiff line change
@@ -2032,7 +2032,7 @@ trait InitialRemoteCommitmentReceiver<SP: Deref> where SP::Target: SignerProvide
20322032
fn received_msg(&self) -> &'static str;
20332033

20342034
fn check_counterparty_commitment_signature<L: Deref>(
2035-
&self, sig: &Signature, holder_commitment_point: &mut HolderCommitmentPoint, logger: &L
2035+
&self, sig: &Signature, holder_commitment_point: &HolderCommitmentPoint, logger: &L
20362036
) -> Result<CommitmentTransaction, ChannelError> where L::Target: Logger {
20372037
let funding_script = self.funding().get_funding_redeemscript();
20382038

@@ -3457,6 +3457,132 @@ impl<SP: Deref> ChannelContext<SP> where SP::Target: SignerProvider {
34573457
!matches!(self.channel_state, ChannelState::AwaitingChannelReady(flags) if flags.is_set(AwaitingChannelReadyFlags::WAITING_FOR_BATCH))
34583458
}
34593459

3460+
fn validate_commitment_signed<L: Deref>(
3461+
&self, funding: &FundingScope, holder_commitment_point: &HolderCommitmentPoint,
3462+
msg: &msgs::CommitmentSigned, logger: &L,
3463+
) -> Result<LatestHolderCommitmentTXInfo, ChannelError>
3464+
where
3465+
L::Target: Logger,
3466+
{
3467+
let funding_script = funding.get_funding_redeemscript();
3468+
3469+
let keys = self.build_holder_transaction_keys(funding, holder_commitment_point.current_point());
3470+
3471+
let commitment_stats = self.build_commitment_transaction(funding, holder_commitment_point.transaction_number(), &keys, true, false, logger);
3472+
let commitment_txid = {
3473+
let trusted_tx = commitment_stats.tx.trust();
3474+
let bitcoin_tx = trusted_tx.built_transaction();
3475+
let sighash = bitcoin_tx.get_sighash_all(&funding_script, funding.get_value_satoshis());
3476+
3477+
log_trace!(logger, "Checking commitment tx signature {} by key {} against tx {} (sighash {}) with redeemscript {} in channel {}",
3478+
log_bytes!(msg.signature.serialize_compact()[..]),
3479+
log_bytes!(funding.counterparty_funding_pubkey().serialize()), encode::serialize_hex(&bitcoin_tx.transaction),
3480+
log_bytes!(sighash[..]), encode::serialize_hex(&funding_script), &self.channel_id());
3481+
if let Err(_) = self.secp_ctx.verify_ecdsa(&sighash, &msg.signature, &funding.counterparty_funding_pubkey()) {
3482+
return Err(ChannelError::close("Invalid commitment tx signature from peer".to_owned()));
3483+
}
3484+
bitcoin_tx.txid
3485+
};
3486+
let mut htlcs_cloned: Vec<_> = commitment_stats.htlcs_included.iter().map(|htlc| (htlc.0.clone(), htlc.1.map(|h| h.clone()))).collect();
3487+
3488+
// If our counterparty updated the channel fee in this commitment transaction, check that
3489+
// they can actually afford the new fee now.
3490+
let update_fee = if let Some((_, update_state)) = self.pending_update_fee {
3491+
update_state == FeeUpdateState::RemoteAnnounced
3492+
} else { false };
3493+
if update_fee {
3494+
debug_assert!(!funding.is_outbound());
3495+
let counterparty_reserve_we_require_msat = funding.holder_selected_channel_reserve_satoshis * 1000;
3496+
if commitment_stats.remote_balance_msat < commitment_stats.total_fee_sat * 1000 + counterparty_reserve_we_require_msat {
3497+
return Err(ChannelError::close("Funding remote cannot afford proposed new fee".to_owned()));
3498+
}
3499+
}
3500+
#[cfg(any(test, fuzzing))]
3501+
{
3502+
if funding.is_outbound() {
3503+
let projected_commit_tx_info = funding.next_local_commitment_tx_fee_info_cached.lock().unwrap().take();
3504+
*funding.next_remote_commitment_tx_fee_info_cached.lock().unwrap() = None;
3505+
if let Some(info) = projected_commit_tx_info {
3506+
let total_pending_htlcs = self.pending_inbound_htlcs.len() + self.pending_outbound_htlcs.len()
3507+
+ self.holding_cell_htlc_updates.len();
3508+
if info.total_pending_htlcs == total_pending_htlcs
3509+
&& info.next_holder_htlc_id == self.next_holder_htlc_id
3510+
&& info.next_counterparty_htlc_id == self.next_counterparty_htlc_id
3511+
&& info.feerate == self.feerate_per_kw {
3512+
assert_eq!(commitment_stats.total_fee_sat, info.fee / 1000);
3513+
}
3514+
}
3515+
}
3516+
}
3517+
3518+
if msg.htlc_signatures.len() != commitment_stats.num_nondust_htlcs {
3519+
return Err(ChannelError::close(format!("Got wrong number of HTLC signatures ({}) from remote. It must be {}", msg.htlc_signatures.len(), commitment_stats.num_nondust_htlcs)));
3520+
}
3521+
3522+
// Up to LDK 0.0.115, HTLC information was required to be duplicated in the
3523+
// `htlcs_and_sigs` vec and in the `holder_commitment_tx` itself, both of which were passed
3524+
// in the `ChannelMonitorUpdate`. In 0.0.115, support for having a separate set of
3525+
// outbound-non-dust-HTLCSources in the `ChannelMonitorUpdate` was added, however for
3526+
// backwards compatibility, we never use it in production. To provide test coverage, here,
3527+
// we randomly decide (in test/fuzzing builds) to use the new vec sometimes.
3528+
#[allow(unused_assignments, unused_mut)]
3529+
let mut separate_nondust_htlc_sources = false;
3530+
#[cfg(all(feature = "std", any(test, fuzzing)))] {
3531+
use core::hash::{BuildHasher, Hasher};
3532+
// Get a random value using the only std API to do so - the DefaultHasher
3533+
let rand_val = std::collections::hash_map::RandomState::new().build_hasher().finish();
3534+
separate_nondust_htlc_sources = rand_val % 2 == 0;
3535+
}
3536+
3537+
let mut nondust_htlc_sources = Vec::with_capacity(htlcs_cloned.len());
3538+
let mut htlcs_and_sigs = Vec::with_capacity(htlcs_cloned.len());
3539+
for (idx, (htlc, mut source_opt)) in htlcs_cloned.drain(..).enumerate() {
3540+
if let Some(_) = htlc.transaction_output_index {
3541+
let htlc_tx = chan_utils::build_htlc_transaction(&commitment_txid, commitment_stats.feerate_per_kw,
3542+
funding.get_counterparty_selected_contest_delay().unwrap(), &htlc, &self.channel_type,
3543+
&keys.broadcaster_delayed_payment_key, &keys.revocation_key);
3544+
3545+
let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, &self.channel_type, &keys);
3546+
let htlc_sighashtype = if self.channel_type.supports_anchors_zero_fee_htlc_tx() { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
3547+
let htlc_sighash = hash_to_message!(&sighash::SighashCache::new(&htlc_tx).p2wsh_signature_hash(0, &htlc_redeemscript, htlc.to_bitcoin_amount(), htlc_sighashtype).unwrap()[..]);
3548+
log_trace!(logger, "Checking HTLC tx signature {} by key {} against tx {} (sighash {}) with redeemscript {} in channel {}.",
3549+
log_bytes!(msg.htlc_signatures[idx].serialize_compact()[..]), log_bytes!(keys.countersignatory_htlc_key.to_public_key().serialize()),
3550+
encode::serialize_hex(&htlc_tx), log_bytes!(htlc_sighash[..]), encode::serialize_hex(&htlc_redeemscript), &self.channel_id());
3551+
if let Err(_) = self.secp_ctx.verify_ecdsa(&htlc_sighash, &msg.htlc_signatures[idx], &keys.countersignatory_htlc_key.to_public_key()) {
3552+
return Err(ChannelError::close("Invalid HTLC tx signature from peer".to_owned()));
3553+
}
3554+
if !separate_nondust_htlc_sources {
3555+
htlcs_and_sigs.push((htlc, Some(msg.htlc_signatures[idx]), source_opt.take()));
3556+
}
3557+
} else {
3558+
htlcs_and_sigs.push((htlc, None, source_opt.take()));
3559+
}
3560+
if separate_nondust_htlc_sources {
3561+
if let Some(source) = source_opt.take() {
3562+
nondust_htlc_sources.push(source);
3563+
}
3564+
}
3565+
debug_assert!(source_opt.is_none(), "HTLCSource should have been put somewhere");
3566+
}
3567+
3568+
let holder_commitment_tx = HolderCommitmentTransaction::new(
3569+
commitment_stats.tx,
3570+
msg.signature,
3571+
msg.htlc_signatures.clone(),
3572+
&funding.get_holder_pubkeys().funding_pubkey,
3573+
funding.counterparty_funding_pubkey()
3574+
);
3575+
3576+
self.holder_signer.as_ref().validate_holder_commitment(&holder_commitment_tx, commitment_stats.outbound_htlc_preimages)
3577+
.map_err(|_| ChannelError::close("Failed to validate our commitment".to_owned()))?;
3578+
3579+
Ok(LatestHolderCommitmentTXInfo {
3580+
commitment_tx: holder_commitment_tx,
3581+
htlc_outputs: htlcs_and_sigs,
3582+
nondust_htlc_sources,
3583+
})
3584+
}
3585+
34603586
/// Transaction nomenclature is somewhat confusing here as there are many different cases - a
34613587
/// transaction is referred to as "a's transaction" implying that a will be able to broadcast
34623588
/// the transaction. Thus, b will generally be sending a signature over such a transaction to
@@ -4711,6 +4837,14 @@ struct CommitmentTxInfoCached {
47114837
feerate: u32,
47124838
}
47134839

4840+
/// Partial data from ChannelMonitorUpdateStep::LatestHolderCommitmentTXInfo used to simplify the
4841+
/// return type of `ChannelContext::validate_commitment_signed`.
4842+
struct LatestHolderCommitmentTXInfo {
4843+
pub commitment_tx: HolderCommitmentTransaction,
4844+
pub htlc_outputs: Vec<(HTLCOutputInCommitment, Option<Signature>, Option<HTLCSource>)>,
4845+
pub nondust_htlc_sources: Vec<HTLCSource>,
4846+
}
4847+
47144848
/// Contents of a wire message that fails an HTLC backwards. Useful for [`FundedChannel::fail_htlc`] to
47154849
/// fail with either [`msgs::UpdateFailMalformedHTLC`] or [`msgs::UpdateFailHTLC`] as needed.
47164850
trait FailHTLCContents {
@@ -5515,118 +5649,22 @@ impl<SP: Deref> FundedChannel<SP> where
55155649
return Err(ChannelError::close("Peer sent commitment_signed after we'd started exchanging closing_signeds".to_owned()));
55165650
}
55175651

5518-
let funding_script = self.funding.get_funding_redeemscript();
5519-
5520-
let keys = self.context.build_holder_transaction_keys(&self.funding, self.holder_commitment_point.current_point());
5521-
5522-
let commitment_stats = self.context.build_commitment_transaction(&self.funding, self.holder_commitment_point.transaction_number(), &keys, true, false, logger);
5523-
let commitment_txid = {
5524-
let trusted_tx = commitment_stats.tx.trust();
5525-
let bitcoin_tx = trusted_tx.built_transaction();
5526-
let sighash = bitcoin_tx.get_sighash_all(&funding_script, self.funding.get_value_satoshis());
5527-
5528-
log_trace!(logger, "Checking commitment tx signature {} by key {} against tx {} (sighash {}) with redeemscript {} in channel {}",
5529-
log_bytes!(msg.signature.serialize_compact()[..]),
5530-
log_bytes!(self.funding.counterparty_funding_pubkey().serialize()), encode::serialize_hex(&bitcoin_tx.transaction),
5531-
log_bytes!(sighash[..]), encode::serialize_hex(&funding_script), &self.context.channel_id());
5532-
if let Err(_) = self.context.secp_ctx.verify_ecdsa(&sighash, &msg.signature, &self.funding.counterparty_funding_pubkey()) {
5533-
return Err(ChannelError::close("Invalid commitment tx signature from peer".to_owned()));
5534-
}
5535-
bitcoin_tx.txid
5536-
};
5537-
let mut htlcs_cloned: Vec<_> = commitment_stats.htlcs_included.iter().map(|htlc| (htlc.0.clone(), htlc.1.map(|h| h.clone()))).collect();
5538-
5539-
// If our counterparty updated the channel fee in this commitment transaction, check that
5540-
// they can actually afford the new fee now.
5541-
let update_fee = if let Some((_, update_state)) = self.context.pending_update_fee {
5542-
update_state == FeeUpdateState::RemoteAnnounced
5543-
} else { false };
5544-
if update_fee {
5545-
debug_assert!(!self.funding.is_outbound());
5546-
let counterparty_reserve_we_require_msat = self.funding.holder_selected_channel_reserve_satoshis * 1000;
5547-
if commitment_stats.remote_balance_msat < commitment_stats.total_fee_sat * 1000 + counterparty_reserve_we_require_msat {
5548-
return Err(ChannelError::close("Funding remote cannot afford proposed new fee".to_owned()));
5549-
}
5550-
}
5551-
#[cfg(any(test, fuzzing))]
5552-
{
5553-
if self.funding.is_outbound() {
5554-
let projected_commit_tx_info = self.funding.next_local_commitment_tx_fee_info_cached.lock().unwrap().take();
5555-
*self.funding.next_remote_commitment_tx_fee_info_cached.lock().unwrap() = None;
5556-
if let Some(info) = projected_commit_tx_info {
5557-
let total_pending_htlcs = self.context.pending_inbound_htlcs.len() + self.context.pending_outbound_htlcs.len()
5558-
+ self.context.holding_cell_htlc_updates.len();
5559-
if info.total_pending_htlcs == total_pending_htlcs
5560-
&& info.next_holder_htlc_id == self.context.next_holder_htlc_id
5561-
&& info.next_counterparty_htlc_id == self.context.next_counterparty_htlc_id
5562-
&& info.feerate == self.context.feerate_per_kw {
5563-
assert_eq!(commitment_stats.total_fee_sat, info.fee / 1000);
5564-
}
5565-
}
5566-
}
5567-
}
5568-
5569-
if msg.htlc_signatures.len() != commitment_stats.num_nondust_htlcs {
5570-
return Err(ChannelError::close(format!("Got wrong number of HTLC signatures ({}) from remote. It must be {}", msg.htlc_signatures.len(), commitment_stats.num_nondust_htlcs)));
5571-
}
5652+
let commitment_tx_info = self.context.validate_commitment_signed(&self.funding, &self.holder_commitment_point, msg, logger)?;
55725653

5573-
// Up to LDK 0.0.115, HTLC information was required to be duplicated in the
5574-
// `htlcs_and_sigs` vec and in the `holder_commitment_tx` itself, both of which were passed
5575-
// in the `ChannelMonitorUpdate`. In 0.0.115, support for having a separate set of
5576-
// outbound-non-dust-HTLCSources in the `ChannelMonitorUpdate` was added, however for
5577-
// backwards compatibility, we never use it in production. To provide test coverage, here,
5578-
// we randomly decide (in test/fuzzing builds) to use the new vec sometimes.
5579-
#[allow(unused_assignments, unused_mut)]
5580-
let mut separate_nondust_htlc_sources = false;
5581-
#[cfg(all(feature = "std", any(test, fuzzing)))] {
5582-
use core::hash::{BuildHasher, Hasher};
5583-
// Get a random value using the only std API to do so - the DefaultHasher
5584-
let rand_val = std::collections::hash_map::RandomState::new().build_hasher().finish();
5585-
separate_nondust_htlc_sources = rand_val % 2 == 0;
5586-
}
5587-
5588-
let mut nondust_htlc_sources = Vec::with_capacity(htlcs_cloned.len());
5589-
let mut htlcs_and_sigs = Vec::with_capacity(htlcs_cloned.len());
5590-
for (idx, (htlc, mut source_opt)) in htlcs_cloned.drain(..).enumerate() {
5591-
if let Some(_) = htlc.transaction_output_index {
5592-
let htlc_tx = chan_utils::build_htlc_transaction(&commitment_txid, commitment_stats.feerate_per_kw,
5593-
self.funding.get_counterparty_selected_contest_delay().unwrap(), &htlc, &self.context.channel_type,
5594-
&keys.broadcaster_delayed_payment_key, &keys.revocation_key);
5595-
5596-
let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, &self.context.channel_type, &keys);
5597-
let htlc_sighashtype = if self.context.channel_type.supports_anchors_zero_fee_htlc_tx() { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
5598-
let htlc_sighash = hash_to_message!(&sighash::SighashCache::new(&htlc_tx).p2wsh_signature_hash(0, &htlc_redeemscript, htlc.to_bitcoin_amount(), htlc_sighashtype).unwrap()[..]);
5599-
log_trace!(logger, "Checking HTLC tx signature {} by key {} against tx {} (sighash {}) with redeemscript {} in channel {}.",
5600-
log_bytes!(msg.htlc_signatures[idx].serialize_compact()[..]), log_bytes!(keys.countersignatory_htlc_key.to_public_key().serialize()),
5601-
encode::serialize_hex(&htlc_tx), log_bytes!(htlc_sighash[..]), encode::serialize_hex(&htlc_redeemscript), &self.context.channel_id());
5602-
if let Err(_) = self.context.secp_ctx.verify_ecdsa(&htlc_sighash, &msg.htlc_signatures[idx], &keys.countersignatory_htlc_key.to_public_key()) {
5603-
return Err(ChannelError::close("Invalid HTLC tx signature from peer".to_owned()));
5604-
}
5605-
if !separate_nondust_htlc_sources {
5606-
htlcs_and_sigs.push((htlc, Some(msg.htlc_signatures[idx]), source_opt.take()));
5607-
}
5608-
} else {
5609-
htlcs_and_sigs.push((htlc, None, source_opt.take()));
5610-
}
5611-
if separate_nondust_htlc_sources {
5612-
if let Some(source) = source_opt.take() {
5613-
nondust_htlc_sources.push(source);
5614-
}
5615-
}
5616-
debug_assert!(source_opt.is_none(), "HTLCSource should have been put somewhere");
5654+
if self.holder_commitment_point.advance(&self.context.holder_signer, &self.context.secp_ctx, logger).is_err() {
5655+
// We only fail to advance our commitment point/number if we're currently
5656+
// waiting for our signer to unblock and provide a commitment point.
5657+
// During post-funding channel operation, we only advance our point upon
5658+
// receiving a commitment_signed, and our counterparty cannot send us
5659+
// another commitment signed until we've provided a new commitment point
5660+
// in revoke_and_ack, which requires unblocking our signer and completing
5661+
// the advance to the next point. This should be unreachable since
5662+
// a new commitment_signed should fail at our signature checks in
5663+
// validate_commitment_signed.
5664+
debug_assert!(false, "We should be ready to advance our commitment point by the time we receive commitment_signed");
5665+
return Err(ChannelError::close("Failed to advance our commitment point".to_owned()));
56175666
}
56185667

5619-
let holder_commitment_tx = HolderCommitmentTransaction::new(
5620-
commitment_stats.tx,
5621-
msg.signature,
5622-
msg.htlc_signatures.clone(),
5623-
&self.funding.get_holder_pubkeys().funding_pubkey,
5624-
self.funding.counterparty_funding_pubkey()
5625-
);
5626-
5627-
self.context.holder_signer.as_ref().validate_holder_commitment(&holder_commitment_tx, commitment_stats.outbound_htlc_preimages)
5628-
.map_err(|_| ChannelError::close("Failed to validate our commitment".to_owned()))?;
5629-
56305668
// Update state now that we've passed all the can-fail calls...
56315669
let mut need_commitment = false;
56325670
if let &mut Some((_, ref mut update_state)) = &mut self.context.pending_update_fee {
@@ -5666,31 +5704,22 @@ impl<SP: Deref> FundedChannel<SP> where
56665704
}
56675705
}
56685706

5707+
let LatestHolderCommitmentTXInfo {
5708+
commitment_tx, htlc_outputs, nondust_htlc_sources,
5709+
} = commitment_tx_info;
56695710
self.context.latest_monitor_update_id += 1;
56705711
let mut monitor_update = ChannelMonitorUpdate {
56715712
update_id: self.context.latest_monitor_update_id,
56725713
counterparty_node_id: Some(self.context.counterparty_node_id),
56735714
updates: vec![ChannelMonitorUpdateStep::LatestHolderCommitmentTXInfo {
5674-
commitment_tx: holder_commitment_tx,
5675-
htlc_outputs: htlcs_and_sigs,
5715+
commitment_tx,
5716+
htlc_outputs,
56765717
claimed_htlcs,
56775718
nondust_htlc_sources,
56785719
}],
56795720
channel_id: Some(self.context.channel_id()),
56805721
};
56815722

5682-
if self.holder_commitment_point.advance(&self.context.holder_signer, &self.context.secp_ctx, logger).is_err() {
5683-
// We only fail to advance our commitment point/number if we're currently
5684-
// waiting for our signer to unblock and provide a commitment point.
5685-
// During post-funding channel operation, we only advance our point upon
5686-
// receiving a commitment_signed, and our counterparty cannot send us
5687-
// another commitment signed until we've provided a new commitment point
5688-
// in revoke_and_ack, which requires unblocking our signer and completing
5689-
// the advance to the next point. This should be unreachable since
5690-
// a new commitment_signed should fail at our signature checks above.
5691-
debug_assert!(false, "We should be ready to advance our commitment point by the time we receive commitment_signed");
5692-
return Err(ChannelError::close("Failed to advance our commitment point".to_owned()));
5693-
}
56945723
self.context.expecting_peer_commitment_signed = false;
56955724
// Note that if we need_commitment & !AwaitingRemoteRevoke we'll call
56965725
// build_commitment_no_status_check() next which will reset this to RAAFirst.

0 commit comments

Comments
 (0)