}
}
-/// An entry for an [`OnchainEvent`], stating the block height when the event was observed and the
-/// transaction causing it.
+/// An entry for an [`OnchainEvent`], stating the block height and hash when the event was
+/// observed, as well as the transaction causing it.
///
/// Used to determine when the on-chain event can be considered safe from a chain reorganization.
#[derive(PartialEq, Eq)]
struct OnchainEventEntry {
txid: Txid,
height: u32,
+ block_hash: Option<BlockHash>, // Added as optional, will be filled in for any entry generated on 0.0.113 or after
event: OnchainEvent,
transaction: Option<Transaction>, // Added as optional, but always filled in, in LDK 0.0.110
}
(0, self.txid, required),
(1, self.transaction, option),
(2, self.height, required),
+ (3, self.block_hash, option),
(4, self.event, required),
});
Ok(())
fn read<R: io::Read>(reader: &mut R) -> Result<Option<Self>, DecodeError> {
let mut txid = Txid::all_zeros();
let mut transaction = None;
+ let mut block_hash = None;
let mut height = 0;
let mut event = None;
read_tlv_fields!(reader, {
(0, txid, required),
(1, transaction, option),
(2, height, required),
+ (3, block_hash, option),
(4, event, ignorable),
});
if let Some(ev) = event {
- Ok(Some(Self { txid, transaction, height, event: ev }))
+ Ok(Some(Self { txid, transaction, height, block_hash, event: ev }))
} else {
Ok(None)
}
}
/// Returns the set of txids that should be monitored for re-organization out of the chain.
- pub fn get_relevant_txids(&self) -> Vec<Txid> {
+ pub fn get_relevant_txids(&self) -> Vec<(Txid, Option<BlockHash>)> {
let inner = self.inner.lock().unwrap();
- let mut txids: Vec<Txid> = inner.onchain_events_awaiting_threshold_conf
+ let mut txids: Vec<(Txid, Option<BlockHash>)> = inner.onchain_events_awaiting_threshold_conf
.iter()
- .map(|entry| entry.txid)
+ .map(|entry| (entry.txid, entry.block_hash))
.chain(inner.onchain_tx_handler.get_relevant_txids().into_iter())
.collect();
txids.sort_unstable();
/// been revoked yet, the previous one, we we will never "forget" to resolve an HTLC.
macro_rules! fail_unbroadcast_htlcs {
($self: expr, $commitment_tx_type: expr, $commitment_txid_confirmed: expr, $commitment_tx_confirmed: expr,
- $commitment_tx_conf_height: expr, $confirmed_htlcs_list: expr, $logger: expr) => { {
+ $commitment_tx_conf_height: expr, $commitment_tx_conf_hash: expr, $confirmed_htlcs_list: expr, $logger: expr) => { {
debug_assert_eq!($commitment_tx_confirmed.txid(), $commitment_txid_confirmed);
macro_rules! check_htlc_fails {
txid: $commitment_txid_confirmed,
transaction: Some($commitment_tx_confirmed.clone()),
height: $commitment_tx_conf_height,
+ block_hash: Some(*$commitment_tx_conf_hash),
event: OnchainEvent::HTLCUpdate {
source: (**source).clone(),
payment_hash: htlc.payment_hash.clone(),
macro_rules! claim_htlcs {
($commitment_number: expr, $txid: expr) => {
let (htlc_claim_reqs, _) = self.get_counterparty_output_claim_info($commitment_number, $txid, None);
- self.onchain_tx_handler.update_claims_view(&Vec::new(), htlc_claim_reqs, self.best_block.height(), self.best_block.height(), broadcaster, fee_estimator, logger);
+ self.onchain_tx_handler.update_claims_view_from_requests(htlc_claim_reqs, self.best_block.height(), self.best_block.height(), broadcaster, fee_estimator, logger);
}
}
if let Some(txid) = self.current_counterparty_commitment_txid {
// block. Even if not, its a reasonable metric for the bump criteria on the HTLC
// transactions.
let (claim_reqs, _) = self.get_broadcasted_holder_claims(&self.current_holder_commitment_tx, self.best_block.height());
- self.onchain_tx_handler.update_claims_view(&Vec::new(), claim_reqs, self.best_block.height(), self.best_block.height(), broadcaster, fee_estimator, logger);
+ self.onchain_tx_handler.update_claims_view_from_requests(claim_reqs, self.best_block.height(), self.best_block.height(), broadcaster, fee_estimator, logger);
if let Some(ref tx) = self.prev_holder_signed_commitment_tx {
let (claim_reqs, _) = self.get_broadcasted_holder_claims(&tx, self.best_block.height());
- self.onchain_tx_handler.update_claims_view(&Vec::new(), claim_reqs, self.best_block.height(), self.best_block.height(), broadcaster, fee_estimator, logger);
+ self.onchain_tx_handler.update_claims_view_from_requests(claim_reqs, self.best_block.height(), self.best_block.height(), broadcaster, fee_estimator, logger);
}
}
}
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,
+ self.onchain_tx_handler.update_claims_view_from_requests(
+ vec![commitment_package], best_block_height, best_block_height,
broadcaster, &bounded_fee_estimator, logger,
);
}
/// Returns packages to claim the revoked output(s), as well as additional outputs to watch and
/// general information about the output that is to the counterparty in the commitment
/// transaction.
- fn check_spend_counterparty_transaction<L: Deref>(&mut self, tx: &Transaction, height: u32, logger: &L)
+ fn check_spend_counterparty_transaction<L: Deref>(&mut self, tx: &Transaction, height: u32, block_hash: &BlockHash, logger: &L)
-> (Vec<PackageTemplate>, TransactionOutputs, CommitmentTxCounterpartyOutputInfo)
where L::Target: Logger {
// Most secp and related errors trying to create keys means we have no hope of constructing
if let Some(per_commitment_data) = per_commitment_option {
fail_unbroadcast_htlcs!(self, "revoked_counterparty", commitment_txid, tx, height,
- per_commitment_data.iter().map(|(htlc, htlc_source)|
+ block_hash, per_commitment_data.iter().map(|(htlc, htlc_source)|
(htlc, htlc_source.as_ref().map(|htlc_source| htlc_source.as_ref()))
), logger);
} else {
debug_assert!(false, "We should have per-commitment option for any recognized old commitment txn");
fail_unbroadcast_htlcs!(self, "revoked counterparty", commitment_txid, tx, height,
- [].iter().map(|reference| *reference), logger);
+ block_hash, [].iter().map(|reference| *reference), logger);
}
}
} else if let Some(per_commitment_data) = per_commitment_option {
self.counterparty_commitment_txn_on_chain.insert(commitment_txid, commitment_number);
log_info!(logger, "Got broadcast of non-revoked counterparty commitment transaction {}", commitment_txid);
- fail_unbroadcast_htlcs!(self, "counterparty", commitment_txid, tx, height,
+ fail_unbroadcast_htlcs!(self, "counterparty", commitment_txid, tx, height, block_hash,
per_commitment_data.iter().map(|(htlc, htlc_source)|
(htlc, htlc_source.as_ref().map(|htlc_source| htlc_source.as_ref()))
), logger);
(claimable_outpoints, Some((htlc_txid, outputs)))
}
- // Returns (1) `PackageTemplate`s that can be given to the OnChainTxHandler, so that the handler can
+ // Returns (1) `PackageTemplate`s that can be given to the OnchainTxHandler, so that the handler can
// broadcast transactions claiming holder HTLC commitment outputs and (2) a holder revokable
// script so we can detect whether a holder transaction has been seen on-chain.
fn get_broadcasted_holder_claims(&self, holder_tx: &HolderSignedTx, conf_height: u32) -> (Vec<PackageTemplate>, Option<(Script, PublicKey, PublicKey)>) {
/// revoked using data in holder_claimable_outpoints.
/// Should not be used if check_spend_revoked_transaction succeeds.
/// Returns None unless the transaction is definitely one of our commitment transactions.
- fn check_spend_holder_transaction<L: Deref>(&mut self, tx: &Transaction, height: u32, logger: &L) -> Option<(Vec<PackageTemplate>, TransactionOutputs)> where L::Target: Logger {
+ fn check_spend_holder_transaction<L: Deref>(&mut self, tx: &Transaction, height: u32, block_hash: &BlockHash, logger: &L) -> Option<(Vec<PackageTemplate>, TransactionOutputs)> where L::Target: Logger {
let commitment_txid = tx.txid();
let mut claim_requests = Vec::new();
let mut watch_outputs = Vec::new();
let mut to_watch = self.get_broadcasted_holder_watch_outputs(&self.current_holder_commitment_tx, tx);
append_onchain_update!(res, to_watch);
fail_unbroadcast_htlcs!(self, "latest holder", commitment_txid, tx, height,
- self.current_holder_commitment_tx.htlc_outputs.iter()
+ block_hash, self.current_holder_commitment_tx.htlc_outputs.iter()
.map(|(htlc, _, htlc_source)| (htlc, htlc_source.as_ref())), logger);
} else if let &Some(ref holder_tx) = &self.prev_holder_signed_commitment_tx {
if holder_tx.txid == commitment_txid {
let res = self.get_broadcasted_holder_claims(holder_tx, height);
let mut to_watch = self.get_broadcasted_holder_watch_outputs(holder_tx, tx);
append_onchain_update!(res, to_watch);
- fail_unbroadcast_htlcs!(self, "previous holder", commitment_txid, tx, height,
+ fail_unbroadcast_htlcs!(self, "previous holder", commitment_txid, tx, height, block_hash,
holder_tx.htlc_outputs.iter().map(|(htlc, _, htlc_source)| (htlc, htlc_source.as_ref())),
logger);
}
if height > self.best_block.height() {
self.best_block = BestBlock::new(block_hash, height);
- self.block_confirmed(height, vec![], vec![], vec![], &broadcaster, &fee_estimator, &logger)
+ self.block_confirmed(height, block_hash, vec![], vec![], vec![], &broadcaster, &fee_estimator, &logger)
} else if block_hash != self.best_block.block_hash() {
self.best_block = BestBlock::new(block_hash, height);
self.onchain_events_awaiting_threshold_conf.retain(|ref entry| entry.height <= height);
let mut commitment_tx_to_counterparty_output = None;
if (tx.input[0].sequence.0 >> 8*3) as u8 == 0x80 && (tx.lock_time.0 >> 8*3) as u8 == 0x20 {
let (mut new_outpoints, new_outputs, counterparty_output_idx_sats) =
- self.check_spend_counterparty_transaction(&tx, height, &logger);
+ self.check_spend_counterparty_transaction(&tx, height, &block_hash, &logger);
commitment_tx_to_counterparty_output = counterparty_output_idx_sats;
if !new_outputs.1.is_empty() {
watch_outputs.push(new_outputs);
}
claimable_outpoints.append(&mut new_outpoints);
if new_outpoints.is_empty() {
- if let Some((mut new_outpoints, new_outputs)) = self.check_spend_holder_transaction(&tx, height, &logger) {
+ if let Some((mut new_outpoints, new_outputs)) = self.check_spend_holder_transaction(&tx, height, &block_hash, &logger) {
debug_assert!(commitment_tx_to_counterparty_output.is_none(),
"A commitment transaction matched as both a counterparty and local commitment tx?");
if !new_outputs.1.is_empty() {
txid,
transaction: Some((*tx).clone()),
height,
+ block_hash: Some(block_hash),
event: OnchainEvent::FundingSpendConfirmation {
on_local_output_csv: balance_spendable_csv,
commitment_tx_to_counterparty_output,
// While all commitment/HTLC-Success/HTLC-Timeout transactions have one input, HTLCs
// can also be resolved in a few other ways which can have more than one output. Thus,
// we call is_resolving_htlc_output here outside of the tx.input.len() == 1 check.
- self.is_resolving_htlc_output(&tx, height, &logger);
+ self.is_resolving_htlc_output(&tx, height, &block_hash, &logger);
- self.is_paying_spendable_output(&tx, height, &logger);
+ self.is_paying_spendable_output(&tx, height, &block_hash, &logger);
}
if height > self.best_block.height() {
self.best_block = BestBlock::new(block_hash, height);
}
- self.block_confirmed(height, txn_matched, watch_outputs, claimable_outpoints, &broadcaster, &fee_estimator, &logger)
+ self.block_confirmed(height, block_hash, txn_matched, watch_outputs, claimable_outpoints, &broadcaster, &fee_estimator, &logger)
}
/// Update state for new block(s)/transaction(s) confirmed. Note that the caller must update
/// `self.best_block` before calling if a new best blockchain tip is available. More
/// concretely, `self.best_block` must never be at a lower height than `conf_height`, avoiding
- /// complexity especially in `OnchainTx::update_claims_view`.
+ /// complexity especially in
+ /// `OnchainTx::update_claims_view_from_requests`/`OnchainTx::update_claims_view_from_matched_txn`.
///
/// `conf_height` should be set to the height at which any new transaction(s)/block(s) were
/// confirmed at, even if it is not the current best height.
fn block_confirmed<B: Deref, F: Deref, L: Deref>(
&mut self,
conf_height: u32,
+ conf_hash: BlockHash,
txn_matched: Vec<&Transaction>,
mut watch_outputs: Vec<TransactionOutputs>,
mut claimable_outpoints: Vec<PackageTemplate>,
}
}
- self.onchain_tx_handler.update_claims_view(&txn_matched, claimable_outpoints, conf_height, self.best_block.height(), broadcaster, fee_estimator, logger);
+ self.onchain_tx_handler.update_claims_view_from_requests(claimable_outpoints, conf_height, self.best_block.height(), broadcaster, fee_estimator, logger);
+ self.onchain_tx_handler.update_claims_view_from_matched_txn(&txn_matched, conf_height, conf_hash, self.best_block.height(), broadcaster, fee_estimator, logger);
// Determine new outputs to watch by comparing against previously known outputs to watch,
// updating the latter in the process.
/// Check if any transaction broadcasted is resolving HTLC output by a success or timeout on a holder
/// or counterparty commitment tx, if so send back the source, preimage if found and payment_hash of resolved HTLC
- fn is_resolving_htlc_output<L: Deref>(&mut self, tx: &Transaction, height: u32, logger: &L) where L::Target: Logger {
+ fn is_resolving_htlc_output<L: Deref>(&mut self, tx: &Transaction, height: u32, block_hash: &BlockHash, logger: &L) where L::Target: Logger {
'outer_loop: for input in &tx.input {
let mut payment_data = None;
let htlc_claim = HTLCClaim::from_witness(&input.witness);
log_claim!($tx_info, $holder_tx, htlc_output, false);
let outbound_htlc = $holder_tx == htlc_output.offered;
self.onchain_events_awaiting_threshold_conf.push(OnchainEventEntry {
- txid: tx.txid(), height, transaction: Some(tx.clone()),
+ txid: tx.txid(), height, block_hash: Some(*block_hash), transaction: Some(tx.clone()),
event: OnchainEvent::HTLCSpendConfirmation {
commitment_tx_output_idx: input.previous_output.vout,
preimage: if accepted_preimage_claim || offered_preimage_claim {
self.onchain_events_awaiting_threshold_conf.push(OnchainEventEntry {
txid: tx.txid(),
height,
+ block_hash: Some(*block_hash),
transaction: Some(tx.clone()),
event: OnchainEvent::HTLCSpendConfirmation {
commitment_tx_output_idx: input.previous_output.vout,
txid: tx.txid(),
transaction: Some(tx.clone()),
height,
+ block_hash: Some(*block_hash),
event: OnchainEvent::HTLCSpendConfirmation {
commitment_tx_output_idx: input.previous_output.vout,
preimage: Some(payment_preimage),
txid: tx.txid(),
transaction: Some(tx.clone()),
height,
+ block_hash: Some(*block_hash),
event: OnchainEvent::HTLCUpdate {
source, payment_hash,
htlc_value_satoshis: Some(amount_msat / 1000),
}
/// Check if any transaction broadcasted is paying fund back to some address we can assume to own
- fn is_paying_spendable_output<L: Deref>(&mut self, tx: &Transaction, height: u32, logger: &L) where L::Target: Logger {
+ fn is_paying_spendable_output<L: Deref>(&mut self, tx: &Transaction, height: u32, block_hash: &BlockHash, logger: &L) where L::Target: Logger {
let mut spendable_output = None;
for (i, outp) in tx.output.iter().enumerate() { // There is max one spendable output for any channel tx, including ones generated by us
if i > ::core::u16::MAX as usize {
txid: tx.txid(),
transaction: Some(tx.clone()),
height,
+ block_hash: Some(*block_hash),
event: OnchainEvent::MaturingOutput { descriptor: spendable_output.clone() },
};
log_info!(logger, "Received spendable output {}, spendable at height {}", log_spendable!(spendable_output), entry.confirmation_threshold());
self.0.best_block_updated(header, height, &*self.1, &*self.2, &*self.3);
}
- fn get_relevant_txids(&self) -> Vec<Txid> {
+ fn get_relevant_txids(&self) -> Vec<(Txid, Option<BlockHash>)> {
self.0.get_relevant_txids()
}
}
use bitcoin::blockdata::transaction::OutPoint as BitcoinOutPoint;
use bitcoin::blockdata::script::Script;
-use bitcoin::hash_types::Txid;
+use bitcoin::hash_types::{Txid, BlockHash};
use bitcoin::secp256k1::{Secp256k1, ecdsa::Signature};
use bitcoin::secp256k1;
struct OnchainEventEntry {
txid: Txid,
height: u32,
+ block_hash: Option<BlockHash>, // Added as optional, will be filled in for any entry generated on 0.0.113 or after
event: OnchainEvent,
}
fn write<W: Writer>(&self, writer: &mut W) -> Result<(), io::Error> {
write_tlv_fields!(writer, {
(0, self.txid, required),
+ (1, self.block_hash, option),
(2, self.height, required),
(4, self.event, required),
});
fn read<R: io::Read>(reader: &mut R) -> Result<Option<Self>, DecodeError> {
let mut txid = Txid::all_zeros();
let mut height = 0;
+ let mut block_hash = None;
let mut event = None;
read_tlv_fields!(reader, {
(0, txid, required),
+ (1, block_hash, option),
(2, height, required),
(4, event, ignorable),
});
if let Some(ev) = event {
- Ok(Some(Self { txid, height, event: ev }))
+ Ok(Some(Self { txid, height, block_hash, event: ev }))
} else {
Ok(None)
}
/// Upon channelmonitor.block_connected(..) or upon provision of a preimage on the forward link
/// for this channel, provide new relevant on-chain transactions and/or new claim requests.
- /// Formerly this was named `block_connected`, but it is now also used for claiming an HTLC output
- /// if we receive a preimage after force-close.
- /// `conf_height` represents the height at which the transactions in `txn_matched` were
- /// confirmed. This does not need to equal the current blockchain tip height, which should be
- /// provided via `cur_height`, however it must never be higher than `cur_height`.
- pub(crate) fn update_claims_view<B: Deref, F: Deref, L: Deref>(&mut self, txn_matched: &[&Transaction], requests: Vec<PackageTemplate>, conf_height: u32, cur_height: u32, broadcaster: &B, fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L)
- where B::Target: BroadcasterInterface,
- F::Target: FeeEstimator,
- L::Target: Logger,
+ /// Together with `update_claims_view_from_matched_txn` this used to be named
+ /// `block_connected`, but it is now also used for claiming an HTLC output if we receive a
+ /// preimage after force-close.
+ ///
+ /// `conf_height` represents the height at which the request was generated. This
+ /// does not need to equal the current blockchain tip height, which should be provided via
+ /// `cur_height`, however it must never be higher than `cur_height`.
+ pub(crate) fn update_claims_view_from_requests<B: Deref, F: Deref, L: Deref>(
+ &mut self, requests: Vec<PackageTemplate>, conf_height: u32, cur_height: u32,
+ broadcaster: &B, fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+ ) where
+ B::Target: BroadcasterInterface,
+ F::Target: FeeEstimator,
+ L::Target: Logger,
{
- log_debug!(logger, "Updating claims view at height {} with {} matched transactions in block {} and {} claim requests", cur_height, txn_matched.len(), conf_height, requests.len());
+ log_debug!(logger, "Updating claims view at height {} with {} claim requests", cur_height, requests.len());
let mut preprocessed_requests = Vec::with_capacity(requests.len());
let mut aggregated_request = None;
self.pending_claim_requests.insert(txid, req);
}
}
+ }
+ /// Upon channelmonitor.block_connected(..) or upon provision of a preimage on the forward link
+ /// for this channel, provide new relevant on-chain transactions and/or new claim requests.
+ /// Together with `update_claims_view_from_requests` this used to be named `block_connected`,
+ /// but it is now also used for claiming an HTLC output if we receive a preimage after force-close.
+ ///
+ /// `conf_height` represents the height at which the transactions in `txn_matched` were
+ /// confirmed. This does not need to equal the current blockchain tip height, which should be
+ /// provided via `cur_height`, however it must never be higher than `cur_height`.
+ pub(crate) fn update_claims_view_from_matched_txn<B: Deref, F: Deref, L: Deref>(
+ &mut self, txn_matched: &[&Transaction], conf_height: u32, conf_hash: BlockHash,
+ cur_height: u32, broadcaster: &B, fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+ ) where
+ B::Target: BroadcasterInterface,
+ F::Target: FeeEstimator,
+ L::Target: Logger,
+ {
+ log_debug!(logger, "Updating claims view at height {} with {} matched transactions in block {}", cur_height, txn_matched.len(), conf_height);
let mut bump_candidates = HashMap::new();
for tx in txn_matched {
// Scan all input to verify is one of the outpoint spent is of interest for us
let entry = OnchainEventEntry {
txid: tx.txid(),
height: conf_height,
+ block_hash: Some(conf_hash),
event: OnchainEvent::Claim { claim_request: first_claim_txid_height.0.clone() }
};
if !self.onchain_events_awaiting_threshold_conf.contains(&entry) {
let entry = OnchainEventEntry {
txid: tx.txid(),
height: conf_height,
+ block_hash: Some(conf_hash),
event: OnchainEvent::ContentiousOutpoint { package },
};
if !self.onchain_events_awaiting_threshold_conf.contains(&entry) {
self.claimable_outpoints.get(outpoint).is_some()
}
- pub(crate) fn get_relevant_txids(&self) -> Vec<Txid> {
- let mut txids: Vec<Txid> = self.onchain_events_awaiting_threshold_conf
+ pub(crate) fn get_relevant_txids(&self) -> Vec<(Txid, Option<BlockHash>)> {
+ let mut txids: Vec<(Txid, Option<BlockHash>)> = self.onchain_events_awaiting_threshold_conf
.iter()
- .map(|entry| entry.txid)
+ .map(|entry| (entry.txid, entry.block_hash))
.collect();
- txids.sort_unstable();
+ txids.sort_unstable_by_key(|(txid, _)| *txid);
txids.dedup();
txids
}