Skip to content

Commit

Permalink
Let ChannelSigner set htlc tx script pubkey
Browse files Browse the repository at this point in the history
This allows the htlc tx output to easily be changed according to the
features of the channel, or the evolution of the LN specification.

The output could even be set to completely arbitrary scripts if
compatibility with the formal LN spec is not required.

Builders of htlc transactions now ask a `ChannelSigner` for the
appropriate revokeable script pubkey to use, and then pass it to the
htlc transaction constructors.
  • Loading branch information
tankyleo committed Dec 15, 2024
1 parent 1155fa0 commit d3181c5
Show file tree
Hide file tree
Showing 7 changed files with 54 additions and 59 deletions.
8 changes: 5 additions & 3 deletions lightning/src/chain/onchaintx.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1210,8 +1210,9 @@ impl<ChannelSigner: EcdsaChannelSigner> OnchainTxHandler<ChannelSigner> {
.find(|(_, htlc)| htlc.transaction_output_index.unwrap() == outp.vout)
.unwrap();
let counterparty_htlc_sig = holder_commitment.counterparty_htlc_sigs[htlc_idx];
let revokeable_spk = self.signer.get_revokeable_spk(true, holder_commitment.commitment_number(), &holder_commitment.per_commitment_point(), &self.secp_ctx);
let mut htlc_tx = trusted_tx.build_unsigned_htlc_tx(
&self.channel_transaction_parameters.as_holder_broadcastable(), htlc_idx, preimage,
htlc_idx, preimage, revokeable_spk,
);

let htlc_descriptor = HTLCDescriptor {
Expand Down Expand Up @@ -1295,7 +1296,7 @@ mod tests {
};
use crate::ln::channel_keys::{DelayedPaymentBasepoint, HtlcBasepoint, RevocationBasepoint};
use crate::ln::functional_test_utils::create_dummy_block;
use crate::sign::InMemorySigner;
use crate::sign::{ChannelSigner, InMemorySigner};
use crate::types::payment::{PaymentHash, PaymentPreimage};
use crate::util::test_utils::{TestBroadcaster, TestFeeEstimator, TestLogger};

Expand All @@ -1307,7 +1308,7 @@ mod tests {
#[test]
fn test_broadcast_height() {
let secp_ctx = Secp256k1::new();
let signer = InMemorySigner::new(
let mut signer = InMemorySigner::new(
&secp_ctx,
SecretKey::from_slice(&[41; 32]).unwrap(),
SecretKey::from_slice(&[41; 32]).unwrap(),
Expand Down Expand Up @@ -1356,6 +1357,7 @@ mod tests {
funding_outpoint: Some(funding_outpoint),
channel_type_features: ChannelTypeFeatures::only_static_remote_key(),
};
signer.provide_channel_parameters(&chan_params);

// Create an OnchainTxHandler for a commitment containing HTLCs with CLTV expiries of 0, 1,
// and 2 blocks.
Expand Down
16 changes: 11 additions & 5 deletions lightning/src/events/bump_transaction.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ use crate::ln::chan_utils::{
};
use crate::prelude::*;
use crate::sign::{
ChannelDerivationParameters, HTLCDescriptor, SignerProvider, P2WPKH_WITNESS_WEIGHT
ChannelDerivationParameters, ChannelSigner, HTLCDescriptor, SignerProvider, P2WPKH_WITNESS_WEIGHT,
};
use crate::sign::ecdsa::EcdsaChannelSigner;
use crate::sync::Mutex;
Expand Down Expand Up @@ -728,6 +728,7 @@ where
output: vec![],
};
let mut must_spend = Vec::with_capacity(htlc_descriptors.len());
let mut signers_and_revokeable_spks = BTreeMap::new();
for htlc_descriptor in htlc_descriptors {
let htlc_input = htlc_descriptor.unsigned_tx_input();
must_spend.push(Input {
Expand All @@ -740,7 +741,13 @@ where
},
});
htlc_tx.input.push(htlc_input);
let htlc_output = htlc_descriptor.tx_output(&self.secp);
let revokeable_spk = signers_and_revokeable_spks.entry(htlc_descriptor.channel_derivation_parameters.keys_id)
.or_insert_with(|| {
let signer = htlc_descriptor.derive_channel_signer(&self.signer_provider);
let revokeable_spk = signer.get_revokeable_spk(true, htlc_descriptor.per_commitment_number, &htlc_descriptor.per_commitment_point, &self.secp);
(signer, revokeable_spk)
}).1.clone();
let htlc_output = htlc_descriptor.tx_output(revokeable_spk);
htlc_tx.output.push(htlc_output);
}

Expand Down Expand Up @@ -789,10 +796,9 @@ where
log_debug!(self.logger, "Signing HTLC transaction {}", htlc_psbt.unsigned_tx.compute_txid());
htlc_tx = self.utxo_source.sign_psbt(htlc_psbt)?;

let mut signers = BTreeMap::new();
for (idx, htlc_descriptor) in htlc_descriptors.iter().enumerate() {
let signer = signers.entry(htlc_descriptor.channel_derivation_parameters.keys_id)
.or_insert_with(|| htlc_descriptor.derive_channel_signer(&self.signer_provider));
// Unwrap because we derived the corresponding signers for all htlc descriptors further above
let signer = &signers_and_revokeable_spks.get(&htlc_descriptor.channel_derivation_parameters.keys_id).unwrap().0;
let htlc_sig = signer.sign_holder_htlc_transaction(&htlc_tx, idx, htlc_descriptor, &self.secp)?;
let witness_script = htlc_descriptor.witness_script(&self.secp);
htlc_tx.input[idx].witness = htlc_descriptor.tx_input_witness(&htlc_sig, &witness_script);
Expand Down
26 changes: 10 additions & 16 deletions lightning/src/ln/chan_utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -704,13 +704,12 @@ pub(crate) fn make_funding_redeemscript_from_slices(broadcaster_funding_key: &[u
///
/// Panics if htlc.transaction_output_index.is_none() (as such HTLCs do not appear in the
/// commitment transaction).
pub fn build_htlc_transaction(commitment_txid: &Txid, feerate_per_kw: u32, contest_delay: u16, htlc: &HTLCOutputInCommitment, channel_type_features: &ChannelTypeFeatures, broadcaster_delayed_payment_key: &DelayedPaymentKey, revocation_key: &RevocationKey) -> Transaction {
let txins= vec![build_htlc_input(commitment_txid, htlc, channel_type_features)];
pub fn build_htlc_transaction(commitment_txid: &Txid, feerate_per_kw: u32, htlc: &HTLCOutputInCommitment, channel_type_features: &ChannelTypeFeatures, revokeable_spk: ScriptBuf) -> Transaction {
let txins = vec![build_htlc_input(commitment_txid, htlc, channel_type_features)];

let mut txouts: Vec<TxOut> = Vec::new();
txouts.push(build_htlc_output(
feerate_per_kw, contest_delay, htlc, channel_type_features,
broadcaster_delayed_payment_key, revocation_key
feerate_per_kw, htlc, channel_type_features, revokeable_spk,
));

Transaction {
Expand All @@ -734,7 +733,7 @@ pub(crate) fn build_htlc_input(commitment_txid: &Txid, htlc: &HTLCOutputInCommit
}

pub(crate) fn build_htlc_output(
feerate_per_kw: u32, contest_delay: u16, htlc: &HTLCOutputInCommitment, channel_type_features: &ChannelTypeFeatures, broadcaster_delayed_payment_key: &DelayedPaymentKey, revocation_key: &RevocationKey
feerate_per_kw: u32, htlc: &HTLCOutputInCommitment, channel_type_features: &ChannelTypeFeatures, revokeable_spk: ScriptBuf
) -> TxOut {
let weight = if htlc.offered {
htlc_timeout_tx_weight(channel_type_features)
Expand All @@ -749,7 +748,7 @@ pub(crate) fn build_htlc_output(
};

TxOut {
script_pubkey: get_revokeable_redeemscript(revocation_key, contest_delay, broadcaster_delayed_payment_key).to_p2wsh(),
script_pubkey: revokeable_spk,
value: output_value,
}
}
Expand Down Expand Up @@ -1740,8 +1739,7 @@ impl<'a> TrustedCommitmentTransaction<'a> {
///
/// This function is only valid in the holder commitment context, it always uses EcdsaSighashType::All.
pub fn get_htlc_sigs<T: secp256k1::Signing, ES: Deref>(
&self, htlc_base_key: &SecretKey, channel_parameters: &DirectedChannelTransactionParameters,
entropy_source: &ES, secp_ctx: &Secp256k1<T>,
&self, htlc_base_key: &SecretKey, entropy_source: &ES, secp_ctx: &Secp256k1<T>, revokeable_spk: ScriptBuf,
) -> Result<Vec<Signature>, ()> where ES::Target: EntropySource {
let inner = self.inner;
let keys = &inner.keys;
Expand All @@ -1751,7 +1749,7 @@ impl<'a> TrustedCommitmentTransaction<'a> {

for this_htlc in inner.htlcs.iter() {
assert!(this_htlc.transaction_output_index.is_some());
let htlc_tx = build_htlc_transaction(&txid, inner.feerate_per_kw, channel_parameters.contest_delay(), &this_htlc, &self.channel_type_features, &keys.broadcaster_delayed_payment_key, &keys.revocation_key);
let htlc_tx = build_htlc_transaction(&txid, inner.feerate_per_kw, &this_htlc, &self.channel_type_features, revokeable_spk.clone());

let htlc_redeemscript = get_htlc_redeemscript_with_explicit_keys(&this_htlc, &self.channel_type_features, &keys.broadcaster_htlc_key, &keys.countersignatory_htlc_key, &keys.revocation_key);

Expand All @@ -1762,11 +1760,7 @@ impl<'a> TrustedCommitmentTransaction<'a> {
}

/// Builds the second-level holder HTLC transaction for the HTLC with index `htlc_index`.
pub(crate) fn build_unsigned_htlc_tx(
&self, channel_parameters: &DirectedChannelTransactionParameters, htlc_index: usize,
preimage: &Option<PaymentPreimage>,
) -> Transaction {
let keys = &self.inner.keys;
pub(crate) fn build_unsigned_htlc_tx(&self, htlc_index: usize, preimage: &Option<PaymentPreimage>, revokeable_spk: ScriptBuf) -> Transaction {
let this_htlc = &self.inner.htlcs[htlc_index];
assert!(this_htlc.transaction_output_index.is_some());
// if we don't have preimage for an HTLC-Success, we can't generate an HTLC transaction.
Expand All @@ -1775,8 +1769,8 @@ impl<'a> TrustedCommitmentTransaction<'a> {
if this_htlc.offered && preimage.is_some() { unreachable!(); }

build_htlc_transaction(
&self.inner.built.txid, self.inner.feerate_per_kw, channel_parameters.contest_delay(), &this_htlc,
&self.channel_type_features, &keys.broadcaster_delayed_payment_key, &keys.revocation_key
&self.inner.built.txid, self.inner.feerate_per_kw, &this_htlc,
&self.channel_type_features, revokeable_spk,
)
}

Expand Down
12 changes: 6 additions & 6 deletions lightning/src/ln/channel.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5130,11 +5130,11 @@ impl<SP: Deref> Channel<SP> where

let mut nondust_htlc_sources = Vec::with_capacity(htlcs_cloned.len());
let mut htlcs_and_sigs = Vec::with_capacity(htlcs_cloned.len());
let revokeable_spk = self.context.holder_signer.as_ref().get_revokeable_spk(true, commitment_stats.tx.commitment_number(), &commitment_stats.tx.per_commitment_point(), &self.context.secp_ctx);
for (idx, (htlc, mut source_opt)) in htlcs_cloned.drain(..).enumerate() {
if let Some(_) = htlc.transaction_output_index {
let htlc_tx = chan_utils::build_htlc_transaction(&commitment_txid, commitment_stats.feerate_per_kw,
self.context.get_counterparty_selected_contest_delay().unwrap(), &htlc, &self.context.channel_type,
&keys.broadcaster_delayed_payment_key, &keys.revocation_key);
&htlc, &self.context.channel_type, revokeable_spk.clone());

let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, &self.context.channel_type, &keys);
let htlc_sighashtype = if self.context.channel_type.supports_anchors_zero_fee_htlc_tx() { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
Expand Down Expand Up @@ -8103,6 +8103,7 @@ impl<SP: Deref> Channel<SP> where
).map_err(|_| ChannelError::Ignore("Failed to get signatures for new commitment_signed".to_owned()))?;
signature = res.0;
htlc_signatures = res.1;
let revokeable_spk = ecdsa.get_revokeable_spk(false, commitment_stats.tx.commitment_number(), &commitment_stats.tx.per_commitment_point(), &self.context.secp_ctx);

log_trace!(logger, "Signed remote commitment tx {} (txid {}) with redeemscript {} -> {} in channel {}",
encode::serialize_hex(&commitment_stats.tx.trust().built_transaction().transaction),
Expand All @@ -8111,7 +8112,7 @@ impl<SP: Deref> Channel<SP> where

for (ref htlc_sig, ref htlc) in htlc_signatures.iter().zip(htlcs) {
log_trace!(logger, "Signed remote HTLC tx {} with redeemscript {} with pubkey {} -> {} in channel {}",
encode::serialize_hex(&chan_utils::build_htlc_transaction(&counterparty_commitment_txid, commitment_stats.feerate_per_kw, self.context.get_holder_selected_contest_delay(), htlc, &self.context.channel_type, &counterparty_keys.broadcaster_delayed_payment_key, &counterparty_keys.revocation_key)),
encode::serialize_hex(&chan_utils::build_htlc_transaction(&counterparty_commitment_txid, commitment_stats.feerate_per_kw, htlc, &self.context.channel_type, revokeable_spk.clone())),
encode::serialize_hex(&chan_utils::get_htlc_redeemscript(&htlc, &self.context.channel_type, &counterparty_keys)),
log_bytes!(counterparty_keys.broadcaster_htlc_key.to_public_key().serialize()),
log_bytes!(htlc_sig.serialize_compact()[..]), &self.context.channel_id());
Expand Down Expand Up @@ -11017,9 +11018,8 @@ mod tests {
let remote_signature = Signature::from_der(&<Vec<u8>>::from_hex($counterparty_htlc_sig_hex).unwrap()[..]).unwrap();

let ref htlc = htlcs[$htlc_idx];
let mut htlc_tx = chan_utils::build_htlc_transaction(&unsigned_tx.txid, chan.context.feerate_per_kw,
chan.context.get_counterparty_selected_contest_delay().unwrap(),
&htlc, $opt_anchors, &keys.broadcaster_delayed_payment_key, &keys.revocation_key);
let revokeable_spk = signer.get_revokeable_spk(true, holder_commitment_tx.commitment_number(), &holder_commitment_tx.per_commitment_point(), &secp_ctx);
let mut htlc_tx = chan_utils::build_htlc_transaction(&unsigned_tx.txid, chan.context.feerate_per_kw, &htlc, $opt_anchors, revokeable_spk);
let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, $opt_anchors, &keys);
let htlc_sighashtype = if $opt_anchors.supports_anchors_zero_fee_htlc_tx() { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
let htlc_sighash = Message::from_digest(sighash::SighashCache::new(&htlc_tx).p2wsh_signature_hash(0, &htlc_redeemscript, htlc.to_bitcoin_amount(), htlc_sighashtype).unwrap().as_raw_hash().to_byte_array());
Expand Down
12 changes: 10 additions & 2 deletions lightning/src/ln/monitor_tests.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,9 @@

//! Further functional tests which test blockchain reorganizations.
use crate::sign::{ecdsa::EcdsaChannelSigner, OutputSpender, SpendableOutputDescriptor};
use alloc::collections::BTreeMap;

use crate::sign::{ecdsa::EcdsaChannelSigner, ChannelSigner, OutputSpender, SpendableOutputDescriptor};
use crate::chain::channelmonitor::{ANTI_REORG_DELAY, LATENCY_GRACE_PERIOD_BLOCKS, Balance, BalanceSource, ChannelMonitorUpdateStep};
use crate::chain::transaction::OutPoint;
use crate::chain::chaininterface::{ConfirmationTarget, LowerBoundedFeeEstimator, compute_feerate_sat_per_1000_weight};
Expand Down Expand Up @@ -2901,6 +2903,7 @@ fn test_anchors_aggregated_revoked_htlc_tx() {
}],
};
let mut descriptors = Vec::with_capacity(4);
let mut revokeable_spks = BTreeMap::new();
for event in events {
// We don't use the `BumpTransactionEventHandler` here because it does not support
// creating one transaction from multiple `HTLCResolution` events.
Expand All @@ -2909,7 +2912,12 @@ fn test_anchors_aggregated_revoked_htlc_tx() {
for htlc_descriptor in &htlc_descriptors {
assert!(!htlc_descriptor.htlc.offered);
htlc_tx.input.push(htlc_descriptor.unsigned_tx_input());
htlc_tx.output.push(htlc_descriptor.tx_output(&secp));
let revokeable_spk = revokeable_spks.entry(htlc_descriptor.channel_derivation_parameters.keys_id)
.or_insert_with(|| {
let signer = htlc_descriptor.derive_channel_signer(&nodes[1].keys_manager);
signer.get_revokeable_spk(true, htlc_descriptor.per_commitment_number, &htlc_descriptor.per_commitment_point, &secp)
}).clone();
htlc_tx.output.push(htlc_descriptor.tx_output(revokeable_spk));
}
descriptors.append(&mut htlc_descriptors);
htlc_tx.lock_time = tx_lock_time;
Expand Down
36 changes: 10 additions & 26 deletions lightning/src/sign/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -631,30 +631,12 @@ impl HTLCDescriptor {

/// Returns the delayed output created as a result of spending the HTLC output in the commitment
/// transaction.
pub fn tx_output<C: secp256k1::Signing + secp256k1::Verification>(
&self, secp: &Secp256k1<C>,
) -> TxOut {
let channel_params =
self.channel_derivation_parameters.transaction_parameters.as_holder_broadcastable();
let broadcaster_keys = channel_params.broadcaster_pubkeys();
let counterparty_keys = channel_params.countersignatory_pubkeys();
let broadcaster_delayed_key = DelayedPaymentKey::from_basepoint(
secp,
&broadcaster_keys.delayed_payment_basepoint,
&self.per_commitment_point,
);
let counterparty_revocation_key = &RevocationKey::from_basepoint(
&secp,
&counterparty_keys.revocation_basepoint,
&self.per_commitment_point,
);
pub fn tx_output(&self, revokeable_spk: ScriptBuf) -> TxOut {
chan_utils::build_htlc_output(
self.feerate_per_kw,
channel_params.contest_delay(),
&self.htlc,
channel_params.channel_type_features(),
&broadcaster_delayed_key,
&counterparty_revocation_key,
&self.channel_derivation_parameters.transaction_parameters.channel_type_features,
revokeable_spk,
)
}

Expand Down Expand Up @@ -1468,19 +1450,21 @@ impl EcdsaChannelSigner for InMemorySigner {
let commitment_txid = built_tx.txid;

let mut htlc_sigs = Vec::with_capacity(commitment_tx.htlcs().len());
let revokeable_spk = self.get_revokeable_spk(
false,
trusted_tx.commitment_number(),
&trusted_tx.per_commitment_point(),
secp_ctx,
);
for htlc in commitment_tx.htlcs() {
let channel_parameters = self.get_channel_parameters().expect(MISSING_PARAMS_ERR);
let holder_selected_contest_delay =
self.holder_selected_contest_delay().expect(MISSING_PARAMS_ERR);
let chan_type = &channel_parameters.channel_type_features;
let htlc_tx = chan_utils::build_htlc_transaction(
&commitment_txid,
commitment_tx.feerate_per_kw(),
holder_selected_contest_delay,
htlc,
chan_type,
&keys.broadcaster_delayed_payment_key,
&keys.revocation_key,
revokeable_spk.clone(),
);
let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, chan_type, &keys);
let htlc_sighashtype = if chan_type.supports_anchors_zero_fee_htlc_tx() {
Expand Down
3 changes: 2 additions & 1 deletion lightning/src/util/test_channel_signer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -308,7 +308,8 @@ impl EcdsaChannelSigner for TestChannelSigner {
}
}
assert_eq!(htlc_tx.input[input], htlc_descriptor.unsigned_tx_input());
assert_eq!(htlc_tx.output[input], htlc_descriptor.tx_output(secp_ctx));
let revokeable_spk = self.get_revokeable_spk(true, htlc_descriptor.per_commitment_number, &htlc_descriptor.per_commitment_point, secp_ctx);
assert_eq!(htlc_tx.output[input], htlc_descriptor.tx_output(revokeable_spk));
{
let witness_script = htlc_descriptor.witness_script(secp_ctx);
let sighash_type = if self.channel_type_features().supports_anchors_zero_fee_htlc_tx() {
Expand Down

0 comments on commit d3181c5

Please sign in to comment.