Merge pull request #1119 from TheBlueMatt/2021-10-less-aggressive-htlc-timeouts
[rust-lightning] / lightning / src / chain / channelmonitor.rs
index d7395c337fc96a6193f61bb5dd4271ced09179e5..425a1890a8f46acfabd68773dbc32f51172e2676 100644 (file)
@@ -115,67 +115,6 @@ impl Readable for ChannelMonitorUpdate {
        }
 }
 
-/// An error enum representing a failure to persist a channel monitor update.
-#[derive(Clone, Copy, Debug, PartialEq)]
-pub enum ChannelMonitorUpdateErr {
-       /// Used to indicate a temporary failure (eg connection to a watchtower or remote backup of
-       /// our state failed, but is expected to succeed at some point in the future).
-       ///
-       /// Such a failure will "freeze" a channel, preventing us from revoking old states or
-       /// submitting new commitment transactions to the counterparty. Once the update(s) which failed
-       /// have been successfully applied, ChannelManager::channel_monitor_updated can be used to
-       /// restore the channel to an operational state.
-       ///
-       /// Note that a given ChannelManager will *never* re-generate a given ChannelMonitorUpdate. If
-       /// you return a TemporaryFailure you must ensure that it is written to disk safely before
-       /// writing out the latest ChannelManager state.
-       ///
-       /// Even when a channel has been "frozen" updates to the ChannelMonitor can continue to occur
-       /// (eg if an inbound HTLC which we forwarded was claimed upstream resulting in us attempting
-       /// to claim it on this channel) and those updates must be applied wherever they can be. At
-       /// least one such updated ChannelMonitor must be persisted otherwise PermanentFailure should
-       /// be returned to get things on-chain ASAP using only the in-memory copy. Obviously updates to
-       /// the channel which would invalidate previous ChannelMonitors are not made when a channel has
-       /// been "frozen".
-       ///
-       /// Note that even if updates made after TemporaryFailure succeed you must still call
-       /// channel_monitor_updated to ensure you have the latest monitor and re-enable normal channel
-       /// operation.
-       ///
-       /// Note that the update being processed here will not be replayed for you when you call
-       /// ChannelManager::channel_monitor_updated, so you must store the update itself along
-       /// with the persisted ChannelMonitor on your own local disk prior to returning a
-       /// TemporaryFailure. You may, of course, employ a journaling approach, storing only the
-       /// ChannelMonitorUpdate on disk without updating the monitor itself, replaying the journal at
-       /// reload-time.
-       ///
-       /// For deployments where a copy of ChannelMonitors and other local state are backed up in a
-       /// remote location (with local copies persisted immediately), it is anticipated that all
-       /// updates will return TemporaryFailure until the remote copies could be updated.
-       TemporaryFailure,
-       /// Used to indicate no further channel monitor updates will be allowed (eg we've moved on to a
-       /// different watchtower and cannot update with all watchtowers that were previously informed
-       /// of this channel).
-       ///
-       /// At reception of this error, ChannelManager will force-close the channel and return at
-       /// least a final ChannelMonitorUpdate::ChannelForceClosed which must be delivered to at
-       /// least one ChannelMonitor copy. Revocation secret MUST NOT be released and offchain channel
-       /// update must be rejected.
-       ///
-       /// This failure may also signal a failure to update the local persisted copy of one of
-       /// the channel monitor instance.
-       ///
-       /// Note that even when you fail a holder commitment transaction update, you must store the
-       /// update to ensure you can claim from it in case of a duplicate copy of this ChannelMonitor
-       /// broadcasts it (e.g distributed channel-monitor deployment)
-       ///
-       /// In case of distributed watchtowers deployment, the new version must be written to disk, as
-       /// state may have been stored but rejected due to a block forcing a commitment broadcast. This
-       /// storage is used to claim outputs of rejected state confirmed onchain by another watchtower,
-       /// lagging behind on block processing.
-       PermanentFailure,
-}
-
 /// General Err type for ChannelMonitor actions. Generally, this implies that the data provided is
 /// inconsistent with the ChannelMonitor being called. eg for ChannelMonitor::update_monitor this
 /// means you tried to update a monitor for a different channel or the ChannelMonitorUpdate was
@@ -192,7 +131,40 @@ pub enum MonitorEvent {
 
        /// A monitor event that the Channel's commitment transaction was confirmed.
        CommitmentTxConfirmed(OutPoint),
+
+       /// Indicates a [`ChannelMonitor`] update has completed. See
+       /// [`ChannelMonitorUpdateErr::TemporaryFailure`] for more information on how this is used.
+       ///
+       /// [`ChannelMonitorUpdateErr::TemporaryFailure`]: super::ChannelMonitorUpdateErr::TemporaryFailure
+       UpdateCompleted {
+               /// The funding outpoint of the [`ChannelMonitor`] that was updated
+               funding_txo: OutPoint,
+               /// The Update ID from [`ChannelMonitorUpdate::update_id`] which was applied or
+               /// [`ChannelMonitor::get_latest_update_id`].
+               ///
+               /// Note that this should only be set to a given update's ID if all previous updates for the
+               /// same [`ChannelMonitor`] have been applied and persisted.
+               monitor_update_id: u64,
+       },
+
+       /// Indicates a [`ChannelMonitor`] update has failed. See
+       /// [`ChannelMonitorUpdateErr::PermanentFailure`] for more information on how this is used.
+       ///
+       /// [`ChannelMonitorUpdateErr::PermanentFailure`]: super::ChannelMonitorUpdateErr::PermanentFailure
+       UpdateFailed(OutPoint),
 }
+impl_writeable_tlv_based_enum_upgradable!(MonitorEvent,
+       // Note that UpdateCompleted and UpdateFailed are currently never serialized to disk as they are
+       // generated only in ChainMonitor
+       (0, UpdateCompleted) => {
+               (0, funding_txo, required),
+               (2, monitor_update_id, required),
+       },
+;
+       (2, HTLCEvent),
+       (4, CommitmentTxConfirmed),
+       (6, UpdateFailed),
+);
 
 /// Simple structure sent back by `chain::Watch` when an HTLC from a forward channel is detected on
 /// chain. Used to update the corresponding HTLC in the backward channel. Failing to pass the
@@ -680,7 +652,17 @@ pub(crate) struct ChannelMonitorImpl<Signer: Sign> {
 
        payment_preimages: HashMap<PaymentHash, PaymentPreimage>,
 
+       // Note that `MonitorEvent`s MUST NOT be generated during update processing, only generated
+       // during chain data processing. This prevents a race in `ChainMonitor::update_channel` (and
+       // presumably user implementations thereof as well) where we update the in-memory channel
+       // object, then before the persistence finishes (as it's all under a read-lock), we return
+       // pending events to the user or to the relevant `ChannelManager`. Then, on reload, we'll have
+       // the pre-event state here, but have processed the event in the `ChannelManager`.
+       // Note that because the `event_lock` in `ChainMonitor` is only taken in
+       // block/transaction-connected events and *not* during block/transaction-disconnected events,
+       // we further MUST NOT generate events during block/transaction-disconnection.
        pending_monitor_events: Vec<MonitorEvent>,
+
        pending_events: Vec<Event>,
 
        // Used to track on-chain events (i.e., transactions part of channels confirmed on chain) on
@@ -906,14 +888,19 @@ impl<Signer: Sign> Writeable for ChannelMonitorImpl<Signer> {
                        writer.write_all(&payment_preimage.0[..])?;
                }
 
-               writer.write_all(&byte_utils::be64_to_array(self.pending_monitor_events.len() as u64))?;
+               writer.write_all(&(self.pending_monitor_events.iter().filter(|ev| match ev {
+                       MonitorEvent::HTLCEvent(_) => true,
+                       MonitorEvent::CommitmentTxConfirmed(_) => true,
+                       _ => false,
+               }).count() as u64).to_be_bytes())?;
                for event in self.pending_monitor_events.iter() {
                        match event {
                                MonitorEvent::HTLCEvent(upd) => {
                                        0u8.write(writer)?;
                                        upd.write(writer)?;
                                },
-                               MonitorEvent::CommitmentTxConfirmed(_) => 1u8.write(writer)?
+                               MonitorEvent::CommitmentTxConfirmed(_) => 1u8.write(writer)?,
+                               _ => {}, // Covered in the TLV writes below
                        }
                }
 
@@ -947,6 +934,7 @@ impl<Signer: Sign> Writeable for ChannelMonitorImpl<Signer> {
                write_tlv_fields!(writer, {
                        (1, self.funding_spend_confirmed, option),
                        (3, self.htlcs_resolved_on_chain, vec_type),
+                       (5, self.pending_monitor_events, vec_type),
                });
 
                Ok(())
@@ -1522,6 +1510,101 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
 
                res
        }
+
+       /// Gets the set of outbound HTLCs which are pending resolution in this channel.
+       /// This is used to reconstruct pending outbound payments on restart in the ChannelManager.
+       pub(crate) fn get_pending_outbound_htlcs(&self) -> HashMap<HTLCSource, HTLCOutputInCommitment> {
+               let mut res = HashMap::new();
+               let us = self.inner.lock().unwrap();
+
+               macro_rules! walk_htlcs {
+                       ($holder_commitment: expr, $htlc_iter: expr) => {
+                               for (htlc, source) in $htlc_iter {
+                                       if us.htlcs_resolved_on_chain.iter().any(|v| Some(v.input_idx) == htlc.transaction_output_index) {
+                                               // We should assert that funding_spend_confirmed is_some() here, but we
+                                               // have some unit tests which violate HTLC transaction CSVs entirely and
+                                               // would fail.
+                                               // TODO: Once tests all connect transactions at consensus-valid times, we
+                                               // should assert here like we do in `get_claimable_balances`.
+                                       } else if htlc.offered == $holder_commitment {
+                                               // If the payment was outbound, check if there's an HTLCUpdate
+                                               // indicating we have spent this HTLC with a timeout, claiming it back
+                                               // and awaiting confirmations on it.
+                                               let htlc_update_confd = us.onchain_events_awaiting_threshold_conf.iter().any(|event| {
+                                                       if let OnchainEvent::HTLCUpdate { input_idx: Some(input_idx), .. } = event.event {
+                                                               // If the HTLC was timed out, we wait for ANTI_REORG_DELAY blocks
+                                                               // before considering it "no longer pending" - this matches when we
+                                                               // provide the ChannelManager an HTLC failure event.
+                                                               Some(input_idx) == htlc.transaction_output_index &&
+                                                                       us.best_block.height() >= event.height + ANTI_REORG_DELAY - 1
+                                                       } else if let OnchainEvent::HTLCSpendConfirmation { input_idx, .. } = event.event {
+                                                               // If the HTLC was fulfilled with a preimage, we consider the HTLC
+                                                               // immediately non-pending, matching when we provide ChannelManager
+                                                               // the preimage.
+                                                               Some(input_idx) == htlc.transaction_output_index
+                                                       } else { false }
+                                               });
+                                               if !htlc_update_confd {
+                                                       res.insert(source.clone(), htlc.clone());
+                                               }
+                                       }
+                               }
+                       }
+               }
+
+               // We're only concerned with the confirmation count of HTLC transactions, and don't
+               // actually care how many confirmations a commitment transaction may or may not have. Thus,
+               // we look for either a FundingSpendConfirmation event or a funding_spend_confirmed.
+               let confirmed_txid = us.funding_spend_confirmed.or_else(|| {
+                       us.onchain_events_awaiting_threshold_conf.iter().find_map(|event| {
+                               if let OnchainEvent::FundingSpendConfirmation { .. } = event.event {
+                                       Some(event.txid)
+                               } else { None }
+                       })
+               });
+               if let Some(txid) = confirmed_txid {
+                       if Some(txid) == us.current_counterparty_commitment_txid || Some(txid) == us.prev_counterparty_commitment_txid {
+                               walk_htlcs!(false, us.counterparty_claimable_outpoints.get(&txid).unwrap().iter().filter_map(|(a, b)| {
+                                       if let &Some(ref source) = b {
+                                               Some((a, &**source))
+                                       } else { None }
+                               }));
+                       } else if txid == us.current_holder_commitment_tx.txid {
+                               walk_htlcs!(true, us.current_holder_commitment_tx.htlc_outputs.iter().filter_map(|(a, _, c)| {
+                                       if let Some(source) = c { Some((a, source)) } else { None }
+                               }));
+                       } else if let Some(prev_commitment) = &us.prev_holder_signed_commitment_tx {
+                               if txid == prev_commitment.txid {
+                                       walk_htlcs!(true, prev_commitment.htlc_outputs.iter().filter_map(|(a, _, c)| {
+                                               if let Some(source) = c { Some((a, source)) } else { None }
+                                       }));
+                               }
+                       }
+               } else {
+                       // If we have not seen a commitment transaction on-chain (ie the channel is not yet
+                       // closed), just examine the available counterparty commitment transactions. See docs
+                       // on `fail_unbroadcast_htlcs`, below, for justification.
+                       macro_rules! walk_counterparty_commitment {
+                               ($txid: expr) => {
+                                       if let Some(ref latest_outpoints) = us.counterparty_claimable_outpoints.get($txid) {
+                                               for &(ref htlc, ref source_option) in latest_outpoints.iter() {
+                                                       if let &Some(ref source) = source_option {
+                                                               res.insert((**source).clone(), htlc.clone());
+                                                       }
+                                               }
+                                       }
+                               }
+                       }
+                       if let Some(ref txid) = us.current_counterparty_commitment_txid {
+                               walk_counterparty_commitment!(txid);
+                       }
+                       if let Some(ref txid) = us.prev_counterparty_commitment_txid {
+                               walk_counterparty_commitment!(txid);
+                       }
+               }
+
+               res
+       }
 }
 
 /// Compares a broadcasted commitment transaction's HTLCs with those in the latest state,
@@ -2610,8 +2693,10 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                        let revocation_sig_claim = (input.witness.len() == 3 && HTLCType::scriptlen_to_htlctype(input.witness[2].len()) == Some(HTLCType::OfferedHTLC) && input.witness[1].len() == 33)
                                || (input.witness.len() == 3 && HTLCType::scriptlen_to_htlctype(input.witness[2].len()) == Some(HTLCType::AcceptedHTLC) && input.witness[1].len() == 33);
                        let accepted_preimage_claim = input.witness.len() == 5 && HTLCType::scriptlen_to_htlctype(input.witness[4].len()) == Some(HTLCType::AcceptedHTLC);
+                       #[cfg(not(fuzzing))]
                        let accepted_timeout_claim = input.witness.len() == 3 && HTLCType::scriptlen_to_htlctype(input.witness[2].len()) == Some(HTLCType::AcceptedHTLC) && !revocation_sig_claim;
                        let offered_preimage_claim = input.witness.len() == 3 && HTLCType::scriptlen_to_htlctype(input.witness[2].len()) == Some(HTLCType::OfferedHTLC) && !revocation_sig_claim;
+                       #[cfg(not(fuzzing))]
                        let offered_timeout_claim = input.witness.len() == 5 && HTLCType::scriptlen_to_htlctype(input.witness[4].len()) == Some(HTLCType::OfferedHTLC);
 
                        let mut payment_preimage = PaymentPreimage([0; 32]);
@@ -2870,53 +2955,6 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
        }
 }
 
-/// `Persist` defines behavior for persisting channel monitors: this could mean
-/// writing once to disk, and/or uploading to one or more backup services.
-///
-/// Note that for every new monitor, you **must** persist the new `ChannelMonitor`
-/// to disk/backups. And, on every update, you **must** persist either the
-/// `ChannelMonitorUpdate` or the updated monitor itself. Otherwise, there is risk
-/// of situations such as revoking a transaction, then crashing before this
-/// revocation can be persisted, then unintentionally broadcasting a revoked
-/// transaction and losing money. This is a risk because previous channel states
-/// are toxic, so it's important that whatever channel state is persisted is
-/// kept up-to-date.
-pub trait Persist<ChannelSigner: Sign> {
-       /// Persist a new channel's data. The data can be stored any way you want, but
-       /// the identifier provided by Rust-Lightning is the channel's outpoint (and
-       /// it is up to you to maintain a correct mapping between the outpoint and the
-       /// stored channel data). Note that you **must** persist every new monitor to
-       /// disk. See the `Persist` trait documentation for more details.
-       ///
-       /// See [`ChannelMonitor::write`] for writing out a `ChannelMonitor`,
-       /// and [`ChannelMonitorUpdateErr`] for requirements when returning errors.
-       fn persist_new_channel(&self, id: OutPoint, data: &ChannelMonitor<ChannelSigner>) -> Result<(), ChannelMonitorUpdateErr>;
-
-       /// Update one channel's data. The provided `ChannelMonitor` has already
-       /// applied the given update.
-       ///
-       /// Note that on every update, you **must** persist either the
-       /// `ChannelMonitorUpdate` or the updated monitor itself to disk/backups. See
-       /// the `Persist` trait documentation for more details.
-       ///
-       /// If an implementer chooses to persist the updates only, they need to make
-       /// sure that all the updates are applied to the `ChannelMonitors` *before*
-       /// the set of channel monitors is given to the `ChannelManager`
-       /// deserialization routine. See [`ChannelMonitor::update_monitor`] for
-       /// applying a monitor update to a monitor. If full `ChannelMonitors` are
-       /// persisted, then there is no need to persist individual updates.
-       ///
-       /// Note that there could be a performance tradeoff between persisting complete
-       /// channel monitors on every update vs. persisting only updates and applying
-       /// them in batches. The size of each monitor grows `O(number of state updates)`
-       /// whereas updates are small and `O(1)`.
-       ///
-       /// See [`ChannelMonitor::write`] for writing out a `ChannelMonitor`,
-       /// [`ChannelMonitorUpdate::write`] for writing out an update, and
-       /// [`ChannelMonitorUpdateErr`] for requirements when returning errors.
-       fn update_persisted_channel(&self, id: OutPoint, update: &ChannelMonitorUpdate, data: &ChannelMonitor<ChannelSigner>) -> Result<(), ChannelMonitorUpdateErr>;
-}
-
 impl<Signer: Sign, T: Deref, F: Deref, L: Deref> chain::Listen for (ChannelMonitor<Signer>, T, F, L)
 where
        T::Target: BroadcasterInterface,
@@ -3101,14 +3139,15 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
                }
 
                let pending_monitor_events_len: u64 = Readable::read(reader)?;
-               let mut pending_monitor_events = Vec::with_capacity(cmp::min(pending_monitor_events_len as usize, MAX_ALLOC_SIZE / (32 + 8*3)));
+               let mut pending_monitor_events = Some(
+                       Vec::with_capacity(cmp::min(pending_monitor_events_len as usize, MAX_ALLOC_SIZE / (32 + 8*3))));
                for _ in 0..pending_monitor_events_len {
                        let ev = match <u8 as Readable>::read(reader)? {
                                0 => MonitorEvent::HTLCEvent(Readable::read(reader)?),
                                1 => MonitorEvent::CommitmentTxConfirmed(funding_info.0),
                                _ => return Err(DecodeError::InvalidValue)
                        };
-                       pending_monitor_events.push(ev);
+                       pending_monitor_events.as_mut().unwrap().push(ev);
                }
 
                let pending_events_len: u64 = Readable::read(reader)?;
@@ -3169,6 +3208,7 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
                read_tlv_fields!(reader, {
                        (1, funding_spend_confirmed, option),
                        (3, htlcs_resolved_on_chain, vec_type),
+                       (5, pending_monitor_events, vec_type),
                });
 
                let mut secp_ctx = Secp256k1::new();
@@ -3208,7 +3248,7 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
                                current_holder_commitment_number,
 
                                payment_preimages,
-                               pending_monitor_events,
+                               pending_monitor_events: pending_monitor_events.unwrap(),
                                pending_events,
 
                                onchain_events_awaiting_threshold_conf,