Merge pull request #1435 from TheBlueMatt/2022-04-1126-first-step
[rust-lightning] / lightning / src / chain / channelmonitor.rs
index 9c381a350309f1f94d099203f6926156aceca9d9..681d895f27e05fa6ef42105f468bae7a1a258250 100644 (file)
@@ -20,7 +20,7 @@
 //! security-domain-separated system design, you should consider having multiple paths for
 //! ChannelMonitors to get out of the HSM and onto monitoring devices.
 
-use bitcoin::blockdata::block::{Block, BlockHeader};
+use bitcoin::blockdata::block::BlockHeader;
 use bitcoin::blockdata::transaction::{TxOut,Transaction};
 use bitcoin::blockdata::script::{Script, Builder};
 use bitcoin::blockdata::opcodes;
@@ -59,7 +59,7 @@ use sync::Mutex;
 
 /// An update generated by the underlying Channel itself which contains some new information the
 /// ChannelMonitor should be made aware of.
-#[cfg_attr(any(test, feature = "fuzztarget", feature = "_test_utils"), derive(PartialEq))]
+#[cfg_attr(any(test, fuzzing, feature = "_test_utils"), derive(PartialEq))]
 #[derive(Clone)]
 #[must_use]
 pub struct ChannelMonitorUpdate {
@@ -115,84 +115,48 @@ 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
-/// corrupted.
-/// Contains a developer-readable error message.
-#[derive(Clone, Debug)]
-pub struct MonitorUpdateError(pub &'static str);
-
 /// An event to be processed by the ChannelManager.
 #[derive(Clone, PartialEq)]
 pub enum MonitorEvent {
        /// A monitor event containing an HTLCUpdate.
        HTLCEvent(HTLCUpdate),
 
-       /// A monitor event that the Channel's commitment transaction was broadcasted.
-       CommitmentTxBroadcasted(OutPoint),
+       /// 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
@@ -232,8 +196,13 @@ pub(crate) const CLTV_CLAIM_BUFFER: u32 = 18;
 /// with at worst this delay, so we are not only using this value as a mercy for them but also
 /// us as a safeguard to delay with enough time.
 pub(crate) const LATENCY_GRACE_PERIOD_BLOCKS: u32 = 3;
-/// Number of blocks we wait on seeing a HTLC output being solved before we fail corresponding inbound
-/// HTLCs. This prevents us from failing backwards and then getting a reorg resulting in us losing money.
+/// Number of blocks we wait on seeing a HTLC output being solved before we fail corresponding
+/// inbound HTLCs. This prevents us from failing backwards and then getting a reorg resulting in us
+/// losing money.
+///
+/// Note that this is a library-wide security assumption. If a reorg deeper than this number of
+/// blocks occurs, counterparties may be able to steal funds or claims made by and balances exposed
+/// by a  [`ChannelMonitor`] may be incorrect.
 // We also use this delay to be sure we can remove our in-flight claim txn from bump candidates buffer.
 // It may cause spurious generation of bumped claim txn but that's alright given the outpoint is already
 // solved by a previous claim tx. What we want to avoid is reorg evicting our claim tx and us not
@@ -248,8 +217,6 @@ pub const ANTI_REORG_DELAY: u32 = 6;
 ///    fail this HTLC,
 /// 2) if we receive an HTLC within this many blocks of its expiry (plus one to avoid a race
 ///    condition with the above), we will fail this HTLC without telling the user we received it,
-/// 3) if we are waiting on a connection or a channel state update to send an HTLC to a peer, and
-///    that HTLC expires within this many blocks, we will simply fail the HTLC instead.
 ///
 /// (1) is all about protecting us - we need enough time to update the channel state before we hit
 /// CLTV_CLAIM_BUFFER, at which point we'd go on chain to claim the HTLC with the preimage.
@@ -257,9 +224,6 @@ pub const ANTI_REORG_DELAY: u32 = 6;
 /// (2) is the same, but with an additional buffer to avoid accepting an HTLC which is immediately
 /// in a race condition between the user connecting a block (which would fail it) and the user
 /// providing us the preimage (which would claim it).
-///
-/// (3) is about our counterparty - we don't want to relay an HTLC to a counterparty when they may
-/// end up force-closing the channel on us to claim it.
 pub(crate) const HTLC_FAIL_BACK_BUFFER: u32 = CLTV_CLAIM_BUFFER + LATENCY_GRACE_PERIOD_BLOCKS;
 
 // TODO(devrandom) replace this with HolderCommitmentTransaction
@@ -272,11 +236,15 @@ struct HolderSignedTx {
        b_htlc_key: PublicKey,
        delayed_payment_key: PublicKey,
        per_commitment_point: PublicKey,
-       feerate_per_kw: u32,
        htlc_outputs: Vec<(HTLCOutputInCommitment, Option<Signature>, Option<HTLCSource>)>,
+       to_self_value_sat: u64,
+       feerate_per_kw: u32,
 }
 impl_writeable_tlv_based!(HolderSignedTx, {
        (0, txid, required),
+       // Note that this is filled in with data from OnchainTxHandler if it's missing.
+       // For HolderSignedTx objects serialized with 0.0.100+, this should be filled in.
+       (1, to_self_value_sat, (default_value, u64::max_value())),
        (2, revocation_key, required),
        (4, a_htlc_key, required),
        (6, b_htlc_key, required),
@@ -286,26 +254,18 @@ impl_writeable_tlv_based!(HolderSignedTx, {
        (14, htlc_outputs, vec_type)
 });
 
-/// We use this to track counterparty commitment transactions and htlcs outputs and
-/// use it to generate any justice or 2nd-stage preimage/timeout transactions.
+/// We use this to track static counterparty commitment transaction data and to generate any
+/// justice or 2nd-stage preimage/timeout transactions.
 #[derive(PartialEq)]
-struct CounterpartyCommitmentTransaction {
+struct CounterpartyCommitmentParameters {
        counterparty_delayed_payment_base_key: PublicKey,
        counterparty_htlc_base_key: PublicKey,
        on_counterparty_tx_csv: u16,
-       per_htlc: HashMap<Txid, Vec<HTLCOutputInCommitment>>
 }
 
-impl Writeable for CounterpartyCommitmentTransaction {
+impl Writeable for CounterpartyCommitmentParameters {
        fn write<W: Writer>(&self, w: &mut W) -> Result<(), io::Error> {
-               w.write_all(&byte_utils::be64_to_array(self.per_htlc.len() as u64))?;
-               for (ref txid, ref htlcs) in self.per_htlc.iter() {
-                       w.write_all(&txid[..])?;
-                       w.write_all(&byte_utils::be64_to_array(htlcs.len() as u64))?;
-                       for &ref htlc in htlcs.iter() {
-                               htlc.write(w)?;
-                       }
-               }
+               w.write_all(&byte_utils::be64_to_array(0))?;
                write_tlv_fields!(w, {
                        (0, self.counterparty_delayed_payment_base_key, required),
                        (2, self.counterparty_htlc_base_key, required),
@@ -314,23 +274,20 @@ impl Writeable for CounterpartyCommitmentTransaction {
                Ok(())
        }
 }
-impl Readable for CounterpartyCommitmentTransaction {
+impl Readable for CounterpartyCommitmentParameters {
        fn read<R: io::Read>(r: &mut R) -> Result<Self, DecodeError> {
                let counterparty_commitment_transaction = {
+                       // Versions prior to 0.0.100 had some per-HTLC state stored here, which is no longer
+                       // used. Read it for compatibility.
                        let per_htlc_len: u64 = Readable::read(r)?;
-                       let mut per_htlc = HashMap::with_capacity(cmp::min(per_htlc_len as usize, MAX_ALLOC_SIZE / 64));
                        for _  in 0..per_htlc_len {
-                               let txid: Txid = Readable::read(r)?;
+                               let _txid: Txid = Readable::read(r)?;
                                let htlcs_count: u64 = Readable::read(r)?;
-                               let mut htlcs = Vec::with_capacity(cmp::min(htlcs_count as usize, MAX_ALLOC_SIZE / 32));
                                for _ in 0..htlcs_count {
-                                       let htlc = Readable::read(r)?;
-                                       htlcs.push(htlc);
-                               }
-                               if let Some(_) = per_htlc.insert(txid, htlcs) {
-                                       return Err(DecodeError::InvalidValue);
+                                       let _htlc: HTLCOutputInCommitment = Readable::read(r)?;
                                }
                        }
+
                        let mut counterparty_delayed_payment_base_key = OptionDeserWrapper(None);
                        let mut counterparty_htlc_base_key = OptionDeserWrapper(None);
                        let mut on_counterparty_tx_csv: u16 = 0;
@@ -339,11 +296,10 @@ impl Readable for CounterpartyCommitmentTransaction {
                                (2, counterparty_htlc_base_key, required),
                                (4, on_counterparty_tx_csv, required),
                        });
-                       CounterpartyCommitmentTransaction {
+                       CounterpartyCommitmentParameters {
                                counterparty_delayed_payment_base_key: counterparty_delayed_payment_base_key.0.unwrap(),
                                counterparty_htlc_base_key: counterparty_htlc_base_key.0.unwrap(),
                                on_counterparty_tx_csv,
-                               per_htlc,
                        }
                };
                Ok(counterparty_commitment_transaction)
@@ -364,12 +320,21 @@ struct OnchainEventEntry {
 impl OnchainEventEntry {
        fn confirmation_threshold(&self) -> u32 {
                let mut conf_threshold = self.height + ANTI_REORG_DELAY - 1;
-               if let OnchainEvent::MaturingOutput {
-                       descriptor: SpendableOutputDescriptor::DelayedPaymentOutput(ref descriptor)
-               } = self.event {
-                       // A CSV'd transaction is confirmable in block (input height) + CSV delay, which means
-                       // it's broadcastable when we see the previous block.
-                       conf_threshold = cmp::max(conf_threshold, self.height + descriptor.to_self_delay as u32 - 1);
+               match self.event {
+                       OnchainEvent::MaturingOutput {
+                               descriptor: SpendableOutputDescriptor::DelayedPaymentOutput(ref descriptor)
+                       } => {
+                               // A CSV'd transaction is confirmable in block (input height) + CSV delay, which means
+                               // it's broadcastable when we see the previous block.
+                               conf_threshold = cmp::max(conf_threshold, self.height + descriptor.to_self_delay as u32 - 1);
+                       },
+                       OnchainEvent::FundingSpendConfirmation { on_local_output_csv: Some(csv), .. } |
+                       OnchainEvent::HTLCSpendConfirmation { on_to_local_output_csv: Some(csv), .. } => {
+                               // A CSV'd transaction is confirmable in block (input height) + CSV delay, which means
+                               // it's broadcastable when we see the previous block.
+                               conf_threshold = cmp::max(conf_threshold, self.height + csv as u32 - 1);
+                       },
+                       _ => {},
                }
                conf_threshold
        }
@@ -383,17 +348,47 @@ impl OnchainEventEntry {
 /// once they mature to enough confirmations (ANTI_REORG_DELAY)
 #[derive(PartialEq)]
 enum OnchainEvent {
-       /// HTLC output getting solved by a timeout, at maturation we pass upstream payment source information to solve
-       /// inbound HTLC in backward channel. Note, in case of preimage, we pass info to upstream without delay as we can
-       /// only win from it, so it's never an OnchainEvent
+       /// An outbound HTLC failing after a transaction is confirmed. Used
+       ///  * when an outbound HTLC output is spent by us after the HTLC timed out
+       ///  * an outbound HTLC which was not present in the commitment transaction which appeared
+       ///    on-chain (either because it was not fully committed to or it was dust).
+       /// Note that this is *not* used for preimage claims, as those are passed upstream immediately,
+       /// appearing only as an `HTLCSpendConfirmation`, below.
        HTLCUpdate {
                source: HTLCSource,
                payment_hash: PaymentHash,
                onchain_value_satoshis: Option<u64>,
+               /// None in the second case, above, ie when there is no relevant output in the commitment
+               /// transaction which appeared on chain.
+               input_idx: Option<u32>,
        },
        MaturingOutput {
                descriptor: SpendableOutputDescriptor,
        },
+       /// A spend of the funding output, either a commitment transaction or a cooperative closing
+       /// transaction.
+       FundingSpendConfirmation {
+               /// The CSV delay for the output of the funding spend transaction (implying it is a local
+               /// commitment transaction, and this is the delay on the to_self output).
+               on_local_output_csv: Option<u16>,
+       },
+       /// A spend of a commitment transaction HTLC output, set in the cases where *no* `HTLCUpdate`
+       /// is constructed. This is used when
+       ///  * an outbound HTLC is claimed by our counterparty with a preimage, causing us to
+       ///    immediately claim the HTLC on the inbound edge and track the resolution here,
+       ///  * an inbound HTLC is claimed by our counterparty (with a timeout),
+       ///  * an inbound HTLC is claimed by us (with a preimage).
+       ///  * a revoked-state HTLC transaction was broadcasted, which was claimed by the revocation
+       ///    signature.
+       HTLCSpendConfirmation {
+               input_idx: u32,
+               /// If the claim was made by either party with a preimage, this is filled in
+               preimage: Option<PaymentPreimage>,
+               /// If the claim was made by us on an inbound HTLC against a local commitment transaction,
+               /// we set this to the output CSV value which we will have to wait until to spend the
+               /// output (and generate a SpendableOutput event).
+               on_to_local_output_csv: Option<u16>,
+       },
 }
 
 impl Writeable for OnchainEventEntry {
@@ -430,13 +425,23 @@ impl_writeable_tlv_based_enum_upgradable!(OnchainEvent,
                (0, source, required),
                (1, onchain_value_satoshis, option),
                (2, payment_hash, required),
+               (3, input_idx, option),
        },
        (1, MaturingOutput) => {
                (0, descriptor, required),
        },
+       (3, FundingSpendConfirmation) => {
+               (0, on_local_output_csv, option),
+       },
+       (5, HTLCSpendConfirmation) => {
+               (0, input_idx, required),
+               (2, preimage, option),
+               (4, on_to_local_output_csv, option),
+       },
+
 );
 
-#[cfg_attr(any(test, feature = "fuzztarget", feature = "_test_utils"), derive(PartialEq))]
+#[cfg_attr(any(test, fuzzing, feature = "_test_utils"), derive(PartialEq))]
 #[derive(Clone)]
 pub(crate) enum ChannelMonitorUpdateStep {
        LatestHolderCommitmentTXInfo {
@@ -468,6 +473,19 @@ pub(crate) enum ChannelMonitorUpdateStep {
        },
 }
 
+impl ChannelMonitorUpdateStep {
+       fn variant_name(&self) -> &'static str {
+               match self {
+                       ChannelMonitorUpdateStep::LatestHolderCommitmentTXInfo { .. } => "LatestHolderCommitmentTXInfo",
+                       ChannelMonitorUpdateStep::LatestCounterpartyCommitmentTXInfo { .. } => "LatestCounterpartyCommitmentTXInfo",
+                       ChannelMonitorUpdateStep::PaymentPreimage { .. } => "PaymentPreimage",
+                       ChannelMonitorUpdateStep::CommitmentSecret { .. } => "CommitmentSecret",
+                       ChannelMonitorUpdateStep::ChannelForceClosed { .. } => "ChannelForceClosed",
+                       ChannelMonitorUpdateStep::ShutdownScript { .. } => "ShutdownScript",
+               }
+       }
+}
+
 impl_writeable_tlv_based_enum_upgradable!(ChannelMonitorUpdateStep,
        (0, LatestHolderCommitmentTXInfo) => {
                (0, commitment_tx, required),
@@ -494,6 +512,72 @@ impl_writeable_tlv_based_enum_upgradable!(ChannelMonitorUpdateStep,
        },
 );
 
+/// Details about the balance(s) available for spending once the channel appears on chain.
+///
+/// See [`ChannelMonitor::get_claimable_balances`] for more details on when these will or will not
+/// be provided.
+#[derive(Clone, Debug, PartialEq, Eq)]
+#[cfg_attr(test, derive(PartialOrd, Ord))]
+pub enum Balance {
+       /// The channel is not yet closed (or the commitment or closing transaction has not yet
+       /// appeared in a block). The given balance is claimable (less on-chain fees) if the channel is
+       /// force-closed now.
+       ClaimableOnChannelClose {
+               /// The amount available to claim, in satoshis, excluding the on-chain fees which will be
+               /// required to do so.
+               claimable_amount_satoshis: u64,
+       },
+       /// The channel has been closed, and the given balance is ours but awaiting confirmations until
+       /// we consider it spendable.
+       ClaimableAwaitingConfirmations {
+               /// The amount available to claim, in satoshis, possibly excluding the on-chain fees which
+               /// were spent in broadcasting the transaction.
+               claimable_amount_satoshis: u64,
+               /// The height at which an [`Event::SpendableOutputs`] event will be generated for this
+               /// amount.
+               confirmation_height: u32,
+       },
+       /// The channel has been closed, and the given balance should be ours but awaiting spending
+       /// transaction confirmation. If the spending transaction does not confirm in time, it is
+       /// possible our counterparty can take the funds by broadcasting an HTLC timeout on-chain.
+       ///
+       /// Once the spending transaction confirms, before it has reached enough confirmations to be
+       /// considered safe from chain reorganizations, the balance will instead be provided via
+       /// [`Balance::ClaimableAwaitingConfirmations`].
+       ContentiousClaimable {
+               /// The amount available to claim, in satoshis, excluding the on-chain fees which will be
+               /// required to do so.
+               claimable_amount_satoshis: u64,
+               /// The height at which the counterparty may be able to claim the balance if we have not
+               /// done so.
+               timeout_height: u32,
+       },
+       /// HTLCs which we sent to our counterparty which are claimable after a timeout (less on-chain
+       /// fees) if the counterparty does not know the preimage for the HTLCs. These are somewhat
+       /// likely to be claimed by our counterparty before we do.
+       MaybeClaimableHTLCAwaitingTimeout {
+               /// The amount available to claim, in satoshis, excluding the on-chain fees which will be
+               /// required to do so.
+               claimable_amount_satoshis: u64,
+               /// The height at which we will be able to claim the balance if our counterparty has not
+               /// done so.
+               claimable_height: u32,
+       },
+}
+
+/// An HTLC which has been irrevocably resolved on-chain, and has reached ANTI_REORG_DELAY.
+#[derive(PartialEq)]
+struct IrrevocablyResolvedHTLC {
+       input_idx: u32,
+       /// Only set if the HTLC claim was ours using a payment preimage
+       payment_preimage: Option<PaymentPreimage>,
+}
+
+impl_writeable_tlv_based!(IrrevocablyResolvedHTLC, {
+       (0, input_idx, required),
+       (2, payment_preimage, option),
+});
+
 /// A ChannelMonitor handles chain events (blocks connected and disconnected) and generates
 /// on-chain transactions to ensure no loss of funds occurs.
 ///
@@ -532,7 +616,7 @@ pub(crate) struct ChannelMonitorImpl<Signer: Sign> {
        current_counterparty_commitment_txid: Option<Txid>,
        prev_counterparty_commitment_txid: Option<Txid>,
 
-       counterparty_tx_cache: CounterpartyCommitmentTransaction,
+       counterparty_commitment_params: CounterpartyCommitmentParameters,
        funding_redeemscript: Script,
        channel_value_satoshis: u64,
        // first is the idx of the first of the two revocation points
@@ -573,7 +657,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
@@ -606,6 +700,17 @@ pub(crate) struct ChannelMonitorImpl<Signer: Sign> {
        // remote monitor out-of-order with regards to the block view.
        holder_tx_signed: bool,
 
+       // If a spend of the funding output is seen, we set this to true and reject any further
+       // updates. This prevents any further changes in the offchain state no matter the order
+       // of block connection between ChannelMonitors and the ChannelManager.
+       funding_spend_seen: bool,
+
+       funding_spend_confirmed: Option<Txid>,
+       /// The set of HTLCs which have been either claimed or failed on chain and have reached
+       /// the requisite confirmations on the claim/fail transaction (either ANTI_REORG_DELAY or the
+       /// spending CSV for revocable outputs).
+       htlcs_resolved_on_chain: Vec<IrrevocablyResolvedHTLC>,
+
        // We simply modify best_block in Channel's block_connected so that serialization is
        // consistent but hopefully the users' copy handles block_connected in a consistent way.
        // (we do *not*, however, update them in update_monitor to ensure any local user copies keep
@@ -619,9 +724,9 @@ pub(crate) struct ChannelMonitorImpl<Signer: Sign> {
 /// Transaction outputs to watch for on-chain spends.
 pub type TransactionOutputs = (Txid, Vec<(u32, TxOut)>);
 
-#[cfg(any(test, feature = "fuzztarget", feature = "_test_utils"))]
-/// Used only in testing and fuzztarget to check serialization roundtrips don't change the
-/// underlying object
+#[cfg(any(test, fuzzing, feature = "_test_utils"))]
+/// Used only in testing and fuzzing to check serialization roundtrips don't change the underlying
+/// object
 impl<Signer: Sign> PartialEq for ChannelMonitor<Signer> {
        fn eq(&self, other: &Self) -> bool {
                let inner = self.inner.lock().unwrap();
@@ -630,9 +735,9 @@ impl<Signer: Sign> PartialEq for ChannelMonitor<Signer> {
        }
 }
 
-#[cfg(any(test, feature = "fuzztarget", feature = "_test_utils"))]
-/// Used only in testing and fuzztarget to check serialization roundtrips don't change the
-/// underlying object
+#[cfg(any(test, fuzzing, feature = "_test_utils"))]
+/// Used only in testing and fuzzing to check serialization roundtrips don't change the underlying
+/// object
 impl<Signer: Sign> PartialEq for ChannelMonitorImpl<Signer> {
        fn eq(&self, other: &Self) -> bool {
                if self.latest_update_id != other.latest_update_id ||
@@ -645,7 +750,7 @@ impl<Signer: Sign> PartialEq for ChannelMonitorImpl<Signer> {
                        self.funding_info != other.funding_info ||
                        self.current_counterparty_commitment_txid != other.current_counterparty_commitment_txid ||
                        self.prev_counterparty_commitment_txid != other.prev_counterparty_commitment_txid ||
-                       self.counterparty_tx_cache != other.counterparty_tx_cache ||
+                       self.counterparty_commitment_params != other.counterparty_commitment_params ||
                        self.funding_redeemscript != other.funding_redeemscript ||
                        self.channel_value_satoshis != other.channel_value_satoshis ||
                        self.their_cur_revocation_points != other.their_cur_revocation_points ||
@@ -664,7 +769,10 @@ impl<Signer: Sign> PartialEq for ChannelMonitorImpl<Signer> {
                        self.onchain_events_awaiting_threshold_conf != other.onchain_events_awaiting_threshold_conf ||
                        self.outputs_to_watch != other.outputs_to_watch ||
                        self.lockdown_from_offchain != other.lockdown_from_offchain ||
-                       self.holder_tx_signed != other.holder_tx_signed
+                       self.holder_tx_signed != other.holder_tx_signed ||
+                       self.funding_spend_seen != other.funding_spend_seen ||
+                       self.funding_spend_confirmed != other.funding_spend_confirmed ||
+                       self.htlcs_resolved_on_chain != other.htlcs_resolved_on_chain
                {
                        false
                } else {
@@ -716,7 +824,7 @@ impl<Signer: Sign> Writeable for ChannelMonitorImpl<Signer> {
                self.current_counterparty_commitment_txid.write(writer)?;
                self.prev_counterparty_commitment_txid.write(writer)?;
 
-               self.counterparty_tx_cache.write(writer)?;
+               self.counterparty_commitment_params.write(writer)?;
                self.funding_redeemscript.write(writer)?;
                self.channel_value_satoshis.write(writer)?;
 
@@ -791,14 +899,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::CommitmentTxBroadcasted(_) => 1u8.write(writer)?
+                               MonitorEvent::CommitmentTxConfirmed(_) => 1u8.write(writer)?,
+                               _ => {}, // Covered in the TLV writes below
                        }
                }
 
@@ -829,7 +942,12 @@ impl<Signer: Sign> Writeable for ChannelMonitorImpl<Signer> {
                self.lockdown_from_offchain.write(writer)?;
                self.holder_tx_signed.write(writer)?;
 
-               write_tlv_fields!(writer, {});
+               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),
+                       (7, self.funding_spend_seen, required),
+               });
 
                Ok(())
        }
@@ -851,7 +969,7 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
                let counterparty_channel_parameters = channel_parameters.counterparty_parameters.as_ref().unwrap();
                let counterparty_delayed_payment_base_key = counterparty_channel_parameters.pubkeys.delayed_payment_basepoint;
                let counterparty_htlc_base_key = counterparty_channel_parameters.pubkeys.htlc_basepoint;
-               let counterparty_tx_cache = CounterpartyCommitmentTransaction { counterparty_delayed_payment_base_key, counterparty_htlc_base_key, on_counterparty_tx_csv, per_htlc: HashMap::new() };
+               let counterparty_commitment_params = CounterpartyCommitmentParameters { counterparty_delayed_payment_base_key, counterparty_htlc_base_key, on_counterparty_tx_csv };
 
                let channel_keys_id = keys.channel_keys_id();
                let holder_revocation_basepoint = keys.pubkeys().revocation_basepoint;
@@ -869,8 +987,9 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
                                b_htlc_key: tx_keys.countersignatory_htlc_key,
                                delayed_payment_key: tx_keys.broadcaster_delayed_payment_key,
                                per_commitment_point: tx_keys.per_commitment_point,
-                               feerate_per_kw: trusted_tx.feerate_per_kw(),
                                htlc_outputs: Vec::new(), // There are never any HTLCs in the initial commitment transactions
+                               to_self_value_sat: initial_holder_commitment_tx.to_broadcaster_value_sat(),
+                               feerate_per_kw: trusted_tx.feerate_per_kw(),
                        };
                        (holder_commitment_tx, trusted_tx.commitment_number())
                };
@@ -898,7 +1017,7 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
                                current_counterparty_commitment_txid: None,
                                prev_counterparty_commitment_txid: None,
 
-                               counterparty_tx_cache,
+                               counterparty_commitment_params,
                                funding_redeemscript,
                                channel_value_satoshis,
                                their_cur_revocation_points: None,
@@ -926,6 +1045,9 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
 
                                lockdown_from_offchain: false,
                                holder_tx_signed: false,
+                               funding_spend_seen: false,
+                               funding_spend_confirmed: None,
+                               htlcs_resolved_on_chain: Vec::new(),
 
                                best_block,
 
@@ -935,7 +1057,7 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
        }
 
        #[cfg(test)]
-       fn provide_secret(&self, idx: u64, secret: [u8; 32]) -> Result<(), MonitorUpdateError> {
+       fn provide_secret(&self, idx: u64, secret: [u8; 32]) -> Result<(), &'static str> {
                self.inner.lock().unwrap().provide_secret(idx, secret)
        }
 
@@ -957,12 +1079,10 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
 
        #[cfg(test)]
        fn provide_latest_holder_commitment_tx(
-               &self,
-               holder_commitment_tx: HolderCommitmentTransaction,
+               &self, holder_commitment_tx: HolderCommitmentTransaction,
                htlc_outputs: Vec<(HTLCOutputInCommitment, Option<Signature>, Option<HTLCSource>)>,
-       ) -> Result<(), MonitorUpdateError> {
-               self.inner.lock().unwrap().provide_latest_holder_commitment_tx(
-                       holder_commitment_tx, htlc_outputs)
+       ) -> Result<(), ()> {
+               self.inner.lock().unwrap().provide_latest_holder_commitment_tx(holder_commitment_tx, htlc_outputs).map_err(|_| ())
        }
 
        #[cfg(test)]
@@ -1003,7 +1123,7 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
                broadcaster: &B,
                fee_estimator: &F,
                logger: &L,
-       ) -> Result<(), MonitorUpdateError>
+       ) -> Result<(), ()>
        where
                B::Target: BroadcasterInterface,
                F::Target: FeeEstimator,
@@ -1234,6 +1354,283 @@ impl<Signer: Sign> ChannelMonitor<Signer> {
        pub fn current_best_block(&self) -> BestBlock {
                self.inner.lock().unwrap().best_block.clone()
        }
+
+       /// Gets the balances in this channel which are either claimable by us if we were to
+       /// force-close the channel now or which are claimable on-chain (possibly awaiting
+       /// confirmation).
+       ///
+       /// Any balances in the channel which are available on-chain (excluding on-chain fees) are
+       /// included here until an [`Event::SpendableOutputs`] event has been generated for the
+       /// balance, or until our counterparty has claimed the balance and accrued several
+       /// confirmations on the claim transaction.
+       ///
+       /// Note that the balances available when you or your counterparty have broadcasted revoked
+       /// state(s) may not be fully captured here.
+       // TODO, fix that ^
+       ///
+       /// See [`Balance`] for additional details on the types of claimable balances which
+       /// may be returned here and their meanings.
+       pub fn get_claimable_balances(&self) -> Vec<Balance> {
+               let mut res = Vec::new();
+               let us = self.inner.lock().unwrap();
+
+               let mut confirmed_txid = us.funding_spend_confirmed;
+               let mut pending_commitment_tx_conf_thresh = None;
+               let funding_spend_pending = us.onchain_events_awaiting_threshold_conf.iter().find_map(|event| {
+                       if let OnchainEvent::FundingSpendConfirmation { .. } = event.event {
+                               Some((event.txid, event.confirmation_threshold()))
+                       } else { None }
+               });
+               if let Some((txid, conf_thresh)) = funding_spend_pending {
+                       debug_assert!(us.funding_spend_confirmed.is_none(),
+                               "We have a pending funding spend awaiting anti-reorg confirmation, we can't have confirmed it already!");
+                       confirmed_txid = Some(txid);
+                       pending_commitment_tx_conf_thresh = Some(conf_thresh);
+               }
+
+               macro_rules! walk_htlcs {
+                       ($holder_commitment: expr, $htlc_iter: expr) => {
+                               for htlc in $htlc_iter {
+                                       if let Some(htlc_input_idx) = htlc.transaction_output_index {
+                                               if let Some(conf_thresh) = us.onchain_events_awaiting_threshold_conf.iter().find_map(|event| {
+                                                       if let OnchainEvent::MaturingOutput { descriptor: SpendableOutputDescriptor::DelayedPaymentOutput(descriptor) } = &event.event {
+                                                               if descriptor.outpoint.index as u32 == htlc_input_idx { Some(event.confirmation_threshold()) } else { None }
+                                                       } else { None }
+                                               }) {
+                                                       debug_assert!($holder_commitment);
+                                                       res.push(Balance::ClaimableAwaitingConfirmations {
+                                                               claimable_amount_satoshis: htlc.amount_msat / 1000,
+                                                               confirmation_height: conf_thresh,
+                                                       });
+                                               } else if us.htlcs_resolved_on_chain.iter().any(|v| v.input_idx == htlc_input_idx) {
+                                                       // Funding transaction spends should be fully confirmed by the time any
+                                                       // HTLC transactions are resolved, unless we're talking about a holder
+                                                       // commitment tx, whose resolution is delayed until the CSV timeout is
+                                                       // reached, even though HTLCs may be resolved after only
+                                                       // ANTI_REORG_DELAY confirmations.
+                                                       debug_assert!($holder_commitment || us.funding_spend_confirmed.is_some());
+                                               } 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_pending = us.onchain_events_awaiting_threshold_conf.iter().find_map(|event| {
+                                                               if let OnchainEvent::HTLCUpdate { input_idx: Some(input_idx), .. } = event.event {
+                                                                       if input_idx == htlc_input_idx { Some(event.confirmation_threshold()) } else { None }
+                                                               } else { None }
+                                                       });
+                                                       if let Some(conf_thresh) = htlc_update_pending {
+                                                               res.push(Balance::ClaimableAwaitingConfirmations {
+                                                                       claimable_amount_satoshis: htlc.amount_msat / 1000,
+                                                                       confirmation_height: conf_thresh,
+                                                               });
+                                                       } else {
+                                                               res.push(Balance::MaybeClaimableHTLCAwaitingTimeout {
+                                                                       claimable_amount_satoshis: htlc.amount_msat / 1000,
+                                                                       claimable_height: htlc.cltv_expiry,
+                                                               });
+                                                       }
+                                               } else if us.payment_preimages.get(&htlc.payment_hash).is_some() {
+                                                       // Otherwise (the payment was inbound), only expose it as claimable if
+                                                       // we know the preimage.
+                                                       // Note that if there is a pending claim, but it did not use the
+                                                       // preimage, we lost funds to our counterparty! We will then continue
+                                                       // to show it as ContentiousClaimable until ANTI_REORG_DELAY.
+                                                       let htlc_spend_pending = us.onchain_events_awaiting_threshold_conf.iter().find_map(|event| {
+                                                               if let OnchainEvent::HTLCSpendConfirmation { input_idx, preimage, .. } = event.event {
+                                                                       if input_idx == htlc_input_idx {
+                                                                               Some((event.confirmation_threshold(), preimage.is_some()))
+                                                                       } else { None }
+                                                               } else { None }
+                                                       });
+                                                       if let Some((conf_thresh, true)) = htlc_spend_pending {
+                                                               res.push(Balance::ClaimableAwaitingConfirmations {
+                                                                       claimable_amount_satoshis: htlc.amount_msat / 1000,
+                                                                       confirmation_height: conf_thresh,
+                                                               });
+                                                       } else {
+                                                               res.push(Balance::ContentiousClaimable {
+                                                                       claimable_amount_satoshis: htlc.amount_msat / 1000,
+                                                                       timeout_height: htlc.cltv_expiry,
+                                                               });
+                                                       }
+                                               }
+                                       }
+                               }
+                       }
+               }
+
+               if let Some(txid) = confirmed_txid {
+                       let mut found_commitment_tx = false;
+                       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().map(|(a, _)| a));
+                               if let Some(conf_thresh) = pending_commitment_tx_conf_thresh {
+                                       if let Some(value) = us.onchain_events_awaiting_threshold_conf.iter().find_map(|event| {
+                                               if let OnchainEvent::MaturingOutput {
+                                                       descriptor: SpendableOutputDescriptor::StaticPaymentOutput(descriptor)
+                                               } = &event.event {
+                                                       Some(descriptor.output.value)
+                                               } else { None }
+                                       }) {
+                                               res.push(Balance::ClaimableAwaitingConfirmations {
+                                                       claimable_amount_satoshis: value,
+                                                       confirmation_height: conf_thresh,
+                                               });
+                                       } else {
+                                               // If a counterparty commitment transaction is awaiting confirmation, we
+                                               // should either have a StaticPaymentOutput MaturingOutput event awaiting
+                                               // confirmation with the same height or have never met our dust amount.
+                                       }
+                               }
+                               found_commitment_tx = true;
+                       } else if txid == us.current_holder_commitment_tx.txid {
+                               walk_htlcs!(true, us.current_holder_commitment_tx.htlc_outputs.iter().map(|(a, _, _)| a));
+                               if let Some(conf_thresh) = pending_commitment_tx_conf_thresh {
+                                       res.push(Balance::ClaimableAwaitingConfirmations {
+                                               claimable_amount_satoshis: us.current_holder_commitment_tx.to_self_value_sat,
+                                               confirmation_height: conf_thresh,
+                                       });
+                               }
+                               found_commitment_tx = true;
+                       } 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().map(|(a, _, _)| a));
+                                       if let Some(conf_thresh) = pending_commitment_tx_conf_thresh {
+                                               res.push(Balance::ClaimableAwaitingConfirmations {
+                                                       claimable_amount_satoshis: prev_commitment.to_self_value_sat,
+                                                       confirmation_height: conf_thresh,
+                                               });
+                                       }
+                                       found_commitment_tx = true;
+                               }
+                       }
+                       if !found_commitment_tx {
+                               if let Some(conf_thresh) = pending_commitment_tx_conf_thresh {
+                                       // We blindly assume this is a cooperative close transaction here, and that
+                                       // neither us nor our counterparty misbehaved. At worst we've under-estimated
+                                       // the amount we can claim as we'll punish a misbehaving counterparty.
+                                       res.push(Balance::ClaimableAwaitingConfirmations {
+                                               claimable_amount_satoshis: us.current_holder_commitment_tx.to_self_value_sat,
+                                               confirmation_height: conf_thresh,
+                                       });
+                               }
+                       }
+                       // TODO: Add logic to provide claimable balances for counterparty broadcasting revoked
+                       // outputs.
+               } else {
+                       let mut claimable_inbound_htlc_value_sat = 0;
+                       for (htlc, _, _) in us.current_holder_commitment_tx.htlc_outputs.iter() {
+                               if htlc.transaction_output_index.is_none() { continue; }
+                               if htlc.offered {
+                                       res.push(Balance::MaybeClaimableHTLCAwaitingTimeout {
+                                               claimable_amount_satoshis: htlc.amount_msat / 1000,
+                                               claimable_height: htlc.cltv_expiry,
+                                       });
+                               } else if us.payment_preimages.get(&htlc.payment_hash).is_some() {
+                                       claimable_inbound_htlc_value_sat += htlc.amount_msat / 1000;
+                               }
+                       }
+                       res.push(Balance::ClaimableOnChannelClose {
+                               claimable_amount_satoshis: us.current_holder_commitment_tx.to_self_value_sat + claimable_inbound_htlc_value_sat,
+                       });
+               }
+
+               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,
@@ -1292,6 +1689,7 @@ macro_rules! fail_unbroadcast_htlcs {
                                                                        source: (**source).clone(),
                                                                        payment_hash: htlc.payment_hash.clone(),
                                                                        onchain_value_satoshis: Some(htlc.amount_msat / 1000),
+                                                                       input_idx: None,
                                                                },
                                                        };
                                                        log_trace!($logger, "Failing HTLC with payment_hash {} from {} counterparty commitment tx due to broadcast of {} commitment transaction, waiting for confirmation (at height {})",
@@ -1315,9 +1713,9 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
        /// Inserts a revocation secret into this channel monitor. Prunes old preimages if neither
        /// needed by holder commitment transactions HTCLs nor by counterparty ones. Unless we haven't already seen
        /// counterparty commitment transaction's secret, they are de facto pruned (we can use revocation key).
-       fn provide_secret(&mut self, idx: u64, secret: [u8; 32]) -> Result<(), MonitorUpdateError> {
+       fn provide_secret(&mut self, idx: u64, secret: [u8; 32]) -> Result<(), &'static str> {
                if let Err(()) = self.commitment_secrets.provide_secret(idx, secret) {
-                       return Err(MonitorUpdateError("Previous secret did not match new one"));
+                       return Err("Previous secret did not match new one");
                }
 
                // Prune HTLCs from the previous counterparty commitment tx so we don't generate failure/fulfill
@@ -1402,7 +1800,6 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                htlcs.push(htlc.0);
                        }
                }
-               self.counterparty_tx_cache.per_htlc.insert(txid, htlcs);
        }
 
        /// Informs this monitor of the latest holder (ie broadcastable) commitment transaction. The
@@ -1410,7 +1807,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
        /// is important that any clones of this channel monitor (including remote clones) by kept
        /// up-to-date as our holder commitment transaction is updated.
        /// Panics if set_on_holder_tx_csv has never been called.
-       fn provide_latest_holder_commitment_tx(&mut self, holder_commitment_tx: HolderCommitmentTransaction, htlc_outputs: Vec<(HTLCOutputInCommitment, Option<Signature>, Option<HTLCSource>)>) -> Result<(), MonitorUpdateError> {
+       fn provide_latest_holder_commitment_tx(&mut self, holder_commitment_tx: HolderCommitmentTransaction, htlc_outputs: Vec<(HTLCOutputInCommitment, Option<Signature>, Option<HTLCSource>)>) -> Result<(), &'static str> {
                // block for Rust 1.34 compat
                let mut new_holder_commitment_tx = {
                        let trusted_tx = holder_commitment_tx.trust();
@@ -1424,15 +1821,16 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                b_htlc_key: tx_keys.countersignatory_htlc_key,
                                delayed_payment_key: tx_keys.broadcaster_delayed_payment_key,
                                per_commitment_point: tx_keys.per_commitment_point,
-                               feerate_per_kw: trusted_tx.feerate_per_kw(),
                                htlc_outputs,
+                               to_self_value_sat: holder_commitment_tx.to_broadcaster_value_sat(),
+                               feerate_per_kw: trusted_tx.feerate_per_kw(),
                        }
                };
                self.onchain_tx_handler.provide_latest_holder_tx(holder_commitment_tx);
                mem::swap(&mut new_holder_commitment_tx, &mut self.current_holder_commitment_tx);
                self.prev_holder_signed_commitment_tx = Some(new_holder_commitment_tx);
                if self.holder_tx_signed {
-                       return Err(MonitorUpdateError("Latest holder commitment signed has already been signed, update is rejected"));
+                       return Err("Latest holder commitment signed has already been signed, update is rejected");
                }
                Ok(())
        }
@@ -1493,33 +1891,43 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                        log_info!(logger, "Broadcasting local {}", log_tx!(tx));
                        broadcaster.broadcast_transaction(tx);
                }
-               self.pending_monitor_events.push(MonitorEvent::CommitmentTxBroadcasted(self.funding_info.0));
+               self.pending_monitor_events.push(MonitorEvent::CommitmentTxConfirmed(self.funding_info.0));
        }
 
-       pub fn update_monitor<B: Deref, F: Deref, L: Deref>(&mut self, updates: &ChannelMonitorUpdate, broadcaster: &B, fee_estimator: &F, logger: &L) -> Result<(), MonitorUpdateError>
+       pub fn update_monitor<B: Deref, F: Deref, L: Deref>(&mut self, updates: &ChannelMonitorUpdate, broadcaster: &B, fee_estimator: &F, logger: &L) -> Result<(), ()>
        where B::Target: BroadcasterInterface,
                    F::Target: FeeEstimator,
                    L::Target: Logger,
        {
+               log_info!(logger, "Applying update to monitor {}, bringing update_id from {} to {} with {} changes.",
+                       log_funding_info!(self), self.latest_update_id, updates.update_id, updates.updates.len());
                // ChannelMonitor updates may be applied after force close if we receive a
                // preimage for a broadcasted commitment transaction HTLC output that we'd
                // like to claim on-chain. If this is the case, we no longer have guaranteed
                // access to the monitor's update ID, so we use a sentinel value instead.
                if updates.update_id == CLOSED_CHANNEL_UPDATE_ID {
+                       assert_eq!(updates.updates.len(), 1);
                        match updates.updates[0] {
                                ChannelMonitorUpdateStep::PaymentPreimage { .. } => {},
-                               _ => panic!("Attempted to apply post-force-close ChannelMonitorUpdate that wasn't providing a payment preimage"),
+                               _ => {
+                                       log_error!(logger, "Attempted to apply post-force-close ChannelMonitorUpdate of type {}", updates.updates[0].variant_name());
+                                       panic!("Attempted to apply post-force-close ChannelMonitorUpdate that wasn't providing a payment preimage");
+                               },
                        }
-                       assert_eq!(updates.updates.len(), 1);
                } else if self.latest_update_id + 1 != updates.update_id {
                        panic!("Attempted to apply ChannelMonitorUpdates out of order, check the update_id before passing an update to update_monitor!");
                }
+               let mut ret = Ok(());
                for update in updates.updates.iter() {
                        match update {
                                ChannelMonitorUpdateStep::LatestHolderCommitmentTXInfo { commitment_tx, htlc_outputs } => {
                                        log_trace!(logger, "Updating ChannelMonitor with latest holder commitment transaction info");
                                        if self.lockdown_from_offchain { panic!(); }
-                                       self.provide_latest_holder_commitment_tx(commitment_tx.clone(), htlc_outputs.clone())?
+                                       if let Err(e) = self.provide_latest_holder_commitment_tx(commitment_tx.clone(), htlc_outputs.clone()) {
+                                               log_error!(logger, "Providing latest holder commitment transaction failed/was refused:");
+                                               log_error!(logger, "    {}", e);
+                                               ret = Err(());
+                                       }
                                }
                                ChannelMonitorUpdateStep::LatestCounterpartyCommitmentTXInfo { commitment_txid, htlc_outputs, commitment_number, their_revocation_point } => {
                                        log_trace!(logger, "Updating ChannelMonitor with latest counterparty commitment transaction info");
@@ -1531,7 +1939,11 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                },
                                ChannelMonitorUpdateStep::CommitmentSecret { idx, secret } => {
                                        log_trace!(logger, "Updating ChannelMonitor with commitment secret");
-                                       self.provide_secret(*idx, *secret)?
+                                       if let Err(e) = self.provide_secret(*idx, *secret) {
+                                               log_error!(logger, "Providing latest counterparty commitment secret failed/was refused:");
+                                               log_error!(logger, "    {}", e);
+                                               ret = Err(());
+                                       }
                                },
                                ChannelMonitorUpdateStep::ChannelForceClosed { should_broadcast } => {
                                        log_trace!(logger, "Updating ChannelMonitor: channel force closed, should broadcast: {}", should_broadcast);
@@ -1541,7 +1953,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                        } else if !self.holder_tx_signed {
                                                log_error!(logger, "You have a toxic holder commitment transaction avaible in channel monitor, read comment in ChannelMonitor::get_latest_holder_commitment_txn to be informed of manual action to take");
                                        } else {
-                                               // If we generated a MonitorEvent::CommitmentTxBroadcasted, the ChannelManager
+                                               // If we generated a MonitorEvent::CommitmentTxConfirmed, the ChannelManager
                                                // will still give us a ChannelForceClosed event with !should_broadcast, but we
                                                // shouldn't print the scary warning above.
                                                log_info!(logger, "Channel off-chain state closed after we broadcasted our latest commitment transaction.");
@@ -1556,7 +1968,11 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                        }
                }
                self.latest_update_id = updates.update_id;
-               Ok(())
+
+               if ret.is_ok() && self.funding_spend_seen {
+                       log_error!(logger, "Refusing Channel Monitor Update as counterparty attempted to update commitment after funding was spent");
+                       Err(())
+               } else { ret }
        }
 
        pub fn get_latest_update_id(&self) -> u64 {
@@ -1636,16 +2052,16 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                        let per_commitment_key = ignore_error!(SecretKey::from_slice(&secret));
                        let per_commitment_point = PublicKey::from_secret_key(&self.secp_ctx, &per_commitment_key);
                        let revocation_pubkey = ignore_error!(chan_utils::derive_public_revocation_key(&self.secp_ctx, &per_commitment_point, &self.holder_revocation_basepoint));
-                       let delayed_key = ignore_error!(chan_utils::derive_public_key(&self.secp_ctx, &PublicKey::from_secret_key(&self.secp_ctx, &per_commitment_key), &self.counterparty_tx_cache.counterparty_delayed_payment_base_key));
+                       let delayed_key = ignore_error!(chan_utils::derive_public_key(&self.secp_ctx, &PublicKey::from_secret_key(&self.secp_ctx, &per_commitment_key), &self.counterparty_commitment_params.counterparty_delayed_payment_base_key));
 
-                       let revokeable_redeemscript = chan_utils::get_revokeable_redeemscript(&revocation_pubkey, self.counterparty_tx_cache.on_counterparty_tx_csv, &delayed_key);
+                       let revokeable_redeemscript = chan_utils::get_revokeable_redeemscript(&revocation_pubkey, self.counterparty_commitment_params.on_counterparty_tx_csv, &delayed_key);
                        let revokeable_p2wsh = revokeable_redeemscript.to_v0_p2wsh();
 
                        // First, process non-htlc outputs (to_holder & to_counterparty)
                        for (idx, outp) in tx.output.iter().enumerate() {
                                if outp.script_pubkey == revokeable_p2wsh {
-                                       let revk_outp = RevokedOutput::build(per_commitment_point, self.counterparty_tx_cache.counterparty_delayed_payment_base_key, self.counterparty_tx_cache.counterparty_htlc_base_key, per_commitment_key, outp.value, self.counterparty_tx_cache.on_counterparty_tx_csv);
-                                       let justice_package = PackageTemplate::build_package(commitment_txid, idx as u32, PackageSolvingData::RevokedOutput(revk_outp), height + self.counterparty_tx_cache.on_counterparty_tx_csv as u32, true, height);
+                                       let revk_outp = RevokedOutput::build(per_commitment_point, self.counterparty_commitment_params.counterparty_delayed_payment_base_key, self.counterparty_commitment_params.counterparty_htlc_base_key, per_commitment_key, outp.value, self.counterparty_commitment_params.on_counterparty_tx_csv);
+                                       let justice_package = PackageTemplate::build_package(commitment_txid, idx as u32, PackageSolvingData::RevokedOutput(revk_outp), height + self.counterparty_commitment_params.on_counterparty_tx_csv as u32, true, height);
                                        claimable_outpoints.push(justice_package);
                                }
                        }
@@ -1658,7 +2074,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                                                tx.output[transaction_output_index as usize].value != htlc.amount_msat / 1000 {
                                                        return (claimable_outpoints, (commitment_txid, watch_outputs)); // Corrupted per_commitment_data, fuck this user
                                                }
-                                               let revk_htlc_outp = RevokedHTLCOutput::build(per_commitment_point, self.counterparty_tx_cache.counterparty_delayed_payment_base_key, self.counterparty_tx_cache.counterparty_htlc_base_key, per_commitment_key, htlc.amount_msat / 1000, htlc.clone());
+                                               let revk_htlc_outp = RevokedHTLCOutput::build(per_commitment_point, self.counterparty_commitment_params.counterparty_delayed_payment_base_key, self.counterparty_commitment_params.counterparty_htlc_base_key, per_commitment_key, htlc.amount_msat / 1000, htlc.clone(), self.onchain_tx_handler.channel_transaction_parameters.opt_anchors.is_some());
                                                let justice_package = PackageTemplate::build_package(commitment_txid, transaction_output_index, PackageSolvingData::RevokedHTLCOutput(revk_htlc_outp), htlc.cltv_expiry, true, height);
                                                claimable_outpoints.push(justice_package);
                                        }
@@ -1726,7 +2142,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                                        }
                                                        let preimage = if htlc.offered { if let Some(p) = self.payment_preimages.get(&htlc.payment_hash) { Some(*p) } else { None } } else { None };
                                                        if preimage.is_some() || !htlc.offered {
-                                                               let counterparty_htlc_outp = if htlc.offered { PackageSolvingData::CounterpartyOfferedHTLCOutput(CounterpartyOfferedHTLCOutput::build(*revocation_point, self.counterparty_tx_cache.counterparty_delayed_payment_base_key, self.counterparty_tx_cache.counterparty_htlc_base_key, preimage.unwrap(), htlc.clone())) } else { PackageSolvingData::CounterpartyReceivedHTLCOutput(CounterpartyReceivedHTLCOutput::build(*revocation_point, self.counterparty_tx_cache.counterparty_delayed_payment_base_key, self.counterparty_tx_cache.counterparty_htlc_base_key, htlc.clone())) };
+                                                               let counterparty_htlc_outp = if htlc.offered { PackageSolvingData::CounterpartyOfferedHTLCOutput(CounterpartyOfferedHTLCOutput::build(*revocation_point, self.counterparty_commitment_params.counterparty_delayed_payment_base_key, self.counterparty_commitment_params.counterparty_htlc_base_key, preimage.unwrap(), htlc.clone())) } else { PackageSolvingData::CounterpartyReceivedHTLCOutput(CounterpartyReceivedHTLCOutput::build(*revocation_point, self.counterparty_commitment_params.counterparty_delayed_payment_base_key, self.counterparty_commitment_params.counterparty_htlc_base_key, htlc.clone())) };
                                                                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);
                                                                claimable_outpoints.push(counterparty_package);
@@ -1760,8 +2176,8 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                let per_commitment_point = PublicKey::from_secret_key(&self.secp_ctx, &per_commitment_key);
 
                log_error!(logger, "Got broadcast of revoked counterparty HTLC transaction, spending {}:{}", htlc_txid, 0);
-               let revk_outp = RevokedOutput::build(per_commitment_point, self.counterparty_tx_cache.counterparty_delayed_payment_base_key, self.counterparty_tx_cache.counterparty_htlc_base_key, per_commitment_key, tx.output[0].value, self.counterparty_tx_cache.on_counterparty_tx_csv);
-               let justice_package = PackageTemplate::build_package(htlc_txid, 0, PackageSolvingData::RevokedOutput(revk_outp), height + self.counterparty_tx_cache.on_counterparty_tx_csv as u32, true, height);
+               let revk_outp = RevokedOutput::build(per_commitment_point, self.counterparty_commitment_params.counterparty_delayed_payment_base_key, self.counterparty_commitment_params.counterparty_htlc_base_key, per_commitment_key, tx.output[0].value, self.counterparty_commitment_params.on_counterparty_tx_csv);
+               let justice_package = PackageTemplate::build_package(htlc_txid, 0, PackageSolvingData::RevokedOutput(revk_outp), height + self.counterparty_commitment_params.on_counterparty_tx_csv as u32, true, height);
                let claimable_outpoints = vec!(justice_package);
                let outputs = vec![(0, tx.output[0].clone())];
                (claimable_outpoints, Some((htlc_txid, outputs)))
@@ -1811,7 +2227,8 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
        /// Attempts to claim any claimable HTLCs in a commitment transaction which was not (yet)
        /// revoked using data in holder_claimable_outpoints.
        /// Should not be used if check_spend_revoked_transaction succeeds.
-       fn check_spend_holder_transaction<L: Deref>(&mut self, tx: &Transaction, height: u32, logger: &L) -> (Vec<PackageTemplate>, TransactionOutputs) where L::Target: Logger {
+       /// 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 {
                let commitment_txid = tx.txid();
                let mut claim_requests = Vec::new();
                let mut watch_outputs = Vec::new();
@@ -1846,9 +2263,10 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                }
 
                if is_holder_tx {
+                       Some((claim_requests, (commitment_txid, watch_outputs)))
+               } else {
+                       None
                }
-
-               (claim_requests, (commitment_txid, watch_outputs))
        }
 
        pub fn get_latest_holder_commitment_txn<L: Deref>(&mut self, logger: &L) -> Vec<Transaction> where L::Target: Logger {
@@ -1913,7 +2331,6 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                        L::Target: Logger,
        {
                let block_hash = header.block_hash();
-               log_trace!(logger, "New best block {} at height {}", block_hash, height);
                self.best_block = BestBlock::new(block_hash, height);
 
                self.transactions_confirmed(header, txdata, height, broadcaster, fee_estimator, logger)
@@ -1933,7 +2350,6 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                L::Target: Logger,
        {
                let block_hash = header.block_hash();
-               log_trace!(logger, "New best block {} at height {}", block_hash, height);
 
                if height > self.best_block.height() {
                        self.best_block = BestBlock::new(block_hash, height);
@@ -1971,7 +2387,6 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                }
 
                let block_hash = header.block_hash();
-               log_trace!(logger, "Block {} at height {} connected with {} txn matched", block_hash, height, txn_matched.len());
 
                let mut watch_outputs = Vec::new();
                let mut claimable_outpoints = Vec::new();
@@ -1983,20 +2398,34 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                // filters.
                                let prevout = &tx.input[0].previous_output;
                                if prevout.txid == self.funding_info.0.txid && prevout.vout == self.funding_info.0.index as u32 {
+                                       let mut balance_spendable_csv = None;
+                                       log_info!(logger, "Channel {} closed by funding output spend in txid {}.",
+                                               log_bytes!(self.funding_info.0.to_channel_id()), tx.txid());
+                                       self.funding_spend_seen = true;
                                        if (tx.input[0].sequence >> 8*3) as u8 == 0x80 && (tx.lock_time >> 8*3) as u8 == 0x20 {
                                                let (mut new_outpoints, new_outputs) = self.check_spend_counterparty_transaction(&tx, height, &logger);
                                                if !new_outputs.1.is_empty() {
                                                        watch_outputs.push(new_outputs);
                                                }
+                                               claimable_outpoints.append(&mut new_outpoints);
                                                if new_outpoints.is_empty() {
-                                                       let (mut new_outpoints, new_outputs) = self.check_spend_holder_transaction(&tx, height, &logger);
-                                                       if !new_outputs.1.is_empty() {
-                                                               watch_outputs.push(new_outputs);
+                                                       if let Some((mut new_outpoints, new_outputs)) = self.check_spend_holder_transaction(&tx, height, &logger) {
+                                                               if !new_outputs.1.is_empty() {
+                                                                       watch_outputs.push(new_outputs);
+                                                               }
+                                                               claimable_outpoints.append(&mut new_outpoints);
+                                                               balance_spendable_csv = Some(self.on_holder_tx_csv);
                                                        }
-                                                       claimable_outpoints.append(&mut new_outpoints);
                                                }
-                                               claimable_outpoints.append(&mut new_outpoints);
                                        }
+                                       let txid = tx.txid();
+                                       self.onchain_events_awaiting_threshold_conf.push(OnchainEventEntry {
+                                               txid,
+                                               height: height,
+                                               event: OnchainEvent::FundingSpendConfirmation {
+                                                       on_local_output_csv: balance_spendable_csv,
+                                               },
+                                       });
                                } else {
                                        if let Some(&commitment_number) = self.counterparty_commitment_txn_on_chain.get(&prevout.txid) {
                                                let (mut new_outpoints, new_outputs_option) = self.check_spend_counterparty_htlc(&tx, commitment_number, height, &logger);
@@ -2044,6 +2473,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                F::Target: FeeEstimator,
                L::Target: Logger,
        {
+               log_trace!(logger, "Processing {} matched transactions for block at height {}.", txn_matched.len(), conf_height);
                debug_assert!(self.best_block.height() >= conf_height);
 
                let should_broadcast = self.should_broadcast_holder_commitment_txn(logger);
@@ -2051,7 +2481,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                        let funding_outp = HolderFundingOutput::build(self.funding_redeemscript.clone());
                        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::CommitmentTxBroadcasted(self.funding_info.0));
+                       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
@@ -2083,7 +2513,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                        .iter()
                        .filter_map(|entry| match &entry.event {
                                OnchainEvent::HTLCUpdate { source, .. } => Some(source),
-                               OnchainEvent::MaturingOutput { .. } => None,
+                               _ => None,
                        })
                        .collect();
                #[cfg(debug_assertions)]
@@ -2092,7 +2522,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                // Produce actionable events from on-chain events having reached their threshold.
                for entry in onchain_events_reaching_threshold_conf.drain(..) {
                        match entry.event {
-                               OnchainEvent::HTLCUpdate { ref source, payment_hash, onchain_value_satoshis } => {
+                               OnchainEvent::HTLCUpdate { ref source, payment_hash, onchain_value_satoshis, input_idx } => {
                                        // Check for duplicate HTLC resolutions.
                                        #[cfg(debug_assertions)]
                                        {
@@ -2116,13 +2546,22 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                                source: source.clone(),
                                                onchain_value_satoshis,
                                        }));
+                                       if let Some(idx) = input_idx {
+                                               self.htlcs_resolved_on_chain.push(IrrevocablyResolvedHTLC { input_idx: idx, payment_preimage: None });
+                                       }
                                },
                                OnchainEvent::MaturingOutput { descriptor } => {
                                        log_debug!(logger, "Descriptor {} has got enough confirmations to be passed upstream", log_spendable!(descriptor));
                                        self.pending_events.push(Event::SpendableOutputs {
                                                outputs: vec![descriptor]
                                        });
-                               }
+                               },
+                               OnchainEvent::HTLCSpendConfirmation { input_idx, preimage, .. } => {
+                                       self.htlcs_resolved_on_chain.push(IrrevocablyResolvedHTLC { input_idx, payment_preimage: preimage });
+                               },
+                               OnchainEvent::FundingSpendConfirmation { .. } => {
+                                       self.funding_spend_confirmed = Some(entry.txid);
+                               },
                        }
                }
 
@@ -2300,15 +2739,34 @@ 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);
-                       let offered_preimage_claim = input.witness.len() == 3 && HTLCType::scriptlen_to_htlctype(input.witness[2].len()) == Some(HTLCType::OfferedHTLC);
+                       #[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]);
+                       if accepted_preimage_claim {
+                               payment_preimage.0.copy_from_slice(&input.witness[3]);
+                       } else if offered_preimage_claim {
+                               payment_preimage.0.copy_from_slice(&input.witness[1]);
+                       }
 
                        macro_rules! log_claim {
                                ($tx_info: expr, $holder_tx: expr, $htlc: expr, $source_avail: expr) => {
-                                       // We found the output in question, but aren't failing it backwards
-                                       // as we have no corresponding source and no valid counterparty commitment txid
-                                       // to try a weak source binding with same-hash, same-value still-valid offered HTLC.
-                                       // This implies either it is an inbound HTLC or an outbound HTLC on a revoked transaction.
                                        let outbound_htlc = $holder_tx == $htlc.offered;
+                                       // HTLCs must either be claimed by a matching script type or through the
+                                       // revocation path:
+                                       #[cfg(not(fuzzing))] // Note that the fuzzer is not bound by pesky things like "signatures"
+                                       debug_assert!(!$htlc.offered || offered_preimage_claim || offered_timeout_claim || revocation_sig_claim);
+                                       #[cfg(not(fuzzing))] // Note that the fuzzer is not bound by pesky things like "signatures"
+                                       debug_assert!($htlc.offered || accepted_preimage_claim || accepted_timeout_claim || revocation_sig_claim);
+                                       // Further, only exactly one of the possible spend paths should have been
+                                       // matched by any HTLC spend:
+                                       #[cfg(not(fuzzing))] // Note that the fuzzer is not bound by pesky things like "signatures"
+                                       debug_assert_eq!(accepted_preimage_claim as u8 + accepted_timeout_claim as u8 +
+                                                        offered_preimage_claim as u8 + offered_timeout_claim as u8 +
+                                                        revocation_sig_claim as u8, 1);
                                        if ($holder_tx && revocation_sig_claim) ||
                                                        (outbound_htlc && !$source_avail && (accepted_preimage_claim || offered_preimage_claim)) {
                                                log_error!(logger, "Input spending {} ({}:{}) in {} resolves {} HTLC with payment hash {} with {}!",
@@ -2353,13 +2811,37 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                                                // resolve the source HTLC with the original sender.
                                                                payment_data = Some(((*source).clone(), htlc_output.payment_hash, htlc_output.amount_msat));
                                                        } else if !$holder_tx {
-                                                                       check_htlc_valid_counterparty!(self.current_counterparty_commitment_txid, htlc_output);
+                                                               check_htlc_valid_counterparty!(self.current_counterparty_commitment_txid, htlc_output);
                                                                if payment_data.is_none() {
                                                                        check_htlc_valid_counterparty!(self.prev_counterparty_commitment_txid, htlc_output);
                                                                }
                                                        }
                                                        if payment_data.is_none() {
                                                                log_claim!($tx_info, $holder_tx, htlc_output, false);
+                                                               let outbound_htlc = $holder_tx == htlc_output.offered;
+                                                               if !outbound_htlc || revocation_sig_claim {
+                                                                       self.onchain_events_awaiting_threshold_conf.push(OnchainEventEntry {
+                                                                               txid: tx.txid(), height,
+                                                                               event: OnchainEvent::HTLCSpendConfirmation {
+                                                                                       input_idx: input.previous_output.vout,
+                                                                                       preimage: if accepted_preimage_claim || offered_preimage_claim {
+                                                                                               Some(payment_preimage) } else { None },
+                                                                                       // If this is a payment to us (!outbound_htlc, above),
+                                                                                       // wait for the CSV delay before dropping the HTLC from
+                                                                                       // claimable balance if the claim was an HTLC-Success
+                                                                                       // transaction.
+                                                                                       on_to_local_output_csv: if accepted_preimage_claim {
+                                                                                               Some(self.on_holder_tx_csv) } else { None },
+                                                                               },
+                                                                       });
+                                                               } else {
+                                                                       // Outbound claims should always have payment_data, unless
+                                                                       // we've already failed the HTLC as the commitment transaction
+                                                                       // which was broadcasted was revoked. In that case, we should
+                                                                       // spend the HTLC output here immediately, and expose that fact
+                                                                       // as a Balance, something which we do not yet do.
+                                                                       // TODO: Track the above as claimable!
+                                                               }
                                                                continue 'outer_loop;
                                                        }
                                                }
@@ -2385,11 +2867,18 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                        // Check that scan_commitment, above, decided there is some source worth relaying an
                        // HTLC resolution backwards to and figure out whether we learned a preimage from it.
                        if let Some((source, payment_hash, amount_msat)) = payment_data {
-                               let mut payment_preimage = PaymentPreimage([0; 32]);
                                if accepted_preimage_claim {
                                        if !self.pending_monitor_events.iter().any(
                                                |update| if let &MonitorEvent::HTLCEvent(ref upd) = update { upd.source == source } else { false }) {
-                                               payment_preimage.0.copy_from_slice(&input.witness[3]);
+                                               self.onchain_events_awaiting_threshold_conf.push(OnchainEventEntry {
+                                                       txid: tx.txid(),
+                                                       height,
+                                                       event: OnchainEvent::HTLCSpendConfirmation {
+                                                               input_idx: input.previous_output.vout,
+                                                               preimage: Some(payment_preimage),
+                                                               on_to_local_output_csv: None,
+                                                       },
+                                               });
                                                self.pending_monitor_events.push(MonitorEvent::HTLCEvent(HTLCUpdate {
                                                        source,
                                                        payment_preimage: Some(payment_preimage),
@@ -2402,7 +2891,15 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                                |update| if let &MonitorEvent::HTLCEvent(ref upd) = update {
                                                        upd.source == source
                                                } else { false }) {
-                                               payment_preimage.0.copy_from_slice(&input.witness[1]);
+                                               self.onchain_events_awaiting_threshold_conf.push(OnchainEventEntry {
+                                                       txid: tx.txid(),
+                                                       height,
+                                                       event: OnchainEvent::HTLCSpendConfirmation {
+                                                               input_idx: input.previous_output.vout,
+                                                               preimage: Some(payment_preimage),
+                                                               on_to_local_output_csv: None,
+                                                       },
+                                               });
                                                self.pending_monitor_events.push(MonitorEvent::HTLCEvent(HTLCUpdate {
                                                        source,
                                                        payment_preimage: Some(payment_preimage),
@@ -2426,6 +2923,7 @@ impl<Signer: Sign> ChannelMonitorImpl<Signer> {
                                                event: OnchainEvent::HTLCUpdate {
                                                        source, payment_hash,
                                                        onchain_value_satoshis: Some(amount_msat / 1000),
+                                                       input_idx: Some(input.previous_output.vout),
                                                },
                                        };
                                        log_info!(logger, "Failing HTLC with payment_hash {} timeout by a spend tx, waiting for confirmation (at height {})", log_bytes!(payment_hash.0), entry.confirmation_threshold());
@@ -2503,62 +3001,14 @@ 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,
        F::Target: FeeEstimator,
        L::Target: Logger,
 {
-       fn block_connected(&self, block: &Block, height: u32) {
-               let txdata: Vec<_> = block.txdata.iter().enumerate().collect();
-               self.0.block_connected(&block.header, &txdata, height, &*self.1, &*self.2, &*self.3);
+       fn filtered_block_connected(&self, header: &BlockHeader, txdata: &TransactionData, height: u32) {
+               self.0.block_connected(header, txdata, height, &*self.1, &*self.2, &*self.3);
        }
 
        fn block_disconnected(&self, header: &BlockHeader, height: u32) {
@@ -2637,7 +3087,7 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
                let current_counterparty_commitment_txid = Readable::read(reader)?;
                let prev_counterparty_commitment_txid = Readable::read(reader)?;
 
-               let counterparty_tx_cache = Readable::read(reader)?;
+               let counterparty_commitment_params = Readable::read(reader)?;
                let funding_redeemscript = Readable::read(reader)?;
                let channel_value_satoshis = Readable::read(reader)?;
 
@@ -2710,14 +3160,15 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
                        }
                }
 
-               let prev_holder_signed_commitment_tx = match <u8 as Readable>::read(reader)? {
-                       0 => None,
-                       1 => {
-                               Some(Readable::read(reader)?)
-                       },
-                       _ => return Err(DecodeError::InvalidValue),
-               };
-               let current_holder_commitment_tx = Readable::read(reader)?;
+               let mut prev_holder_signed_commitment_tx: Option<HolderSignedTx> =
+                       match <u8 as Readable>::read(reader)? {
+                               0 => None,
+                               1 => {
+                                       Some(Readable::read(reader)?)
+                               },
+                               _ => return Err(DecodeError::InvalidValue),
+                       };
+               let mut current_holder_commitment_tx: HolderSignedTx = Readable::read(reader)?;
 
                let current_counterparty_commitment_number = <U48 as Readable>::read(reader)?.0;
                let current_holder_commitment_number = <U48 as Readable>::read(reader)?.0;
@@ -2733,14 +3184,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::CommitmentTxBroadcasted(funding_info.0),
+                               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)?;
@@ -2774,12 +3226,37 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
                                return Err(DecodeError::InvalidValue);
                        }
                }
-               let onchain_tx_handler = ReadableArgs::read(reader, keys_manager)?;
+               let onchain_tx_handler: OnchainTxHandler<Signer> = ReadableArgs::read(reader, keys_manager)?;
 
                let lockdown_from_offchain = Readable::read(reader)?;
                let holder_tx_signed = Readable::read(reader)?;
 
-               read_tlv_fields!(reader, {});
+               if let Some(prev_commitment_tx) = prev_holder_signed_commitment_tx.as_mut() {
+                       let prev_holder_value = onchain_tx_handler.get_prev_holder_commitment_to_self_value();
+                       if prev_holder_value.is_none() { return Err(DecodeError::InvalidValue); }
+                       if prev_commitment_tx.to_self_value_sat == u64::max_value() {
+                               prev_commitment_tx.to_self_value_sat = prev_holder_value.unwrap();
+                       } else if prev_commitment_tx.to_self_value_sat != prev_holder_value.unwrap() {
+                               return Err(DecodeError::InvalidValue);
+                       }
+               }
+
+               let cur_holder_value = onchain_tx_handler.get_cur_holder_commitment_to_self_value();
+               if current_holder_commitment_tx.to_self_value_sat == u64::max_value() {
+                       current_holder_commitment_tx.to_self_value_sat = cur_holder_value;
+               } else if current_holder_commitment_tx.to_self_value_sat != cur_holder_value {
+                       return Err(DecodeError::InvalidValue);
+               }
+
+               let mut funding_spend_confirmed = None;
+               let mut htlcs_resolved_on_chain = Some(Vec::new());
+               let mut funding_spend_seen = Some(false);
+               read_tlv_fields!(reader, {
+                       (1, funding_spend_confirmed, option),
+                       (3, htlcs_resolved_on_chain, vec_type),
+                       (5, pending_monitor_events, vec_type),
+                       (7, funding_spend_seen, option),
+               });
 
                let mut secp_ctx = Secp256k1::new();
                secp_ctx.seeded_randomize(&keys_manager.get_secure_random_bytes());
@@ -2800,7 +3277,7 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
                                current_counterparty_commitment_txid,
                                prev_counterparty_commitment_txid,
 
-                               counterparty_tx_cache,
+                               counterparty_commitment_params,
                                funding_redeemscript,
                                channel_value_satoshis,
                                their_cur_revocation_points,
@@ -2818,7 +3295,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,
@@ -2828,6 +3305,9 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
 
                                lockdown_from_offchain,
                                holder_tx_signed,
+                               funding_spend_seen: funding_spend_seen.unwrap(),
+                               funding_spend_confirmed,
+                               htlcs_resolved_on_chain: htlcs_resolved_on_chain.unwrap(),
 
                                best_block,
 
@@ -2839,6 +3319,7 @@ impl<'a, Signer: Sign, K: KeysInterface<Signer = Signer>> ReadableArgs<&'a K>
 
 #[cfg(test)]
 mod tests {
+       use bitcoin::blockdata::block::BlockHeader;
        use bitcoin::blockdata::script::{Script, Builder};
        use bitcoin::blockdata::opcodes;
        use bitcoin::blockdata::transaction::{Transaction, TxIn, TxOut, SigHashType};
@@ -2847,24 +3328,128 @@ mod tests {
        use bitcoin::hashes::Hash;
        use bitcoin::hashes::sha256::Hash as Sha256;
        use bitcoin::hashes::hex::FromHex;
-       use bitcoin::hash_types::Txid;
+       use bitcoin::hash_types::{BlockHash, Txid};
        use bitcoin::network::constants::Network;
+       use bitcoin::secp256k1::key::{SecretKey,PublicKey};
+       use bitcoin::secp256k1::Secp256k1;
+
        use hex;
-       use chain::BestBlock;
+
+       use super::ChannelMonitorUpdateStep;
+       use ::{check_added_monitors, check_closed_broadcast, check_closed_event, check_spends, get_local_commitment_txn, get_monitor, get_route_and_payment_hash, unwrap_send_err};
+       use chain::{BestBlock, Confirm};
        use chain::channelmonitor::ChannelMonitor;
-       use chain::package::{WEIGHT_OFFERED_HTLC, WEIGHT_RECEIVED_HTLC, WEIGHT_REVOKED_OFFERED_HTLC, WEIGHT_REVOKED_RECEIVED_HTLC, WEIGHT_REVOKED_OUTPUT};
+       use chain::package::{weight_offered_htlc, weight_received_htlc, weight_revoked_offered_htlc, weight_revoked_received_htlc, WEIGHT_REVOKED_OUTPUT};
        use chain::transaction::OutPoint;
+       use chain::keysinterface::InMemorySigner;
        use ln::{PaymentPreimage, PaymentHash};
        use ln::chan_utils;
        use ln::chan_utils::{HTLCOutputInCommitment, ChannelPublicKeys, ChannelTransactionParameters, HolderCommitmentTransaction, CounterpartyChannelTransactionParameters};
+       use ln::channelmanager::PaymentSendFailure;
+       use ln::features::InitFeatures;
+       use ln::functional_test_utils::*;
        use ln::script::ShutdownScript;
+       use util::errors::APIError;
+       use util::events::{ClosureReason, MessageSendEventsProvider};
        use util::test_utils::{TestLogger, TestBroadcaster, TestFeeEstimator};
-       use bitcoin::secp256k1::key::{SecretKey,PublicKey};
-       use bitcoin::secp256k1::Secp256k1;
+       use util::ser::{ReadableArgs, Writeable};
        use sync::{Arc, Mutex};
-       use chain::keysinterface::InMemorySigner;
+       use io;
        use prelude::*;
 
+       fn do_test_funding_spend_refuses_updates(use_local_txn: bool) {
+               // Previously, monitor updates were allowed freely even after a funding-spend transaction
+               // confirmed. This would allow a race condition where we could receive a payment (including
+               // the counterparty revoking their broadcasted state!) and accept it without recourse as
+               // long as the ChannelMonitor receives the block first, the full commitment update dance
+               // occurs after the block is connected, and before the ChannelManager receives the block.
+               // Obviously this is an incredibly contrived race given the counterparty would be risking
+               // their full channel balance for it, but its worth fixing nonetheless as it makes the
+               // potential ChannelMonitor states simpler to reason about.
+               //
+               // This test checks said behavior, as well as ensuring a ChannelMonitorUpdate with multiple
+               // updates is handled correctly in such conditions.
+               let chanmon_cfgs = create_chanmon_cfgs(3);
+               let node_cfgs = create_node_cfgs(3, &chanmon_cfgs);
+               let node_chanmgrs = create_node_chanmgrs(3, &node_cfgs, &[None, None, None]);
+               let nodes = create_network(3, &node_cfgs, &node_chanmgrs);
+               let channel = create_announced_chan_between_nodes(
+                       &nodes, 0, 1, InitFeatures::known(), InitFeatures::known());
+               create_announced_chan_between_nodes(
+                       &nodes, 1, 2, InitFeatures::known(), InitFeatures::known());
+
+               // Rebalance somewhat
+               send_payment(&nodes[0], &[&nodes[1]], 10_000_000);
+
+               // First route two payments for testing at the end
+               let payment_preimage_1 = route_payment(&nodes[0], &[&nodes[1], &nodes[2]], 1_000_000).0;
+               let payment_preimage_2 = route_payment(&nodes[0], &[&nodes[1], &nodes[2]], 1_000_000).0;
+
+               let local_txn = get_local_commitment_txn!(nodes[1], channel.2);
+               assert_eq!(local_txn.len(), 1);
+               let remote_txn = get_local_commitment_txn!(nodes[0], channel.2);
+               assert_eq!(remote_txn.len(), 3); // Commitment and two HTLC-Timeouts
+               check_spends!(remote_txn[1], remote_txn[0]);
+               check_spends!(remote_txn[2], remote_txn[0]);
+               let broadcast_tx = if use_local_txn { &local_txn[0] } else { &remote_txn[0] };
+
+               // Connect a commitment transaction, but only to the ChainMonitor/ChannelMonitor. The
+               // channel is now closed, but the ChannelManager doesn't know that yet.
+               let new_header = BlockHeader {
+                       version: 2, time: 0, bits: 0, nonce: 0,
+                       prev_blockhash: nodes[0].best_block_info().0,
+                       merkle_root: Default::default() };
+               let conf_height = nodes[0].best_block_info().1 + 1;
+               nodes[1].chain_monitor.chain_monitor.transactions_confirmed(&new_header,
+                       &[(0, broadcast_tx)], conf_height);
+
+               let (_, pre_update_monitor) = <(BlockHash, ChannelMonitor<InMemorySigner>)>::read(
+                                               &mut io::Cursor::new(&get_monitor!(nodes[1], channel.2).encode()),
+                                               &nodes[1].keys_manager.backing).unwrap();
+
+               // If the ChannelManager tries to update the channel, however, the ChainMonitor will pass
+               // the update through to the ChannelMonitor which will refuse it (as the channel is closed).
+               let (route, payment_hash, _, payment_secret) = get_route_and_payment_hash!(nodes[1], nodes[0], 100_000);
+               unwrap_send_err!(nodes[1].node.send_payment(&route, payment_hash, &Some(payment_secret)),
+                       true, APIError::ChannelUnavailable { ref err },
+                       assert!(err.contains("ChannelMonitor storage failure")));
+               check_added_monitors!(nodes[1], 2); // After the failure we generate a close-channel monitor update
+               check_closed_broadcast!(nodes[1], true);
+               check_closed_event!(nodes[1], 1, ClosureReason::ProcessingError { err: "ChannelMonitor storage failure".to_string() });
+
+               // Build a new ChannelMonitorUpdate which contains both the failing commitment tx update
+               // and provides the claim preimages for the two pending HTLCs. The first update generates
+               // an error, but the point of this test is to ensure the later updates are still applied.
+               let monitor_updates = nodes[1].chain_monitor.monitor_updates.lock().unwrap();
+               let mut replay_update = monitor_updates.get(&channel.2).unwrap().iter().rev().skip(1).next().unwrap().clone();
+               assert_eq!(replay_update.updates.len(), 1);
+               if let ChannelMonitorUpdateStep::LatestCounterpartyCommitmentTXInfo { .. } = replay_update.updates[0] {
+               } else { panic!(); }
+               replay_update.updates.push(ChannelMonitorUpdateStep::PaymentPreimage { payment_preimage: payment_preimage_1 });
+               replay_update.updates.push(ChannelMonitorUpdateStep::PaymentPreimage { payment_preimage: payment_preimage_2 });
+
+               let broadcaster = TestBroadcaster::new(Arc::clone(&nodes[1].blocks));
+               assert!(
+                       pre_update_monitor.update_monitor(&replay_update, &&broadcaster, &&chanmon_cfgs[1].fee_estimator, &nodes[1].logger)
+                       .is_err());
+               // Even though we error'd on the first update, we should still have generated an HTLC claim
+               // transaction
+               let txn_broadcasted = broadcaster.txn_broadcasted.lock().unwrap().split_off(0);
+               assert!(txn_broadcasted.len() >= 2);
+               let htlc_txn = txn_broadcasted.iter().filter(|tx| {
+                       assert_eq!(tx.input.len(), 1);
+                       tx.input[0].previous_output.txid == broadcast_tx.txid()
+               }).collect::<Vec<_>>();
+               assert_eq!(htlc_txn.len(), 2);
+               check_spends!(htlc_txn[0], broadcast_tx);
+               check_spends!(htlc_txn[1], broadcast_tx);
+       }
+       #[test]
+       fn test_funding_spend_refuses_updates() {
+               do_test_funding_spend_refuses_updates(true);
+               do_test_funding_spend_refuses_updates(false);
+       }
+
        #[test]
        fn test_prune_preimages() {
                let secp_ctx = Secp256k1::new();
@@ -2926,6 +3511,7 @@ mod tests {
                        SecretKey::from_slice(&[41; 32]).unwrap(),
                        SecretKey::from_slice(&[41; 32]).unwrap(),
                        SecretKey::from_slice(&[41; 32]).unwrap(),
+                       SecretKey::from_slice(&[41; 32]).unwrap(),
                        [41; 32],
                        0,
                        [0; 32]
@@ -2948,6 +3534,7 @@ mod tests {
                                selected_contest_delay: 67,
                        }),
                        funding_outpoint: Some(funding_outpoint),
+                       opt_anchors: None,
                };
                // Prune with one old state and a holder commitment tx holding a few overlaps with the
                // old state.
@@ -3010,28 +3597,27 @@ mod tests {
                let secp_ctx = Secp256k1::new();
                let privkey = SecretKey::from_slice(&hex::decode("0101010101010101010101010101010101010101010101010101010101010101").unwrap()[..]).unwrap();
                let pubkey = PublicKey::from_secret_key(&secp_ctx, &privkey);
-               let mut sum_actual_sigs = 0;
 
                macro_rules! sign_input {
-                       ($sighash_parts: expr, $idx: expr, $amount: expr, $weight: expr, $sum_actual_sigs: expr) => {
+                       ($sighash_parts: expr, $idx: expr, $amount: expr, $weight: expr, $sum_actual_sigs: expr, $opt_anchors: expr) => {
                                let htlc = HTLCOutputInCommitment {
-                                       offered: if *$weight == WEIGHT_REVOKED_OFFERED_HTLC || *$weight == WEIGHT_OFFERED_HTLC { true } else { false },
+                                       offered: if *$weight == weight_revoked_offered_htlc($opt_anchors) || *$weight == weight_offered_htlc($opt_anchors) { true } else { false },
                                        amount_msat: 0,
                                        cltv_expiry: 2 << 16,
                                        payment_hash: PaymentHash([1; 32]),
                                        transaction_output_index: Some($idx as u32),
                                };
-                               let redeem_script = if *$weight == WEIGHT_REVOKED_OUTPUT { chan_utils::get_revokeable_redeemscript(&pubkey, 256, &pubkey) } else { chan_utils::get_htlc_redeemscript_with_explicit_keys(&htlc, &pubkey, &pubkey, &pubkey) };
+                               let redeem_script = if *$weight == WEIGHT_REVOKED_OUTPUT { chan_utils::get_revokeable_redeemscript(&pubkey, 256, &pubkey) } else { chan_utils::get_htlc_redeemscript_with_explicit_keys(&htlc, $opt_anchors, &pubkey, &pubkey, &pubkey) };
                                let sighash = hash_to_message!(&$sighash_parts.signature_hash($idx, &redeem_script, $amount, SigHashType::All)[..]);
                                let sig = secp_ctx.sign(&sighash, &privkey);
                                $sighash_parts.access_witness($idx).push(sig.serialize_der().to_vec());
                                $sighash_parts.access_witness($idx)[0].push(SigHashType::All as u8);
-                               sum_actual_sigs += $sighash_parts.access_witness($idx)[0].len();
+                               $sum_actual_sigs += $sighash_parts.access_witness($idx)[0].len();
                                if *$weight == WEIGHT_REVOKED_OUTPUT {
                                        $sighash_parts.access_witness($idx).push(vec!(1));
-                               } else if *$weight == WEIGHT_REVOKED_OFFERED_HTLC || *$weight == WEIGHT_REVOKED_RECEIVED_HTLC {
+                               } else if *$weight == weight_revoked_offered_htlc($opt_anchors) || *$weight == weight_revoked_received_htlc($opt_anchors) {
                                        $sighash_parts.access_witness($idx).push(pubkey.clone().serialize().to_vec());
-                               } else if *$weight == WEIGHT_RECEIVED_HTLC {
+                               } else if *$weight == weight_received_htlc($opt_anchors) {
                                        $sighash_parts.access_witness($idx).push(vec![0]);
                                } else {
                                        $sighash_parts.access_witness($idx).push(PaymentPreimage([1; 32]).0.to_vec());
@@ -3047,83 +3633,98 @@ mod tests {
                let txid = Txid::from_hex("56944c5d3f98413ef45cf54545538103cc9f298e0575820ad3591376e2e0f65d").unwrap();
 
                // Justice tx with 1 to_holder, 2 revoked offered HTLCs, 1 revoked received HTLCs
-               let mut claim_tx = Transaction { version: 0, lock_time: 0, input: Vec::new(), output: Vec::new() };
-               for i in 0..4 {
-                       claim_tx.input.push(TxIn {
-                               previous_output: BitcoinOutPoint {
-                                       txid,
-                                       vout: i,
-                               },
-                               script_sig: Script::new(),
-                               sequence: 0xfffffffd,
-                               witness: Vec::new(),
+               for &opt_anchors in [false, true].iter() {
+                       let mut claim_tx = Transaction { version: 0, lock_time: 0, input: Vec::new(), output: Vec::new() };
+                       let mut sum_actual_sigs = 0;
+                       for i in 0..4 {
+                               claim_tx.input.push(TxIn {
+                                       previous_output: BitcoinOutPoint {
+                                               txid,
+                                               vout: i,
+                                       },
+                                       script_sig: Script::new(),
+                                       sequence: 0xfffffffd,
+                                       witness: Vec::new(),
+                               });
+                       }
+                       claim_tx.output.push(TxOut {
+                               script_pubkey: script_pubkey.clone(),
+                               value: 0,
                        });
-               }
-               claim_tx.output.push(TxOut {
-                       script_pubkey: script_pubkey.clone(),
-                       value: 0,
-               });
-               let base_weight = claim_tx.get_weight();
-               let inputs_weight = vec![WEIGHT_REVOKED_OUTPUT, WEIGHT_REVOKED_OFFERED_HTLC, WEIGHT_REVOKED_OFFERED_HTLC, WEIGHT_REVOKED_RECEIVED_HTLC];
-               let mut inputs_total_weight = 2; // count segwit flags
-               {
-                       let mut sighash_parts = bip143::SigHashCache::new(&mut claim_tx);
-                       for (idx, inp) in inputs_weight.iter().enumerate() {
-                               sign_input!(sighash_parts, idx, 0, inp, sum_actual_sigs);
-                               inputs_total_weight += inp;
+                       let base_weight = claim_tx.get_weight();
+                       let inputs_weight = vec![WEIGHT_REVOKED_OUTPUT, weight_revoked_offered_htlc(opt_anchors), weight_revoked_offered_htlc(opt_anchors), weight_revoked_received_htlc(opt_anchors)];
+                       let mut inputs_total_weight = 2; // count segwit flags
+                       {
+                               let mut sighash_parts = bip143::SigHashCache::new(&mut claim_tx);
+                               for (idx, inp) in inputs_weight.iter().enumerate() {
+                                       sign_input!(sighash_parts, idx, 0, inp, sum_actual_sigs, opt_anchors);
+                                       inputs_total_weight += inp;
+                               }
                        }
+                       assert_eq!(base_weight + inputs_total_weight as usize,  claim_tx.get_weight() + /* max_length_sig */ (73 * inputs_weight.len() - sum_actual_sigs));
                }
-               assert_eq!(base_weight + inputs_total_weight as usize,  claim_tx.get_weight() + /* max_length_sig */ (73 * inputs_weight.len() - sum_actual_sigs));
 
                // Claim tx with 1 offered HTLCs, 3 received HTLCs
-               claim_tx.input.clear();
-               sum_actual_sigs = 0;
-               for i in 0..4 {
+               for &opt_anchors in [false, true].iter() {
+                       let mut claim_tx = Transaction { version: 0, lock_time: 0, input: Vec::new(), output: Vec::new() };
+                       let mut sum_actual_sigs = 0;
+                       for i in 0..4 {
+                               claim_tx.input.push(TxIn {
+                                       previous_output: BitcoinOutPoint {
+                                               txid,
+                                               vout: i,
+                                       },
+                                       script_sig: Script::new(),
+                                       sequence: 0xfffffffd,
+                                       witness: Vec::new(),
+                               });
+                       }
+                       claim_tx.output.push(TxOut {
+                               script_pubkey: script_pubkey.clone(),
+                               value: 0,
+                       });
+                       let base_weight = claim_tx.get_weight();
+                       let inputs_weight = vec![weight_offered_htlc(opt_anchors), weight_received_htlc(opt_anchors), weight_received_htlc(opt_anchors), weight_received_htlc(opt_anchors)];
+                       let mut inputs_total_weight = 2; // count segwit flags
+                       {
+                               let mut sighash_parts = bip143::SigHashCache::new(&mut claim_tx);
+                               for (idx, inp) in inputs_weight.iter().enumerate() {
+                                       sign_input!(sighash_parts, idx, 0, inp, sum_actual_sigs, opt_anchors);
+                                       inputs_total_weight += inp;
+                               }
+                       }
+                       assert_eq!(base_weight + inputs_total_weight as usize,  claim_tx.get_weight() + /* max_length_sig */ (73 * inputs_weight.len() - sum_actual_sigs));
+               }
+
+               // Justice tx with 1 revoked HTLC-Success tx output
+               for &opt_anchors in [false, true].iter() {
+                       let mut claim_tx = Transaction { version: 0, lock_time: 0, input: Vec::new(), output: Vec::new() };
+                       let mut sum_actual_sigs = 0;
                        claim_tx.input.push(TxIn {
                                previous_output: BitcoinOutPoint {
                                        txid,
-                                       vout: i,
+                                       vout: 0,
                                },
                                script_sig: Script::new(),
                                sequence: 0xfffffffd,
                                witness: Vec::new(),
                        });
-               }
-               let base_weight = claim_tx.get_weight();
-               let inputs_weight = vec![WEIGHT_OFFERED_HTLC, WEIGHT_RECEIVED_HTLC, WEIGHT_RECEIVED_HTLC, WEIGHT_RECEIVED_HTLC];
-               let mut inputs_total_weight = 2; // count segwit flags
-               {
-                       let mut sighash_parts = bip143::SigHashCache::new(&mut claim_tx);
-                       for (idx, inp) in inputs_weight.iter().enumerate() {
-                               sign_input!(sighash_parts, idx, 0, inp, sum_actual_sigs);
-                               inputs_total_weight += inp;
-                       }
-               }
-               assert_eq!(base_weight + inputs_total_weight as usize,  claim_tx.get_weight() + /* max_length_sig */ (73 * inputs_weight.len() - sum_actual_sigs));
-
-               // Justice tx with 1 revoked HTLC-Success tx output
-               claim_tx.input.clear();
-               sum_actual_sigs = 0;
-               claim_tx.input.push(TxIn {
-                       previous_output: BitcoinOutPoint {
-                               txid,
-                               vout: 0,
-                       },
-                       script_sig: Script::new(),
-                       sequence: 0xfffffffd,
-                       witness: Vec::new(),
-               });
-               let base_weight = claim_tx.get_weight();
-               let inputs_weight = vec![WEIGHT_REVOKED_OUTPUT];
-               let mut inputs_total_weight = 2; // count segwit flags
-               {
-                       let mut sighash_parts = bip143::SigHashCache::new(&mut claim_tx);
-                       for (idx, inp) in inputs_weight.iter().enumerate() {
-                               sign_input!(sighash_parts, idx, 0, inp, sum_actual_sigs);
-                               inputs_total_weight += inp;
+                       claim_tx.output.push(TxOut {
+                               script_pubkey: script_pubkey.clone(),
+                               value: 0,
+                       });
+                       let base_weight = claim_tx.get_weight();
+                       let inputs_weight = vec![WEIGHT_REVOKED_OUTPUT];
+                       let mut inputs_total_weight = 2; // count segwit flags
+                       {
+                               let mut sighash_parts = bip143::SigHashCache::new(&mut claim_tx);
+                               for (idx, inp) in inputs_weight.iter().enumerate() {
+                                       sign_input!(sighash_parts, idx, 0, inp, sum_actual_sigs, opt_anchors);
+                                       inputs_total_weight += inp;
+                               }
                        }
+                       assert_eq!(base_weight + inputs_total_weight as usize, claim_tx.get_weight() + /* max_length_isg */ (73 * inputs_weight.len() - sum_actual_sigs));
                }
-               assert_eq!(base_weight + inputs_total_weight as usize, claim_tx.get_weight() + /* max_length_isg */ (73 * inputs_weight.len() - sum_actual_sigs));
        }
 
        // Further testing is done in the ChannelManager integration tests.