Merge pull request #2439 from tnull/2023-05-fix-0conf-sigs-racing-confirms
[rust-lightning] / lightning / src / ln / channel.rs
index 6df539b537cff8035f50a4625317c785f8cb31f3..e559ac3355007808a18abe58acb562cf8e2c112f 100644 (file)
@@ -27,7 +27,7 @@ use crate::ln::features::{ChannelTypeFeatures, InitFeatures};
 use crate::ln::msgs;
 use crate::ln::msgs::DecodeError;
 use crate::ln::script::{self, ShutdownScript};
-use crate::ln::channelmanager::{self, CounterpartyForwardingInfo, PendingHTLCStatus, HTLCSource, SentHTLCId, HTLCFailureMsg, PendingHTLCInfo, RAACommitmentOrder, BREAKDOWN_TIMEOUT, MIN_CLTV_EXPIRY_DELTA, MAX_LOCAL_BREAKDOWN_TIMEOUT};
+use crate::ln::channelmanager::{self, CounterpartyForwardingInfo, PendingHTLCStatus, HTLCSource, SentHTLCId, HTLCFailureMsg, PendingHTLCInfo, RAACommitmentOrder, BREAKDOWN_TIMEOUT, MIN_CLTV_EXPIRY_DELTA, MAX_LOCAL_BREAKDOWN_TIMEOUT, ChannelShutdownState};
 use crate::ln::chan_utils::{CounterpartyCommitmentSecrets, TxCreationKeys, HTLCOutputInCommitment, htlc_success_tx_weight, htlc_timeout_tx_weight, make_funding_redeemscript, ChannelPublicKeys, CommitmentTransaction, HolderCommitmentTransaction, ChannelTransactionParameters, CounterpartyChannelTransactionParameters, MAX_HTLCS, get_commitment_transaction_number_obscure_factor, ClosingTransaction};
 use crate::ln::chan_utils;
 use crate::ln::onion_utils::HTLCFailReason;
@@ -41,7 +41,7 @@ use crate::routing::gossip::NodeId;
 use crate::util::ser::{Readable, ReadableArgs, Writeable, Writer, VecWriter};
 use crate::util::logger::Logger;
 use crate::util::errors::APIError;
-use crate::util::config::{UserConfig, ChannelConfig, LegacyChannelConfig, ChannelHandshakeConfig, ChannelHandshakeLimits};
+use crate::util::config::{UserConfig, ChannelConfig, LegacyChannelConfig, ChannelHandshakeConfig, ChannelHandshakeLimits, MaxDustHTLCExposure};
 use crate::util::scid_utils::scid_from_parts;
 
 use crate::io;
@@ -224,6 +224,7 @@ struct OutboundHTLCOutput {
        payment_hash: PaymentHash,
        state: OutboundHTLCState,
        source: HTLCSource,
+       skimmed_fee_msat: Option<u64>,
 }
 
 /// See AwaitingRemoteRevoke ChannelState for more info
@@ -235,6 +236,8 @@ enum HTLCUpdateAwaitingACK {
                payment_hash: PaymentHash,
                source: HTLCSource,
                onion_routing_packet: msgs::OnionPacket,
+               // The extra fee we're skimming off the top of this HTLC.
+               skimmed_fee_msat: Option<u64>,
        },
        ClaimHTLC {
                payment_preimage: PaymentPreimage,
@@ -306,10 +309,10 @@ pub const INITIAL_COMMITMENT_NUMBER: u64 = (1 << 48) - 1;
 
 pub const DEFAULT_MAX_HTLCS: u16 = 50;
 
-pub(crate) fn commitment_tx_base_weight(opt_anchors: bool) -> u64 {
+pub(crate) fn commitment_tx_base_weight(channel_type_features: &ChannelTypeFeatures) -> u64 {
        const COMMITMENT_TX_BASE_WEIGHT: u64 = 724;
        const COMMITMENT_TX_BASE_ANCHOR_WEIGHT: u64 = 1124;
-       if opt_anchors { COMMITMENT_TX_BASE_ANCHOR_WEIGHT } else { COMMITMENT_TX_BASE_WEIGHT }
+       if channel_type_features.supports_anchors_zero_fee_htlc_tx() { COMMITMENT_TX_BASE_ANCHOR_WEIGHT } else { COMMITMENT_TX_BASE_WEIGHT }
 }
 
 #[cfg(not(test))]
@@ -485,13 +488,13 @@ enum UpdateFulfillFetch {
 }
 
 /// The return type of get_update_fulfill_htlc_and_commit.
-pub enum UpdateFulfillCommitFetch<'a> {
+pub enum UpdateFulfillCommitFetch {
        /// Indicates the HTLC fulfill is new, and either generated an update_fulfill message, placed
        /// it in the holding cell, or re-generated the update_fulfill message after the same claim was
        /// previously placed in the holding cell (and has since been removed).
        NewClaim {
                /// The ChannelMonitorUpdate which places the new payment preimage in the channel monitor
-               monitor_update: &'a ChannelMonitorUpdate,
+               monitor_update: ChannelMonitorUpdate,
                /// The value of the HTLC which was claimed, in msat.
                htlc_value_msat: u64,
        },
@@ -524,6 +527,10 @@ pub(super) struct ReestablishResponses {
 }
 
 /// The return type of `force_shutdown`
+///
+/// Contains a (counterparty_node_id, funding_txo, [`ChannelMonitorUpdate`]) tuple
+/// followed by a list of HTLCs to fail back in the form of the (source, payment hash, and this
+/// channel's counterparty_node_id and channel_id).
 pub(crate) type ShutdownResult = (
        Option<(PublicKey, OutPoint, ChannelMonitorUpdate)>,
        Vec<(HTLCSource, PaymentHash, PublicKey, [u8; 32])>
@@ -583,21 +590,41 @@ pub(crate) const EXPIRE_PREV_CONFIG_TICKS: usize = 5;
 /// See [`ChannelContext::sent_message_awaiting_response`] for more information.
 pub(crate) const DISCONNECT_PEER_AWAITING_RESPONSE_TICKS: usize = 2;
 
+/// The number of ticks that may elapse while we're waiting for an unfunded outbound/inbound channel
+/// to be promoted to a [`Channel`] since the unfunded channel was created. An unfunded channel
+/// exceeding this age limit will be force-closed and purged from memory.
+pub(crate) const UNFUNDED_CHANNEL_AGE_LIMIT_TICKS: usize = 60;
+
 struct PendingChannelMonitorUpdate {
        update: ChannelMonitorUpdate,
-       /// In some cases we need to delay letting the [`ChannelMonitorUpdate`] go until after an
-       /// `Event` is processed by the user. This bool indicates the [`ChannelMonitorUpdate`] is
-       /// blocked on some external event and the [`ChannelManager`] will update us when we're ready.
-       ///
-       /// [`ChannelManager`]: super::channelmanager::ChannelManager
-       blocked: bool,
 }
 
 impl_writeable_tlv_based!(PendingChannelMonitorUpdate, {
        (0, update, required),
-       (2, blocked, required),
 });
 
+/// Contains all state common to unfunded inbound/outbound channels.
+pub(super) struct UnfundedChannelContext {
+       /// A counter tracking how many ticks have elapsed since this unfunded channel was
+       /// created. If this unfunded channel reaches peer has yet to respond after reaching
+       /// `UNFUNDED_CHANNEL_AGE_LIMIT_TICKS`, it will be force-closed and purged from memory.
+       ///
+       /// This is so that we don't keep channels around that haven't progressed to a funded state
+       /// in a timely manner.
+       unfunded_channel_age_ticks: usize,
+}
+
+impl UnfundedChannelContext {
+       /// Determines whether we should force-close and purge this unfunded channel from memory due to it
+       /// having reached the unfunded channel age limit.
+       ///
+       /// This should be called on every [`super::channelmanager::ChannelManager::timer_tick_occurred`].
+       pub fn should_expire_unfunded_channel(&mut self) -> bool {
+               self.unfunded_channel_age_ticks += 1;
+               self.unfunded_channel_age_ticks >= UNFUNDED_CHANNEL_AGE_LIMIT_TICKS
+       }
+}
+
 /// Contains everything about the channel including state, and various flags.
 pub(super) struct ChannelContext<Signer: ChannelSigner> {
        config: LegacyChannelConfig,
@@ -866,18 +893,12 @@ pub(super) struct ChannelContext<Signer: ChannelSigner> {
        /// [`SignerProvider::derive_channel_signer`].
        channel_keys_id: [u8; 32],
 
-       /// When we generate [`ChannelMonitorUpdate`]s to persist, they may not be persisted immediately.
-       /// If we then persist the [`channelmanager::ChannelManager`] and crash before the persistence
-       /// completes we still need to be able to complete the persistence. Thus, we have to keep a
-       /// copy of the [`ChannelMonitorUpdate`] here until it is complete.
-       pending_monitor_updates: Vec<PendingChannelMonitorUpdate>,
+       /// If we can't release a [`ChannelMonitorUpdate`] until some external action completes, we
+       /// store it here and only release it to the `ChannelManager` once it asks for it.
+       blocked_monitor_updates: Vec<PendingChannelMonitorUpdate>,
 }
 
 impl<Signer: ChannelSigner> ChannelContext<Signer> {
-       pub(crate) fn opt_anchors(&self) -> bool {
-               self.channel_transaction_parameters.opt_anchors.is_some()
-       }
-
        /// Allowed in any state (including after shutdown)
        pub fn get_update_time_counter(&self) -> u32 {
                self.update_time_counter
@@ -913,6 +934,34 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                (self.channel_state & mask) == (ChannelState::ChannelReady as u32) && !self.monitor_pending_channel_ready
        }
 
+       /// shutdown state returns the state of the channel in its various stages of shutdown
+       pub fn shutdown_state(&self) -> ChannelShutdownState {
+               if self.channel_state & (ChannelState::ShutdownComplete as u32) != 0 {
+                       return ChannelShutdownState::ShutdownComplete;
+               }
+               if self.channel_state & (ChannelState::LocalShutdownSent as u32) != 0 &&  self.channel_state & (ChannelState::RemoteShutdownSent as u32) == 0 {
+                       return ChannelShutdownState::ShutdownInitiated;
+               }
+               if (self.channel_state & BOTH_SIDES_SHUTDOWN_MASK != 0) && !self.closing_negotiation_ready() {
+                       return ChannelShutdownState::ResolvingHTLCs;
+               }
+               if (self.channel_state & BOTH_SIDES_SHUTDOWN_MASK != 0) && self.closing_negotiation_ready() {
+                       return ChannelShutdownState::NegotiatingClosingFee;
+               }
+               return ChannelShutdownState::NotShuttingDown;
+       }
+
+       fn closing_negotiation_ready(&self) -> bool {
+               self.pending_inbound_htlcs.is_empty() &&
+               self.pending_outbound_htlcs.is_empty() &&
+               self.pending_update_fee.is_none() &&
+               self.channel_state &
+               (BOTH_SIDES_SHUTDOWN_MASK |
+                       ChannelState::AwaitingRemoteRevoke as u32 |
+                       ChannelState::PeerDisconnected as u32 |
+                       ChannelState::MonitorUpdateInProgress as u32) == BOTH_SIDES_SHUTDOWN_MASK
+       }
+
        /// Returns true if this channel is currently available for use. This is a superset of
        /// is_usable() and considers things like the channel being temporarily disabled.
        /// Allowed in any state (including after shutdown)
@@ -973,7 +1022,7 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
        }
 
        /// Returns the funding_txo we either got from our peer, or were given by
-       /// get_outbound_funding_created.
+       /// get_funding_created.
        pub fn get_funding_txo(&self) -> Option<OutPoint> {
                self.channel_transaction_parameters.funding_outpoint
        }
@@ -1069,8 +1118,18 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                cmp::max(self.config.options.cltv_expiry_delta, MIN_CLTV_EXPIRY_DELTA)
        }
 
-       pub fn get_max_dust_htlc_exposure_msat(&self) -> u64 {
-               self.config.options.max_dust_htlc_exposure_msat
+       pub fn get_max_dust_htlc_exposure_msat<F: Deref>(&self,
+               fee_estimator: &LowerBoundedFeeEstimator<F>) -> u64
+       where F::Target: FeeEstimator
+       {
+               match self.config.options.max_dust_htlc_exposure {
+                       MaxDustHTLCExposure::FeeRateMultiplier(multiplier) => {
+                               let feerate_per_kw = fee_estimator.bounded_sat_per_1000_weight(
+                                       ConfirmationTarget::HighPriority);
+                               feerate_per_kw as u64 * multiplier
+                       },
+                       MaxDustHTLCExposure::FixedLimitMsat(limit) => limit,
+               }
        }
 
        /// Returns the previous [`ChannelConfig`] applied to this channel, if any.
@@ -1204,10 +1263,10 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                        ($htlc: expr, $outbound: expr, $source: expr, $state_name: expr) => {
                                if $outbound == local { // "offered HTLC output"
                                        let htlc_in_tx = get_htlc_in_commitment!($htlc, true);
-                                       let htlc_tx_fee = if self.opt_anchors() {
+                                       let htlc_tx_fee = if self.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                                                0
                                        } else {
-                                               feerate_per_kw as u64 * htlc_timeout_tx_weight(false) / 1000
+                                               feerate_per_kw as u64 * htlc_timeout_tx_weight(self.get_channel_type()) / 1000
                                        };
                                        if $htlc.amount_msat / 1000 >= broadcaster_dust_limit_satoshis + htlc_tx_fee {
                                                log_trace!(logger, "   ...including {} {} HTLC {} (hash {}) with value {}", if $outbound { "outbound" } else { "inbound" }, $state_name, $htlc.htlc_id, log_bytes!($htlc.payment_hash.0), $htlc.amount_msat);
@@ -1218,10 +1277,10 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                                        }
                                } else {
                                        let htlc_in_tx = get_htlc_in_commitment!($htlc, false);
-                                       let htlc_tx_fee = if self.opt_anchors() {
+                                       let htlc_tx_fee = if self.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                                                0
                                        } else {
-                                               feerate_per_kw as u64 * htlc_success_tx_weight(false) / 1000
+                                               feerate_per_kw as u64 * htlc_success_tx_weight(self.get_channel_type()) / 1000
                                        };
                                        if $htlc.amount_msat / 1000 >= broadcaster_dust_limit_satoshis + htlc_tx_fee {
                                                log_trace!(logger, "   ...including {} {} HTLC {} (hash {}) with value {}", if $outbound { "outbound" } else { "inbound" }, $state_name, $htlc.htlc_id, log_bytes!($htlc.payment_hash.0), $htlc.amount_msat);
@@ -1326,8 +1385,8 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                        broadcaster_max_commitment_tx_output.1 = cmp::max(broadcaster_max_commitment_tx_output.1, value_to_remote_msat as u64);
                }
 
-               let total_fee_sat = commit_tx_fee_sat(feerate_per_kw, included_non_dust_htlcs.len(), self.channel_transaction_parameters.opt_anchors.is_some());
-               let anchors_val = if self.channel_transaction_parameters.opt_anchors.is_some() { ANCHOR_OUTPUT_VALUE_SATOSHI * 2 } else { 0 } as i64;
+               let total_fee_sat = commit_tx_fee_sat(feerate_per_kw, included_non_dust_htlcs.len(), &self.channel_transaction_parameters.channel_type_features);
+               let anchors_val = if self.channel_transaction_parameters.channel_type_features.supports_anchors_zero_fee_htlc_tx() { ANCHOR_OUTPUT_VALUE_SATOSHI * 2 } else { 0 } as i64;
                let (value_to_self, value_to_remote) = if self.is_outbound() {
                        (value_to_self_msat / 1000 - anchors_val - total_fee_sat as i64, value_to_remote_msat / 1000)
                } else {
@@ -1362,7 +1421,6 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                let tx = CommitmentTransaction::new_with_auxiliary_htlc_data(commitment_number,
                                                                             value_to_a as u64,
                                                                             value_to_b as u64,
-                                                                            self.channel_transaction_parameters.opt_anchors.is_some(),
                                                                             funding_pubkey_a,
                                                                             funding_pubkey_b,
                                                                             keys.clone(),
@@ -1409,7 +1467,7 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
        #[inline]
        /// Creates a set of keys for build_commitment_transaction to generate a transaction which we
        /// will sign and send to our counterparty.
-       /// If an Err is returned, it is a ChannelError::Close (for get_outbound_funding_created)
+       /// If an Err is returned, it is a ChannelError::Close (for get_funding_created)
        fn build_remote_transaction_keys(&self) -> TxCreationKeys {
                //TODO: Ensure that the payment_key derived here ends up in the library users' wallet as we
                //may see payments to it!
@@ -1470,12 +1528,12 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                        on_holder_tx_holding_cell_htlcs_count: 0,
                };
 
-               let (htlc_timeout_dust_limit, htlc_success_dust_limit) = if context.opt_anchors() {
+               let (htlc_timeout_dust_limit, htlc_success_dust_limit) = if context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                        (0, 0)
                } else {
                        let dust_buffer_feerate = context.get_dust_buffer_feerate(outbound_feerate_update) as u64;
-                       (dust_buffer_feerate * htlc_timeout_tx_weight(false) / 1000,
-                               dust_buffer_feerate * htlc_success_tx_weight(false) / 1000)
+                       (dust_buffer_feerate * htlc_timeout_tx_weight(context.get_channel_type()) / 1000,
+                               dust_buffer_feerate * htlc_success_tx_weight(context.get_channel_type()) / 1000)
                };
                let counterparty_dust_limit_timeout_sat = htlc_timeout_dust_limit + context.counterparty_dust_limit_satoshis;
                let holder_dust_limit_success_sat = htlc_success_dust_limit + context.holder_dust_limit_satoshis;
@@ -1503,12 +1561,12 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                        on_holder_tx_holding_cell_htlcs_count: 0,
                };
 
-               let (htlc_timeout_dust_limit, htlc_success_dust_limit) = if context.opt_anchors() {
+               let (htlc_timeout_dust_limit, htlc_success_dust_limit) = if context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                        (0, 0)
                } else {
                        let dust_buffer_feerate = context.get_dust_buffer_feerate(outbound_feerate_update) as u64;
-                       (dust_buffer_feerate * htlc_timeout_tx_weight(false) / 1000,
-                               dust_buffer_feerate * htlc_success_tx_weight(false) / 1000)
+                       (dust_buffer_feerate * htlc_timeout_tx_weight(context.get_channel_type()) / 1000,
+                               dust_buffer_feerate * htlc_success_tx_weight(context.get_channel_type()) / 1000)
                };
                let counterparty_dust_limit_success_sat = htlc_success_dust_limit + context.counterparty_dust_limit_satoshis;
                let holder_dust_limit_timeout_sat = htlc_timeout_dust_limit + context.holder_dust_limit_satoshis;
@@ -1544,7 +1602,10 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
        /// Doesn't bother handling the
        /// if-we-removed-it-already-but-haven't-fully-resolved-they-can-still-send-an-inbound-HTLC
        /// corner case properly.
-       pub fn get_available_balances(&self) -> AvailableBalances {
+       pub fn get_available_balances<F: Deref>(&self, fee_estimator: &LowerBoundedFeeEstimator<F>)
+       -> AvailableBalances
+       where F::Target: FeeEstimator
+       {
                let context = &self;
                // Note that we have to handle overflow due to the above case.
                let inbound_stats = context.get_inbound_pending_htlc_stats(None);
@@ -1574,8 +1635,8 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                        // dependency.
                        // This complicates the computation around dust-values, up to the one-htlc-value.
                        let mut real_dust_limit_timeout_sat = context.holder_dust_limit_satoshis;
-                       if !context.opt_anchors() {
-                               real_dust_limit_timeout_sat += context.feerate_per_kw as u64 * htlc_timeout_tx_weight(false) / 1000;
+                       if !context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
+                               real_dust_limit_timeout_sat += context.feerate_per_kw as u64 * htlc_timeout_tx_weight(context.get_channel_type()) / 1000;
                        }
 
                        let htlc_above_dust = HTLCCandidate::new(real_dust_limit_timeout_sat * 1000, HTLCInitiator::LocalOffered);
@@ -1600,8 +1661,8 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                        // If the channel is inbound (i.e. counterparty pays the fee), we need to make sure
                        // sending a new HTLC won't reduce their balance below our reserve threshold.
                        let mut real_dust_limit_success_sat = context.counterparty_dust_limit_satoshis;
-                       if !context.opt_anchors() {
-                               real_dust_limit_success_sat += context.feerate_per_kw as u64 * htlc_success_tx_weight(false) / 1000;
+                       if !context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
+                               real_dust_limit_success_sat += context.feerate_per_kw as u64 * htlc_success_tx_weight(context.get_channel_type()) / 1000;
                        }
 
                        let htlc_above_dust = HTLCCandidate::new(real_dust_limit_success_sat * 1000, HTLCInitiator::LocalOffered);
@@ -1626,26 +1687,27 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                // send above the dust limit (as the router can always overpay to meet the dust limit).
                let mut remaining_msat_below_dust_exposure_limit = None;
                let mut dust_exposure_dust_limit_msat = 0;
+               let max_dust_htlc_exposure_msat = context.get_max_dust_htlc_exposure_msat(fee_estimator);
 
-               let (htlc_success_dust_limit, htlc_timeout_dust_limit) = if context.opt_anchors() {
+               let (htlc_success_dust_limit, htlc_timeout_dust_limit) = if context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                        (context.counterparty_dust_limit_satoshis, context.holder_dust_limit_satoshis)
                } else {
                        let dust_buffer_feerate = context.get_dust_buffer_feerate(None) as u64;
-                       (context.counterparty_dust_limit_satoshis + dust_buffer_feerate * htlc_success_tx_weight(false) / 1000,
-                        context.holder_dust_limit_satoshis       + dust_buffer_feerate * htlc_timeout_tx_weight(false) / 1000)
+                       (context.counterparty_dust_limit_satoshis + dust_buffer_feerate * htlc_success_tx_weight(context.get_channel_type()) / 1000,
+                        context.holder_dust_limit_satoshis       + dust_buffer_feerate * htlc_timeout_tx_weight(context.get_channel_type()) / 1000)
                };
                let on_counterparty_dust_htlc_exposure_msat = inbound_stats.on_counterparty_tx_dust_exposure_msat + outbound_stats.on_counterparty_tx_dust_exposure_msat;
-               if on_counterparty_dust_htlc_exposure_msat as i64 + htlc_success_dust_limit as i64 * 1000 - 1 > context.get_max_dust_htlc_exposure_msat() as i64 {
+               if on_counterparty_dust_htlc_exposure_msat as i64 + htlc_success_dust_limit as i64 * 1000 - 1 > max_dust_htlc_exposure_msat as i64 {
                        remaining_msat_below_dust_exposure_limit =
-                               Some(context.get_max_dust_htlc_exposure_msat().saturating_sub(on_counterparty_dust_htlc_exposure_msat));
+                               Some(max_dust_htlc_exposure_msat.saturating_sub(on_counterparty_dust_htlc_exposure_msat));
                        dust_exposure_dust_limit_msat = cmp::max(dust_exposure_dust_limit_msat, htlc_success_dust_limit * 1000);
                }
 
                let on_holder_dust_htlc_exposure_msat = inbound_stats.on_holder_tx_dust_exposure_msat + outbound_stats.on_holder_tx_dust_exposure_msat;
-               if on_holder_dust_htlc_exposure_msat as i64 + htlc_timeout_dust_limit as i64 * 1000 - 1 > context.get_max_dust_htlc_exposure_msat() as i64 {
+               if on_holder_dust_htlc_exposure_msat as i64 + htlc_timeout_dust_limit as i64 * 1000 - 1 > max_dust_htlc_exposure_msat as i64 {
                        remaining_msat_below_dust_exposure_limit = Some(cmp::min(
                                remaining_msat_below_dust_exposure_limit.unwrap_or(u64::max_value()),
-                               context.get_max_dust_htlc_exposure_msat().saturating_sub(on_holder_dust_htlc_exposure_msat)));
+                               max_dust_htlc_exposure_msat.saturating_sub(on_holder_dust_htlc_exposure_msat)));
                        dust_exposure_dust_limit_msat = cmp::max(dust_exposure_dust_limit_msat, htlc_timeout_dust_limit * 1000);
                }
 
@@ -1696,11 +1758,11 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                let context = &self;
                assert!(context.is_outbound());
 
-               let (htlc_success_dust_limit, htlc_timeout_dust_limit) = if context.opt_anchors() {
+               let (htlc_success_dust_limit, htlc_timeout_dust_limit) = if context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                        (0, 0)
                } else {
-                       (context.feerate_per_kw as u64 * htlc_success_tx_weight(false) / 1000,
-                               context.feerate_per_kw as u64 * htlc_timeout_tx_weight(false) / 1000)
+                       (context.feerate_per_kw as u64 * htlc_success_tx_weight(context.get_channel_type()) / 1000,
+                               context.feerate_per_kw as u64 * htlc_timeout_tx_weight(context.get_channel_type()) / 1000)
                };
                let real_dust_limit_success_sat = htlc_success_dust_limit + context.holder_dust_limit_satoshis;
                let real_dust_limit_timeout_sat = htlc_timeout_dust_limit + context.holder_dust_limit_satoshis;
@@ -1759,12 +1821,12 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                }
 
                let num_htlcs = included_htlcs + addl_htlcs;
-               let res = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs, context.opt_anchors());
+               let res = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs, &context.channel_type);
                #[cfg(any(test, fuzzing))]
                {
                        let mut fee = res;
                        if fee_spike_buffer_htlc.is_some() {
-                               fee = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs - 1, context.opt_anchors());
+                               fee = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs - 1, &context.channel_type);
                        }
                        let total_pending_htlcs = context.pending_inbound_htlcs.len() + context.pending_outbound_htlcs.len()
                                + context.holding_cell_htlc_updates.len();
@@ -1800,11 +1862,11 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                let context = &self;
                assert!(!context.is_outbound());
 
-               let (htlc_success_dust_limit, htlc_timeout_dust_limit) = if context.opt_anchors() {
+               let (htlc_success_dust_limit, htlc_timeout_dust_limit) = if context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                        (0, 0)
                } else {
-                       (context.feerate_per_kw as u64 * htlc_success_tx_weight(false) / 1000,
-                               context.feerate_per_kw as u64 * htlc_timeout_tx_weight(false) / 1000)
+                       (context.feerate_per_kw as u64 * htlc_success_tx_weight(context.get_channel_type()) / 1000,
+                               context.feerate_per_kw as u64 * htlc_timeout_tx_weight(context.get_channel_type()) / 1000)
                };
                let real_dust_limit_success_sat = htlc_success_dust_limit + context.counterparty_dust_limit_satoshis;
                let real_dust_limit_timeout_sat = htlc_timeout_dust_limit + context.counterparty_dust_limit_satoshis;
@@ -1850,12 +1912,12 @@ impl<Signer: ChannelSigner> ChannelContext<Signer> {
                }
 
                let num_htlcs = included_htlcs + addl_htlcs;
-               let res = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs, context.opt_anchors());
+               let res = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs, &context.channel_type);
                #[cfg(any(test, fuzzing))]
                {
                        let mut fee = res;
                        if fee_spike_buffer_htlc.is_some() {
-                               fee = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs - 1, context.opt_anchors());
+                               fee = commit_tx_fee_msat(context.feerate_per_kw, num_htlcs - 1, &context.channel_type);
                        }
                        let total_pending_htlcs = context.pending_inbound_htlcs.len() + context.pending_outbound_htlcs.len();
                        let commitment_tx_info = CommitmentTxInfoCached {
@@ -1977,21 +2039,21 @@ pub(crate) fn get_legacy_default_holder_selected_channel_reserve_satoshis(channe
 // Get the fee cost in SATS of a commitment tx with a given number of HTLC outputs.
 // Note that num_htlcs should not include dust HTLCs.
 #[inline]
-fn commit_tx_fee_sat(feerate_per_kw: u32, num_htlcs: usize, opt_anchors: bool) -> u64 {
-       feerate_per_kw as u64 * (commitment_tx_base_weight(opt_anchors) + num_htlcs as u64 * COMMITMENT_TX_WEIGHT_PER_HTLC) / 1000
+fn commit_tx_fee_sat(feerate_per_kw: u32, num_htlcs: usize, channel_type_features: &ChannelTypeFeatures) -> u64 {
+       feerate_per_kw as u64 * (commitment_tx_base_weight(channel_type_features) + num_htlcs as u64 * COMMITMENT_TX_WEIGHT_PER_HTLC) / 1000
 }
 
 // Get the fee cost in MSATS of a commitment tx with a given number of HTLC outputs.
 // Note that num_htlcs should not include dust HTLCs.
-fn commit_tx_fee_msat(feerate_per_kw: u32, num_htlcs: usize, opt_anchors: bool) -> u64 {
+fn commit_tx_fee_msat(feerate_per_kw: u32, num_htlcs: usize, channel_type_features: &ChannelTypeFeatures) -> u64 {
        // Note that we need to divide before multiplying to round properly,
        // since the lowest denomination of bitcoin on-chain is the satoshi.
-       (commitment_tx_base_weight(opt_anchors) + num_htlcs as u64 * COMMITMENT_TX_WEIGHT_PER_HTLC) * feerate_per_kw as u64 / 1000 * 1000
+       (commitment_tx_base_weight(channel_type_features) + num_htlcs as u64 * COMMITMENT_TX_WEIGHT_PER_HTLC) * feerate_per_kw as u64 / 1000 * 1000
 }
 
 // TODO: We should refactor this to be an Inbound/OutboundChannel until initial setup handshaking
 // has been completed, and then turn into a Channel to get compiler-time enforcement of things like
-// calling channel_id() before we're set up or things like get_outbound_funding_signed on an
+// calling channel_id() before we're set up or things like get_funding_signed on an
 // inbound channel.
 //
 // Holder designates channel data owned for the benefit of the user client.
@@ -2010,20 +2072,35 @@ struct CommitmentTxInfoCached {
 }
 
 impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
-       fn check_remote_fee<F: Deref, L: Deref>(fee_estimator: &LowerBoundedFeeEstimator<F>,
-               feerate_per_kw: u32, cur_feerate_per_kw: Option<u32>, logger: &L)
-               -> Result<(), ChannelError> where F::Target: FeeEstimator, L::Target: Logger,
+       fn check_remote_fee<F: Deref, L: Deref>(
+               channel_type: &ChannelTypeFeatures, fee_estimator: &LowerBoundedFeeEstimator<F>,
+               feerate_per_kw: u32, cur_feerate_per_kw: Option<u32>, logger: &L
+       ) -> Result<(), ChannelError> where F::Target: FeeEstimator, L::Target: Logger,
        {
                // We only bound the fee updates on the upper side to prevent completely absurd feerates,
                // always accepting up to 25 sat/vByte or 10x our fee estimator's "High Priority" fee.
                // We generally don't care too much if they set the feerate to something very high, but it
-               // could result in the channel being useless due to everything being dust.
-               let upper_limit = cmp::max(250 * 25,
-                       fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::HighPriority) as u64 * 10);
-               if feerate_per_kw as u64 > upper_limit {
-                       return Err(ChannelError::Close(format!("Peer's feerate much too high. Actual: {}. Our expected upper limit: {}", feerate_per_kw, upper_limit)));
-               }
-               let lower_limit = fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::Background);
+               // could result in the channel being useless due to everything being dust. This doesn't
+               // apply to channels supporting anchor outputs since HTLC transactions are pre-signed with a
+               // zero fee, so their fee is no longer considered to determine dust limits.
+               if !channel_type.supports_anchors_zero_fee_htlc_tx() {
+                       let upper_limit = cmp::max(250 * 25,
+                               fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::HighPriority) as u64 * 10);
+                       if feerate_per_kw as u64 > upper_limit {
+                               return Err(ChannelError::Close(format!("Peer's feerate much too high. Actual: {}. Our expected upper limit: {}", feerate_per_kw, upper_limit)));
+                       }
+               }
+
+               // We can afford to use a lower bound with anchors than previously since we can now bump
+               // fees when broadcasting our commitment. However, we must still make sure we meet the
+               // minimum mempool feerate, until package relay is deployed, such that we can ensure the
+               // commitment transaction propagates throughout node mempools on its own.
+               let lower_limit_conf_target = if channel_type.supports_anchors_zero_fee_htlc_tx() {
+                       ConfirmationTarget::MempoolMinimum
+               } else {
+                       ConfirmationTarget::Background
+               };
+               let lower_limit = fee_estimator.bounded_sat_per_1000_weight(lower_limit_conf_target);
                // Some fee estimators round up to the next full sat/vbyte (ie 250 sats per kw), causing
                // occasional issues with feerate disagreements between an initiator that wants a feerate
                // of 1.1 sat/vbyte and a receiver that wants 1.1 rounded up to 2. Thus, we always add 250
@@ -2261,7 +2338,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        }
 
        pub fn get_update_fulfill_htlc_and_commit<L: Deref>(&mut self, htlc_id: u64, payment_preimage: PaymentPreimage, logger: &L) -> UpdateFulfillCommitFetch where L::Target: Logger {
-               let release_cs_monitor = self.context.pending_monitor_updates.iter().all(|upd| !upd.blocked);
+               let release_cs_monitor = self.context.blocked_monitor_updates.is_empty();
                match self.get_update_fulfill_htlc(htlc_id, payment_preimage, logger) {
                        UpdateFulfillFetch::NewClaim { mut monitor_update, htlc_value_msat, msg } => {
                                // Even if we aren't supposed to let new monitor updates with commitment state
@@ -2269,43 +2346,30 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                // matter what. Sadly, to push a new monitor update which flies before others
                                // already queued, we have to insert it into the pending queue and update the
                                // update_ids of all the following monitors.
-                               let unblocked_update_pos = if release_cs_monitor && msg.is_some() {
+                               if release_cs_monitor && msg.is_some() {
                                        let mut additional_update = self.build_commitment_no_status_check(logger);
                                        // build_commitment_no_status_check may bump latest_monitor_id but we want them
                                        // to be strictly increasing by one, so decrement it here.
                                        self.context.latest_monitor_update_id = monitor_update.update_id;
                                        monitor_update.updates.append(&mut additional_update.updates);
-                                       self.context.pending_monitor_updates.push(PendingChannelMonitorUpdate {
-                                               update: monitor_update, blocked: false,
-                                       });
-                                       self.context.pending_monitor_updates.len() - 1
                                } else {
-                                       let insert_pos = self.context.pending_monitor_updates.iter().position(|upd| upd.blocked)
-                                               .unwrap_or(self.context.pending_monitor_updates.len());
-                                       let new_mon_id = self.context.pending_monitor_updates.get(insert_pos)
+                                       let new_mon_id = self.context.blocked_monitor_updates.get(0)
                                                .map(|upd| upd.update.update_id).unwrap_or(monitor_update.update_id);
                                        monitor_update.update_id = new_mon_id;
-                                       self.context.pending_monitor_updates.insert(insert_pos, PendingChannelMonitorUpdate {
-                                               update: monitor_update, blocked: false,
-                                       });
-                                       for held_update in self.context.pending_monitor_updates.iter_mut().skip(insert_pos + 1) {
+                                       for held_update in self.context.blocked_monitor_updates.iter_mut() {
                                                held_update.update.update_id += 1;
                                        }
                                        if msg.is_some() {
                                                debug_assert!(false, "If there is a pending blocked monitor we should have MonitorUpdateInProgress set");
                                                let update = self.build_commitment_no_status_check(logger);
-                                               self.context.pending_monitor_updates.push(PendingChannelMonitorUpdate {
-                                                       update, blocked: true,
+                                               self.context.blocked_monitor_updates.push(PendingChannelMonitorUpdate {
+                                                       update,
                                                });
                                        }
-                                       insert_pos
-                               };
-                               self.monitor_updating_paused(false, msg.is_some(), false, Vec::new(), Vec::new(), Vec::new());
-                               UpdateFulfillCommitFetch::NewClaim {
-                                       monitor_update: &self.context.pending_monitor_updates.get(unblocked_update_pos)
-                                               .expect("We just pushed the monitor update").update,
-                                       htlc_value_msat,
                                }
+
+                               self.monitor_updating_paused(false, msg.is_some(), false, Vec::new(), Vec::new(), Vec::new());
+                               UpdateFulfillCommitFetch::NewClaim { monitor_update, htlc_value_msat, }
                        },
                        UpdateFulfillFetch::DuplicateClaim {} => UpdateFulfillCommitFetch::DuplicateClaim {},
                }
@@ -2576,8 +2640,13 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                Ok(self.get_announcement_sigs(node_signer, genesis_block_hash, user_config, best_block.height(), logger))
        }
 
-       pub fn update_add_htlc<F, L: Deref>(&mut self, msg: &msgs::UpdateAddHTLC, mut pending_forward_status: PendingHTLCStatus, create_pending_htlc_status: F, logger: &L) -> Result<(), ChannelError>
-       where F: for<'a> Fn(&'a Self, PendingHTLCStatus, u16) -> PendingHTLCStatus, L::Target: Logger {
+       pub fn update_add_htlc<F, FE: Deref, L: Deref>(
+               &mut self, msg: &msgs::UpdateAddHTLC, mut pending_forward_status: PendingHTLCStatus,
+               create_pending_htlc_status: F, fee_estimator: &LowerBoundedFeeEstimator<FE>, logger: &L
+       ) -> Result<(), ChannelError>
+       where F: for<'a> Fn(&'a Self, PendingHTLCStatus, u16) -> PendingHTLCStatus,
+               FE::Target: FeeEstimator, L::Target: Logger,
+       {
                // We can't accept HTLCs sent after we've sent a shutdown.
                let local_sent_shutdown = (self.context.channel_state & (ChannelState::ChannelReady as u32 | ChannelState::LocalShutdownSent as u32)) != (ChannelState::ChannelReady as u32);
                if local_sent_shutdown {
@@ -2630,19 +2699,20 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                        }
                }
 
-               let (htlc_timeout_dust_limit, htlc_success_dust_limit) = if self.context.opt_anchors() {
+               let max_dust_htlc_exposure_msat = self.context.get_max_dust_htlc_exposure_msat(fee_estimator);
+               let (htlc_timeout_dust_limit, htlc_success_dust_limit) = if self.context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
                        (0, 0)
                } else {
                        let dust_buffer_feerate = self.context.get_dust_buffer_feerate(None) as u64;
-                       (dust_buffer_feerate * htlc_timeout_tx_weight(false) / 1000,
-                               dust_buffer_feerate * htlc_success_tx_weight(false) / 1000)
+                       (dust_buffer_feerate * htlc_timeout_tx_weight(self.context.get_channel_type()) / 1000,
+                               dust_buffer_feerate * htlc_success_tx_weight(self.context.get_channel_type()) / 1000)
                };
                let exposure_dust_limit_timeout_sats = htlc_timeout_dust_limit + self.context.counterparty_dust_limit_satoshis;
                if msg.amount_msat / 1000 < exposure_dust_limit_timeout_sats {
                        let on_counterparty_tx_dust_htlc_exposure_msat = inbound_stats.on_counterparty_tx_dust_exposure_msat + outbound_stats.on_counterparty_tx_dust_exposure_msat + msg.amount_msat;
-                       if on_counterparty_tx_dust_htlc_exposure_msat > self.context.get_max_dust_htlc_exposure_msat() {
+                       if on_counterparty_tx_dust_htlc_exposure_msat > max_dust_htlc_exposure_msat {
                                log_info!(logger, "Cannot accept value that would put our exposure to dust HTLCs at {} over the limit {} on counterparty commitment tx",
-                                       on_counterparty_tx_dust_htlc_exposure_msat, self.context.get_max_dust_htlc_exposure_msat());
+                                       on_counterparty_tx_dust_htlc_exposure_msat, max_dust_htlc_exposure_msat);
                                pending_forward_status = create_pending_htlc_status(self, pending_forward_status, 0x1000|7);
                        }
                }
@@ -2650,9 +2720,9 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                let exposure_dust_limit_success_sats = htlc_success_dust_limit + self.context.holder_dust_limit_satoshis;
                if msg.amount_msat / 1000 < exposure_dust_limit_success_sats {
                        let on_holder_tx_dust_htlc_exposure_msat = inbound_stats.on_holder_tx_dust_exposure_msat + outbound_stats.on_holder_tx_dust_exposure_msat + msg.amount_msat;
-                       if on_holder_tx_dust_htlc_exposure_msat > self.context.get_max_dust_htlc_exposure_msat() {
+                       if on_holder_tx_dust_htlc_exposure_msat > max_dust_htlc_exposure_msat {
                                log_info!(logger, "Cannot accept value that would put our exposure to dust HTLCs at {} over the limit {} on holder commitment tx",
-                                       on_holder_tx_dust_htlc_exposure_msat, self.context.get_max_dust_htlc_exposure_msat());
+                                       on_holder_tx_dust_htlc_exposure_msat, max_dust_htlc_exposure_msat);
                                pending_forward_status = create_pending_htlc_status(self, pending_forward_status, 0x1000|7);
                        }
                }
@@ -2795,7 +2865,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                Ok(())
        }
 
-       pub fn commitment_signed<L: Deref>(&mut self, msg: &msgs::CommitmentSigned, logger: &L) -> Result<Option<&ChannelMonitorUpdate>, ChannelError>
+       pub fn commitment_signed<L: Deref>(&mut self, msg: &msgs::CommitmentSigned, logger: &L) -> Result<Option<ChannelMonitorUpdate>, ChannelError>
                where L::Target: Logger
        {
                if (self.context.channel_state & (ChannelState::ChannelReady as u32)) != (ChannelState::ChannelReady as u32) {
@@ -2883,11 +2953,11 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                for (idx, (htlc, mut source_opt)) in htlcs_cloned.drain(..).enumerate() {
                        if let Some(_) = htlc.transaction_output_index {
                                let htlc_tx = chan_utils::build_htlc_transaction(&commitment_txid, commitment_stats.feerate_per_kw,
-                                       self.context.get_counterparty_selected_contest_delay().unwrap(), &htlc, self.context.opt_anchors(),
-                                       false, &keys.broadcaster_delayed_payment_key, &keys.revocation_key);
+                                       self.context.get_counterparty_selected_contest_delay().unwrap(), &htlc, &self.context.channel_type,
+                                       &keys.broadcaster_delayed_payment_key, &keys.revocation_key);
 
-                               let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, self.context.opt_anchors(), &keys);
-                               let htlc_sighashtype = if self.context.opt_anchors() { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
+                               let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, &self.context.channel_type, &keys);
+                               let htlc_sighashtype = if self.context.channel_type.supports_anchors_zero_fee_htlc_tx() { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
                                let htlc_sighash = hash_to_message!(&sighash::SighashCache::new(&htlc_tx).segwit_signature_hash(0, &htlc_redeemscript, htlc.amount_msat / 1000, htlc_sighashtype).unwrap()[..]);
                                log_trace!(logger, "Checking HTLC tx signature {} by key {} against tx {} (sighash {}) with redeemscript {} in channel {}.",
                                        log_bytes!(msg.htlc_signatures[idx].serialize_compact()[..]), log_bytes!(keys.countersignatory_htlc_key.serialize()),
@@ -3019,16 +3089,24 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        /// Public version of the below, checking relevant preconditions first.
        /// If we're not in a state where freeing the holding cell makes sense, this is a no-op and
        /// returns `(None, Vec::new())`.
-       pub fn maybe_free_holding_cell_htlcs<L: Deref>(&mut self, logger: &L) -> (Option<&ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>) where L::Target: Logger {
+       pub fn maybe_free_holding_cell_htlcs<F: Deref, L: Deref>(
+               &mut self, fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+       ) -> (Option<ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>)
+       where F::Target: FeeEstimator, L::Target: Logger
+       {
                if self.context.channel_state >= ChannelState::ChannelReady as u32 &&
                   (self.context.channel_state & (ChannelState::AwaitingRemoteRevoke as u32 | ChannelState::PeerDisconnected as u32 | ChannelState::MonitorUpdateInProgress as u32)) == 0 {
-                       self.free_holding_cell_htlcs(logger)
+                       self.free_holding_cell_htlcs(fee_estimator, logger)
                } else { (None, Vec::new()) }
        }
 
        /// Frees any pending commitment updates in the holding cell, generating the relevant messages
        /// for our counterparty.
-       fn free_holding_cell_htlcs<L: Deref>(&mut self, logger: &L) -> (Option<&ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>) where L::Target: Logger {
+       fn free_holding_cell_htlcs<F: Deref, L: Deref>(
+               &mut self, fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+       ) -> (Option<ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>)
+       where F::Target: FeeEstimator, L::Target: Logger
+       {
                assert_eq!(self.context.channel_state & ChannelState::MonitorUpdateInProgress as u32, 0);
                if self.context.holding_cell_htlc_updates.len() != 0 || self.context.holding_cell_update_fee.is_some() {
                        log_trace!(logger, "Freeing holding cell with {} HTLC updates{} in channel {}", self.context.holding_cell_htlc_updates.len(),
@@ -3052,8 +3130,13 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                // handling this case better and maybe fulfilling some of the HTLCs while attempting
                                // to rebalance channels.
                                match &htlc_update {
-                                       &HTLCUpdateAwaitingACK::AddHTLC {amount_msat, cltv_expiry, ref payment_hash, ref source, ref onion_routing_packet, ..} => {
-                                               match self.send_htlc(amount_msat, *payment_hash, cltv_expiry, source.clone(), onion_routing_packet.clone(), false, logger) {
+                                       &HTLCUpdateAwaitingACK::AddHTLC {
+                                               amount_msat, cltv_expiry, ref payment_hash, ref source, ref onion_routing_packet,
+                                               skimmed_fee_msat, ..
+                                       } => {
+                                               match self.send_htlc(amount_msat, *payment_hash, cltv_expiry, source.clone(),
+                                                       onion_routing_packet.clone(), false, skimmed_fee_msat, fee_estimator, logger)
+                                               {
                                                        Ok(update_add_msg_option) => update_add_htlcs.push(update_add_msg_option.unwrap()),
                                                        Err(e) => {
                                                                match e {
@@ -3112,7 +3195,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                return (None, htlcs_to_fail);
                        }
                        let update_fee = if let Some(feerate) = self.context.holding_cell_update_fee.take() {
-                               self.send_update_fee(feerate, false, logger)
+                               self.send_update_fee(feerate, false, fee_estimator, logger)
                        } else {
                                None
                        };
@@ -3139,8 +3222,10 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        /// waiting on this revoke_and_ack. The generation of this new commitment_signed may also fail,
        /// generating an appropriate error *after* the channel state has been updated based on the
        /// revoke_and_ack message.
-       pub fn revoke_and_ack<L: Deref>(&mut self, msg: &msgs::RevokeAndACK, logger: &L) -> Result<(Vec<(HTLCSource, PaymentHash)>, Option<&ChannelMonitorUpdate>), ChannelError>
-               where L::Target: Logger,
+       pub fn revoke_and_ack<F: Deref, L: Deref>(&mut self, msg: &msgs::RevokeAndACK,
+               fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+       ) -> Result<(Vec<(HTLCSource, PaymentHash)>, Option<ChannelMonitorUpdate>), ChannelError>
+       where F::Target: FeeEstimator, L::Target: Logger,
        {
                if (self.context.channel_state & (ChannelState::ChannelReady as u32)) != (ChannelState::ChannelReady as u32) {
                        return Err(ChannelError::Close("Got revoke/ACK message when channel was not in an operational state".to_owned()));
@@ -3340,9 +3425,8 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                        return Ok((Vec::new(), self.push_ret_blockable_mon_update(monitor_update)));
                }
 
-               match self.free_holding_cell_htlcs(logger) {
-                       (Some(_), htlcs_to_fail) => {
-                               let mut additional_update = self.context.pending_monitor_updates.pop().unwrap().update;
+               match self.free_holding_cell_htlcs(fee_estimator, logger) {
+                       (Some(mut additional_update), htlcs_to_fail) => {
                                // free_holding_cell_htlcs may bump latest_monitor_id multiple times but we want them to be
                                // strictly increasing by one, so decrement it here.
                                self.context.latest_monitor_update_id = monitor_update.update_id;
@@ -3376,8 +3460,11 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        /// Queues up an outbound update fee by placing it in the holding cell. You should call
        /// [`Self::maybe_free_holding_cell_htlcs`] in order to actually generate and send the
        /// commitment update.
-       pub fn queue_update_fee<L: Deref>(&mut self, feerate_per_kw: u32, logger: &L) where L::Target: Logger {
-               let msg_opt = self.send_update_fee(feerate_per_kw, true, logger);
+       pub fn queue_update_fee<F: Deref, L: Deref>(&mut self, feerate_per_kw: u32,
+               fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L)
+       where F::Target: FeeEstimator, L::Target: Logger
+       {
+               let msg_opt = self.send_update_fee(feerate_per_kw, true, fee_estimator, logger);
                assert!(msg_opt.is_none(), "We forced holding cell?");
        }
 
@@ -3388,7 +3475,12 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        ///
        /// You MUST call [`Self::send_commitment_no_state_update`] prior to any other calls on this
        /// [`Channel`] if `force_holding_cell` is false.
-       fn send_update_fee<L: Deref>(&mut self, feerate_per_kw: u32, mut force_holding_cell: bool, logger: &L) -> Option<msgs::UpdateFee> where L::Target: Logger {
+       fn send_update_fee<F: Deref, L: Deref>(
+               &mut self, feerate_per_kw: u32, mut force_holding_cell: bool,
+               fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+       ) -> Option<msgs::UpdateFee>
+       where F::Target: FeeEstimator, L::Target: Logger
+       {
                if !self.context.is_outbound() {
                        panic!("Cannot send fee from inbound channel");
                }
@@ -3404,7 +3496,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                let outbound_stats = self.context.get_outbound_pending_htlc_stats(Some(feerate_per_kw));
                let keys = self.context.build_holder_transaction_keys(self.context.cur_holder_commitment_transaction_number);
                let commitment_stats = self.context.build_commitment_transaction(self.context.cur_holder_commitment_transaction_number, &keys, true, true, logger);
-               let buffer_fee_msat = commit_tx_fee_sat(feerate_per_kw, commitment_stats.num_nondust_htlcs + outbound_stats.on_holder_tx_holding_cell_htlcs_count as usize + CONCURRENT_INBOUND_HTLC_FEE_BUFFER as usize, self.context.opt_anchors()) * 1000;
+               let buffer_fee_msat = commit_tx_fee_sat(feerate_per_kw, commitment_stats.num_nondust_htlcs + outbound_stats.on_holder_tx_holding_cell_htlcs_count as usize + CONCURRENT_INBOUND_HTLC_FEE_BUFFER as usize, self.context.get_channel_type()) * 1000;
                let holder_balance_msat = commitment_stats.local_balance_msat - outbound_stats.holding_cell_msat;
                if holder_balance_msat < buffer_fee_msat  + self.context.counterparty_selected_channel_reserve_satoshis.unwrap() * 1000 {
                        //TODO: auto-close after a number of failures?
@@ -3415,11 +3507,12 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                // Note, we evaluate pending htlc "preemptive" trimmed-to-dust threshold at the proposed `feerate_per_kw`.
                let holder_tx_dust_exposure = inbound_stats.on_holder_tx_dust_exposure_msat + outbound_stats.on_holder_tx_dust_exposure_msat;
                let counterparty_tx_dust_exposure = inbound_stats.on_counterparty_tx_dust_exposure_msat + outbound_stats.on_counterparty_tx_dust_exposure_msat;
-               if holder_tx_dust_exposure > self.context.get_max_dust_htlc_exposure_msat() {
+               let max_dust_htlc_exposure_msat = self.context.get_max_dust_htlc_exposure_msat(fee_estimator);
+               if holder_tx_dust_exposure > max_dust_htlc_exposure_msat {
                        log_debug!(logger, "Cannot afford to send new feerate at {} without infringing max dust htlc exposure", feerate_per_kw);
                        return None;
                }
-               if counterparty_tx_dust_exposure > self.context.get_max_dust_htlc_exposure_msat() {
+               if counterparty_tx_dust_exposure > max_dust_htlc_exposure_msat {
                        log_debug!(logger, "Cannot afford to send new feerate at {} without infringing max dust htlc exposure", feerate_per_kw);
                        return None;
                }
@@ -3558,12 +3651,6 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        {
                assert_eq!(self.context.channel_state & ChannelState::MonitorUpdateInProgress as u32, ChannelState::MonitorUpdateInProgress as u32);
                self.context.channel_state &= !(ChannelState::MonitorUpdateInProgress as u32);
-               let mut found_blocked = false;
-               self.context.pending_monitor_updates.retain(|upd| {
-                       if found_blocked { debug_assert!(upd.blocked, "No mons may be unblocked after a blocked one"); }
-                       if upd.blocked { found_blocked = true; }
-                       upd.blocked
-               });
 
                // If we're past (or at) the FundingSent stage on an outbound channel, try to
                // (re-)broadcast the funding transaction as we may have declined to broadcast it when we
@@ -3643,7 +3730,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                if self.context.channel_state & (ChannelState::PeerDisconnected as u32) == ChannelState::PeerDisconnected as u32 {
                        return Err(ChannelError::Close("Peer sent update_fee when we needed a channel_reestablish".to_owned()));
                }
-               Channel::<Signer>::check_remote_fee(fee_estimator, msg.feerate_per_kw, Some(self.context.feerate_per_kw), logger)?;
+               Channel::<Signer>::check_remote_fee(&self.context.channel_type, fee_estimator, msg.feerate_per_kw, Some(self.context.feerate_per_kw), logger)?;
                let feerate_over_dust_buffer = msg.feerate_per_kw > self.context.get_dust_buffer_feerate(None);
 
                self.context.pending_update_fee = Some((msg.feerate_per_kw, FeeUpdateState::RemoteAnnounced));
@@ -3656,11 +3743,12 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                        let outbound_stats = self.context.get_outbound_pending_htlc_stats(None);
                        let holder_tx_dust_exposure = inbound_stats.on_holder_tx_dust_exposure_msat + outbound_stats.on_holder_tx_dust_exposure_msat;
                        let counterparty_tx_dust_exposure = inbound_stats.on_counterparty_tx_dust_exposure_msat + outbound_stats.on_counterparty_tx_dust_exposure_msat;
-                       if holder_tx_dust_exposure > self.context.get_max_dust_htlc_exposure_msat() {
+                       let max_dust_htlc_exposure_msat = self.context.get_max_dust_htlc_exposure_msat(fee_estimator);
+                       if holder_tx_dust_exposure > max_dust_htlc_exposure_msat {
                                return Err(ChannelError::Close(format!("Peer sent update_fee with a feerate ({}) which may over-expose us to dust-in-flight on our own transactions (totaling {} msat)",
                                        msg.feerate_per_kw, holder_tx_dust_exposure)));
                        }
-                       if counterparty_tx_dust_exposure > self.context.get_max_dust_htlc_exposure_msat() {
+                       if counterparty_tx_dust_exposure > max_dust_htlc_exposure_msat {
                                return Err(ChannelError::Close(format!("Peer sent update_fee with a feerate ({}) which may over-expose us to dust-in-flight on our counterparty's transactions (totaling {} msat)",
                                        msg.feerate_per_kw, counterparty_tx_dust_exposure)));
                        }
@@ -3695,6 +3783,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                        payment_hash: htlc.payment_hash,
                                        cltv_expiry: htlc.cltv_expiry,
                                        onion_routing_packet: (**onion_packet).clone(),
+                                       skimmed_fee_msat: htlc.skimmed_fee_msat,
                                });
                        }
                }
@@ -3981,12 +4070,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        /// this point if we're the funder we should send the initial closing_signed, and in any case
        /// shutdown should complete within a reasonable timeframe.
        fn closing_negotiation_ready(&self) -> bool {
-               self.context.pending_inbound_htlcs.is_empty() && self.context.pending_outbound_htlcs.is_empty() &&
-                       self.context.channel_state &
-                               (BOTH_SIDES_SHUTDOWN_MASK | ChannelState::AwaitingRemoteRevoke as u32 |
-                                ChannelState::PeerDisconnected as u32 | ChannelState::MonitorUpdateInProgress as u32)
-                               == BOTH_SIDES_SHUTDOWN_MASK &&
-                       self.context.pending_update_fee.is_none()
+               self.context.closing_negotiation_ready()
        }
 
        /// Checks if the closing_signed negotiation is making appropriate progress, possibly returning
@@ -4066,7 +4150,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
 
        pub fn shutdown<SP: Deref>(
                &mut self, signer_provider: &SP, their_features: &InitFeatures, msg: &msgs::Shutdown
-       ) -> Result<(Option<msgs::Shutdown>, Option<&ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>), ChannelError>
+       ) -> Result<(Option<msgs::Shutdown>, Option<ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>), ChannelError>
        where SP::Target: SignerProvider
        {
                if self.context.channel_state & (ChannelState::PeerDisconnected as u32) == ChannelState::PeerDisconnected as u32 {
@@ -4132,9 +4216,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                }],
                        };
                        self.monitor_updating_paused(false, false, false, Vec::new(), Vec::new(), Vec::new());
-                       if self.push_blockable_mon_update(monitor_update) {
-                               self.context.pending_monitor_updates.last().map(|upd| &upd.update)
-                       } else { None }
+                       self.push_ret_blockable_mon_update(monitor_update)
                } else { None };
                let shutdown = if send_shutdown {
                        Some(msgs::Shutdown {
@@ -4424,64 +4506,37 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                (self.context.channel_state & ChannelState::MonitorUpdateInProgress as u32) != 0
        }
 
-       pub fn get_latest_complete_monitor_update_id(&self) -> u64 {
-               if self.context.pending_monitor_updates.is_empty() { return self.context.get_latest_monitor_update_id(); }
-               self.context.pending_monitor_updates[0].update.update_id - 1
+       /// Gets the latest [`ChannelMonitorUpdate`] ID which has been released and is in-flight.
+       pub fn get_latest_unblocked_monitor_update_id(&self) -> u64 {
+               if self.context.blocked_monitor_updates.is_empty() { return self.context.get_latest_monitor_update_id(); }
+               self.context.blocked_monitor_updates[0].update.update_id - 1
        }
 
        /// Returns the next blocked monitor update, if one exists, and a bool which indicates a
        /// further blocked monitor update exists after the next.
-       pub fn unblock_next_blocked_monitor_update(&mut self) -> Option<(&ChannelMonitorUpdate, bool)> {
-               for i in 0..self.context.pending_monitor_updates.len() {
-                       if self.context.pending_monitor_updates[i].blocked {
-                               self.context.pending_monitor_updates[i].blocked = false;
-                               return Some((&self.context.pending_monitor_updates[i].update,
-                                       self.context.pending_monitor_updates.len() > i + 1));
-                       }
-               }
-               None
-       }
-
-       /// Pushes a new monitor update into our monitor update queue, returning whether it should be
-       /// immediately given to the user for persisting or if it should be held as blocked.
-       fn push_blockable_mon_update(&mut self, update: ChannelMonitorUpdate) -> bool {
-               let release_monitor = self.context.pending_monitor_updates.iter().all(|upd| !upd.blocked);
-               self.context.pending_monitor_updates.push(PendingChannelMonitorUpdate {
-                       update, blocked: !release_monitor
-               });
-               release_monitor
+       pub fn unblock_next_blocked_monitor_update(&mut self) -> Option<(ChannelMonitorUpdate, bool)> {
+               if self.context.blocked_monitor_updates.is_empty() { return None; }
+               Some((self.context.blocked_monitor_updates.remove(0).update,
+                       !self.context.blocked_monitor_updates.is_empty()))
        }
 
-       /// Pushes a new monitor update into our monitor update queue, returning a reference to it if
-       /// it should be immediately given to the user for persisting or `None` if it should be held as
-       /// blocked.
+       /// Pushes a new monitor update into our monitor update queue, returning it if it should be
+       /// immediately given to the user for persisting or `None` if it should be held as blocked.
        fn push_ret_blockable_mon_update(&mut self, update: ChannelMonitorUpdate)
-       -> Option<&ChannelMonitorUpdate> {
-               let release_monitor = self.push_blockable_mon_update(update);
-               if release_monitor { self.context.pending_monitor_updates.last().map(|upd| &upd.update) } else { None }
-       }
-
-       pub fn no_monitor_updates_pending(&self) -> bool {
-               self.context.pending_monitor_updates.is_empty()
-       }
-
-       pub fn complete_all_mon_updates_through(&mut self, update_id: u64) {
-               self.context.pending_monitor_updates.retain(|upd| {
-                       if upd.update.update_id <= update_id {
-                               assert!(!upd.blocked, "Completed update must have flown");
-                               false
-                       } else { true }
-               });
-       }
-
-       pub fn complete_one_mon_update(&mut self, update_id: u64) {
-               self.context.pending_monitor_updates.retain(|upd| upd.update.update_id != update_id);
+       -> Option<ChannelMonitorUpdate> {
+               let release_monitor = self.context.blocked_monitor_updates.is_empty();
+               if !release_monitor {
+                       self.context.blocked_monitor_updates.push(PendingChannelMonitorUpdate {
+                               update,
+                       });
+                       None
+               } else {
+                       Some(update)
+               }
        }
 
-       /// Returns an iterator over all unblocked monitor updates which have not yet completed.
-       pub fn uncompleted_unblocked_mon_updates(&self) -> impl Iterator<Item=&ChannelMonitorUpdate> {
-               self.context.pending_monitor_updates.iter()
-                       .filter_map(|upd| if upd.blocked { None } else { Some(&upd.update) })
+       pub fn blocked_monitor_updates_pending(&self) -> usize {
+               self.context.blocked_monitor_updates.len()
        }
 
        /// Returns true if the channel is awaiting the persistence of the initial ChannelMonitor.
@@ -5049,11 +5104,16 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        /// commitment update.
        ///
        /// `Err`s will only be [`ChannelError::Ignore`].
-       pub fn queue_add_htlc<L: Deref>(&mut self, amount_msat: u64, payment_hash: PaymentHash, cltv_expiry: u32, source: HTLCSource,
-               onion_routing_packet: msgs::OnionPacket, logger: &L)
-       -> Result<(), ChannelError> where L::Target: Logger {
+       pub fn queue_add_htlc<F: Deref, L: Deref>(
+               &mut self, amount_msat: u64, payment_hash: PaymentHash, cltv_expiry: u32, source: HTLCSource,
+               onion_routing_packet: msgs::OnionPacket, skimmed_fee_msat: Option<u64>,
+               fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+       ) -> Result<(), ChannelError>
+       where F::Target: FeeEstimator, L::Target: Logger
+       {
                self
-                       .send_htlc(amount_msat, payment_hash, cltv_expiry, source, onion_routing_packet, true, logger)
+                       .send_htlc(amount_msat, payment_hash, cltv_expiry, source, onion_routing_packet, true,
+                               skimmed_fee_msat, fee_estimator, logger)
                        .map(|msg_opt| assert!(msg_opt.is_none(), "We forced holding cell?"))
                        .map_err(|err| {
                                if let ChannelError::Ignore(_) = err { /* fine */ }
@@ -5078,9 +5138,13 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        /// on this [`Channel`] if `force_holding_cell` is false.
        ///
        /// `Err`s will only be [`ChannelError::Ignore`].
-       fn send_htlc<L: Deref>(&mut self, amount_msat: u64, payment_hash: PaymentHash, cltv_expiry: u32, source: HTLCSource,
-               onion_routing_packet: msgs::OnionPacket, mut force_holding_cell: bool, logger: &L)
-       -> Result<Option<msgs::UpdateAddHTLC>, ChannelError> where L::Target: Logger {
+       fn send_htlc<F: Deref, L: Deref>(
+               &mut self, amount_msat: u64, payment_hash: PaymentHash, cltv_expiry: u32, source: HTLCSource,
+               onion_routing_packet: msgs::OnionPacket, mut force_holding_cell: bool,
+               skimmed_fee_msat: Option<u64>, fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+       ) -> Result<Option<msgs::UpdateAddHTLC>, ChannelError>
+       where F::Target: FeeEstimator, L::Target: Logger
+       {
                if (self.context.channel_state & (ChannelState::ChannelReady as u32 | BOTH_SIDES_SHUTDOWN_MASK)) != (ChannelState::ChannelReady as u32) {
                        return Err(ChannelError::Ignore("Cannot send HTLC until channel is fully established and we haven't started shutting down".to_owned()));
                }
@@ -5093,7 +5157,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                        return Err(ChannelError::Ignore("Cannot send 0-msat HTLC".to_owned()));
                }
 
-               let available_balances = self.context.get_available_balances();
+               let available_balances = self.context.get_available_balances(fee_estimator);
                if amount_msat < available_balances.next_outbound_htlc_minimum_msat {
                        return Err(ChannelError::Ignore(format!("Cannot send less than our next-HTLC minimum - {} msat",
                                available_balances.next_outbound_htlc_minimum_msat)));
@@ -5132,6 +5196,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                cltv_expiry,
                                source,
                                onion_routing_packet,
+                               skimmed_fee_msat,
                        });
                        return Ok(None);
                }
@@ -5143,6 +5208,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                        cltv_expiry,
                        state: OutboundHTLCState::LocalAnnounced(Box::new(onion_routing_packet.clone())),
                        source,
+                       skimmed_fee_msat,
                });
 
                let res = msgs::UpdateAddHTLC {
@@ -5152,6 +5218,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                        payment_hash,
                        cltv_expiry,
                        onion_routing_packet,
+                       skimmed_fee_msat,
                };
                self.context.next_holder_htlc_id += 1;
 
@@ -5229,7 +5296,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                                && info.next_holder_htlc_id == self.context.next_holder_htlc_id
                                                && info.next_counterparty_htlc_id == self.context.next_counterparty_htlc_id
                                                && info.feerate == self.context.feerate_per_kw {
-                                                       let actual_fee = commit_tx_fee_msat(self.context.feerate_per_kw, commitment_stats.num_nondust_htlcs, self.context.opt_anchors());
+                                                       let actual_fee = commit_tx_fee_msat(self.context.feerate_per_kw, commitment_stats.num_nondust_htlcs, self.context.get_channel_type());
                                                        assert_eq!(actual_fee, info.fee);
                                                }
                                }
@@ -5269,8 +5336,8 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
 
                        for (ref htlc_sig, ref htlc) in htlc_signatures.iter().zip(htlcs) {
                                log_trace!(logger, "Signed remote HTLC tx {} with redeemscript {} with pubkey {} -> {} in channel {}",
-                                       encode::serialize_hex(&chan_utils::build_htlc_transaction(&counterparty_commitment_txid, commitment_stats.feerate_per_kw, self.context.get_holder_selected_contest_delay(), htlc, self.context.opt_anchors(), false, &counterparty_keys.broadcaster_delayed_payment_key, &counterparty_keys.revocation_key)),
-                                       encode::serialize_hex(&chan_utils::get_htlc_redeemscript(&htlc, self.context.opt_anchors(), &counterparty_keys)),
+                                       encode::serialize_hex(&chan_utils::build_htlc_transaction(&counterparty_commitment_txid, commitment_stats.feerate_per_kw, self.context.get_holder_selected_contest_delay(), htlc, &self.context.channel_type, &counterparty_keys.broadcaster_delayed_payment_key, &counterparty_keys.revocation_key)),
+                                       encode::serialize_hex(&chan_utils::get_htlc_redeemscript(&htlc, &self.context.channel_type, &counterparty_keys)),
                                        log_bytes!(counterparty_keys.broadcaster_htlc_key.serialize()),
                                        log_bytes!(htlc_sig.serialize_compact()[..]), log_bytes!(self.context.channel_id()));
                        }
@@ -5290,8 +5357,15 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        ///
        /// Shorthand for calling [`Self::send_htlc`] followed by a commitment update, see docs on
        /// [`Self::send_htlc`] and [`Self::build_commitment_no_state_update`] for more info.
-       pub fn send_htlc_and_commit<L: Deref>(&mut self, amount_msat: u64, payment_hash: PaymentHash, cltv_expiry: u32, source: HTLCSource, onion_routing_packet: msgs::OnionPacket, logger: &L) -> Result<Option<&ChannelMonitorUpdate>, ChannelError> where L::Target: Logger {
-               let send_res = self.send_htlc(amount_msat, payment_hash, cltv_expiry, source, onion_routing_packet, false, logger);
+       pub fn send_htlc_and_commit<F: Deref, L: Deref>(
+               &mut self, amount_msat: u64, payment_hash: PaymentHash, cltv_expiry: u32,
+               source: HTLCSource, onion_routing_packet: msgs::OnionPacket, skimmed_fee_msat: Option<u64>,
+               fee_estimator: &LowerBoundedFeeEstimator<F>, logger: &L
+       ) -> Result<Option<ChannelMonitorUpdate>, ChannelError>
+       where F::Target: FeeEstimator, L::Target: Logger
+       {
+               let send_res = self.send_htlc(amount_msat, payment_hash, cltv_expiry, source,
+                       onion_routing_packet, false, skimmed_fee_msat, fee_estimator, logger);
                if let Err(e) = &send_res { if let ChannelError::Ignore(_) = e {} else { debug_assert!(false, "Sending cannot trigger channel failure"); } }
                match send_res? {
                        Some(_) => {
@@ -5323,7 +5397,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
        /// [`ChannelMonitorUpdate`] will be returned).
        pub fn get_shutdown<SP: Deref>(&mut self, signer_provider: &SP, their_features: &InitFeatures,
                target_feerate_sats_per_kw: Option<u32>, override_shutdown_script: Option<ShutdownScript>)
-       -> Result<(msgs::Shutdown, Option<&ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>), APIError>
+       -> Result<(msgs::Shutdown, Option<ChannelMonitorUpdate>, Vec<(HTLCSource, PaymentHash)>), APIError>
        where SP::Target: SignerProvider {
                for htlc in self.context.pending_outbound_htlcs.iter() {
                        if let OutboundHTLCState::LocalAnnounced(_) = htlc.state {
@@ -5394,9 +5468,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
                                }],
                        };
                        self.monitor_updating_paused(false, false, false, Vec::new(), Vec::new(), Vec::new());
-                       if self.push_blockable_mon_update(monitor_update) {
-                               self.context.pending_monitor_updates.last().map(|upd| &upd.update)
-                       } else { None }
+                       self.push_ret_blockable_mon_update(monitor_update)
                } else { None };
                let shutdown = msgs::Shutdown {
                        channel_id: self.context.channel_id,
@@ -5439,6 +5511,7 @@ impl<Signer: WriteableEcdsaChannelSigner> Channel<Signer> {
 /// A not-yet-funded outbound (from holder) channel using V1 channel establishment.
 pub(super) struct OutboundV1Channel<Signer: ChannelSigner> {
        pub context: ChannelContext<Signer>,
+       pub unfunded_context: UnfundedChannelContext,
 }
 
 impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
@@ -5479,10 +5552,15 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                let channel_type = Self::get_initial_channel_type(&config, their_features);
                debug_assert!(channel_type.is_subset(&channelmanager::provided_channel_type_features(&config)));
 
-               let feerate = fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::Normal);
+               let commitment_conf_target = if channel_type.supports_anchors_zero_fee_htlc_tx() {
+                       ConfirmationTarget::MempoolMinimum
+               } else {
+                       ConfirmationTarget::Normal
+               };
+               let commitment_feerate = fee_estimator.bounded_sat_per_1000_weight(commitment_conf_target);
 
                let value_to_self_msat = channel_value_satoshis * 1000 - push_msat;
-               let commitment_tx_fee = commit_tx_fee_msat(feerate, MIN_AFFORDABLE_HTLC_COUNT, channel_type.requires_anchors_zero_fee_htlc_tx());
+               let commitment_tx_fee = commit_tx_fee_msat(commitment_feerate, MIN_AFFORDABLE_HTLC_COUNT, &channel_type);
                if value_to_self_msat < commitment_tx_fee {
                        return Err(APIError::APIMisuseError{ err: format!("Funding amount ({}) can't even pay fee for initial commitment transaction fee of {}.", value_to_self_msat / 1000, commitment_tx_fee / 1000) });
                }
@@ -5576,7 +5654,7 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                                short_channel_id: None,
                                channel_creation_height: current_chain_height,
 
-                               feerate_per_kw: feerate,
+                               feerate_per_kw: commitment_feerate,
                                counterparty_dust_limit_satoshis: 0,
                                holder_dust_limit_satoshis: MIN_CHAN_DUST_LIMIT_SATOSHIS,
                                counterparty_max_htlc_value_in_flight_msat: 0,
@@ -5597,8 +5675,7 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                                        is_outbound_from_holder: true,
                                        counterparty_parameters: None,
                                        funding_outpoint: None,
-                                       opt_anchors: if channel_type.requires_anchors_zero_fee_htlc_tx() { Some(()) } else { None },
-                                       opt_non_zero_fee_anchors: None
+                                       channel_type_features: channel_type.clone()
                                },
                                funding_transaction: None,
 
@@ -5635,13 +5712,14 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                                channel_type,
                                channel_keys_id,
 
-                               pending_monitor_updates: Vec::new(),
-                       }
+                               blocked_monitor_updates: Vec::new(),
+                       },
+                       unfunded_context: UnfundedChannelContext { unfunded_channel_age_ticks: 0 }
                })
        }
 
-       /// If an Err is returned, it is a ChannelError::Close (for get_outbound_funding_created)
-       fn get_outbound_funding_created_signature<L: Deref>(&mut self, logger: &L) -> Result<Signature, ChannelError> where L::Target: Logger {
+       /// If an Err is returned, it is a ChannelError::Close (for get_funding_created)
+       fn get_funding_created_signature<L: Deref>(&mut self, logger: &L) -> Result<Signature, ChannelError> where L::Target: Logger {
                let counterparty_keys = self.context.build_remote_transaction_keys();
                let counterparty_initial_commitment_tx = self.context.build_commitment_transaction(self.context.cur_counterparty_commitment_transaction_number, &counterparty_keys, false, false, logger).tx;
                Ok(self.context.holder_signer.sign_counterparty_commitment(&counterparty_initial_commitment_tx, Vec::new(), &self.context.secp_ctx)
@@ -5655,7 +5733,7 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
        /// Note that channel_id changes during this call!
        /// Do NOT broadcast the funding transaction until after a successful funding_signed call!
        /// If an Err is returned, it is a ChannelError::Close.
-       pub fn get_outbound_funding_created<L: Deref>(mut self, funding_transaction: Transaction, funding_txo: OutPoint, logger: &L)
+       pub fn get_funding_created<L: Deref>(mut self, funding_transaction: Transaction, funding_txo: OutPoint, logger: &L)
        -> Result<(Channel<Signer>, msgs::FundingCreated), (Self, ChannelError)> where L::Target: Logger {
                if !self.context.is_outbound() {
                        panic!("Tried to create outbound funding_created message on an inbound channel!");
@@ -5672,7 +5750,7 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                self.context.channel_transaction_parameters.funding_outpoint = Some(funding_txo);
                self.context.holder_signer.provide_channel_parameters(&self.context.channel_transaction_parameters);
 
-               let signature = match self.get_outbound_funding_created_signature(logger) {
+               let signature = match self.get_funding_created_signature(logger) {
                        Ok(res) => res,
                        Err(e) => {
                                log_error!(logger, "Got bad signatures: {:?}!", e);
@@ -5720,12 +5798,9 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                // Optionally, if the user would like to negotiate the `anchors_zero_fee_htlc_tx` option, we
                // set it now. If they don't understand it, we'll fall back to our default of
                // `only_static_remotekey`.
-               #[cfg(anchors)]
-               { // Attributes are not allowed on if expressions on our current MSRV of 1.41.
-                       if config.channel_handshake_config.negotiate_anchors_zero_fee_htlc_tx &&
-                               their_features.supports_anchors_zero_fee_htlc_tx() {
-                               ret.set_anchors_zero_fee_htlc_tx_required();
-                       }
+               if config.channel_handshake_config.negotiate_anchors_zero_fee_htlc_tx &&
+                       their_features.supports_anchors_zero_fee_htlc_tx() {
+                       ret.set_anchors_zero_fee_htlc_tx_required();
                }
 
                ret
@@ -5734,7 +5809,12 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
        /// If we receive an error message, it may only be a rejection of the channel type we tried,
        /// not of our ability to open any channel at all. Thus, on error, we should first call this
        /// and see if we get a new `OpenChannel` message, otherwise the channel is failed.
-       pub(crate) fn maybe_handle_error_without_close(&mut self, chain_hash: BlockHash) -> Result<msgs::OpenChannel, ()> {
+       pub(crate) fn maybe_handle_error_without_close<F: Deref>(
+               &mut self, chain_hash: BlockHash, fee_estimator: &LowerBoundedFeeEstimator<F>
+       ) -> Result<msgs::OpenChannel, ()>
+       where
+               F::Target: FeeEstimator
+       {
                if !self.context.is_outbound() || self.context.channel_state != ChannelState::OurInitSent as u32 { return Err(()); }
                if self.context.channel_type == ChannelTypeFeatures::only_static_remote_key() {
                        // We've exhausted our options
@@ -5751,13 +5831,14 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                // whatever reason.
                if self.context.channel_type.supports_anchors_zero_fee_htlc_tx() {
                        self.context.channel_type.clear_anchors_zero_fee_htlc_tx();
-                       assert!(self.context.channel_transaction_parameters.opt_non_zero_fee_anchors.is_none());
-                       self.context.channel_transaction_parameters.opt_anchors = None;
+                       self.context.feerate_per_kw = fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::Normal);
+                       assert!(!self.context.channel_transaction_parameters.channel_type_features.supports_anchors_nonzero_fee_htlc_tx());
                } else if self.context.channel_type.supports_scid_privacy() {
                        self.context.channel_type.clear_scid_privacy();
                } else {
                        self.context.channel_type = ChannelTypeFeatures::only_static_remote_key();
                }
+               self.context.channel_transaction_parameters.channel_type_features = self.context.channel_type.clone();
                Ok(self.get_open_channel(chain_hash))
        }
 
@@ -5876,7 +5957,8 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
                        if channel_type != ChannelTypeFeatures::only_static_remote_key() {
                                return Err(ChannelError::Close("Only static_remote_key is supported for non-negotiated channel types".to_owned()));
                        }
-                       self.context.channel_type = channel_type;
+                       self.context.channel_type = channel_type.clone();
+                       self.context.channel_transaction_parameters.channel_type_features = channel_type;
                }
 
                let counterparty_shutdown_scriptpubkey = if their_features.supports_upfront_shutdown_script() {
@@ -5937,6 +6019,7 @@ impl<Signer: WriteableEcdsaChannelSigner> OutboundV1Channel<Signer> {
 /// A not-yet-funded inbound (from counterparty) channel using V1 channel establishment.
 pub(super) struct InboundV1Channel<Signer: ChannelSigner> {
        pub context: ChannelContext<Signer>,
+       pub unfunded_context: UnfundedChannelContext,
 }
 
 impl<Signer: WriteableEcdsaChannelSigner> InboundV1Channel<Signer> {
@@ -5983,7 +6066,6 @@ impl<Signer: WriteableEcdsaChannelSigner> InboundV1Channel<Signer> {
                        }
                        channel_type
                };
-               let opt_anchors = channel_type.supports_anchors_zero_fee_htlc_tx();
 
                let channel_keys_id = signer_provider.generate_channel_keys_id(true, msg.funding_satoshis, user_id);
                let holder_signer = signer_provider.derive_channel_signer(msg.funding_satoshis, channel_keys_id);
@@ -6020,7 +6102,7 @@ impl<Signer: WriteableEcdsaChannelSigner> InboundV1Channel<Signer> {
                if msg.htlc_minimum_msat >= full_channel_value_msat {
                        return Err(ChannelError::Close(format!("Minimum htlc value ({}) was larger than full channel value ({})", msg.htlc_minimum_msat, full_channel_value_msat)));
                }
-               Channel::<Signer>::check_remote_fee(fee_estimator, msg.feerate_per_kw, None, logger)?;
+               Channel::<Signer>::check_remote_fee(&channel_type, fee_estimator, msg.feerate_per_kw, None, logger)?;
 
                let max_counterparty_selected_contest_delay = u16::min(config.channel_handshake_limits.their_to_self_delay, MAX_LOCAL_BREAKDOWN_TIMEOUT);
                if msg.to_self_delay > max_counterparty_selected_contest_delay {
@@ -6084,7 +6166,7 @@ impl<Signer: WriteableEcdsaChannelSigner> InboundV1Channel<Signer> {
                // check if the funder's amount for the initial commitment tx is sufficient
                // for full fee payment plus a few HTLCs to ensure the channel will be useful.
                let funders_amount_msat = msg.funding_satoshis * 1000 - msg.push_msat;
-               let commitment_tx_fee = commit_tx_fee_msat(msg.feerate_per_kw, MIN_AFFORDABLE_HTLC_COUNT, opt_anchors) / 1000;
+               let commitment_tx_fee = commit_tx_fee_msat(msg.feerate_per_kw, MIN_AFFORDABLE_HTLC_COUNT, &channel_type) / 1000;
                if funders_amount_msat / 1000 < commitment_tx_fee {
                        return Err(ChannelError::Close(format!("Funding amount ({} sats) can't even pay fee for initial commitment transaction fee of {} sats.", funders_amount_msat / 1000, commitment_tx_fee)));
                }
@@ -6227,8 +6309,7 @@ impl<Signer: WriteableEcdsaChannelSigner> InboundV1Channel<Signer> {
                                                pubkeys: counterparty_pubkeys,
                                        }),
                                        funding_outpoint: None,
-                                       opt_anchors: if opt_anchors { Some(()) } else { None },
-                                       opt_non_zero_fee_anchors: None
+                                       channel_type_features: channel_type.clone()
                                },
                                funding_transaction: None,
 
@@ -6265,8 +6346,9 @@ impl<Signer: WriteableEcdsaChannelSigner> InboundV1Channel<Signer> {
                                channel_type,
                                channel_keys_id,
 
-                               pending_monitor_updates: Vec::new(),
-                       }
+                               blocked_monitor_updates: Vec::new(),
+                       },
+                       unfunded_context: UnfundedChannelContext { unfunded_channel_age_ticks: 0 }
                };
 
                Ok(chan)
@@ -6616,9 +6698,10 @@ impl<Signer: WriteableEcdsaChannelSigner> Writeable for Channel<Signer> {
                }
 
                let mut preimages: Vec<&Option<PaymentPreimage>> = vec![];
+               let mut pending_outbound_skimmed_fees: Vec<Option<u64>> = Vec::new();
 
                (self.context.pending_outbound_htlcs.len() as u64).write(writer)?;
-               for htlc in self.context.pending_outbound_htlcs.iter() {
+               for (idx, htlc) in self.context.pending_outbound_htlcs.iter().enumerate() {
                        htlc.htlc_id.write(writer)?;
                        htlc.amount_msat.write(writer)?;
                        htlc.cltv_expiry.write(writer)?;
@@ -6654,18 +6737,37 @@ impl<Signer: WriteableEcdsaChannelSigner> Writeable for Channel<Signer> {
                                        reason.write(writer)?;
                                }
                        }
+                       if let Some(skimmed_fee) = htlc.skimmed_fee_msat {
+                               if pending_outbound_skimmed_fees.is_empty() {
+                                       for _ in 0..idx { pending_outbound_skimmed_fees.push(None); }
+                               }
+                               pending_outbound_skimmed_fees.push(Some(skimmed_fee));
+                       } else if !pending_outbound_skimmed_fees.is_empty() {
+                               pending_outbound_skimmed_fees.push(None);
+                       }
                }
 
+               let mut holding_cell_skimmed_fees: Vec<Option<u64>> = Vec::new();
                (self.context.holding_cell_htlc_updates.len() as u64).write(writer)?;
-               for update in self.context.holding_cell_htlc_updates.iter() {
+               for (idx, update) in self.context.holding_cell_htlc_updates.iter().enumerate() {
                        match update {
-                               &HTLCUpdateAwaitingACK::AddHTLC { ref amount_msat, ref cltv_expiry, ref payment_hash, ref source, ref onion_routing_packet } => {
+                               &HTLCUpdateAwaitingACK::AddHTLC {
+                                       ref amount_msat, ref cltv_expiry, ref payment_hash, ref source, ref onion_routing_packet,
+                                       skimmed_fee_msat,
+                               } => {
                                        0u8.write(writer)?;
                                        amount_msat.write(writer)?;
                                        cltv_expiry.write(writer)?;
                                        payment_hash.write(writer)?;
                                        source.write(writer)?;
                                        onion_routing_packet.write(writer)?;
+
+                                       if let Some(skimmed_fee) = skimmed_fee_msat {
+                                               if holding_cell_skimmed_fees.is_empty() {
+                                                       for _ in 0..idx { holding_cell_skimmed_fees.push(None); }
+                                               }
+                                               holding_cell_skimmed_fees.push(Some(skimmed_fee));
+                                       } else if !holding_cell_skimmed_fees.is_empty() { holding_cell_skimmed_fees.push(None); }
                                },
                                &HTLCUpdateAwaitingACK::ClaimHTLC { ref payment_preimage, ref htlc_id } => {
                                        1u8.write(writer)?;
@@ -6818,10 +6920,11 @@ impl<Signer: WriteableEcdsaChannelSigner> Writeable for Channel<Signer> {
                        (5, self.context.config, required),
                        (6, serialized_holder_htlc_max_in_flight, option),
                        (7, self.context.shutdown_scriptpubkey, option),
+                       (8, self.context.blocked_monitor_updates, optional_vec),
                        (9, self.context.target_closing_feerate_sats_per_kw, option),
-                       (11, self.context.monitor_pending_finalized_fulfills, vec_type),
+                       (11, self.context.monitor_pending_finalized_fulfills, required_vec),
                        (13, self.context.channel_creation_height, required),
-                       (15, preimages, vec_type),
+                       (15, preimages, required_vec),
                        (17, self.context.announcement_sigs_state, required),
                        (19, self.context.latest_inbound_scid_alias, option),
                        (21, self.context.outbound_scid_alias, required),
@@ -6831,7 +6934,8 @@ impl<Signer: WriteableEcdsaChannelSigner> Writeable for Channel<Signer> {
                        (28, holder_max_accepted_htlcs, option),
                        (29, self.context.temporary_channel_id, option),
                        (31, channel_pending_event_emitted, option),
-                       (33, self.context.pending_monitor_updates, vec_type),
+                       (35, pending_outbound_skimmed_fees, optional_vec),
+                       (37, holding_cell_skimmed_fees, optional_vec),
                });
 
                Ok(())
@@ -6942,6 +7046,7 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                                        },
                                        _ => return Err(DecodeError::InvalidValue),
                                },
+                               skimmed_fee_msat: None,
                        });
                }
 
@@ -6955,6 +7060,7 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                                        payment_hash: Readable::read(reader)?,
                                        source: Readable::read(reader)?,
                                        onion_routing_packet: Readable::read(reader)?,
+                                       skimmed_fee_msat: None,
                                },
                                1 => HTLCUpdateAwaitingACK::ClaimHTLC {
                                        payment_preimage: Readable::read(reader)?,
@@ -7051,7 +7157,7 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                        _ => return Err(DecodeError::InvalidValue),
                };
 
-               let channel_parameters: ChannelTransactionParameters = Readable::read(reader)?;
+               let mut channel_parameters: ChannelTransactionParameters = Readable::read(reader)?;
                let funding_transaction = Readable::read(reader)?;
 
                let counterparty_cur_commitment_point = Readable::read(reader)?;
@@ -7108,7 +7214,10 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                let mut temporary_channel_id: Option<[u8; 32]> = None;
                let mut holder_max_accepted_htlcs: Option<u16> = None;
 
-               let mut pending_monitor_updates = Some(Vec::new());
+               let mut blocked_monitor_updates = Some(Vec::new());
+
+               let mut pending_outbound_skimmed_fees_opt: Option<Vec<Option<u64>>> = None;
+               let mut holding_cell_skimmed_fees_opt: Option<Vec<Option<u64>>> = None;
 
                read_tlv_fields!(reader, {
                        (0, announcement_sigs, option),
@@ -7119,10 +7228,11 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                        (5, config, option), // Note that if none is provided we will *not* overwrite the existing one.
                        (6, holder_max_htlc_value_in_flight_msat, option),
                        (7, shutdown_scriptpubkey, option),
+                       (8, blocked_monitor_updates, optional_vec),
                        (9, target_closing_feerate_sats_per_kw, option),
-                       (11, monitor_pending_finalized_fulfills, vec_type),
+                       (11, monitor_pending_finalized_fulfills, optional_vec),
                        (13, channel_creation_height, option),
-                       (15, preimages_opt, vec_type),
+                       (15, preimages_opt, optional_vec),
                        (17, announcement_sigs_state, option),
                        (19, latest_inbound_scid_alias, option),
                        (21, outbound_scid_alias, option),
@@ -7132,7 +7242,8 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                        (28, holder_max_accepted_htlcs, option),
                        (29, temporary_channel_id, option),
                        (31, channel_pending_event_emitted, option),
-                       (33, pending_monitor_updates, vec_type),
+                       (35, pending_outbound_skimmed_fees_opt, optional_vec),
+                       (37, holding_cell_skimmed_fees_opt, optional_vec),
                });
 
                let (channel_keys_id, holder_signer) = if let Some(channel_keys_id) = channel_keys_id {
@@ -7177,6 +7288,10 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                        return Err(DecodeError::UnknownRequiredFeature);
                }
 
+               // ChannelTransactionParameters may have had an empty features set upon deserialization.
+               // To account for that, we're proactively setting/overriding the field here.
+               channel_parameters.channel_type_features = chan_features.clone();
+
                let mut secp_ctx = Secp256k1::new();
                secp_ctx.seeded_randomize(&entropy_source.get_secure_random_bytes());
 
@@ -7187,6 +7302,25 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
 
                let holder_max_accepted_htlcs = holder_max_accepted_htlcs.unwrap_or(DEFAULT_MAX_HTLCS);
 
+               if let Some(skimmed_fees) = pending_outbound_skimmed_fees_opt {
+                       let mut iter = skimmed_fees.into_iter();
+                       for htlc in pending_outbound_htlcs.iter_mut() {
+                               htlc.skimmed_fee_msat = iter.next().ok_or(DecodeError::InvalidValue)?;
+                       }
+                       // We expect all skimmed fees to be consumed above
+                       if iter.next().is_some() { return Err(DecodeError::InvalidValue) }
+               }
+               if let Some(skimmed_fees) = holding_cell_skimmed_fees_opt {
+                       let mut iter = skimmed_fees.into_iter();
+                       for htlc in holding_cell_htlc_updates.iter_mut() {
+                               if let HTLCUpdateAwaitingACK::AddHTLC { ref mut skimmed_fee_msat, .. } = htlc {
+                                       *skimmed_fee_msat = iter.next().ok_or(DecodeError::InvalidValue)?;
+                               }
+                       }
+                       // We expect all skimmed fees to be consumed above
+                       if iter.next().is_some() { return Err(DecodeError::InvalidValue) }
+               }
+
                Ok(Channel {
                        context: ChannelContext {
                                user_id,
@@ -7304,7 +7438,7 @@ impl<'a, 'b, 'c, ES: Deref, SP: Deref> ReadableArgs<(&'a ES, &'b SP, u32, &'c Ch
                                channel_type: channel_type.unwrap(),
                                channel_keys_id,
 
-                               pending_monitor_updates: pending_monitor_updates.unwrap(),
+                               blocked_monitor_updates: blocked_monitor_updates.unwrap(),
                        }
                })
        }
@@ -7321,7 +7455,6 @@ mod tests {
        use hex;
        use crate::ln::PaymentHash;
        use crate::ln::channelmanager::{self, HTLCSource, PaymentId};
-       #[cfg(anchors)]
        use crate::ln::channel::InitFeatures;
        use crate::ln::channel::{Channel, InboundHTLCOutput, OutboundV1Channel, InboundV1Channel, OutboundHTLCOutput, InboundHTLCState, OutboundHTLCState, HTLCCandidate, HTLCInitiator, commit_tx_fee_msat};
        use crate::ln::channel::{MAX_FUNDING_SATOSHIS_NO_WUMBO, TOTAL_BITCOIN_SUPPLY_SATOSHIS, MIN_THEIR_CHAN_RESERVE_SATOSHIS};
@@ -7372,7 +7505,8 @@ mod tests {
                // arithmetic, causing a panic with debug assertions enabled.
                let fee_est = TestFeeEstimator { fee_est: 42 };
                let bounded_fee_estimator = LowerBoundedFeeEstimator::new(&fee_est);
-               assert!(Channel::<InMemorySigner>::check_remote_fee(&bounded_fee_estimator,
+               assert!(Channel::<InMemorySigner>::check_remote_fee(
+                       &ChannelTypeFeatures::only_static_remote_key(), &bounded_fee_estimator,
                        u32::max_value(), None, &&test_utils::TestLogger::new()).is_err());
        }
 
@@ -7411,7 +7545,7 @@ mod tests {
                }
        }
 
-       #[cfg(not(feature = "grind_signatures"))]
+       #[cfg(all(feature = "_test_vectors", not(feature = "grind_signatures")))]
        fn public_from_secret_hex(secp_ctx: &Secp256k1<bitcoin::secp256k1::All>, hex: &str) -> PublicKey {
                PublicKey::from_secret_key(&secp_ctx, &SecretKey::from_slice(&hex::decode(hex).unwrap()[..]).unwrap())
        }
@@ -7502,7 +7636,7 @@ mod tests {
                        value: 10000000, script_pubkey: output_script.clone(),
                }]};
                let funding_outpoint = OutPoint{ txid: tx.txid(), index: 0 };
-               let (mut node_a_chan, funding_created_msg) = node_a_chan.get_outbound_funding_created(tx.clone(), funding_outpoint, &&logger).map_err(|_| ()).unwrap();
+               let (mut node_a_chan, funding_created_msg) = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, &&logger).map_err(|_| ()).unwrap();
                let (_, funding_signed_msg, _) = node_b_chan.funding_created(&funding_created_msg, best_block, &&keys_provider, &&logger).map_err(|_| ()).unwrap();
 
                // Node B --> Node A: funding signed
@@ -7529,20 +7663,21 @@ mod tests {
                                session_priv: SecretKey::from_slice(&hex::decode("0fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff").unwrap()[..]).unwrap(),
                                first_hop_htlc_msat: 548,
                                payment_id: PaymentId([42; 32]),
-                       }
+                       },
+                       skimmed_fee_msat: None,
                });
 
                // Make sure when Node A calculates their local commitment transaction, none of the HTLCs pass
                // the dust limit check.
                let htlc_candidate = HTLCCandidate::new(htlc_amount_msat, HTLCInitiator::LocalOffered);
                let local_commit_tx_fee = node_a_chan.context.next_local_commit_tx_fee_msat(htlc_candidate, None);
-               let local_commit_fee_0_htlcs = commit_tx_fee_msat(node_a_chan.context.feerate_per_kw, 0, node_a_chan.context.opt_anchors());
+               let local_commit_fee_0_htlcs = commit_tx_fee_msat(node_a_chan.context.feerate_per_kw, 0, node_a_chan.context.get_channel_type());
                assert_eq!(local_commit_tx_fee, local_commit_fee_0_htlcs);
 
                // Finally, make sure that when Node A calculates the remote's commitment transaction fees, all
                // of the HTLCs are seen to be above the dust limit.
                node_a_chan.context.channel_transaction_parameters.is_outbound_from_holder = false;
-               let remote_commit_fee_3_htlcs = commit_tx_fee_msat(node_a_chan.context.feerate_per_kw, 3, node_a_chan.context.opt_anchors());
+               let remote_commit_fee_3_htlcs = commit_tx_fee_msat(node_a_chan.context.feerate_per_kw, 3, node_a_chan.context.get_channel_type());
                let htlc_candidate = HTLCCandidate::new(htlc_amount_msat, HTLCInitiator::LocalOffered);
                let remote_commit_tx_fee = node_a_chan.context.next_remote_commit_tx_fee_msat(htlc_candidate, None);
                assert_eq!(remote_commit_tx_fee, remote_commit_fee_3_htlcs);
@@ -7564,18 +7699,18 @@ mod tests {
                let config = UserConfig::default();
                let mut chan = OutboundV1Channel::<EnforcingSigner>::new(&fee_est, &&keys_provider, &&keys_provider, node_id, &channelmanager::provided_init_features(&config), 10000000, 100000, 42, &config, 0, 42).unwrap();
 
-               let commitment_tx_fee_0_htlcs = commit_tx_fee_msat(chan.context.feerate_per_kw, 0, chan.context.opt_anchors());
-               let commitment_tx_fee_1_htlc = commit_tx_fee_msat(chan.context.feerate_per_kw, 1, chan.context.opt_anchors());
+               let commitment_tx_fee_0_htlcs = commit_tx_fee_msat(chan.context.feerate_per_kw, 0, chan.context.get_channel_type());
+               let commitment_tx_fee_1_htlc = commit_tx_fee_msat(chan.context.feerate_per_kw, 1, chan.context.get_channel_type());
 
                // If HTLC_SUCCESS_TX_WEIGHT and HTLC_TIMEOUT_TX_WEIGHT were swapped: then this HTLC would be
                // counted as dust when it shouldn't be.
-               let htlc_amt_above_timeout = ((253 * htlc_timeout_tx_weight(chan.context.opt_anchors()) / 1000) + chan.context.holder_dust_limit_satoshis + 1) * 1000;
+               let htlc_amt_above_timeout = ((253 * htlc_timeout_tx_weight(chan.context.get_channel_type()) / 1000) + chan.context.holder_dust_limit_satoshis + 1) * 1000;
                let htlc_candidate = HTLCCandidate::new(htlc_amt_above_timeout, HTLCInitiator::LocalOffered);
                let commitment_tx_fee = chan.context.next_local_commit_tx_fee_msat(htlc_candidate, None);
                assert_eq!(commitment_tx_fee, commitment_tx_fee_1_htlc);
 
                // If swapped: this HTLC would be counted as non-dust when it shouldn't be.
-               let dust_htlc_amt_below_success = ((253 * htlc_success_tx_weight(chan.context.opt_anchors()) / 1000) + chan.context.holder_dust_limit_satoshis - 1) * 1000;
+               let dust_htlc_amt_below_success = ((253 * htlc_success_tx_weight(chan.context.get_channel_type()) / 1000) + chan.context.holder_dust_limit_satoshis - 1) * 1000;
                let htlc_candidate = HTLCCandidate::new(dust_htlc_amt_below_success, HTLCInitiator::RemoteOffered);
                let commitment_tx_fee = chan.context.next_local_commit_tx_fee_msat(htlc_candidate, None);
                assert_eq!(commitment_tx_fee, commitment_tx_fee_0_htlcs);
@@ -7583,13 +7718,13 @@ mod tests {
                chan.context.channel_transaction_parameters.is_outbound_from_holder = false;
 
                // If swapped: this HTLC would be counted as non-dust when it shouldn't be.
-               let dust_htlc_amt_above_timeout = ((253 * htlc_timeout_tx_weight(chan.context.opt_anchors()) / 1000) + chan.context.counterparty_dust_limit_satoshis + 1) * 1000;
+               let dust_htlc_amt_above_timeout = ((253 * htlc_timeout_tx_weight(chan.context.get_channel_type()) / 1000) + chan.context.counterparty_dust_limit_satoshis + 1) * 1000;
                let htlc_candidate = HTLCCandidate::new(dust_htlc_amt_above_timeout, HTLCInitiator::LocalOffered);
                let commitment_tx_fee = chan.context.next_remote_commit_tx_fee_msat(htlc_candidate, None);
                assert_eq!(commitment_tx_fee, commitment_tx_fee_0_htlcs);
 
                // If swapped: this HTLC would be counted as dust when it shouldn't be.
-               let htlc_amt_below_success = ((253 * htlc_success_tx_weight(chan.context.opt_anchors()) / 1000) + chan.context.counterparty_dust_limit_satoshis - 1) * 1000;
+               let htlc_amt_below_success = ((253 * htlc_success_tx_weight(chan.context.get_channel_type()) / 1000) + chan.context.counterparty_dust_limit_satoshis - 1) * 1000;
                let htlc_candidate = HTLCCandidate::new(htlc_amt_below_success, HTLCInitiator::RemoteOffered);
                let commitment_tx_fee = chan.context.next_remote_commit_tx_fee_msat(htlc_candidate, None);
                assert_eq!(commitment_tx_fee, commitment_tx_fee_1_htlc);
@@ -7628,7 +7763,7 @@ mod tests {
                        value: 10000000, script_pubkey: output_script.clone(),
                }]};
                let funding_outpoint = OutPoint{ txid: tx.txid(), index: 0 };
-               let (mut node_a_chan, funding_created_msg) = node_a_chan.get_outbound_funding_created(tx.clone(), funding_outpoint, &&logger).map_err(|_| ()).unwrap();
+               let (mut node_a_chan, funding_created_msg) = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, &&logger).map_err(|_| ()).unwrap();
                let (mut node_b_chan, funding_signed_msg, _) = node_b_chan.funding_created(&funding_created_msg, best_block, &&keys_provider, &&logger).map_err(|_| ()).unwrap();
 
                // Node B --> Node A: funding signed
@@ -7816,7 +7951,7 @@ mod tests {
                        value: 10000000, script_pubkey: output_script.clone(),
                }]};
                let funding_outpoint = OutPoint{ txid: tx.txid(), index: 0 };
-               let (mut node_a_chan, funding_created_msg) = node_a_chan.get_outbound_funding_created(tx.clone(), funding_outpoint, &&logger).map_err(|_| ()).unwrap();
+               let (mut node_a_chan, funding_created_msg) = node_a_chan.get_funding_created(tx.clone(), funding_outpoint, &&logger).map_err(|_| ()).unwrap();
                let (_, funding_signed_msg, _) = node_b_chan.funding_created(&funding_created_msg, best_block, &&keys_provider, &&logger).map_err(|_| ()).unwrap();
 
                // Node B --> Node A: funding signed
@@ -7937,15 +8072,15 @@ mod tests {
 
                macro_rules! test_commitment {
                        ( $counterparty_sig_hex: expr, $sig_hex: expr, $tx_hex: expr, $($remain:tt)* ) => {
-                               chan.context.channel_transaction_parameters.opt_anchors = None;
-                               test_commitment_common!($counterparty_sig_hex, $sig_hex, $tx_hex, false, $($remain)*);
+                               chan.context.channel_transaction_parameters.channel_type_features = ChannelTypeFeatures::only_static_remote_key();
+                               test_commitment_common!($counterparty_sig_hex, $sig_hex, $tx_hex, &ChannelTypeFeatures::only_static_remote_key(), $($remain)*);
                        };
                }
 
                macro_rules! test_commitment_with_anchors {
                        ( $counterparty_sig_hex: expr, $sig_hex: expr, $tx_hex: expr, $($remain:tt)* ) => {
-                               chan.context.channel_transaction_parameters.opt_anchors = Some(());
-                               test_commitment_common!($counterparty_sig_hex, $sig_hex, $tx_hex, true, $($remain)*);
+                               chan.context.channel_transaction_parameters.channel_type_features = ChannelTypeFeatures::anchors_zero_htlc_fee_and_dependencies();
+                               test_commitment_common!($counterparty_sig_hex, $sig_hex, $tx_hex, &ChannelTypeFeatures::anchors_zero_htlc_fee_and_dependencies(), $($remain)*);
                        };
                }
 
@@ -8004,9 +8139,9 @@ mod tests {
                                        let ref htlc = htlcs[$htlc_idx];
                                        let htlc_tx = chan_utils::build_htlc_transaction(&unsigned_tx.txid, chan.context.feerate_per_kw,
                                                chan.context.get_counterparty_selected_contest_delay().unwrap(),
-                                               &htlc, $opt_anchors, false, &keys.broadcaster_delayed_payment_key, &keys.revocation_key);
+                                               &htlc, $opt_anchors, &keys.broadcaster_delayed_payment_key, &keys.revocation_key);
                                        let htlc_redeemscript = chan_utils::get_htlc_redeemscript(&htlc, $opt_anchors, &keys);
-                                       let htlc_sighashtype = if $opt_anchors { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
+                                       let htlc_sighashtype = if $opt_anchors.supports_anchors_zero_fee_htlc_tx() { EcdsaSighashType::SinglePlusAnyoneCanPay } else { EcdsaSighashType::All };
                                        let htlc_sighash = Message::from_slice(&sighash::SighashCache::new(&htlc_tx).segwit_signature_hash(0, &htlc_redeemscript, htlc.amount_msat / 1000, htlc_sighashtype).unwrap()[..]).unwrap();
                                        assert!(secp_ctx.verify_ecdsa(&htlc_sighash, &remote_signature, &keys.countersignatory_htlc_key).is_ok(), "verify counterparty htlc sig");
 
@@ -8023,7 +8158,7 @@ mod tests {
                                        }
 
                                        let htlc_sig = htlc_sig_iter.next().unwrap();
-                                       let num_anchors = if $opt_anchors { 2 } else { 0 };
+                                       let num_anchors = if $opt_anchors.supports_anchors_zero_fee_htlc_tx() { 2 } else { 0 };
                                        assert_eq!((htlc_sig.0).0.transaction_output_index, Some($htlc_idx + num_anchors), "output index");
 
                                        let signature = Signature::from_der(&hex::decode($htlc_sig_hex).unwrap()[..]).unwrap();
@@ -8086,6 +8221,7 @@ mod tests {
                                payment_hash: PaymentHash([0; 32]),
                                state: OutboundHTLCState::Committed,
                                source: HTLCSource::dummy(),
+                               skimmed_fee_msat: None,
                        };
                        out.payment_hash.0 = Sha256::hash(&hex::decode("0202020202020202020202020202020202020202020202020202020202020202").unwrap()).into_inner();
                        out
@@ -8098,6 +8234,7 @@ mod tests {
                                payment_hash: PaymentHash([0; 32]),
                                state: OutboundHTLCState::Committed,
                                source: HTLCSource::dummy(),
+                               skimmed_fee_msat: None,
                        };
                        out.payment_hash.0 = Sha256::hash(&hex::decode("0303030303030303030303030303030303030303030303030303030303030303").unwrap()).into_inner();
                        out
@@ -8342,6 +8479,8 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 2185;
                chan.context.holder_dust_limit_satoshis = 2001;
+               let cached_channel_type = chan.context.channel_type;
+               chan.context.channel_type = ChannelTypeFeatures::anchors_zero_htlc_fee_and_dependencies();
 
                test_commitment_with_anchors!("3044022040f63a16148cf35c8d3d41827f5ae7f7c3746885bb64d4d1b895892a83812b3e02202fcf95c2bf02c466163b3fa3ced6a24926fbb4035095a96842ef516e86ba54c0",
                                 "3045022100cd8479cfe1edb1e5a1d487391e0451a469c7171e51e680183f19eb4321f20e9b02204eab7d5a6384b1b08e03baa6e4d9748dfd2b5ab2bae7e39604a0d0055bbffdd5",
@@ -8362,6 +8501,7 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 3702;
                chan.context.holder_dust_limit_satoshis = 546;
+               chan.context.channel_type = cached_channel_type.clone();
 
                test_commitment!("304502210092a587aeb777f869e7ff0d7898ea619ee26a3dacd1f3672b945eea600be431100220077ee9eae3528d15251f2a52b607b189820e57a6ccfac8d1af502b132ee40169",
                                 "3045022100e5efb73c32d32da2d79702299b6317de6fb24a60476e3855926d78484dd1b3c802203557cb66a42c944ef06e00bcc4da35a5bcb2f185aab0f8e403e519e1d66aaf75",
@@ -8396,6 +8536,7 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 3687;
                chan.context.holder_dust_limit_satoshis = 3001;
+               chan.context.channel_type = ChannelTypeFeatures::anchors_zero_htlc_fee_and_dependencies();
 
                test_commitment_with_anchors!("3045022100ad6c71569856b2d7ff42e838b4abe74a713426b37f22fa667a195a4c88908c6902202b37272b02a42dc6d9f4f82cab3eaf84ac882d9ed762859e1e75455c2c228377",
                                 "3045022100c970799bcb33f43179eb43b3378a0a61991cf2923f69b36ef12548c3df0e6d500220413dc27d2e39ee583093adfcb7799be680141738babb31cc7b0669a777a31f5d",
@@ -8411,6 +8552,7 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 4914;
                chan.context.holder_dust_limit_satoshis = 546;
+               chan.context.channel_type = cached_channel_type.clone();
 
                test_commitment!("3045022100b4b16d5f8cc9fc4c1aff48831e832a0d8990e133978a66e302c133550954a44d022073573ce127e2200d316f6b612803a5c0c97b8d20e1e44dbe2ac0dd2fb8c95244",
                                 "3045022100d72638bc6308b88bb6d45861aae83e5b9ff6e10986546e13bce769c70036e2620220320be7c6d66d22f30b9fcd52af66531505b1310ca3b848c19285b38d8a1a8c19",
@@ -8435,6 +8577,7 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 4894;
                chan.context.holder_dust_limit_satoshis = 4001;
+               chan.context.channel_type = ChannelTypeFeatures::anchors_zero_htlc_fee_and_dependencies();
 
                test_commitment_with_anchors!("3045022100e784a66b1588575801e237d35e510fd92a81ae3a4a2a1b90c031ad803d07b3f3022021bc5f16501f167607d63b681442da193eb0a76b4b7fd25c2ed4f8b28fd35b95",
                                 "30450221009f16ac85d232e4eddb3fcd750a68ebf0b58e3356eaada45d3513ede7e817bf4c02207c2b043b4e5f971261975406cb955219fa56bffe5d834a833694b5abc1ce4cfd",
@@ -8444,6 +8587,7 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 9651180;
                chan.context.holder_dust_limit_satoshis = 546;
+               chan.context.channel_type = cached_channel_type.clone();
 
                test_commitment!("304402200a8544eba1d216f5c5e530597665fa9bec56943c0f66d98fc3d028df52d84f7002201e45fa5c6bc3a506cc2553e7d1c0043a9811313fc39c954692c0d47cfce2bbd3",
                                 "3045022100e11b638c05c650c2f63a421d36ef8756c5ce82f2184278643520311cdf50aa200220259565fb9c8e4a87ccaf17f27a3b9ca4f20625754a0920d9c6c239d8156a11de",
@@ -8461,6 +8605,7 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 6216010;
                chan.context.holder_dust_limit_satoshis = 4001;
+               chan.context.channel_type = ChannelTypeFeatures::anchors_zero_htlc_fee_and_dependencies();
 
                test_commitment_with_anchors!("30450221008fd5dbff02e4b59020d4cd23a3c30d3e287065fda75a0a09b402980adf68ccda022001e0b8b620cd915ddff11f1de32addf23d81d51b90e6841b2cb8dcaf3faa5ecf",
                                 "30450221009ad80792e3038fe6968d12ff23e6888a565c3ddd065037f357445f01675d63f3022018384915e5f1f4ae157e15debf4f49b61c8d9d2b073c7d6f97c4a68caa3ed4c1",
@@ -8470,6 +8615,7 @@ mod tests {
                chan.context.value_to_self_msat = 6993000000; // 7000000000 - 7000000
                chan.context.feerate_per_kw = 9651936;
                chan.context.holder_dust_limit_satoshis = 546;
+               chan.context.channel_type = cached_channel_type;
 
                test_commitment!("304402202ade0142008309eb376736575ad58d03e5b115499709c6db0b46e36ff394b492022037b63d78d66404d6504d4c4ac13be346f3d1802928a6d3ad95a6a944227161a2",
                                 "304402207e8d51e0c570a5868a78414f4e0cbfaed1106b171b9581542c30718ee4eb95ba02203af84194c97adf98898c9afe2f2ed4a7f8dba05a2dfab28ac9d9c604aa49a379",
@@ -8499,6 +8645,7 @@ mod tests {
                                payment_hash: PaymentHash([0; 32]),
                                state: OutboundHTLCState::Committed,
                                source: HTLCSource::dummy(),
+                               skimmed_fee_msat: None,
                        };
                        out.payment_hash.0 = Sha256::hash(&hex::decode("0505050505050505050505050505050505050505050505050505050505050505").unwrap()).into_inner();
                        out
@@ -8511,6 +8658,7 @@ mod tests {
                                payment_hash: PaymentHash([0; 32]),
                                state: OutboundHTLCState::Committed,
                                source: HTLCSource::dummy(),
+                               skimmed_fee_msat: None,
                        };
                        out.payment_hash.0 = Sha256::hash(&hex::decode("0505050505050505050505050505050505050505050505050505050505050505").unwrap()).into_inner();
                        out
@@ -8534,6 +8682,7 @@ mod tests {
                                  "020000000001014bdccf28653066a2c554cafeffdfe1e678e64a69b056684deb0c4fba909423ec02000000000000000001e1120000000000002200204adb4e2f00643db396dd120d4e7dc17625f5f2c11a40d857accc862d6b7dd80e05004730440220471c9f3ad92e49b13b7b8059f43ecf8f7887b0dccbb9fdb54bfe23d62a8ae332022024bd22fae0740e86a44228c35330da9526fd7306dffb2b9dc362d5e78abef7cc0147304402207157f452f2506d73c315192311893800cfb3cc235cc1185b1cfcc136b55230db022014be242dbc6c5da141fec4034e7f387f74d6ff1899453d72ba957467540e1ecb01008576a91414011f7254d96b819c76986c277d115efce6f7b58763ac67210394854aa6eab5b2a8122cc726e9dded053a2184d88256816826d6231c068d4a5b7c820120876475527c21030d417a46946384f88d5f3337267c5e579765875dc4daca813e21734b140639e752ae67a9142002cc93ebefbb1b73f0af055dcc27a0b504ad7688ac6868fa010000" }
                } );
 
+               chan.context.channel_type = ChannelTypeFeatures::anchors_zero_htlc_fee_and_dependencies();
                test_commitment_with_anchors!("3044022027b38dfb654c34032ffb70bb43022981652fce923cbbe3cbe7394e2ade8b34230220584195b78da6e25c2e8da6b4308d9db25b65b64975db9266163ef592abb7c725",
                                 "3045022100b4014970d9d7962853f3f85196144671d7d5d87426250f0a5fdaf9a55292e92502205360910c9abb397467e19dbd63d081deb4a3240903114c98cec0a23591b79b76",
                                 "02000000000101bef67e4e2fb9ddeeb3461973cd4c62abb35050b1add772995b820b584a488489000000000038b02b80074a010000000000002200202b1b5854183c12d3316565972c4668929d314d81c5dcdbb21cb45fe8a9a8114f4a01000000000000220020e9e86e4823faa62e222ebc858a226636856158f07e69898da3b0d1af0ddb3994d007000000000000220020fe0598d74fee2205cc3672e6e6647706b4f3099713b4661b62482c3addd04a5e881300000000000022002018e40f9072c44350f134bdc887bab4d9bdfc8aa468a25616c80e21757ba5dac7881300000000000022002018e40f9072c44350f134bdc887bab4d9bdfc8aa468a25616c80e21757ba5dac7c0c62d0000000000220020f3394e1e619b0eca1f91be2fb5ab4dfc59ba5b84ebe014ad1d43a564d012994aad9c6a00000000002200204adb4e2f00643db396dd120d4e7dc17625f5f2c11a40d857accc862d6b7dd80e0400483045022100b4014970d9d7962853f3f85196144671d7d5d87426250f0a5fdaf9a55292e92502205360910c9abb397467e19dbd63d081deb4a3240903114c98cec0a23591b79b7601473044022027b38dfb654c34032ffb70bb43022981652fce923cbbe3cbe7394e2ade8b34230220584195b78da6e25c2e8da6b4308d9db25b65b64975db9266163ef592abb7c72501475221023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb21030e9f7b623d2ccc7c9bd44d66d5ce21ce504c0acf6385a132cec6d3c39fa711c152ae3e195220", {
@@ -8630,7 +8779,6 @@ mod tests {
                assert!(res.is_ok());
        }
 
-       #[cfg(anchors)]
        #[test]
        fn test_supports_anchors_zero_htlc_tx_fee() {
                // Tests that if both sides support and negotiate `anchors_zero_fee_htlc_tx`, it is the
@@ -8676,7 +8824,6 @@ mod tests {
                assert_eq!(channel_b.context.channel_type, expected_channel_type);
        }
 
-       #[cfg(anchors)]
        #[test]
        fn test_rejects_implicit_simple_anchors() {
                // Tests that if `option_anchors` is being negotiated implicitly through the intersection of
@@ -8717,7 +8864,6 @@ mod tests {
                assert!(channel_b.is_err());
        }
 
-       #[cfg(anchors)]
        #[test]
        fn test_rejects_simple_anchors_channel_type() {
                // Tests that if `option_anchors` is being negotiated through the `channel_type` feature,
@@ -8739,8 +8885,8 @@ mod tests {
                let simple_anchors_raw_features = static_remote_key_required | simple_anchors_required;
                let simple_anchors_init = InitFeatures::from_le_bytes(simple_anchors_raw_features.to_le_bytes().to_vec());
                let simple_anchors_channel_type = ChannelTypeFeatures::from_le_bytes(simple_anchors_raw_features.to_le_bytes().to_vec());
-               assert!(simple_anchors_init.requires_unknown_bits());
-               assert!(simple_anchors_channel_type.requires_unknown_bits());
+               assert!(!simple_anchors_init.requires_unknown_bits());
+               assert!(!simple_anchors_channel_type.requires_unknown_bits());
 
                // First, we'll try to open a channel between A and B where A requests a channel type for
                // the original `option_anchors` feature (non zero fee htlc tx). This should be rejected by