diff --git a/lightning/src/chain/chainmonitor.rs b/lightning/src/chain/chainmonitor.rs index 2479e8f7e7b..3949810bf3e 100644 --- a/lightning/src/chain/chainmonitor.rs +++ b/lightning/src/chain/chainmonitor.rs @@ -707,6 +707,7 @@ impl even L::Target: Logger, P::Target: Persist, { + #[cfg(not(anchors))] /// Processes [`SpendableOutputs`] events produced from each [`ChannelMonitor`] upon maturity. /// /// An [`EventHandler`] may safely call back to the provider, though this shouldn't be needed in @@ -722,6 +723,29 @@ impl even handler.handle_event(&event); } } + #[cfg(anchors)] + /// Processes [`SpendableOutputs`] events produced from each [`ChannelMonitor`] upon maturity. + /// + /// For channels featuring anchor outputs, this method will also process [`BumpTransaction`] + /// events produced from each [`ChannelMonitor`] while there is a balance to claim onchain + /// within each channel. As the confirmation of a commitment transaction may be critical to the + /// safety of funds, this method must be invoked frequently, ideally once for every chain tip + /// update (block connected or disconnected). + /// + /// An [`EventHandler`] may safely call back to the provider, though this shouldn't be needed in + /// order to handle these events. + /// + /// [`SpendableOutputs`]: events::Event::SpendableOutputs + /// [`BumpTransaction`]: events::Event::BumpTransaction + fn process_pending_events(&self, handler: H) where H::Target: EventHandler { + let mut pending_events = Vec::new(); + for monitor_state in self.monitors.read().unwrap().values() { + pending_events.append(&mut monitor_state.monitor.get_and_clear_pending_events()); + } + for event in pending_events.drain(..) { + handler.handle_event(&event); + } + } } #[cfg(test)] diff --git a/lightning/src/chain/channelmonitor.rs b/lightning/src/chain/channelmonitor.rs index 2c3def4889e..7c6b48d144b 100644 --- a/lightning/src/chain/channelmonitor.rs +++ b/lightning/src/chain/channelmonitor.rs @@ -21,8 +21,7 @@ //! ChannelMonitors to get out of the HSM and onto monitoring devices. use bitcoin::blockdata::block::BlockHeader; -use bitcoin::blockdata::transaction::{TxOut,Transaction}; -use bitcoin::blockdata::transaction::OutPoint as BitcoinOutPoint; +use bitcoin::blockdata::transaction::{OutPoint as BitcoinOutPoint, TxOut, Transaction}; use bitcoin::blockdata::script::{Script, Builder}; use bitcoin::blockdata::opcodes; @@ -44,6 +43,8 @@ use chain::{BestBlock, WatchedOutput}; use chain::chaininterface::{BroadcasterInterface, FeeEstimator, LowerBoundedFeeEstimator}; use chain::transaction::{OutPoint, TransactionData}; use chain::keysinterface::{SpendableOutputDescriptor, StaticPaymentOutputDescriptor, DelayedPaymentOutputDescriptor, Sign, KeysInterface}; +#[cfg(anchors)] +use chain::onchaintx::ClaimEvent; use chain::onchaintx::OnchainTxHandler; use chain::package::{CounterpartyOfferedHTLCOutput, CounterpartyReceivedHTLCOutput, HolderFundingOutput, HolderHTLCOutput, PackageSolvingData, PackageTemplate, RevokedOutput, RevokedHTLCOutput}; use chain::Filter; @@ -51,6 +52,8 @@ use util::logger::Logger; use util::ser::{Readable, ReadableArgs, MaybeReadable, Writer, Writeable, U48, OptionDeserWrapper}; use util::byte_utils; use util::events::Event; +#[cfg(anchors)] +use util::events::{AnchorDescriptor, BumpTransactionEvent}; use prelude::*; use core::{cmp, mem}; @@ -263,6 +266,20 @@ impl_writeable_tlv_based!(HolderSignedTx, { (14, htlc_outputs, vec_type) }); +#[cfg(anchors)] +impl HolderSignedTx { + fn non_dust_htlcs(&self) -> Vec { + self.htlc_outputs.iter().filter_map(|(htlc, _, _)| { + if let Some(_) = htlc.transaction_output_index { + Some(htlc.clone()) + } else { + None + } + }) + .collect() + } +} + /// We use this to track static counterparty commitment transaction data and to generate any /// justice or 2nd-stage preimage/timeout transactions. #[derive(PartialEq, Eq)] @@ -1221,7 +1238,7 @@ impl ChannelMonitor { B::Target: BroadcasterInterface, L::Target: Logger, { - self.inner.lock().unwrap().broadcast_latest_holder_commitment_txn(broadcaster, logger) + self.inner.lock().unwrap().broadcast_latest_holder_commitment_txn(broadcaster, logger); } /// Updates a ChannelMonitor on the basis of some new information provided by the Channel @@ -2222,6 +2239,7 @@ impl ChannelMonitorImpl { panic!("Attempted to apply ChannelMonitorUpdates out of order, check the update_id before passing an update to update_monitor!"); } let mut ret = Ok(()); + let bounded_fee_estimator = LowerBoundedFeeEstimator::new(&*fee_estimator); for update in updates.updates.iter() { match update { ChannelMonitorUpdateStep::LatestHolderCommitmentTXInfo { commitment_tx, htlc_outputs } => { @@ -2239,7 +2257,6 @@ impl ChannelMonitorImpl { }, ChannelMonitorUpdateStep::PaymentPreimage { payment_preimage } => { log_trace!(logger, "Updating ChannelMonitor with payment preimage"); - let bounded_fee_estimator = LowerBoundedFeeEstimator::new(&*fee_estimator); self.provide_payment_preimage(&PaymentHash(Sha256::hash(&payment_preimage.0[..]).into_inner()), &payment_preimage, broadcaster, &bounded_fee_estimator, logger) }, ChannelMonitorUpdateStep::CommitmentSecret { idx, secret } => { @@ -2255,6 +2272,25 @@ impl ChannelMonitorImpl { self.lockdown_from_offchain = true; if *should_broadcast { self.broadcast_latest_holder_commitment_txn(broadcaster, logger); + // If the channel supports anchor outputs, we'll need to emit an external + // event to be consumed such that a child transaction is broadcast with a + // high enough feerate for the parent commitment transaction to confirm. + if self.onchain_tx_handler.opt_anchors() { + let funding_output = HolderFundingOutput::build( + self.funding_redeemscript.clone(), self.channel_value_satoshis, + self.onchain_tx_handler.opt_anchors(), + ); + let best_block_height = self.best_block.height(); + let commitment_package = PackageTemplate::build_package( + self.funding_info.0.txid.clone(), self.funding_info.0.index as u32, + PackageSolvingData::HolderFundingOutput(funding_output), + best_block_height, false, best_block_height, + ); + self.onchain_tx_handler.update_claims_view( + &[], vec![commitment_package], best_block_height, best_block_height, + broadcaster, &bounded_fee_estimator, logger, + ); + } } else if !self.holder_tx_signed { log_error!(logger, "WARNING: You have a potentially-unsafe holder commitment transaction available to broadcast"); log_error!(logger, " in channel monitor for channel {}!", log_bytes!(self.funding_info.0.to_channel_id())); @@ -2309,6 +2345,34 @@ impl ChannelMonitorImpl { pub fn get_and_clear_pending_events(&mut self) -> Vec { let mut ret = Vec::new(); mem::swap(&mut ret, &mut self.pending_events); + #[cfg(anchors)] + for claim_event in self.onchain_tx_handler.get_and_clear_pending_claim_events().drain(..) { + match claim_event { + ClaimEvent::BumpCommitment { + package_target_feerate_sat_per_1000_weight, commitment_tx, anchor_output_idx, + } => { + let commitment_txid = commitment_tx.txid(); + debug_assert_eq!(self.current_holder_commitment_tx.txid, commitment_txid); + let pending_htlcs = self.current_holder_commitment_tx.non_dust_htlcs(); + let commitment_tx_fee_satoshis = self.channel_value_satoshis - + commitment_tx.output.iter().fold(0u64, |sum, output| sum + output.value); + ret.push(Event::BumpTransaction(BumpTransactionEvent::ChannelClose { + package_target_feerate_sat_per_1000_weight, + commitment_tx, + commitment_tx_fee_satoshis, + anchor_descriptor: AnchorDescriptor { + channel_keys_id: self.channel_keys_id, + channel_value_satoshis: self.channel_value_satoshis, + outpoint: BitcoinOutPoint { + txid: commitment_txid, + vout: anchor_output_idx, + }, + }, + pending_htlcs, + })); + }, + } + } ret } @@ -2521,13 +2585,13 @@ impl ChannelMonitorImpl { CounterpartyOfferedHTLCOutput::build(*per_commitment_point, self.counterparty_commitment_params.counterparty_delayed_payment_base_key, self.counterparty_commitment_params.counterparty_htlc_base_key, - preimage.unwrap(), htlc.clone())) + preimage.unwrap(), htlc.clone(), self.onchain_tx_handler.opt_anchors())) } else { PackageSolvingData::CounterpartyReceivedHTLCOutput( CounterpartyReceivedHTLCOutput::build(*per_commitment_point, self.counterparty_commitment_params.counterparty_delayed_payment_base_key, self.counterparty_commitment_params.counterparty_htlc_base_key, - htlc.clone())) + htlc.clone(), self.onchain_tx_handler.opt_anchors())) }; let aggregation = if !htlc.offered { false } else { true }; let counterparty_package = PackageTemplate::build_package(commitment_txid, transaction_output_index, counterparty_htlc_outp, htlc.cltv_expiry,aggregation, 0); @@ -2884,21 +2948,26 @@ impl ChannelMonitorImpl { let should_broadcast = self.should_broadcast_holder_commitment_txn(logger); if should_broadcast { - let funding_outp = HolderFundingOutput::build(self.funding_redeemscript.clone()); + let funding_outp = HolderFundingOutput::build(self.funding_redeemscript.clone(), self.channel_value_satoshis, self.onchain_tx_handler.opt_anchors()); let commitment_package = PackageTemplate::build_package(self.funding_info.0.txid.clone(), self.funding_info.0.index as u32, PackageSolvingData::HolderFundingOutput(funding_outp), self.best_block.height(), false, self.best_block.height()); claimable_outpoints.push(commitment_package); self.pending_monitor_events.push(MonitorEvent::CommitmentTxConfirmed(self.funding_info.0)); let commitment_tx = self.onchain_tx_handler.get_fully_signed_holder_tx(&self.funding_redeemscript); self.holder_tx_signed = true; - // Because we're broadcasting a commitment transaction, we should construct the package - // assuming it gets confirmed in the next block. Sadly, we have code which considers - // "not yet confirmed" things as discardable, so we cannot do that here. - let (mut new_outpoints, _) = self.get_broadcasted_holder_claims(&self.current_holder_commitment_tx, self.best_block.height()); - let new_outputs = self.get_broadcasted_holder_watch_outputs(&self.current_holder_commitment_tx, &commitment_tx); - if !new_outputs.is_empty() { - watch_outputs.push((self.current_holder_commitment_tx.txid.clone(), new_outputs)); + // We can't broadcast our HTLC transactions while the commitment transaction is + // unconfirmed. We'll delay doing so until we detect the confirmed commitment in + // `transactions_confirmed`. + if !self.onchain_tx_handler.opt_anchors() { + // Because we're broadcasting a commitment transaction, we should construct the package + // assuming it gets confirmed in the next block. Sadly, we have code which considers + // "not yet confirmed" things as discardable, so we cannot do that here. + let (mut new_outpoints, _) = self.get_broadcasted_holder_claims(&self.current_holder_commitment_tx, self.best_block.height()); + let new_outputs = self.get_broadcasted_holder_watch_outputs(&self.current_holder_commitment_tx, &commitment_tx); + if !new_outputs.is_empty() { + watch_outputs.push((self.current_holder_commitment_tx.txid.clone(), new_outputs)); + } + claimable_outpoints.append(&mut new_outpoints); } - claimable_outpoints.append(&mut new_outpoints); } // Find which on-chain events have reached their confirmation threshold. diff --git a/lightning/src/chain/keysinterface.rs b/lightning/src/chain/keysinterface.rs index a89cc602bc2..1c64a8bccdf 100644 --- a/lightning/src/chain/keysinterface.rs +++ b/lightning/src/chain/keysinterface.rs @@ -36,6 +36,7 @@ use util::crypto::{hkdf_extract_expand_twice, sign}; use util::ser::{Writeable, Writer, Readable, ReadableArgs}; use chain::transaction::OutPoint; +use ln::channel::ANCHOR_OUTPUT_VALUE_SATOSHI; use ln::{chan_utils, PaymentPreimage}; use ln::chan_utils::{HTLCOutputInCommitment, make_funding_redeemscript, ChannelPublicKeys, HolderCommitmentTransaction, ChannelTransactionParameters, CommitmentTransaction, ClosingTransaction}; use ln::msgs::UnsignedChannelAnnouncement; @@ -348,6 +349,12 @@ pub trait BaseSign { /// chosen to forgo their output as dust. fn sign_closing_transaction(&self, closing_tx: &ClosingTransaction, secp_ctx: &Secp256k1) -> Result; + /// Computes the signature for a commitment transaction's anchor output used as an + /// input within `anchor_tx`, which spends the commitment transaction, at index `input`. + fn sign_holder_anchor_input( + &self, anchor_tx: &mut Transaction, input: usize, secp_ctx: &Secp256k1, + ) -> Result; + /// Signs a channel announcement message with our funding key and our node secret key (aka /// node_id or network_key), proving it comes from one of the channel participants. /// @@ -645,6 +652,7 @@ impl InMemorySigner { witness.push(witness_script.clone().into_bytes()); Ok(witness) } + } impl BaseSign for InMemorySigner { @@ -762,6 +770,16 @@ impl BaseSign for InMemorySigner { Ok(closing_tx.trust().sign(&self.funding_key, &channel_funding_redeemscript, self.channel_value_satoshis, secp_ctx)) } + fn sign_holder_anchor_input( + &self, anchor_tx: &mut Transaction, input: usize, secp_ctx: &Secp256k1, + ) -> Result { + let witness_script = chan_utils::get_anchor_redeemscript(&self.holder_channel_pubkeys.funding_pubkey); + let sighash = sighash::SighashCache::new(&*anchor_tx).segwit_signature_hash( + input, &witness_script, ANCHOR_OUTPUT_VALUE_SATOSHI, EcdsaSighashType::All, + ).unwrap(); + Ok(sign(secp_ctx, &hash_to_message!(&sighash[..]), &self.funding_key)) + } + fn sign_channel_announcement(&self, msg: &UnsignedChannelAnnouncement, secp_ctx: &Secp256k1) -> Result<(Signature, Signature), ()> { let msghash = hash_to_message!(&Sha256dHash::hash(&msg.encode()[..])[..]); diff --git a/lightning/src/chain/onchaintx.rs b/lightning/src/chain/onchaintx.rs index f14c983b6d5..875f4d896d5 100644 --- a/lightning/src/chain/onchaintx.rs +++ b/lightning/src/chain/onchaintx.rs @@ -23,10 +23,16 @@ use bitcoin::secp256k1; use ln::msgs::DecodeError; use ln::PaymentPreimage; +#[cfg(anchors)] +use ln::chan_utils; use ln::chan_utils::{ChannelTransactionParameters, HolderCommitmentTransaction}; +#[cfg(anchors)] +use chain::chaininterface::ConfirmationTarget; use chain::chaininterface::{FeeEstimator, BroadcasterInterface, LowerBoundedFeeEstimator}; use chain::channelmonitor::{ANTI_REORG_DELAY, CLTV_SHARED_CLAIM_BUFFER}; use chain::keysinterface::{Sign, KeysInterface}; +#[cfg(anchors)] +use chain::package::PackageSolvingData; use chain::package::PackageTemplate; use util::logger::Logger; use util::ser::{Readable, ReadableArgs, MaybeReadable, Writer, Writeable, VecWriter}; @@ -38,6 +44,8 @@ use alloc::collections::BTreeMap; use core::cmp; use core::ops::Deref; use core::mem::replace; +#[cfg(anchors)] +use core::mem::swap; use bitcoin::hashes::Hash; const MAX_ALLOC_SIZE: usize = 64*1024; @@ -162,6 +170,29 @@ impl Writeable for Option>> { } } +// Represents the different types of claims for which events are yielded externally to satisfy said +// claims. +#[cfg(anchors)] +pub(crate) enum ClaimEvent { + /// Event yielded to signal that the commitment transaction fee must be bumped to claim any + /// encumbered funds and proceed to HTLC resolution, if any HTLCs exist. + BumpCommitment { + package_target_feerate_sat_per_1000_weight: u32, + commitment_tx: Transaction, + anchor_output_idx: u32, + }, +} + +/// Represents the different ways an output can be claimed (i.e., spent to an address under our +/// control) onchain. +pub(crate) enum OnchainClaim { + /// A finalized transaction pending confirmation spending the output to claim. + Tx(Transaction), + #[cfg(anchors)] + /// An event yielded externally to signal additional inputs must be added to a transaction + /// pending confirmation spending the output to claim. + Event(ClaimEvent), +} /// OnchainTxHandler receives claiming requests, aggregates them if it's sound, broadcast and /// do RBF bumping if possible. @@ -193,6 +224,8 @@ pub struct OnchainTxHandler { pub(crate) pending_claim_requests: HashMap, #[cfg(not(test))] pending_claim_requests: HashMap, + #[cfg(anchors)] + pending_claim_events: HashMap, // Used to link outpoints claimed in a connected block to a pending claim request. // Key is outpoint than monitor parsing has detected we have keys/scripts to claim @@ -342,6 +375,8 @@ impl<'a, K: KeysInterface> ReadableArgs<&'a K> for OnchainTxHandler { locktimed_packages, pending_claim_requests, onchain_events_awaiting_threshold_conf, + #[cfg(anchors)] + pending_claim_events: HashMap::new(), secp_ctx, }) } @@ -361,6 +396,8 @@ impl OnchainTxHandler { claimable_outpoints: HashMap::new(), locktimed_packages: BTreeMap::new(), onchain_events_awaiting_threshold_conf: Vec::new(), + #[cfg(anchors)] + pending_claim_events: HashMap::new(), secp_ctx, } @@ -374,11 +411,22 @@ impl OnchainTxHandler { self.holder_commitment.to_broadcaster_value_sat() } - /// Lightning security model (i.e being able to redeem/timeout HTLC or penalize coutnerparty onchain) lays on the assumption of claim transactions getting confirmed before timelock expiration - /// (CSV or CLTV following cases). In case of high-fee spikes, claim tx may stuck in the mempool, so you need to bump its feerate quickly using Replace-By-Fee or Child-Pay-For-Parent. - /// Panics if there are signing errors, because signing operations in reaction to on-chain events - /// are not expected to fail, and if they do, we may lose funds. - fn generate_claim_tx(&mut self, cur_height: u32, cached_request: &PackageTemplate, fee_estimator: &LowerBoundedFeeEstimator, logger: &L) -> Option<(Option, u64, Transaction)> + #[cfg(anchors)] + pub(crate) fn get_and_clear_pending_claim_events(&mut self) -> Vec { + let mut ret = HashMap::new(); + swap(&mut ret, &mut self.pending_claim_events); + ret.into_iter().map(|(_, event)| event).collect::>() + } + + /// Lightning security model (i.e being able to redeem/timeout HTLC or penalize counterparty + /// onchain) lays on the assumption of claim transactions getting confirmed before timelock + /// expiration (CSV or CLTV following cases). In case of high-fee spikes, claim tx may get stuck + /// in the mempool, so you need to bump its feerate quickly using Replace-By-Fee or + /// Child-Pay-For-Parent. + /// + /// Panics if there are signing errors, because signing operations in reaction to on-chain + /// events are not expected to fail, and if they do, we may lose funds. + fn generate_claim(&mut self, cur_height: u32, cached_request: &PackageTemplate, fee_estimator: &LowerBoundedFeeEstimator, logger: &L) -> Option<(Option, u64, OnchainClaim)> where F::Target: FeeEstimator, L::Target: Logger, { @@ -388,23 +436,71 @@ impl OnchainTxHandler { // didn't receive confirmation of it before, or not enough reorg-safe depth on top of it). let new_timer = Some(cached_request.get_height_timer(cur_height)); if cached_request.is_malleable() { - let predicted_weight = cached_request.package_weight(&self.destination_script, self.channel_transaction_parameters.opt_anchors.is_some()); + let predicted_weight = cached_request.package_weight(&self.destination_script); if let Some((output_value, new_feerate)) = cached_request.compute_package_output(predicted_weight, self.destination_script.dust_value().to_sat(), fee_estimator, logger) { assert!(new_feerate != 0); - let transaction = cached_request.finalize_package(self, output_value, self.destination_script.clone(), logger).unwrap(); + let transaction = cached_request.finalize_malleable_package(self, output_value, self.destination_script.clone(), logger).unwrap(); log_trace!(logger, "...with timer {} and feerate {}", new_timer.unwrap(), new_feerate); assert!(predicted_weight >= transaction.weight()); - return Some((new_timer, new_feerate, transaction)) + return Some((new_timer, new_feerate, OnchainClaim::Tx(transaction))) } } else { - // Note: Currently, amounts of holder outputs spending witnesses aren't used - // as we can't malleate spending package to increase their feerate. This - // should change with the remaining anchor output patchset. - if let Some(transaction) = cached_request.finalize_package(self, 0, self.destination_script.clone(), logger) { - return Some((None, 0, transaction)); + // Untractable packages cannot have their fees bumped through Replace-By-Fee. Some + // packages may support fee bumping through Child-Pays-For-Parent, indicated by those + // which require external funding. + #[cfg(not(anchors))] + let inputs = cached_request.inputs(); + #[cfg(anchors)] + let mut inputs = cached_request.inputs(); + debug_assert_eq!(inputs.len(), 1); + let tx = match cached_request.finalize_untractable_package(self, logger) { + Some(tx) => tx, + None => return None, + }; + if !cached_request.requires_external_funding() { + return Some((None, 0, OnchainClaim::Tx(tx))); } + #[cfg(anchors)] + return inputs.find_map(|input| match input { + // Commitment inputs with anchors support are the only untractable inputs supported + // thus far that require external funding. + PackageSolvingData::HolderFundingOutput(..) => { + debug_assert_eq!(tx.txid(), self.holder_commitment.trust().txid(), + "Holder commitment transaction mismatch"); + // We'll locate an anchor output we can spend within the commitment transaction. + let funding_pubkey = &self.channel_transaction_parameters.holder_pubkeys.funding_pubkey; + match chan_utils::get_anchor_output(&tx, funding_pubkey) { + // An anchor output was found, so we should yield a funding event externally. + Some((idx, _)) => { + // TODO: Use a lower confirmation target when both our and the + // counterparty's latest commitment don't have any HTLCs present. + let conf_target = ConfirmationTarget::HighPriority; + let package_target_feerate_sat_per_1000_weight = cached_request + .compute_package_feerate(fee_estimator, conf_target); + Some(( + new_timer, + package_target_feerate_sat_per_1000_weight as u64, + OnchainClaim::Event(ClaimEvent::BumpCommitment { + package_target_feerate_sat_per_1000_weight, + commitment_tx: tx.clone(), + anchor_output_idx: idx, + }), + )) + }, + // An anchor output was not found. There's nothing we can do other than + // attempt to broadcast the transaction with its current fee rate and hope + // it confirms. This is essentially the same behavior as a commitment + // transaction without anchor outputs. + None => Some((None, 0, OnchainClaim::Tx(tx.clone()))), + } + }, + _ => { + debug_assert!(false, "Only HolderFundingOutput inputs should be untractable and require external funding"); + None + }, + }); } None } @@ -475,17 +571,30 @@ impl OnchainTxHandler { // Generate claim transactions and track them to bump if necessary at // height timer expiration (i.e in how many blocks we're going to take action). for mut req in preprocessed_requests { - if let Some((new_timer, new_feerate, tx)) = self.generate_claim_tx(cur_height, &req, &*fee_estimator, &*logger) { + if let Some((new_timer, new_feerate, claim)) = self.generate_claim(cur_height, &req, &*fee_estimator, &*logger) { req.set_timer(new_timer); req.set_feerate(new_feerate); - let txid = tx.txid(); + let txid = match claim { + OnchainClaim::Tx(tx) => { + log_info!(logger, "Broadcasting onchain {}", log_tx!(tx)); + broadcaster.broadcast_transaction(&tx); + tx.txid() + }, + #[cfg(anchors)] + OnchainClaim::Event(claim_event) => { + log_info!(logger, "Yielding onchain event to spend inputs {:?}", req.outpoints()); + let txid = match claim_event { + ClaimEvent::BumpCommitment { ref commitment_tx, .. } => commitment_tx.txid(), + }; + self.pending_claim_events.insert(txid, claim_event); + txid + }, + }; for k in req.outpoints() { log_info!(logger, "Registering claiming request for {}:{}", k.txid, k.vout); self.claimable_outpoints.insert(k.clone(), (txid, conf_height)); } self.pending_claim_requests.insert(txid, req); - log_info!(logger, "Broadcasting onchain {}", log_tx!(tx)); - broadcaster.broadcast_transaction(&tx); } } @@ -577,6 +686,8 @@ impl OnchainTxHandler { for outpoint in request.outpoints() { log_debug!(logger, "Removing claim tracking for {} due to maturation of claim tx {}.", outpoint, claim_request); self.claimable_outpoints.remove(&outpoint); + #[cfg(anchors)] + self.pending_claim_events.remove(&claim_request); } } }, @@ -603,9 +714,18 @@ impl OnchainTxHandler { // Build, bump and rebroadcast tx accordingly log_trace!(logger, "Bumping {} candidates", bump_candidates.len()); for (first_claim_txid, request) in bump_candidates.iter() { - if let Some((new_timer, new_feerate, bump_tx)) = self.generate_claim_tx(cur_height, &request, &*fee_estimator, &*logger) { - log_info!(logger, "Broadcasting RBF-bumped onchain {}", log_tx!(bump_tx)); - broadcaster.broadcast_transaction(&bump_tx); + if let Some((new_timer, new_feerate, bump_claim)) = self.generate_claim(cur_height, &request, &*fee_estimator, &*logger) { + match bump_claim { + OnchainClaim::Tx(bump_tx) => { + log_info!(logger, "Broadcasting RBF-bumped onchain {}", log_tx!(bump_tx)); + broadcaster.broadcast_transaction(&bump_tx); + }, + #[cfg(anchors)] + OnchainClaim::Event(claim_event) => { + log_info!(logger, "Yielding RBF-bumped onchain event to spend inputs {:?}", request.outpoints()); + self.pending_claim_events.insert(*first_claim_txid, claim_event); + }, + } if let Some(request) = self.pending_claim_requests.get_mut(first_claim_txid) { request.set_timer(new_timer); request.set_feerate(new_feerate); @@ -667,12 +787,21 @@ impl OnchainTxHandler { self.onchain_events_awaiting_threshold_conf.push(entry); } } - for (_, request) in bump_candidates.iter_mut() { - if let Some((new_timer, new_feerate, bump_tx)) = self.generate_claim_tx(height, &request, fee_estimator, &&*logger) { + for (_first_claim_txid_height, request) in bump_candidates.iter_mut() { + if let Some((new_timer, new_feerate, bump_claim)) = self.generate_claim(height, &request, fee_estimator, &&*logger) { request.set_timer(new_timer); request.set_feerate(new_feerate); - log_info!(logger, "Broadcasting onchain {}", log_tx!(bump_tx)); - broadcaster.broadcast_transaction(&bump_tx); + match bump_claim { + OnchainClaim::Tx(bump_tx) => { + log_info!(logger, "Broadcasting onchain {}", log_tx!(bump_tx)); + broadcaster.broadcast_transaction(&bump_tx); + }, + #[cfg(anchors)] + OnchainClaim::Event(claim_event) => { + log_info!(logger, "Yielding onchain event after reorg to spend inputs {:?}", request.outpoints()); + self.pending_claim_events.insert(_first_claim_txid_height.0, claim_event); + }, + } } } for (ancestor_claim_txid, request) in bump_candidates.drain() { diff --git a/lightning/src/chain/package.rs b/lightning/src/chain/package.rs index bc5e997b1d4..5aa55fb1975 100644 --- a/lightning/src/chain/package.rs +++ b/lightning/src/chain/package.rs @@ -34,6 +34,8 @@ use util::ser::{Readable, Writer, Writeable}; use io; use prelude::*; use core::cmp; +#[cfg(anchors)] +use core::convert::TryInto; use core::mem; use core::ops::Deref; use bitcoin::{PackedLockTime, Sequence, Witness}; @@ -177,23 +179,30 @@ pub(crate) struct CounterpartyOfferedHTLCOutput { counterparty_delayed_payment_base_key: PublicKey, counterparty_htlc_base_key: PublicKey, preimage: PaymentPreimage, - htlc: HTLCOutputInCommitment + htlc: HTLCOutputInCommitment, + opt_anchors: Option<()>, } impl CounterpartyOfferedHTLCOutput { - pub(crate) fn build(per_commitment_point: PublicKey, counterparty_delayed_payment_base_key: PublicKey, counterparty_htlc_base_key: PublicKey, preimage: PaymentPreimage, htlc: HTLCOutputInCommitment) -> Self { + pub(crate) fn build(per_commitment_point: PublicKey, counterparty_delayed_payment_base_key: PublicKey, counterparty_htlc_base_key: PublicKey, preimage: PaymentPreimage, htlc: HTLCOutputInCommitment, opt_anchors: bool) -> Self { CounterpartyOfferedHTLCOutput { per_commitment_point, counterparty_delayed_payment_base_key, counterparty_htlc_base_key, preimage, - htlc + htlc, + opt_anchors: if opt_anchors { Some(()) } else { None }, } } + + fn opt_anchors(&self) -> bool { + self.opt_anchors.is_some() + } } impl_writeable_tlv_based!(CounterpartyOfferedHTLCOutput, { (0, per_commitment_point, required), + (1, opt_anchors, option), (2, counterparty_delayed_payment_base_key, required), (4, counterparty_htlc_base_key, required), (6, preimage, required), @@ -209,22 +218,29 @@ pub(crate) struct CounterpartyReceivedHTLCOutput { per_commitment_point: PublicKey, counterparty_delayed_payment_base_key: PublicKey, counterparty_htlc_base_key: PublicKey, - htlc: HTLCOutputInCommitment + htlc: HTLCOutputInCommitment, + opt_anchors: Option<()>, } impl CounterpartyReceivedHTLCOutput { - pub(crate) fn build(per_commitment_point: PublicKey, counterparty_delayed_payment_base_key: PublicKey, counterparty_htlc_base_key: PublicKey, htlc: HTLCOutputInCommitment) -> Self { + pub(crate) fn build(per_commitment_point: PublicKey, counterparty_delayed_payment_base_key: PublicKey, counterparty_htlc_base_key: PublicKey, htlc: HTLCOutputInCommitment, opt_anchors: bool) -> Self { CounterpartyReceivedHTLCOutput { per_commitment_point, counterparty_delayed_payment_base_key, counterparty_htlc_base_key, - htlc + htlc, + opt_anchors: if opt_anchors { Some(()) } else { None }, } } + + fn opt_anchors(&self) -> bool { + self.opt_anchors.is_some() + } } impl_writeable_tlv_based!(CounterpartyReceivedHTLCOutput, { (0, per_commitment_point, required), + (1, opt_anchors, option), (2, counterparty_delayed_payment_base_key, required), (4, counterparty_htlc_base_key, required), (6, htlc, required), @@ -272,18 +288,29 @@ impl_writeable_tlv_based!(HolderHTLCOutput, { #[derive(Clone, PartialEq, Eq)] pub(crate) struct HolderFundingOutput { funding_redeemscript: Script, + funding_amount: Option, + opt_anchors: Option<()>, } + impl HolderFundingOutput { - pub(crate) fn build(funding_redeemscript: Script) -> Self { + pub(crate) fn build(funding_redeemscript: Script, funding_amount: u64, opt_anchors: bool) -> Self { HolderFundingOutput { funding_redeemscript, + funding_amount: Some(funding_amount), + opt_anchors: if opt_anchors { Some(()) } else { None }, } } + + fn opt_anchors(&self) -> bool { + self.opt_anchors.is_some() + } } impl_writeable_tlv_based!(HolderFundingOutput, { (0, funding_redeemscript, required), + (1, opt_anchors, option), + (3, funding_amount, option), }); /// A wrapper encapsulating all in-protocol differing outputs types. @@ -303,24 +330,27 @@ pub(crate) enum PackageSolvingData { impl PackageSolvingData { fn amount(&self) -> u64 { let amt = match self { - PackageSolvingData::RevokedOutput(ref outp) => { outp.amount }, - PackageSolvingData::RevokedHTLCOutput(ref outp) => { outp.amount }, - PackageSolvingData::CounterpartyOfferedHTLCOutput(ref outp) => { outp.htlc.amount_msat / 1000 }, - PackageSolvingData::CounterpartyReceivedHTLCOutput(ref outp) => { outp.htlc.amount_msat / 1000 }, + PackageSolvingData::RevokedOutput(ref outp) => outp.amount, + PackageSolvingData::RevokedHTLCOutput(ref outp) => outp.amount, + PackageSolvingData::CounterpartyOfferedHTLCOutput(ref outp) => outp.htlc.amount_msat / 1000, + PackageSolvingData::CounterpartyReceivedHTLCOutput(ref outp) => outp.htlc.amount_msat / 1000, // Note: Currently, amounts of holder outputs spending witnesses aren't used // as we can't malleate spending package to increase their feerate. This // should change with the remaining anchor output patchset. - PackageSolvingData::HolderHTLCOutput(..) => { unreachable!() }, - PackageSolvingData::HolderFundingOutput(..) => { unreachable!() }, + PackageSolvingData::HolderHTLCOutput(..) => unreachable!(), + PackageSolvingData::HolderFundingOutput(ref outp) => { + debug_assert!(outp.opt_anchors()); + outp.funding_amount.unwrap() + } }; amt } - fn weight(&self, opt_anchors: bool) -> usize { + fn weight(&self) -> usize { let weight = match self { PackageSolvingData::RevokedOutput(ref outp) => { outp.weight as usize }, PackageSolvingData::RevokedHTLCOutput(ref outp) => { outp.weight as usize }, - PackageSolvingData::CounterpartyOfferedHTLCOutput(..) => { weight_offered_htlc(opt_anchors) as usize }, - PackageSolvingData::CounterpartyReceivedHTLCOutput(..) => { weight_received_htlc(opt_anchors) as usize }, + PackageSolvingData::CounterpartyOfferedHTLCOutput(ref outp) => { weight_offered_htlc(outp.opt_anchors()) as usize }, + PackageSolvingData::CounterpartyReceivedHTLCOutput(ref outp) => { weight_received_htlc(outp.opt_anchors()) as usize }, // Note: Currently, weights of holder outputs spending witnesses aren't used // as we can't malleate spending package to increase their feerate. This // should change with the remaining anchor output patchset. @@ -520,6 +550,9 @@ impl PackageTemplate { pub(crate) fn outpoints(&self) -> Vec<&BitcoinOutPoint> { self.inputs.iter().map(|(o, _)| o).collect() } + pub(crate) fn inputs(&self) -> impl ExactSizeIterator { + self.inputs.iter().map(|(_, i)| i) + } pub(crate) fn split_package(&mut self, split_outp: &BitcoinOutPoint) -> Option { match self.malleability { PackageMalleability::Malleable => { @@ -583,7 +616,7 @@ impl PackageTemplate { } /// Gets the amount of all outptus being spent by this package, only valid for malleable /// packages. - fn package_amount(&self) -> u64 { + pub(crate) fn package_amount(&self) -> u64 { let mut amounts = 0; for (_, outp) in self.inputs.iter() { amounts += outp.amount(); @@ -594,13 +627,13 @@ impl PackageTemplate { self.inputs.iter().map(|(_, outp)| outp.absolute_tx_timelock(self.height_original)) .max().expect("There must always be at least one output to spend in a PackageTemplate") } - pub(crate) fn package_weight(&self, destination_script: &Script, opt_anchors: bool) -> usize { + pub(crate) fn package_weight(&self, destination_script: &Script) -> usize { let mut inputs_weight = 0; let mut witnesses_weight = 2; // count segwit flags for (_, outp) in self.inputs.iter() { // previous_out_point: 36 bytes ; var_int: 1 byte ; sequence: 4 bytes inputs_weight += 41 * WITNESS_SCALE_FACTOR; - witnesses_weight += outp.weight(opt_anchors); + witnesses_weight += outp.weight(); } // version: 4 bytes ; count_tx_in: 1 byte ; count_tx_out: 1 byte ; lock_time: 4 bytes let transaction_weight = 10 * WITNESS_SCALE_FACTOR; @@ -608,47 +641,46 @@ impl PackageTemplate { let output_weight = (8 + 1 + destination_script.len()) * WITNESS_SCALE_FACTOR; inputs_weight + witnesses_weight + transaction_weight + output_weight } - pub(crate) fn finalize_package(&self, onchain_handler: &mut OnchainTxHandler, value: u64, destination_script: Script, logger: &L) -> Option - where L::Target: Logger, - { - match self.malleability { - PackageMalleability::Malleable => { - let mut bumped_tx = Transaction { - version: 2, - lock_time: PackedLockTime::ZERO, - input: vec![], - output: vec![TxOut { - script_pubkey: destination_script, - value, - }], - }; - for (outpoint, _) in self.inputs.iter() { - bumped_tx.input.push(TxIn { - previous_output: *outpoint, - script_sig: Script::new(), - sequence: Sequence::ENABLE_RBF_NO_LOCKTIME, - witness: Witness::new(), - }); - } - for (i, (outpoint, out)) in self.inputs.iter().enumerate() { - log_debug!(logger, "Adding claiming input for outpoint {}:{}", outpoint.txid, outpoint.vout); - if !out.finalize_input(&mut bumped_tx, i, onchain_handler) { return None; } - } - log_debug!(logger, "Finalized transaction {} ready to broadcast", bumped_tx.txid()); - return Some(bumped_tx); - }, - PackageMalleability::Untractable => { - debug_assert_eq!(value, 0, "value is ignored for non-malleable packages, should be zero to ensure callsites are correct"); - if let Some((outpoint, outp)) = self.inputs.first() { - if let Some(final_tx) = outp.get_finalized_tx(outpoint, onchain_handler) { - log_debug!(logger, "Adding claiming input for outpoint {}:{}", outpoint.txid, outpoint.vout); - log_debug!(logger, "Finalized transaction {} ready to broadcast", final_tx.txid()); - return Some(final_tx); - } - return None; - } else { panic!("API Error: Package must not be inputs empty"); } - }, + pub(crate) fn finalize_malleable_package( + &self, onchain_handler: &mut OnchainTxHandler, value: u64, destination_script: Script, logger: &L + ) -> Option where L::Target: Logger { + debug_assert!(self.is_malleable()); + let mut bumped_tx = Transaction { + version: 2, + lock_time: PackedLockTime::ZERO, + input: vec![], + output: vec![TxOut { + script_pubkey: destination_script, + value, + }], + }; + for (outpoint, _) in self.inputs.iter() { + bumped_tx.input.push(TxIn { + previous_output: *outpoint, + script_sig: Script::new(), + sequence: Sequence::ENABLE_RBF_NO_LOCKTIME, + witness: Witness::new(), + }); + } + for (i, (outpoint, out)) in self.inputs.iter().enumerate() { + log_debug!(logger, "Adding claiming input for outpoint {}:{}", outpoint.txid, outpoint.vout); + if !out.finalize_input(&mut bumped_tx, i, onchain_handler) { return None; } } + log_debug!(logger, "Finalized transaction {} ready to broadcast", bumped_tx.txid()); + Some(bumped_tx) + } + pub(crate) fn finalize_untractable_package( + &self, onchain_handler: &mut OnchainTxHandler, logger: &L, + ) -> Option where L::Target: Logger { + debug_assert!(!self.is_malleable()); + if let Some((outpoint, outp)) = self.inputs.first() { + if let Some(final_tx) = outp.get_finalized_tx(outpoint, onchain_handler) { + log_debug!(logger, "Adding claiming input for outpoint {}:{}", outpoint.txid, outpoint.vout); + log_debug!(logger, "Finalized transaction {} ready to broadcast", final_tx.txid()); + return Some(final_tx); + } + return None; + } else { panic!("API Error: Package must not be inputs empty"); } } /// In LN, output claimed are time-sensitive, which means we have to spend them before reaching some timelock expiration. At in-channel /// output detection, we generate a first version of a claim tx and associate to it a height timer. A height timer is an absolute block @@ -686,14 +718,45 @@ impl PackageTemplate { } None } + + #[cfg(anchors)] + /// Computes a feerate based on the given confirmation target. If a previous feerate was used, + /// and the new feerate is below it, we'll use a 25% increase of the previous feerate instead of + /// the new one. + pub(crate) fn compute_package_feerate( + &self, fee_estimator: &LowerBoundedFeeEstimator, conf_target: ConfirmationTarget, + ) -> u32 where F::Target: FeeEstimator { + let feerate_estimate = fee_estimator.bounded_sat_per_1000_weight(conf_target); + if self.feerate_previous != 0 { + // If old feerate inferior to actual one given back by Fee Estimator, use it to compute new fee... + if feerate_estimate as u64 > self.feerate_previous { + feerate_estimate + } else { + // ...else just increase the previous feerate by 25% (because that's a nice number) + (self.feerate_previous + (self.feerate_previous / 4)).try_into().unwrap_or(u32::max_value()) + } + } else { + feerate_estimate + } + } + + /// Determines whether a package contains an input which must have additional external inputs + /// attached to help the spending transaction reach confirmation. + pub(crate) fn requires_external_funding(&self) -> bool { + self.inputs.iter().find(|input| match input.1 { + PackageSolvingData::HolderFundingOutput(ref outp) => outp.opt_anchors(), + _ => false, + }).is_some() + } + pub (crate) fn build_package(txid: Txid, vout: u32, input_solving_data: PackageSolvingData, soonest_conf_deadline: u32, aggregable: bool, height_original: u32) -> Self { let malleability = match input_solving_data { - PackageSolvingData::RevokedOutput(..) => { PackageMalleability::Malleable }, - PackageSolvingData::RevokedHTLCOutput(..) => { PackageMalleability::Malleable }, - PackageSolvingData::CounterpartyOfferedHTLCOutput(..) => { PackageMalleability::Malleable }, - PackageSolvingData::CounterpartyReceivedHTLCOutput(..) => { PackageMalleability::Malleable }, - PackageSolvingData::HolderHTLCOutput(..) => { PackageMalleability::Untractable }, - PackageSolvingData::HolderFundingOutput(..) => { PackageMalleability::Untractable }, + PackageSolvingData::RevokedOutput(..) => PackageMalleability::Malleable, + PackageSolvingData::RevokedHTLCOutput(..) => PackageMalleability::Malleable, + PackageSolvingData::CounterpartyOfferedHTLCOutput(..) => PackageMalleability::Malleable, + PackageSolvingData::CounterpartyReceivedHTLCOutput(..) => PackageMalleability::Malleable, + PackageSolvingData::HolderHTLCOutput(..) => PackageMalleability::Untractable, + PackageSolvingData::HolderFundingOutput(..) => PackageMalleability::Untractable, }; let mut inputs = Vec::with_capacity(1); inputs.push((BitcoinOutPoint { txid, vout }, input_solving_data)); @@ -873,26 +936,26 @@ mod tests { } macro_rules! dumb_counterparty_output { - ($secp_ctx: expr, $amt: expr) => { + ($secp_ctx: expr, $amt: expr, $opt_anchors: expr) => { { let dumb_scalar = SecretKey::from_slice(&hex::decode("0101010101010101010101010101010101010101010101010101010101010101").unwrap()[..]).unwrap(); let dumb_point = PublicKey::from_secret_key(&$secp_ctx, &dumb_scalar); let hash = PaymentHash([1; 32]); let htlc = HTLCOutputInCommitment { offered: true, amount_msat: $amt, cltv_expiry: 0, payment_hash: hash, transaction_output_index: None }; - PackageSolvingData::CounterpartyReceivedHTLCOutput(CounterpartyReceivedHTLCOutput::build(dumb_point, dumb_point, dumb_point, htlc)) + PackageSolvingData::CounterpartyReceivedHTLCOutput(CounterpartyReceivedHTLCOutput::build(dumb_point, dumb_point, dumb_point, htlc, $opt_anchors)) } } } macro_rules! dumb_counterparty_offered_output { - ($secp_ctx: expr, $amt: expr) => { + ($secp_ctx: expr, $amt: expr, $opt_anchors: expr) => { { let dumb_scalar = SecretKey::from_slice(&hex::decode("0101010101010101010101010101010101010101010101010101010101010101").unwrap()[..]).unwrap(); let dumb_point = PublicKey::from_secret_key(&$secp_ctx, &dumb_scalar); let hash = PaymentHash([1; 32]); let preimage = PaymentPreimage([2;32]); let htlc = HTLCOutputInCommitment { offered: false, amount_msat: $amt, cltv_expiry: 1000, payment_hash: hash, transaction_output_index: None }; - PackageSolvingData::CounterpartyOfferedHTLCOutput(CounterpartyOfferedHTLCOutput::build(dumb_point, dumb_point, dumb_point, preimage, htlc)) + PackageSolvingData::CounterpartyOfferedHTLCOutput(CounterpartyOfferedHTLCOutput::build(dumb_point, dumb_point, dumb_point, preimage, htlc, $opt_anchors)) } } } @@ -987,7 +1050,7 @@ mod tests { let txid = Txid::from_hex("c2d4449afa8d26140898dd54d3390b057ba2a5afcf03ba29d7dc0d8b9ffe966e").unwrap(); let secp_ctx = Secp256k1::new(); let revk_outp = dumb_revk_output!(secp_ctx); - let counterparty_outp = dumb_counterparty_output!(secp_ctx, 0); + let counterparty_outp = dumb_counterparty_output!(secp_ctx, 0, false); let mut revoked_package = PackageTemplate::build_package(txid, 0, revk_outp, 1000, true, 100); let counterparty_package = PackageTemplate::build_package(txid, 1, counterparty_outp, 1000, true, 100); @@ -1051,7 +1114,7 @@ mod tests { fn test_package_amounts() { let txid = Txid::from_hex("c2d4449afa8d26140898dd54d3390b057ba2a5afcf03ba29d7dc0d8b9ffe966e").unwrap(); let secp_ctx = Secp256k1::new(); - let counterparty_outp = dumb_counterparty_output!(secp_ctx, 1_000_000); + let counterparty_outp = dumb_counterparty_output!(secp_ctx, 1_000_000, false); let package = PackageTemplate::build_package(txid, 0, counterparty_outp, 1000, true, 100); assert_eq!(package.package_amount(), 1000); @@ -1068,24 +1131,22 @@ mod tests { { let revk_outp = dumb_revk_output!(secp_ctx); let package = PackageTemplate::build_package(txid, 0, revk_outp, 0, true, 100); - for &opt_anchors in [false, true].iter() { - assert_eq!(package.package_weight(&Script::new(), opt_anchors), weight_sans_output + WEIGHT_REVOKED_OUTPUT as usize); - } + assert_eq!(package.package_weight(&Script::new()), weight_sans_output + WEIGHT_REVOKED_OUTPUT as usize); } { - let counterparty_outp = dumb_counterparty_output!(secp_ctx, 1_000_000); - let package = PackageTemplate::build_package(txid, 0, counterparty_outp, 1000, true, 100); for &opt_anchors in [false, true].iter() { - assert_eq!(package.package_weight(&Script::new(), opt_anchors), weight_sans_output + weight_received_htlc(opt_anchors) as usize); + let counterparty_outp = dumb_counterparty_output!(secp_ctx, 1_000_000, opt_anchors); + let package = PackageTemplate::build_package(txid, 0, counterparty_outp, 1000, true, 100); + assert_eq!(package.package_weight(&Script::new()), weight_sans_output + weight_received_htlc(opt_anchors) as usize); } } { - let counterparty_outp = dumb_counterparty_offered_output!(secp_ctx, 1_000_000); - let package = PackageTemplate::build_package(txid, 0, counterparty_outp, 1000, true, 100); for &opt_anchors in [false, true].iter() { - assert_eq!(package.package_weight(&Script::new(), opt_anchors), weight_sans_output + weight_offered_htlc(opt_anchors) as usize); + let counterparty_outp = dumb_counterparty_offered_output!(secp_ctx, 1_000_000, opt_anchors); + let package = PackageTemplate::build_package(txid, 0, counterparty_outp, 1000, true, 100); + assert_eq!(package.package_weight(&Script::new()), weight_sans_output + weight_offered_htlc(opt_anchors) as usize); } } } diff --git a/lightning/src/ln/chan_utils.rs b/lightning/src/ln/chan_utils.rs index 69ca4fb83f9..15bc0d0e23e 100644 --- a/lightning/src/ln/chan_utils.rs +++ b/lightning/src/ln/chan_utils.rs @@ -526,8 +526,8 @@ pub fn get_revokeable_redeemscript(revocation_key: &PublicKey, contest_delay: u1 res } -#[derive(Clone, PartialEq, Eq)] /// Information about an HTLC as it appears in a commitment transaction +#[derive(Clone, Debug, PartialEq, Eq)] pub struct HTLCOutputInCommitment { /// Whether the HTLC was "offered" (ie outbound in relation to this commitment transaction). /// Note that this is not the same as whether it is ountbound *from us*. To determine that you @@ -727,6 +727,23 @@ pub fn get_anchor_redeemscript(funding_pubkey: &PublicKey) -> Script { .into_script() } +#[cfg(anchors)] +/// Locates the output with an anchor script paying to `funding_pubkey` within `commitment_tx`. +pub(crate) fn get_anchor_output<'a>(commitment_tx: &'a Transaction, funding_pubkey: &PublicKey) -> Option<(u32, &'a TxOut)> { + let anchor_script = chan_utils::get_anchor_redeemscript(funding_pubkey).to_v0_p2wsh(); + commitment_tx.output.iter().enumerate() + .find(|(_, txout)| txout.script_pubkey == anchor_script) + .map(|(idx, txout)| (idx as u32, txout)) +} + +/// Returns the witness required to satisfy and spend an anchor input. +pub fn build_anchor_input_witness(funding_key: &PublicKey, funding_sig: &Signature) -> Witness { + let anchor_redeem_script = chan_utils::get_anchor_redeemscript(funding_key); + let mut funding_sig = funding_sig.serialize_der().to_vec(); + funding_sig.push(EcdsaSighashType::All as u8); + Witness::from_vec(vec![funding_sig, anchor_redeem_script.to_bytes()]) +} + /// Per-channel data used to build transactions in conjunction with the per-commitment data (CommitmentTransaction). /// The fields are organized by holder/counterparty. /// diff --git a/lightning/src/util/enforcing_trait_impls.rs b/lightning/src/util/enforcing_trait_impls.rs index f4450cc129d..34b5954d485 100644 --- a/lightning/src/util/enforcing_trait_impls.rs +++ b/lightning/src/util/enforcing_trait_impls.rs @@ -7,6 +7,7 @@ // You may not use this file except in accordance with one or both of these // licenses. +use ln::channel::{ANCHOR_OUTPUT_VALUE_SATOSHI, MIN_CHAN_DUST_LIMIT_SATOSHIS}; use ln::chan_utils::{HTLCOutputInCommitment, ChannelPublicKeys, HolderCommitmentTransaction, CommitmentTransaction, ChannelTransactionParameters, TrustedCommitmentTransaction, ClosingTransaction}; use ln::{chan_utils, msgs, PaymentPreimage}; use chain::keysinterface::{Sign, InMemorySigner, BaseSign}; @@ -199,6 +200,16 @@ impl BaseSign for EnforcingSigner { Ok(self.inner.sign_closing_transaction(closing_tx, secp_ctx).unwrap()) } + fn sign_holder_anchor_input( + &self, anchor_tx: &mut Transaction, input: usize, secp_ctx: &Secp256k1, + ) -> Result { + debug_assert!(MIN_CHAN_DUST_LIMIT_SATOSHIS > ANCHOR_OUTPUT_VALUE_SATOSHI); + // As long as our minimum dust limit is enforced and is greater than our anchor output + // value, an anchor output can only have an index within [0, 1]. + assert!(anchor_tx.input[input].previous_output.vout == 0 || anchor_tx.input[input].previous_output.vout == 1); + self.inner.sign_holder_anchor_input(anchor_tx, input, secp_ctx) + } + fn sign_channel_announcement(&self, msg: &msgs::UnsignedChannelAnnouncement, secp_ctx: &Secp256k1) -> Result<(Signature, Signature), ()> { self.inner.sign_channel_announcement(msg, secp_ctx) diff --git a/lightning/src/util/events.rs b/lightning/src/util/events.rs index 455263ee26f..20f1c5b786c 100644 --- a/lightning/src/util/events.rs +++ b/lightning/src/util/events.rs @@ -15,6 +15,7 @@ //! few other things. use chain::keysinterface::SpendableOutputDescriptor; +use ln::chan_utils::HTLCOutputInCommitment; use ln::channelmanager::PaymentId; use ln::channel::FUNDING_CONF_DEADLINE_BLOCKS; use ln::features::ChannelTypeFeatures; @@ -25,7 +26,7 @@ use routing::gossip::NetworkUpdate; use util::ser::{BigSize, FixedLengthReader, Writeable, Writer, MaybeReadable, Readable, VecReadWrapper, VecWriteWrapper}; use routing::router::{RouteHop, RouteParameters}; -use bitcoin::{PackedLockTime, Transaction}; +use bitcoin::{PackedLockTime, Transaction, OutPoint}; use bitcoin::blockdata::script::Script; use bitcoin::hashes::Hash; use bitcoin::hashes::sha256::Hash as Sha256; @@ -196,6 +197,84 @@ impl_writeable_tlv_based_enum_upgradable!(HTLCDestination, } ); +/// A descriptor used to sign for a commitment transaction's anchor output. +#[derive(Clone, Debug)] +pub struct AnchorDescriptor { + /// A unique identifier used along with `channel_value_satoshis` to re-derive the + /// [`InMemorySigner`] required to sign `input`. + /// + /// [`InMemorySigner`]: crate::chain::keysinterface::InMemorySigner + pub channel_keys_id: [u8; 32], + /// The value in satoshis of the channel we're attempting to spend the anchor output of. This is + /// used along with `channel_keys_id` to re-derive the [`InMemorySigner`] required to sign + /// `input`. + /// + /// [`InMemorySigner`]: crate::chain::keysinterface::InMemorySigner + pub channel_value_satoshis: u64, + /// The transaction input's outpoint corresponding to the commitment transaction's anchor + /// output. + pub outpoint: OutPoint, +} + +/// Represents the different types of transactions, originating from LDK, to be bumped. +#[derive(Clone, Debug)] +pub enum BumpTransactionEvent { + /// Indicates that a channel featuring anchor outputs is to be closed by broadcasting the local + /// commitment transaction. Since commitment transactions have a static feerate pre-agreed upon, + /// they may need additional fees to be attached through a child transaction using the popular + /// [Child-Pays-For-Parent](https://bitcoinops.org/en/topics/cpfp) fee bumping technique. This + /// child transaction must include the anchor input described within `anchor_descriptor` along + /// with additional inputs to meet the target feerate. Failure to meet the target feerate + /// decreases the confirmation odds of the transaction package (which includes the commitment + /// and child anchor transactions), possibly resulting in a loss of funds. Once the transaction + /// is constructed, it must be fully signed for and broadcasted by the consumer of the event + /// along with the `commitment_tx` enclosed. Note that the `commitment_tx` must always be + /// broadcast first, as the child anchor transaction depends on it. + /// + /// The consumer should be able to sign for any of the additional inputs included within the + /// child anchor transaction. To sign its anchor input, an [`InMemorySigner`] should be + /// re-derived through [`KeysManager::derive_channel_keys`] with the help of + /// [`AnchorDescriptor::channel_keys_id`] and [`AnchorDescriptor::channel_value_satoshis`]. + /// + /// It is possible to receive more than one instance of this event if a valid child anchor + /// transaction is never broadcast or is but not with a sufficient fee to be mined. Care should + /// be taken by the consumer of the event to ensure any future iterations of the child anchor + /// transaction adhere to the [Replace-By-Fee + /// rules](https://github.com/bitcoin/bitcoin/blob/master/doc/policy/mempool-replacements.md) + /// for fee bumps to be accepted into the mempool, and eventually the chain. As the frequency of + /// these events is not user-controlled, users may ignore/drop the event if they are no longer + /// able to commit external confirmed funds to the child anchor transaction. + /// + /// The set of `pending_htlcs` on the commitment transaction to be broadcast can be inspected to + /// determine whether a significant portion of the channel's funds are allocated to HTLCs, + /// enabling users to make their own decisions regarding the importance of the commitment + /// transaction's confirmation. Note that this is not required, but simply exists as an option + /// for users to override LDK's behavior. On commitments with no HTLCs (indicated by those with + /// an empty `pending_htlcs`), confirmation of the commitment transaction can be considered to + /// be not urgent. + /// + /// [`InMemorySigner`]: crate::chain::keysinterface::InMemorySigner + /// [`KeysManager::derive_channel_keys`]: crate::chain::keysinterface::KeysManager::derive_channel_keys + ChannelClose { + /// The target feerate that the transaction package, which consists of the commitment + /// transaction and the to-be-crafted child anchor transaction, must meet. + package_target_feerate_sat_per_1000_weight: u32, + /// The channel's commitment transaction to bump the fee of. This transaction should be + /// broadcast along with the anchor transaction constructed as a result of consuming this + /// event. + commitment_tx: Transaction, + /// The absolute fee in satoshis of the commitment transaction. This can be used along the + /// with weight of the commitment transaction to determine its feerate. + commitment_tx_fee_satoshis: u64, + /// The descriptor to sign the anchor input of the anchor transaction constructed as a + /// result of consuming this event. + anchor_descriptor: AnchorDescriptor, + /// The set of pending HTLCs on the commitment transaction that need to be resolved once the + /// commitment transaction confirms. + pending_htlcs: Vec, + }, +} + /// An Event which you should probably take some action in response to. /// /// Note that while Writeable and Readable are implemented for Event, you probably shouldn't use @@ -602,6 +681,13 @@ pub enum Event { /// Destination of the HTLC that failed to be processed. failed_next_destination: HTLCDestination, }, + #[cfg(anchors)] + /// Indicates that a transaction originating from LDK needs to have its fee bumped. This event + /// requires confirmed external funds to be readily available to spend. + /// + /// LDK does not currently generate this event. It is limited to the scope of channels with + /// anchor outputs, which will be introduced in a future release. + BumpTransaction(BumpTransactionEvent), } impl Writeable for Event { @@ -753,6 +839,15 @@ impl Writeable for Event { (2, failed_next_destination, required), }) }, + #[cfg(anchors)] + &Event::BumpTransaction(ref event)=> { + 27u8.write(writer)?; + match event { + // We never write the ChannelClose events as they'll be replayed upon restarting + // anyway if the commitment transaction remains unconfirmed. + BumpTransactionEvent::ChannelClose { .. } => {} + } + } // Note that, going forward, all new events must only write data inside of // `write_tlv_fields`. Versions 0.0.101+ will ignore odd-numbered events that write // data via `write_tlv_fields`.