Send warning messages when repeating shutdown messages at volume
[rust-lightning] / lightning / src / ln / channelmanager.rs
index bf17464253aad4e09ecec60a921b6285880bc682..15f536789bf43713279e5e231e415d26bf5b1129 100644 (file)
@@ -40,12 +40,12 @@ use crate::events::{Event, EventHandler, EventsProvider, MessageSendEvent, Messa
 // Since this struct is returned in `list_channels` methods, expose it here in case users want to
 // construct one themselves.
 use crate::ln::{inbound_payment, PaymentHash, PaymentPreimage, PaymentSecret};
-use crate::ln::channel::{Channel, ChannelContext, ChannelError, ChannelUpdateStatus, ShutdownResult, UpdateFulfillCommitFetch, OutboundV1Channel, InboundV1Channel};
+use crate::ln::channel::{Channel, ChannelContext, ChannelError, ChannelUpdateStatus, ShutdownResult, UnfundedChannelContext, UpdateFulfillCommitFetch, OutboundV1Channel, InboundV1Channel};
 use crate::ln::features::{ChannelFeatures, ChannelTypeFeatures, InitFeatures, NodeFeatures};
 #[cfg(any(feature = "_test_utils", test))]
-use crate::ln::features::InvoiceFeatures;
+use crate::ln::features::Bolt11InvoiceFeatures;
 use crate::routing::gossip::NetworkGraph;
-use crate::routing::router::{BlindedTail, DefaultRouter, InFlightHtlcs, Path, Payee, PaymentParameters, Route, RouteHop, RouteParameters, Router};
+use crate::routing::router::{BlindedTail, DefaultRouter, InFlightHtlcs, Path, Payee, PaymentParameters, Route, RouteParameters, Router};
 use crate::routing::scoring::{ProbabilisticScorer, ProbabilisticScoringFeeParameters};
 use crate::ln::msgs;
 use crate::ln::onion_utils;
@@ -53,7 +53,7 @@ use crate::ln::onion_utils::HTLCFailReason;
 use crate::ln::msgs::{ChannelMessageHandler, DecodeError, LightningError};
 #[cfg(test)]
 use crate::ln::outbound_payment;
-use crate::ln::outbound_payment::{OutboundPayments, PaymentAttempts, PendingOutboundPayment};
+use crate::ln::outbound_payment::{OutboundPayments, PaymentAttempts, PendingOutboundPayment, SendAlongPathArgs};
 use crate::ln::wire::Encode;
 use crate::sign::{EntropySource, KeysManager, NodeSigner, Recipient, SignerProvider, ChannelSigner, WriteableEcdsaChannelSigner};
 use crate::util::config::{UserConfig, ChannelConfig, ChannelConfigUpdate};
@@ -110,6 +110,8 @@ pub(super) enum PendingHTLCRouting {
                payment_metadata: Option<Vec<u8>>,
                incoming_cltv_expiry: u32, // Used to track when we should expire pending HTLCs that go unclaimed
                phantom_shared_secret: Option<[u8; 32]>,
+               /// See [`RecipientOnionFields::custom_tlvs`] for more info.
+               custom_tlvs: Vec<(u64, Vec<u8>)>,
        },
        ReceiveKeysend {
                /// This was added in 0.0.116 and will break deserialization on downgrades.
@@ -117,6 +119,8 @@ pub(super) enum PendingHTLCRouting {
                payment_preimage: PaymentPreimage,
                payment_metadata: Option<Vec<u8>>,
                incoming_cltv_expiry: u32, // Used to track when we should expire pending HTLCs that go unclaimed
+               /// See [`RecipientOnionFields::custom_tlvs`] for more info.
+               custom_tlvs: Vec<(u64, Vec<u8>)>,
        },
 }
 
@@ -341,7 +345,7 @@ impl HTLCSource {
        }
 }
 
-struct ReceiveError {
+struct InboundOnionErr {
        err_code: u16,
        err_data: Vec<u8>,
        msg: &'static str,
@@ -355,15 +359,32 @@ struct ReceiveError {
 pub enum FailureCode {
        /// We had a temporary error processing the payment. Useful if no other error codes fit
        /// and you want to indicate that the payer may want to retry.
-       TemporaryNodeFailure             = 0x2000 | 2,
+       TemporaryNodeFailure,
        /// We have a required feature which was not in this onion. For example, you may require
        /// some additional metadata that was not provided with this payment.
-       RequiredNodeFeatureMissing       = 0x4000 | 0x2000 | 3,
+       RequiredNodeFeatureMissing,
        /// You may wish to use this when a `payment_preimage` is unknown, or the CLTV expiry of
        /// the HTLC is too close to the current block height for safe handling.
        /// Using this failure code in [`ChannelManager::fail_htlc_backwards_with_reason`] is
        /// equivalent to calling [`ChannelManager::fail_htlc_backwards`].
-       IncorrectOrUnknownPaymentDetails = 0x4000 | 15,
+       IncorrectOrUnknownPaymentDetails,
+       /// We failed to process the payload after the onion was decrypted. You may wish to
+       /// use this when receiving custom HTLC TLVs with even type numbers that you don't recognize.
+       ///
+       /// If available, the tuple data may include the type number and byte offset in the
+       /// decrypted byte stream where the failure occurred.
+       InvalidOnionPayload(Option<(u64, u16)>),
+}
+
+impl Into<u16> for FailureCode {
+    fn into(self) -> u16 {
+               match self {
+                       FailureCode::TemporaryNodeFailure => 0x2000 | 2,
+                       FailureCode::RequiredNodeFeatureMissing => 0x4000 | 0x2000 | 3,
+                       FailureCode::IncorrectOrUnknownPaymentDetails => 0x4000 | 15,
+                       FailureCode::InvalidOnionPayload(_) => 0x4000 | 22,
+               }
+       }
 }
 
 /// Error type returned across the peer_state mutex boundary. When an Err is generated for a
@@ -376,6 +397,7 @@ struct MsgHandleErrInternal {
        err: msgs::LightningError,
        chan_id: Option<([u8; 32], u128)>, // If Some a channel of ours has been closed
        shutdown_finish: Option<(ShutdownResult, Option<msgs::ChannelUpdate>)>,
+       channel_capacity: Option<u64>,
 }
 impl MsgHandleErrInternal {
        #[inline]
@@ -392,14 +414,15 @@ impl MsgHandleErrInternal {
                        },
                        chan_id: None,
                        shutdown_finish: None,
+                       channel_capacity: None,
                }
        }
        #[inline]
        fn from_no_close(err: msgs::LightningError) -> Self {
-               Self { err, chan_id: None, shutdown_finish: None }
+               Self { err, chan_id: None, shutdown_finish: None, channel_capacity: None }
        }
        #[inline]
-       fn from_finish_shutdown(err: String, channel_id: [u8; 32], user_channel_id: u128, shutdown_res: ShutdownResult, channel_update: Option<msgs::ChannelUpdate>) -> Self {
+       fn from_finish_shutdown(err: String, channel_id: [u8; 32], user_channel_id: u128, shutdown_res: ShutdownResult, channel_update: Option<msgs::ChannelUpdate>, channel_capacity: u64) -> Self {
                Self {
                        err: LightningError {
                                err: err.clone(),
@@ -412,6 +435,7 @@ impl MsgHandleErrInternal {
                        },
                        chan_id: Some((channel_id, user_channel_id)),
                        shutdown_finish: Some((shutdown_res, channel_update)),
+                       channel_capacity: Some(channel_capacity)
                }
        }
        #[inline]
@@ -444,6 +468,7 @@ impl MsgHandleErrInternal {
                        },
                        chan_id: None,
                        shutdown_finish: None,
+                       channel_capacity: None,
                }
        }
 }
@@ -507,19 +532,19 @@ struct ClaimablePayments {
 /// running normally, and specifically must be processed before any other non-background
 /// [`ChannelMonitorUpdate`]s are applied.
 enum BackgroundEvent {
-       /// Handle a ChannelMonitorUpdate which closes the channel. This is only separated from
-       /// [`Self::MonitorUpdateRegeneratedOnStartup`] as the maybe-non-closing variant needs a public
-       /// key to handle channel resumption, whereas if the channel has been force-closed we do not
-       /// need the counterparty node_id.
+       /// Handle a ChannelMonitorUpdate which closes the channel or for an already-closed channel.
+       /// This is only separated from [`Self::MonitorUpdateRegeneratedOnStartup`] as the
+       /// maybe-non-closing variant needs a public key to handle channel resumption, whereas if the
+       /// channel has been force-closed we do not need the counterparty node_id.
        ///
        /// Note that any such events are lost on shutdown, so in general they must be updates which
        /// are regenerated on startup.
-       ClosingMonitorUpdateRegeneratedOnStartup((OutPoint, ChannelMonitorUpdate)),
+       ClosedMonitorUpdateRegeneratedOnStartup((OutPoint, ChannelMonitorUpdate)),
        /// Handle a ChannelMonitorUpdate which may or may not close the channel and may unblock the
        /// channel to continue normal operation.
        ///
        /// In general this should be used rather than
-       /// [`Self::ClosingMonitorUpdateRegeneratedOnStartup`], however in cases where the
+       /// [`Self::ClosedMonitorUpdateRegeneratedOnStartup`], however in cases where the
        /// `counterparty_node_id` is not available as the channel has closed from a [`ChannelMonitor`]
        /// error the other variant is acceptable.
        ///
@@ -530,6 +555,13 @@ enum BackgroundEvent {
                funding_txo: OutPoint,
                update: ChannelMonitorUpdate
        },
+       /// Some [`ChannelMonitorUpdate`] (s) completed before we were serialized but we still have
+       /// them marked pending, thus we need to run any [`MonitorUpdateCompletionAction`] (s) pending
+       /// on a channel.
+       MonitorUpdatesComplete {
+               counterparty_node_id: PublicKey,
+               channel_id: [u8; 32],
+       },
 }
 
 #[derive(Debug)]
@@ -628,6 +660,13 @@ pub(super) struct PeerState<Signer: ChannelSigner> {
        /// been assigned a `channel_id`, the entry in this map is removed and one is created in
        /// `channel_by_id`.
        pub(super) inbound_v1_channel_by_id: HashMap<[u8; 32], InboundV1Channel<Signer>>,
+       /// `temporary_channel_id` -> `InboundChannelRequest`.
+       ///
+       /// When manual channel acceptance is enabled, this holds all unaccepted inbound channels where
+       /// the peer is the counterparty. If the channel is accepted, then the entry in this table is
+       /// removed, and an InboundV1Channel is created and placed in the `inbound_v1_channel_by_id` table. If
+       /// the channel is rejected, then the entry is simply removed.
+       pub(super) inbound_channel_request_by_id: HashMap<[u8; 32], InboundChannelRequest>,
        /// The latest `InitFeatures` we heard from the peer.
        latest_features: InitFeatures,
        /// Messages to send to the peer - pushed to in the same lock that they are generated in (except
@@ -678,21 +717,36 @@ impl <Signer: ChannelSigner> PeerState<Signer> {
                        && self.in_flight_monitor_updates.is_empty()
        }
 
-       // Returns a count of all channels we have with this peer, including pending channels.
+       // Returns a count of all channels we have with this peer, including unfunded channels.
        fn total_channel_count(&self) -> usize {
                self.channel_by_id.len() +
                        self.outbound_v1_channel_by_id.len() +
-                       self.inbound_v1_channel_by_id.len()
+                       self.inbound_v1_channel_by_id.len() +
+                       self.inbound_channel_request_by_id.len()
        }
 
        // Returns a bool indicating if the given `channel_id` matches a channel we have with this peer.
        fn has_channel(&self, channel_id: &[u8; 32]) -> bool {
                self.channel_by_id.contains_key(channel_id) ||
                        self.outbound_v1_channel_by_id.contains_key(channel_id) ||
-                       self.inbound_v1_channel_by_id.contains_key(channel_id)
+                       self.inbound_v1_channel_by_id.contains_key(channel_id) ||
+                       self.inbound_channel_request_by_id.contains_key(channel_id)
        }
 }
 
+/// A not-yet-accepted inbound (from counterparty) channel. Once
+/// accepted, the parameters will be used to construct a channel.
+pub(super) struct InboundChannelRequest {
+       /// The original OpenChannel message.
+       pub open_channel_msg: msgs::OpenChannel,
+       /// The number of ticks remaining before the request expires.
+       pub ticks_remaining: i32,
+}
+
+/// The number of ticks that may elapse while we're waiting for an unaccepted inbound channel to be
+/// accepted. An unaccepted channel that exceeds this limit will be abandoned.
+const UNACCEPTED_INBOUND_CHANNEL_AGE_LIMIT_TICKS: i32 = 2;
+
 /// Stores a PaymentSecret and any other data we may need to validate an inbound payment is
 /// actually ours and not some duplicate HTLC sent to us by a node along the route.
 ///
@@ -752,7 +806,23 @@ pub type SimpleArcChannelManager<M, T, F, L> = ChannelManager<
 /// of [`KeysManager`] and [`DefaultRouter`].
 ///
 /// This is not exported to bindings users as Arcs don't make sense in bindings
-pub type SimpleRefChannelManager<'a, 'b, 'c, 'd, 'e, 'f, 'g, 'h, M, T, F, L> = ChannelManager<&'a M, &'b T, &'c KeysManager, &'c KeysManager, &'c KeysManager, &'d F, &'e DefaultRouter<&'f NetworkGraph<&'g L>, &'g L, &'h Mutex<ProbabilisticScorer<&'f NetworkGraph<&'g L>, &'g L>>, ProbabilisticScoringFeeParameters, ProbabilisticScorer<&'f NetworkGraph<&'g L>, &'g L>>, &'g L>;
+pub type SimpleRefChannelManager<'a, 'b, 'c, 'd, 'e, 'f, 'g, 'h, M, T, F, L> =
+       ChannelManager<
+               &'a M,
+               &'b T,
+               &'c KeysManager,
+               &'c KeysManager,
+               &'c KeysManager,
+               &'d F,
+               &'e DefaultRouter<
+                       &'f NetworkGraph<&'g L>,
+                       &'g L,
+                       &'h Mutex<ProbabilisticScorer<&'f NetworkGraph<&'g L>, &'g L>>,
+                       ProbabilisticScoringFeeParameters,
+                       ProbabilisticScorer<&'f NetworkGraph<&'g L>, &'g L>
+               >,
+               &'g L
+       >;
 
 macro_rules! define_test_pub_trait { ($vis: vis) => {
 /// A trivial trait which describes any [`ChannelManager`] used in testing.
@@ -1309,6 +1379,12 @@ pub struct ChannelCounterparty {
 }
 
 /// Details of a channel, as returned by [`ChannelManager::list_channels`] and [`ChannelManager::list_usable_channels`]
+///
+/// Balances of a channel are available through [`ChainMonitor::get_claimable_balances`] and
+/// [`ChannelMonitor::get_claimable_balances`], calculated with respect to the corresponding on-chain
+/// transactions.
+///
+/// [`ChainMonitor::get_claimable_balances`]: crate::chain::chainmonitor::ChainMonitor::get_claimable_balances
 #[derive(Clone, Debug, PartialEq)]
 pub struct ChannelDetails {
        /// The channel's ID (prior to funding transaction generation, this is a random 32 bytes,
@@ -1375,33 +1451,26 @@ pub struct ChannelDetails {
        ///
        /// [`outbound_capacity_msat`]: ChannelDetails::outbound_capacity_msat
        pub unspendable_punishment_reserve: Option<u64>,
-       /// The `user_channel_id` passed in to create_channel, or a random value if the channel was
-       /// inbound. This may be zero for inbound channels serialized with LDK versions prior to
-       /// 0.0.113.
+       /// The `user_channel_id` value passed in to [`ChannelManager::create_channel`] for outbound
+       /// channels, or to [`ChannelManager::accept_inbound_channel`] for inbound channels if
+       /// [`UserConfig::manually_accept_inbound_channels`] config flag is set to true. Otherwise
+       /// `user_channel_id` will be randomized for an inbound channel.  This may be zero for objects
+       /// serialized with LDK versions prior to 0.0.113.
+       ///
+       /// [`ChannelManager::create_channel`]: crate::ln::channelmanager::ChannelManager::create_channel
+       /// [`ChannelManager::accept_inbound_channel`]: crate::ln::channelmanager::ChannelManager::accept_inbound_channel
+       /// [`UserConfig::manually_accept_inbound_channels`]: crate::util::config::UserConfig::manually_accept_inbound_channels
        pub user_channel_id: u128,
        /// The currently negotiated fee rate denominated in satoshi per 1000 weight units,
        /// which is applied to commitment and HTLC transactions.
        ///
        /// This value will be `None` for objects serialized with LDK versions prior to 0.0.115.
        pub feerate_sat_per_1000_weight: Option<u32>,
-       /// Our total balance.  This is the amount we would get if we close the channel.
-       /// This value is not exact. Due to various in-flight changes and feerate changes, exactly this
-       /// amount is not likely to be recoverable on close.
-       ///
-       /// This does not include any pending HTLCs which are not yet fully resolved (and, thus, whose
-       /// balance is not available for inclusion in new outbound HTLCs). This further does not include
-       /// any pending outgoing HTLCs which are awaiting some other resolution to be sent.
-       /// This does not consider any on-chain fees.
-       ///
-       /// See also [`ChannelDetails::outbound_capacity_msat`]
-       pub balance_msat: u64,
        /// The available outbound capacity for sending HTLCs to the remote peer. This does not include
        /// any pending HTLCs which are not yet fully resolved (and, thus, whose balance is not
        /// available for inclusion in new outbound HTLCs). This further does not include any pending
        /// outgoing HTLCs which are awaiting some other resolution to be sent.
        ///
-       /// See also [`ChannelDetails::balance_msat`]
-       ///
        /// This value is not exact. Due to various in-flight changes, feerate changes, and our
        /// conflict-avoidance policy, exactly this amount is not likely to be spendable. However, we
        /// should be able to spend nearly this amount.
@@ -1411,8 +1480,8 @@ pub struct ChannelDetails {
        /// the current state and per-HTLC limit(s). This is intended for use when routing, allowing us
        /// to use a limit as close as possible to the HTLC limit we can currently send.
        ///
-       /// See also [`ChannelDetails::next_outbound_htlc_minimum_msat`],
-       /// [`ChannelDetails::balance_msat`], and [`ChannelDetails::outbound_capacity_msat`].
+       /// See also [`ChannelDetails::next_outbound_htlc_minimum_msat`] and
+       /// [`ChannelDetails::outbound_capacity_msat`].
        pub next_outbound_htlc_limit_msat: u64,
        /// The minimum value for sending a single HTLC to the remote peer. This is the equivalent of
        /// [`ChannelDetails::next_outbound_htlc_limit_msat`] but represents a lower-bound, rather than
@@ -1463,6 +1532,9 @@ pub struct ChannelDetails {
        ///
        /// [`confirmations_required`]: ChannelDetails::confirmations_required
        pub is_channel_ready: bool,
+       /// The stage of the channel's shutdown.
+       /// `None` for `ChannelDetails` serialized on LDK versions prior to 0.0.116.
+       pub channel_shutdown_state: Option<ChannelShutdownState>,
        /// True if the channel is (a) confirmed and channel_ready messages have been exchanged, (b)
        /// the peer is connected, and (c) the channel is not currently negotiating a shutdown.
        ///
@@ -1502,10 +1574,13 @@ impl ChannelDetails {
                self.short_channel_id.or(self.outbound_scid_alias)
        }
 
-       fn from_channel_context<Signer: WriteableEcdsaChannelSigner>(context: &ChannelContext<Signer>,
-               best_block_height: u32, latest_features: InitFeatures) -> Self {
-
-               let balance = context.get_available_balances();
+       fn from_channel_context<Signer: WriteableEcdsaChannelSigner, F: Deref>(
+               context: &ChannelContext<Signer>, best_block_height: u32, latest_features: InitFeatures,
+               fee_estimator: &LowerBoundedFeeEstimator<F>
+       ) -> Self
+       where F::Target: FeeEstimator
+       {
+               let balance = context.get_available_balances(fee_estimator);
                let (to_remote_reserve_satoshis, to_self_reserve_satoshis) =
                        context.get_holder_counterparty_selected_channel_reserve_satoshis();
                ChannelDetails {
@@ -1534,7 +1609,6 @@ impl ChannelDetails {
                        channel_value_satoshis: context.get_value_satoshis(),
                        feerate_sat_per_1000_weight: Some(context.get_feerate_sat_per_1000_weight()),
                        unspendable_punishment_reserve: to_self_reserve_satoshis,
-                       balance_msat: balance.balance_msat,
                        inbound_capacity_msat: balance.inbound_capacity_msat,
                        outbound_capacity_msat: balance.outbound_capacity_msat,
                        next_outbound_htlc_limit_msat: balance.next_outbound_htlc_limit_msat,
@@ -1550,10 +1624,33 @@ impl ChannelDetails {
                        inbound_htlc_minimum_msat: Some(context.get_holder_htlc_minimum_msat()),
                        inbound_htlc_maximum_msat: context.get_holder_htlc_maximum_msat(),
                        config: Some(context.config()),
+                       channel_shutdown_state: Some(context.shutdown_state()),
                }
        }
 }
 
+#[derive(Clone, Copy, Debug, PartialEq, Eq)]
+/// Further information on the details of the channel shutdown.
+/// Upon channels being forced closed (i.e. commitment transaction confirmation detected
+/// by `ChainMonitor`), ChannelShutdownState will be set to `ShutdownComplete` or
+/// the channel will be removed shortly.
+/// Also note, that in normal operation, peers could disconnect at any of these states
+/// and require peer re-connection before making progress onto other states
+pub enum ChannelShutdownState {
+       /// Channel has not sent or received a shutdown message.
+       NotShuttingDown,
+       /// Local node has sent a shutdown message for this channel.
+       ShutdownInitiated,
+       /// Shutdown message exchanges have concluded and the channels are in the midst of
+       /// resolving all existing open HTLCs before closing can continue.
+       ResolvingHTLCs,
+       /// All HTLCs have been resolved, nodes are currently negotiating channel close onchain fee rates.
+       NegotiatingClosingFee,
+       /// We've successfully negotiated a closing_signed dance. At this point `ChannelManager` is about
+       /// to drop the channel.
+       ShutdownComplete,
+}
+
 /// Used by [`ChannelManager::list_recent_payments`] to express the status of recent payments.
 /// These include payments that have yet to find a successful path, or have unresolved HTLCs.
 #[derive(Debug, PartialEq)]
@@ -1607,7 +1704,7 @@ macro_rules! handle_error {
 
                match $internal {
                        Ok(msg) => Ok(msg),
-                       Err(MsgHandleErrInternal { err, chan_id, shutdown_finish }) => {
+                       Err(MsgHandleErrInternal { err, chan_id, shutdown_finish, channel_capacity }) => {
                                let mut msg_events = Vec::with_capacity(2);
 
                                if let Some((shutdown_res, update_option)) = shutdown_finish {
@@ -1620,7 +1717,9 @@ macro_rules! handle_error {
                                        if let Some((channel_id, user_channel_id)) = chan_id {
                                                $self.pending_events.lock().unwrap().push_back((events::Event::ChannelClosed {
                                                        channel_id, user_channel_id,
-                                                       reason: ClosureReason::ProcessingError { err: err.err.clone() }
+                                                       reason: ClosureReason::ProcessingError { err: err.err.clone() },
+                                                       counterparty_node_id: Some($counterparty_node_id),
+                                                       channel_capacity_sats: channel_capacity,
                                                }, None));
                                        }
                                }
@@ -1693,20 +1792,20 @@ macro_rules! convert_chan_err {
                                update_maps_on_chan_removal!($self, &$channel.context);
                                let shutdown_res = $channel.context.force_shutdown(true);
                                (true, MsgHandleErrInternal::from_finish_shutdown(msg, *$channel_id, $channel.context.get_user_id(),
-                                       shutdown_res, $self.get_channel_update_for_broadcast(&$channel).ok()))
+                                       shutdown_res, $self.get_channel_update_for_broadcast(&$channel).ok(), $channel.context.get_value_satoshis()))
                        },
                }
        };
-       ($self: ident, $err: expr, $channel_context: expr, $channel_id: expr, PREFUNDED) => {
+       ($self: ident, $err: expr, $channel_context: expr, $channel_id: expr, UNFUNDED) => {
                match $err {
-                       // We should only ever have `ChannelError::Close` when prefunded channels error.
+                       // We should only ever have `ChannelError::Close` when unfunded channels error.
                        // In any case, just close the channel.
                        ChannelError::Warn(msg) | ChannelError::Ignore(msg) | ChannelError::Close(msg) => {
-                               log_error!($self.logger, "Closing prefunded channel {} due to an error: {}", log_bytes!($channel_id[..]), msg);
+                               log_error!($self.logger, "Closing unfunded channel {} due to an error: {}", log_bytes!($channel_id[..]), msg);
                                update_maps_on_chan_removal!($self, &$channel_context);
                                let shutdown_res = $channel_context.force_shutdown(false);
                                (true, MsgHandleErrInternal::from_finish_shutdown(msg, *$channel_id, $channel_context.get_user_id(),
-                                       shutdown_res, None))
+                                       shutdown_res, None, $channel_context.get_value_satoshis()))
                        },
                }
        }
@@ -1732,7 +1831,7 @@ macro_rules! try_v1_outbound_chan_entry {
                match $res {
                        Ok(res) => res,
                        Err(e) => {
-                               let (drop, res) = convert_chan_err!($self, e, $entry.get_mut().context, $entry.key(), PREFUNDED);
+                               let (drop, res) = convert_chan_err!($self, e, $entry.get_mut().context, $entry.key(), UNFUNDED);
                                if drop {
                                        $entry.remove_entry();
                                }
@@ -1885,7 +1984,7 @@ macro_rules! handle_new_monitor_update {
                                let res = Err(MsgHandleErrInternal::from_finish_shutdown(
                                        "ChannelMonitor storage failure".to_owned(), $chan.context.channel_id(),
                                        $chan.context.get_user_id(), $chan.context.force_shutdown(false),
-                                       $self.get_channel_update_for_broadcast(&$chan).ok()));
+                                       $self.get_channel_update_for_broadcast(&$chan).ok(), $chan.context.get_value_satoshis()));
                                $remove;
                                res
                        },
@@ -1975,6 +2074,8 @@ macro_rules! process_events_body {
                                let mut pending_events = $self.pending_events.lock().unwrap();
                                pending_events.drain(..num_events);
                                processed_all_events = pending_events.is_empty();
+                               // Note that `push_pending_forwards_ev` relies on `pending_events_processor` being
+                               // updated here with the `pending_events` lock acquired.
                                $self.pending_events_processor.store(false, Ordering::Release);
                        }
 
@@ -2004,6 +2105,8 @@ where
 {
        /// Constructs a new `ChannelManager` to hold several channels and route between them.
        ///
+       /// The current time or latest block header time can be provided as the `current_timestamp`.
+       ///
        /// This is the main "logic hub" for all channel-related actions, and implements
        /// [`ChannelMessageHandler`].
        ///
@@ -2017,7 +2120,11 @@ where
        /// [`block_connected`]: chain::Listen::block_connected
        /// [`block_disconnected`]: chain::Listen::block_disconnected
        /// [`params.best_block.block_hash`]: chain::BestBlock::block_hash
-       pub fn new(fee_est: F, chain_monitor: M, tx_broadcaster: T, router: R, logger: L, entropy_source: ES, node_signer: NS, signer_provider: SP, config: UserConfig, params: ChainParameters) -> Self {
+       pub fn new(
+               fee_est: F, chain_monitor: M, tx_broadcaster: T, router: R, logger: L, entropy_source: ES,
+               node_signer: NS, signer_provider: SP, config: UserConfig, params: ChainParameters,
+               current_timestamp: u32,
+       ) -> Self {
                let mut secp_ctx = Secp256k1::new();
                secp_ctx.seeded_randomize(&entropy_source.get_secure_random_bytes());
                let inbound_pmt_key_material = node_signer.get_inbound_payment_key_material();
@@ -2049,7 +2156,7 @@ where
 
                        probing_cookie_secret: entropy_source.get_secure_random_bytes(),
 
-                       highest_seen_timestamp: AtomicUsize::new(0),
+                       highest_seen_timestamp: AtomicUsize::new(current_timestamp as usize),
 
                        per_peer_state: FairRwLock::new(HashMap::new()),
 
@@ -2186,9 +2293,10 @@ where
                        for (_cp_id, peer_state_mutex) in per_peer_state.iter() {
                                let mut peer_state_lock = peer_state_mutex.lock().unwrap();
                                let peer_state = &mut *peer_state_lock;
+                               // Only `Channels` in the channel_by_id map can be considered funded.
                                for (_channel_id, channel) in peer_state.channel_by_id.iter().filter(f) {
                                        let details = ChannelDetails::from_channel_context(&channel.context, best_block_height,
-                                               peer_state.latest_features.clone());
+                                               peer_state.latest_features.clone(), &self.fee_estimator);
                                        res.push(details);
                                }
                        }
@@ -2214,17 +2322,17 @@ where
                                let peer_state = &mut *peer_state_lock;
                                for (_channel_id, channel) in peer_state.channel_by_id.iter() {
                                        let details = ChannelDetails::from_channel_context(&channel.context, best_block_height,
-                                               peer_state.latest_features.clone());
+                                               peer_state.latest_features.clone(), &self.fee_estimator);
                                        res.push(details);
                                }
                                for (_channel_id, channel) in peer_state.inbound_v1_channel_by_id.iter() {
                                        let details = ChannelDetails::from_channel_context(&channel.context, best_block_height,
-                                               peer_state.latest_features.clone());
+                                               peer_state.latest_features.clone(), &self.fee_estimator);
                                        res.push(details);
                                }
                                for (_channel_id, channel) in peer_state.outbound_v1_channel_by_id.iter() {
                                        let details = ChannelDetails::from_channel_context(&channel.context, best_block_height,
-                                               peer_state.latest_features.clone());
+                                               peer_state.latest_features.clone(), &self.fee_estimator);
                                        res.push(details);
                                }
                        }
@@ -2254,10 +2362,15 @@ where
                        let mut peer_state_lock = peer_state_mutex.lock().unwrap();
                        let peer_state = &mut *peer_state_lock;
                        let features = &peer_state.latest_features;
+                       let chan_context_to_details = |context| {
+                               ChannelDetails::from_channel_context(context, best_block_height, features.clone(), &self.fee_estimator)
+                       };
                        return peer_state.channel_by_id
                                .iter()
-                               .map(|(_, channel)|
-                                       ChannelDetails::from_channel_context(&channel.context, best_block_height, features.clone()))
+                               .map(|(_, channel)| &channel.context)
+                               .chain(peer_state.outbound_v1_channel_by_id.iter().map(|(_, channel)| &channel.context))
+                               .chain(peer_state.inbound_v1_channel_by_id.iter().map(|(_, channel)| &channel.context))
+                               .map(chan_context_to_details)
                                .collect();
                }
                vec![]
@@ -2305,7 +2418,9 @@ where
                pending_events_lock.push_back((events::Event::ChannelClosed {
                        channel_id: context.channel_id(),
                        user_channel_id: context.get_user_id(),
-                       reason: closure_reason
+                       reason: closure_reason,
+                       counterparty_node_id: Some(context.get_counterparty_node_id()),
+                       channel_capacity_sats: Some(context.get_value_satoshis()),
                }, None));
        }
 
@@ -2314,48 +2429,58 @@ where
 
                let mut failed_htlcs: Vec<(HTLCSource, PaymentHash)>;
                let result: Result<(), _> = loop {
-                       let per_peer_state = self.per_peer_state.read().unwrap();
+                       {
+                               let per_peer_state = self.per_peer_state.read().unwrap();
 
-                       let peer_state_mutex = per_peer_state.get(counterparty_node_id)
-                               .ok_or_else(|| APIError::ChannelUnavailable { err: format!("Can't find a peer matching the passed counterparty node_id {}", counterparty_node_id) })?;
+                               let peer_state_mutex = per_peer_state.get(counterparty_node_id)
+                                       .ok_or_else(|| APIError::ChannelUnavailable { err: format!("Can't find a peer matching the passed counterparty node_id {}", counterparty_node_id) })?;
 
-                       let mut peer_state_lock = peer_state_mutex.lock().unwrap();
-                       let peer_state = &mut *peer_state_lock;
-                       match peer_state.channel_by_id.entry(channel_id.clone()) {
-                               hash_map::Entry::Occupied(mut chan_entry) => {
-                                       let funding_txo_opt = chan_entry.get().context.get_funding_txo();
-                                       let their_features = &peer_state.latest_features;
-                                       let (shutdown_msg, mut monitor_update_opt, htlcs) = chan_entry.get_mut()
-                                               .get_shutdown(&self.signer_provider, their_features, target_feerate_sats_per_1000_weight, override_shutdown_script)?;
-                                       failed_htlcs = htlcs;
+                               let mut peer_state_lock = peer_state_mutex.lock().unwrap();
+                               let peer_state = &mut *peer_state_lock;
 
-                                       // We can send the `shutdown` message before updating the `ChannelMonitor`
-                                       // here as we don't need the monitor update to complete until we send a
-                                       // `shutdown_signed`, which we'll delay if we're pending a monitor update.
-                                       peer_state.pending_msg_events.push(events::MessageSendEvent::SendShutdown {
-                                               node_id: *counterparty_node_id,
-                                               msg: shutdown_msg,
-                                       });
+                               match peer_state.channel_by_id.entry(channel_id.clone()) {
+                                       hash_map::Entry::Occupied(mut chan_entry) => {
+                                               let funding_txo_opt = chan_entry.get().context.get_funding_txo();
+                                               let their_features = &peer_state.latest_features;
+                                               let (shutdown_msg, mut monitor_update_opt, htlcs) = chan_entry.get_mut()
+                                                       .get_shutdown(&self.signer_provider, their_features, target_feerate_sats_per_1000_weight, override_shutdown_script)?;
+                                               failed_htlcs = htlcs;
 
-                                       // Update the monitor with the shutdown script if necessary.
-                                       if let Some(monitor_update) = monitor_update_opt.take() {
-                                               break handle_new_monitor_update!(self, funding_txo_opt.unwrap(), monitor_update,
-                                                       peer_state_lock, peer_state, per_peer_state, chan_entry).map(|_| ());
-                                       }
+                                               // We can send the `shutdown` message before updating the `ChannelMonitor`
+                                               // here as we don't need the monitor update to complete until we send a
+                                               // `shutdown_signed`, which we'll delay if we're pending a monitor update.
+                                               peer_state.pending_msg_events.push(events::MessageSendEvent::SendShutdown {
+                                                       node_id: *counterparty_node_id,
+                                                       msg: shutdown_msg,
+                                               });
 
-                                       if chan_entry.get().is_shutdown() {
-                                               let channel = remove_channel!(self, chan_entry);
-                                               if let Ok(channel_update) = self.get_channel_update_for_broadcast(&channel) {
-                                                       peer_state.pending_msg_events.push(events::MessageSendEvent::BroadcastChannelUpdate {
-                                                               msg: channel_update
-                                                       });
+                                               // Update the monitor with the shutdown script if necessary.
+                                               if let Some(monitor_update) = monitor_update_opt.take() {
+                                                       break handle_new_monitor_update!(self, funding_txo_opt.unwrap(), monitor_update,
+                                                               peer_state_lock, peer_state, per_peer_state, chan_entry).map(|_| ());
                                                }
-                                               self.issue_channel_close_events(&channel.context, ClosureReason::HolderForceClosed);
-                                       }
-                                       break Ok(());
-                               },
-                               hash_map::Entry::Vacant(_) => return Err(APIError::ChannelUnavailable{err: format!("Channel with id {} not found for the passed counterparty node_id {}", log_bytes!(*channel_id), counterparty_node_id) })
+
+                                               if chan_entry.get().is_shutdown() {
+                                                       let channel = remove_channel!(self, chan_entry);
+                                                       if let Ok(channel_update) = self.get_channel_update_for_broadcast(&channel) {
+                                                               peer_state.pending_msg_events.push(events::MessageSendEvent::BroadcastChannelUpdate {
+                                                                       msg: channel_update
+                                                               });
+                                                       }
+                                                       self.issue_channel_close_events(&channel.context, ClosureReason::HolderForceClosed);
+                                               }
+                                               break Ok(());
+                                       },
+                                       hash_map::Entry::Vacant(_) => (),
+                               }
                        }
+                       // If we reach this point, it means that the channel_id either refers to an unfunded channel or
+                       // it does not exist for this peer. Either way, we can attempt to force-close it.
+                       //
+                       // An appropriate error will be returned for non-existence of the channel if that's the case.
+                       return self.force_close_channel_with_peer(&channel_id, counterparty_node_id, None, false).map(|_| ())
+                       // TODO(dunxen): This is still not ideal as we're doing some extra lookups.
+                       // Fix this with https://github.com/lightningdevkit/rust-lightning/issues/2422
                };
 
                for htlc_source in failed_htlcs.drain(..) {
@@ -2474,15 +2599,21 @@ where
                                self.issue_channel_close_events(&chan.get().context, closure_reason);
                                let mut chan = remove_channel!(self, chan);
                                self.finish_force_close_channel(chan.context.force_shutdown(false));
-                               // Prefunded channel has no update
+                               // Unfunded channel has no update
                                (None, chan.context.get_counterparty_node_id())
                        } else if let hash_map::Entry::Occupied(chan) = peer_state.inbound_v1_channel_by_id.entry(channel_id.clone()) {
                                log_error!(self.logger, "Force-closing channel {}", log_bytes!(channel_id[..]));
                                self.issue_channel_close_events(&chan.get().context, closure_reason);
                                let mut chan = remove_channel!(self, chan);
                                self.finish_force_close_channel(chan.context.force_shutdown(false));
-                               // Prefunded channel has no update
+                               // Unfunded channel has no update
                                (None, chan.context.get_counterparty_node_id())
+                       } else if peer_state.inbound_channel_request_by_id.remove(channel_id).is_some() {
+                               log_error!(self.logger, "Force-closing channel {}", log_bytes!(channel_id[..]));
+                               // N.B. that we don't send any channel close event here: we
+                               // don't have a user_channel_id, and we never sent any opening
+                               // events anyway.
+                               (None, *peer_node_id)
                        } else {
                                return Err(APIError::ChannelUnavailable{ err: format!("Channel with id {} not found for the passed counterparty node_id {}", log_bytes!(*channel_id), peer_node_id) });
                        }
@@ -2555,14 +2686,64 @@ where
                }
        }
 
+       fn construct_fwd_pending_htlc_info(
+               &self, msg: &msgs::UpdateAddHTLC, hop_data: msgs::InboundOnionPayload, hop_hmac: [u8; 32],
+               new_packet_bytes: [u8; onion_utils::ONION_DATA_LEN], shared_secret: [u8; 32],
+               next_packet_pubkey_opt: Option<Result<PublicKey, secp256k1::Error>>
+       ) -> Result<PendingHTLCInfo, InboundOnionErr> {
+               debug_assert!(next_packet_pubkey_opt.is_some());
+               let outgoing_packet = msgs::OnionPacket {
+                       version: 0,
+                       public_key: next_packet_pubkey_opt.unwrap_or(Err(secp256k1::Error::InvalidPublicKey)),
+                       hop_data: new_packet_bytes,
+                       hmac: hop_hmac,
+               };
+
+               let (short_channel_id, amt_to_forward, outgoing_cltv_value) = match hop_data {
+                       msgs::InboundOnionPayload::Forward { short_channel_id, amt_to_forward, outgoing_cltv_value } =>
+                               (short_channel_id, amt_to_forward, outgoing_cltv_value),
+                       msgs::InboundOnionPayload::Receive { .. } =>
+                               return Err(InboundOnionErr {
+                                       msg: "Final Node OnionHopData provided for us as an intermediary node",
+                                       err_code: 0x4000 | 22,
+                                       err_data: Vec::new(),
+                               }),
+               };
+
+               Ok(PendingHTLCInfo {
+                       routing: PendingHTLCRouting::Forward {
+                               onion_packet: outgoing_packet,
+                               short_channel_id,
+                       },
+                       payment_hash: msg.payment_hash,
+                       incoming_shared_secret: shared_secret,
+                       incoming_amt_msat: Some(msg.amount_msat),
+                       outgoing_amt_msat: amt_to_forward,
+                       outgoing_cltv_value,
+                       skimmed_fee_msat: None,
+               })
+       }
+
        fn construct_recv_pending_htlc_info(
-               &self, hop_data: msgs::OnionHopData, shared_secret: [u8; 32], payment_hash: PaymentHash,
+               &self, hop_data: msgs::InboundOnionPayload, shared_secret: [u8; 32], payment_hash: PaymentHash,
                amt_msat: u64, cltv_expiry: u32, phantom_shared_secret: Option<[u8; 32]>, allow_underpay: bool,
                counterparty_skimmed_fee_msat: Option<u64>,
-       ) -> Result<PendingHTLCInfo, ReceiveError> {
+       ) -> Result<PendingHTLCInfo, InboundOnionErr> {
+               let (payment_data, keysend_preimage, custom_tlvs, onion_amt_msat, outgoing_cltv_value, payment_metadata) = match hop_data {
+                       msgs::InboundOnionPayload::Receive {
+                               payment_data, keysend_preimage, custom_tlvs, amt_msat, outgoing_cltv_value, payment_metadata, ..
+                       } =>
+                               (payment_data, keysend_preimage, custom_tlvs, amt_msat, outgoing_cltv_value, payment_metadata),
+                       _ =>
+                               return Err(InboundOnionErr {
+                                       err_code: 0x4000|22,
+                                       err_data: Vec::new(),
+                                       msg: "Got non final data with an HMAC of 0",
+                               }),
+               };
                // final_incorrect_cltv_expiry
-               if hop_data.outgoing_cltv_value > cltv_expiry {
-                       return Err(ReceiveError {
+               if outgoing_cltv_value > cltv_expiry {
+                       return Err(InboundOnionErr {
                                msg: "Upstream node set CLTV to less than the CLTV set by the sender",
                                err_code: 18,
                                err_data: cltv_expiry.to_be_bytes().to_vec()
@@ -2576,85 +2757,76 @@ where
                // payment logic has enough time to fail the HTLC backward before our onchain logic triggers a
                // channel closure (see HTLC_FAIL_BACK_BUFFER rationale).
                let current_height: u32 = self.best_block.read().unwrap().height();
-               if (hop_data.outgoing_cltv_value as u64) <= current_height as u64 + HTLC_FAIL_BACK_BUFFER as u64 + 1 {
+               if (outgoing_cltv_value as u64) <= current_height as u64 + HTLC_FAIL_BACK_BUFFER as u64 + 1 {
                        let mut err_data = Vec::with_capacity(12);
                        err_data.extend_from_slice(&amt_msat.to_be_bytes());
                        err_data.extend_from_slice(&current_height.to_be_bytes());
-                       return Err(ReceiveError {
+                       return Err(InboundOnionErr {
                                err_code: 0x4000 | 15, err_data,
                                msg: "The final CLTV expiry is too soon to handle",
                        });
                }
-               if (!allow_underpay && hop_data.amt_to_forward > amt_msat) ||
-                       (allow_underpay && hop_data.amt_to_forward >
+               if (!allow_underpay && onion_amt_msat > amt_msat) ||
+                       (allow_underpay && onion_amt_msat >
                         amt_msat.saturating_add(counterparty_skimmed_fee_msat.unwrap_or(0)))
                {
-                       return Err(ReceiveError {
+                       return Err(InboundOnionErr {
                                err_code: 19,
                                err_data: amt_msat.to_be_bytes().to_vec(),
                                msg: "Upstream node sent less than we were supposed to receive in payment",
                        });
                }
 
-               let routing = match hop_data.format {
-                       msgs::OnionHopDataFormat::NonFinalNode { .. } => {
-                               return Err(ReceiveError {
+               let routing = if let Some(payment_preimage) = keysend_preimage {
+                       // We need to check that the sender knows the keysend preimage before processing this
+                       // payment further. Otherwise, an intermediary routing hop forwarding non-keysend-HTLC X
+                       // could discover the final destination of X, by probing the adjacent nodes on the route
+                       // with a keysend payment of identical payment hash to X and observing the processing
+                       // time discrepancies due to a hash collision with X.
+                       let hashed_preimage = PaymentHash(Sha256::hash(&payment_preimage.0).into_inner());
+                       if hashed_preimage != payment_hash {
+                               return Err(InboundOnionErr {
                                        err_code: 0x4000|22,
                                        err_data: Vec::new(),
-                                       msg: "Got non final data with an HMAC of 0",
+                                       msg: "Payment preimage didn't match payment hash",
                                });
-                       },
-                       msgs::OnionHopDataFormat::FinalNode { payment_data, keysend_preimage, payment_metadata } => {
-                               if let Some(payment_preimage) = keysend_preimage {
-                                       // We need to check that the sender knows the keysend preimage before processing this
-                                       // payment further. Otherwise, an intermediary routing hop forwarding non-keysend-HTLC X
-                                       // could discover the final destination of X, by probing the adjacent nodes on the route
-                                       // with a keysend payment of identical payment hash to X and observing the processing
-                                       // time discrepancies due to a hash collision with X.
-                                       let hashed_preimage = PaymentHash(Sha256::hash(&payment_preimage.0).into_inner());
-                                       if hashed_preimage != payment_hash {
-                                               return Err(ReceiveError {
-                                                       err_code: 0x4000|22,
-                                                       err_data: Vec::new(),
-                                                       msg: "Payment preimage didn't match payment hash",
-                                               });
-                                       }
-                                       if !self.default_configuration.accept_mpp_keysend && payment_data.is_some() {
-                                               return Err(ReceiveError {
-                                                       err_code: 0x4000|22,
-                                                       err_data: Vec::new(),
-                                                       msg: "We don't support MPP keysend payments",
-                                               });
-                                       }
-                                       PendingHTLCRouting::ReceiveKeysend {
-                                               payment_data,
-                                               payment_preimage,
-                                               payment_metadata,
-                                               incoming_cltv_expiry: hop_data.outgoing_cltv_value,
-                                       }
-                               } else if let Some(data) = payment_data {
-                                       PendingHTLCRouting::Receive {
-                                               payment_data: data,
-                                               payment_metadata,
-                                               incoming_cltv_expiry: hop_data.outgoing_cltv_value,
-                                               phantom_shared_secret,
-                                       }
-                               } else {
-                                       return Err(ReceiveError {
-                                               err_code: 0x4000|0x2000|3,
-                                               err_data: Vec::new(),
-                                               msg: "We require payment_secrets",
-                                       });
-                               }
-                       },
+                       }
+                       if !self.default_configuration.accept_mpp_keysend && payment_data.is_some() {
+                               return Err(InboundOnionErr {
+                                       err_code: 0x4000|22,
+                                       err_data: Vec::new(),
+                                       msg: "We don't support MPP keysend payments",
+                               });
+                       }
+                       PendingHTLCRouting::ReceiveKeysend {
+                               payment_data,
+                               payment_preimage,
+                               payment_metadata,
+                               incoming_cltv_expiry: outgoing_cltv_value,
+                               custom_tlvs,
+                       }
+               } else if let Some(data) = payment_data {
+                       PendingHTLCRouting::Receive {
+                               payment_data: data,
+                               payment_metadata,
+                               incoming_cltv_expiry: outgoing_cltv_value,
+                               phantom_shared_secret,
+                               custom_tlvs,
+                       }
+               } else {
+                       return Err(InboundOnionErr {
+                               err_code: 0x4000|0x2000|3,
+                               err_data: Vec::new(),
+                               msg: "We require payment_secrets",
+                       });
                };
                Ok(PendingHTLCInfo {
                        routing,
                        payment_hash,
                        incoming_shared_secret: shared_secret,
                        incoming_amt_msat: Some(amt_msat),
-                       outgoing_amt_msat: hop_data.amt_to_forward,
-                       outgoing_cltv_value: hop_data.outgoing_cltv_value,
+                       outgoing_amt_msat: onion_amt_msat,
+                       outgoing_cltv_value,
                        skimmed_fee_msat: counterparty_skimmed_fee_msat,
                })
        }
@@ -2718,9 +2890,8 @@ where
                };
                let (outgoing_scid, outgoing_amt_msat, outgoing_cltv_value, next_packet_pk_opt) = match next_hop {
                        onion_utils::Hop::Forward {
-                               next_hop_data: msgs::OnionHopData {
-                                       format: msgs::OnionHopDataFormat::NonFinalNode { short_channel_id }, amt_to_forward,
-                                       outgoing_cltv_value,
+                               next_hop_data: msgs::InboundOnionPayload::Forward {
+                                       short_channel_id, amt_to_forward, outgoing_cltv_value
                                }, ..
                        } => {
                                let next_pk = onion_utils::next_hop_packet_pubkey(&self.secp_ctx,
@@ -2730,9 +2901,7 @@ where
                        // We'll do receive checks in [`Self::construct_pending_htlc_info`] so we have access to the
                        // inbound channel's state.
                        onion_utils::Hop::Receive { .. } => return Ok((next_hop, shared_secret, None)),
-                       onion_utils::Hop::Forward {
-                               next_hop_data: msgs::OnionHopData { format: msgs::OnionHopDataFormat::FinalNode { .. }, .. }, ..
-                       } => {
+                       onion_utils::Hop::Forward { next_hop_data: msgs::InboundOnionPayload::Receive { .. }, .. } => {
                                return_err!("Final Node OnionHopData provided for us as an intermediary node", 0x4000 | 22, &[0; 0]);
                        }
                };
@@ -2903,37 +3072,15 @@ where
                                                // delay) once they've send us a commitment_signed!
                                                PendingHTLCStatus::Forward(info)
                                        },
-                                       Err(ReceiveError { err_code, err_data, msg }) => return_err!(msg, err_code, &err_data)
+                                       Err(InboundOnionErr { err_code, err_data, msg }) => return_err!(msg, err_code, &err_data)
                                }
                        },
                        onion_utils::Hop::Forward { next_hop_data, next_hop_hmac, new_packet_bytes } => {
-                               debug_assert!(next_packet_pubkey_opt.is_some());
-                               let outgoing_packet = msgs::OnionPacket {
-                                       version: 0,
-                                       public_key: next_packet_pubkey_opt.unwrap_or(Err(secp256k1::Error::InvalidPublicKey)),
-                                       hop_data: new_packet_bytes,
-                                       hmac: next_hop_hmac.clone(),
-                               };
-
-                               let short_channel_id = match next_hop_data.format {
-                                       msgs::OnionHopDataFormat::NonFinalNode { short_channel_id } => short_channel_id,
-                                       msgs::OnionHopDataFormat::FinalNode { .. } => {
-                                               return_err!("Final Node OnionHopData provided for us as an intermediary node", 0x4000 | 22, &[0;0]);
-                                       },
-                               };
-
-                               PendingHTLCStatus::Forward(PendingHTLCInfo {
-                                       routing: PendingHTLCRouting::Forward {
-                                               onion_packet: outgoing_packet,
-                                               short_channel_id,
-                                       },
-                                       payment_hash: msg.payment_hash.clone(),
-                                       incoming_shared_secret: shared_secret,
-                                       incoming_amt_msat: Some(msg.amount_msat),
-                                       outgoing_amt_msat: next_hop_data.amt_to_forward,
-                                       outgoing_cltv_value: next_hop_data.outgoing_cltv_value,
-                                       skimmed_fee_msat: None,
-                               })
+                               match self.construct_fwd_pending_htlc_info(msg, next_hop_data, next_hop_hmac,
+                                       new_packet_bytes, shared_secret, next_packet_pubkey_opt) {
+                                       Ok(info) => PendingHTLCStatus::Forward(info),
+                                       Err(InboundOnionErr { err_code, err_data, msg }) => return_err!(msg, err_code, &err_data)
+                               }
                        }
                }
        }
@@ -3021,10 +3168,17 @@ where
        #[cfg(test)]
        pub(crate) fn test_send_payment_along_path(&self, path: &Path, payment_hash: &PaymentHash, recipient_onion: RecipientOnionFields, total_value: u64, cur_height: u32, payment_id: PaymentId, keysend_preimage: &Option<PaymentPreimage>, session_priv_bytes: [u8; 32]) -> Result<(), APIError> {
                let _lck = self.total_consistency_lock.read().unwrap();
-               self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv_bytes)
+               self.send_payment_along_path(SendAlongPathArgs {
+                       path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage,
+                       session_priv_bytes
+               })
        }
 
-       fn send_payment_along_path(&self, path: &Path, payment_hash: &PaymentHash, recipient_onion: RecipientOnionFields, total_value: u64, cur_height: u32, payment_id: PaymentId, keysend_preimage: &Option<PaymentPreimage>, session_priv_bytes: [u8; 32]) -> Result<(), APIError> {
+       fn send_payment_along_path(&self, args: SendAlongPathArgs) -> Result<(), APIError> {
+               let SendAlongPathArgs {
+                       path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage,
+                       session_priv_bytes
+               } = args;
                // The top-level caller should hold the total_consistency_lock read lock.
                debug_assert!(self.total_consistency_lock.try_write().is_err());
 
@@ -3061,7 +3215,7 @@ where
                                                session_priv: session_priv.clone(),
                                                first_hop_htlc_msat: htlc_msat,
                                                payment_id,
-                                       }, onion_packet, None, &self.logger);
+                                       }, onion_packet, None, &self.fee_estimator, &self.logger);
                                match break_chan_entry!(self, send_res, chan) {
                                        Some(monitor_update) => {
                                                match handle_new_monitor_update!(self, funding_txo, monitor_update, peer_state_lock, peer_state, per_peer_state, chan) {
@@ -3144,6 +3298,7 @@ where
        /// irrevocably committed to on our end. In such a case, do NOT retry the payment with a
        /// different route unless you intend to pay twice!
        ///
+       /// [`RouteHop`]: crate::routing::router::RouteHop
        /// [`Event::PaymentSent`]: events::Event::PaymentSent
        /// [`Event::PaymentFailed`]: events::Event::PaymentFailed
        /// [`UpdateHTLCs`]: events::MessageSendEvent::UpdateHTLCs
@@ -3153,9 +3308,9 @@ where
                let best_block_height = self.best_block.read().unwrap().height();
                let _persistence_guard = PersistenceNotifierGuard::notify_on_drop(self);
                self.pending_outbound_payments
-                       .send_payment_with_route(route, payment_hash, recipient_onion, payment_id, &self.entropy_source, &self.node_signer, best_block_height,
-                               |path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv|
-                               self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv))
+                       .send_payment_with_route(route, payment_hash, recipient_onion, payment_id,
+                               &self.entropy_source, &self.node_signer, best_block_height,
+                               |args| self.send_payment_along_path(args))
        }
 
        /// Similar to [`ChannelManager::send_payment_with_route`], but will automatically find a route based on
@@ -3167,18 +3322,16 @@ where
                        .send_payment(payment_hash, recipient_onion, payment_id, retry_strategy, route_params,
                                &self.router, self.list_usable_channels(), || self.compute_inflight_htlcs(),
                                &self.entropy_source, &self.node_signer, best_block_height, &self.logger,
-                               &self.pending_events,
-                               |path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv|
-                               self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv))
+                               &self.pending_events, |args| self.send_payment_along_path(args))
        }
 
        #[cfg(test)]
        pub(super) fn test_send_payment_internal(&self, route: &Route, payment_hash: PaymentHash, recipient_onion: RecipientOnionFields, keysend_preimage: Option<PaymentPreimage>, payment_id: PaymentId, recv_value_msat: Option<u64>, onion_session_privs: Vec<[u8; 32]>) -> Result<(), PaymentSendFailure> {
                let best_block_height = self.best_block.read().unwrap().height();
                let _persistence_guard = PersistenceNotifierGuard::notify_on_drop(self);
-               self.pending_outbound_payments.test_send_payment_internal(route, payment_hash, recipient_onion, keysend_preimage, payment_id, recv_value_msat, onion_session_privs, &self.node_signer, best_block_height,
-                       |path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv|
-                       self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv))
+               self.pending_outbound_payments.test_send_payment_internal(route, payment_hash, recipient_onion,
+                       keysend_preimage, payment_id, recv_value_msat, onion_session_privs, &self.node_signer,
+                       best_block_height, |args| self.send_payment_along_path(args))
        }
 
        #[cfg(test)]
@@ -3232,9 +3385,7 @@ where
                let _persistence_guard = PersistenceNotifierGuard::notify_on_drop(self);
                self.pending_outbound_payments.send_spontaneous_payment_with_route(
                        route, payment_preimage, recipient_onion, payment_id, &self.entropy_source,
-                       &self.node_signer, best_block_height,
-                       |path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv|
-                       self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv))
+                       &self.node_signer, best_block_height, |args| self.send_payment_along_path(args))
        }
 
        /// Similar to [`ChannelManager::send_spontaneous_payment`], but will automatically find a route
@@ -3250,9 +3401,7 @@ where
                self.pending_outbound_payments.send_spontaneous_payment(payment_preimage, recipient_onion,
                        payment_id, retry_strategy, route_params, &self.router, self.list_usable_channels(),
                        || self.compute_inflight_htlcs(),  &self.entropy_source, &self.node_signer, best_block_height,
-                       &self.logger, &self.pending_events,
-                       |path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv|
-                       self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv))
+                       &self.logger, &self.pending_events, |args| self.send_payment_along_path(args))
        }
 
        /// Send a payment that is probing the given route for liquidity. We calculate the
@@ -3261,9 +3410,9 @@ where
        pub fn send_probe(&self, path: Path) -> Result<(PaymentHash, PaymentId), PaymentSendFailure> {
                let best_block_height = self.best_block.read().unwrap().height();
                let _persistence_guard = PersistenceNotifierGuard::notify_on_drop(self);
-               self.pending_outbound_payments.send_probe(path, self.probing_cookie_secret, &self.entropy_source, &self.node_signer, best_block_height,
-                       |path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv|
-                       self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv))
+               self.pending_outbound_payments.send_probe(path, self.probing_cookie_secret,
+                       &self.entropy_source, &self.node_signer, best_block_height,
+                       |args| self.send_payment_along_path(args))
        }
 
        /// Returns whether a payment with the given [`PaymentHash`] and [`PaymentId`] is, in fact, a
@@ -3288,12 +3437,13 @@ where
                        Some(chan) => {
                                let funding_txo = find_funding_output(&chan, &funding_transaction)?;
 
-                               let funding_res = chan.get_outbound_funding_created(funding_transaction, funding_txo, &self.logger)
+                               let funding_res = chan.get_funding_created(funding_transaction, funding_txo, &self.logger)
                                        .map_err(|(mut chan, e)| if let ChannelError::Close(msg) = e {
                                                let channel_id = chan.context.channel_id();
                                                let user_id = chan.context.get_user_id();
                                                let shutdown_res = chan.context.force_shutdown(false);
-                                               (chan, MsgHandleErrInternal::from_finish_shutdown(msg, channel_id, user_id, shutdown_res, None))
+                                               let channel_capacity = chan.context.get_value_satoshis();
+                                               (chan, MsgHandleErrInternal::from_finish_shutdown(msg, channel_id, user_id, shutdown_res, None, channel_capacity))
                                        } else { unreachable!(); });
                                match funding_res {
                                        Ok((chan, funding_msg)) => (chan, funding_msg),
@@ -3461,27 +3611,48 @@ where
                let mut peer_state_lock = peer_state_mutex.lock().unwrap();
                let peer_state = &mut *peer_state_lock;
                for channel_id in channel_ids {
-                       if !peer_state.channel_by_id.contains_key(channel_id) {
+                       if !peer_state.has_channel(channel_id) {
                                return Err(APIError::ChannelUnavailable {
                                        err: format!("Channel with ID {} was not found for the passed counterparty_node_id {}", log_bytes!(*channel_id), counterparty_node_id),
                                });
-                       }
+                       };
                }
                for channel_id in channel_ids {
-                       let channel = peer_state.channel_by_id.get_mut(channel_id).unwrap();
-                       let mut config = channel.context.config();
-                       config.apply(config_update);
-                       if !channel.context.update_config(&config) {
+                       if let Some(channel) = peer_state.channel_by_id.get_mut(channel_id) {
+                               let mut config = channel.context.config();
+                               config.apply(config_update);
+                               if !channel.context.update_config(&config) {
+                                       continue;
+                               }
+                               if let Ok(msg) = self.get_channel_update_for_broadcast(channel) {
+                                       peer_state.pending_msg_events.push(events::MessageSendEvent::BroadcastChannelUpdate { msg });
+                               } else if let Ok(msg) = self.get_channel_update_for_unicast(channel) {
+                                       peer_state.pending_msg_events.push(events::MessageSendEvent::SendChannelUpdate {
+                                               node_id: channel.context.get_counterparty_node_id(),
+                                               msg,
+                                       });
+                               }
                                continue;
                        }
-                       if let Ok(msg) = self.get_channel_update_for_broadcast(channel) {
-                               peer_state.pending_msg_events.push(events::MessageSendEvent::BroadcastChannelUpdate { msg });
-                       } else if let Ok(msg) = self.get_channel_update_for_unicast(channel) {
-                               peer_state.pending_msg_events.push(events::MessageSendEvent::SendChannelUpdate {
-                                       node_id: channel.context.get_counterparty_node_id(),
-                                       msg,
+
+                       let context = if let Some(channel) = peer_state.inbound_v1_channel_by_id.get_mut(channel_id) {
+                               &mut channel.context
+                       } else if let Some(channel) = peer_state.outbound_v1_channel_by_id.get_mut(channel_id) {
+                               &mut channel.context
+                       } else {
+                               // This should not be reachable as we've already checked for non-existence in the previous channel_id loop.
+                               debug_assert!(false);
+                               return Err(APIError::ChannelUnavailable {
+                                       err: format!(
+                                               "Channel with ID {} for passed counterparty_node_id {} disappeared after we confirmed its existence - this should not be reachable!",
+                                               log_bytes!(*channel_id), counterparty_node_id),
                                });
-                       }
+                       };
+                       let mut config = context.config();
+                       config.apply(config_update);
+                       // We update the config, but we MUST NOT broadcast a `channel_update` before `channel_ready`
+                       // which would be the case for pending inbound/outbound channels.
+                       context.update_config(&config);
                }
                Ok(())
        }
@@ -3715,7 +3886,7 @@ where
                                                                                                                        outgoing_cltv_value, Some(phantom_shared_secret), false, None)
                                                                                                                {
                                                                                                                        Ok(info) => phantom_receives.push((prev_short_channel_id, prev_funding_outpoint, prev_user_channel_id, vec![(info, prev_htlc_id)])),
-                                                                                                                       Err(ReceiveError { err_code, err_data, msg }) => failed_payment!(msg, err_code, err_data, Some(phantom_shared_secret))
+                                                                                                                       Err(InboundOnionErr { err_code, err_data, msg }) => failed_payment!(msg, err_code, err_data, Some(phantom_shared_secret))
                                                                                                                }
                                                                                                        },
                                                                                                        _ => panic!(),
@@ -3778,7 +3949,8 @@ where
                                                                                });
                                                                                if let Err(e) = chan.get_mut().queue_add_htlc(outgoing_amt_msat,
                                                                                        payment_hash, outgoing_cltv_value, htlc_source.clone(),
-                                                                                       onion_packet, skimmed_fee_msat, &self.logger)
+                                                                                       onion_packet, skimmed_fee_msat, &self.fee_estimator,
+                                                                                       &self.logger)
                                                                                {
                                                                                        if let ChannelError::Ignore(msg) = e {
                                                                                                log_trace!(self.logger, "Failed to forward HTLC with payment_hash {}: {}", log_bytes!(payment_hash.0), msg);
@@ -3827,17 +3999,18 @@ where
                                                                }
                                                        }) => {
                                                                let (cltv_expiry, onion_payload, payment_data, phantom_shared_secret, mut onion_fields) = match routing {
-                                                                       PendingHTLCRouting::Receive { payment_data, payment_metadata, incoming_cltv_expiry, phantom_shared_secret } => {
+                                                                       PendingHTLCRouting::Receive { payment_data, payment_metadata, incoming_cltv_expiry, phantom_shared_secret, custom_tlvs } => {
                                                                                let _legacy_hop_data = Some(payment_data.clone());
-                                                                               let onion_fields =
-                                                                                       RecipientOnionFields { payment_secret: Some(payment_data.payment_secret), payment_metadata };
+                                                                               let onion_fields = RecipientOnionFields { payment_secret: Some(payment_data.payment_secret),
+                                                                                               payment_metadata, custom_tlvs };
                                                                                (incoming_cltv_expiry, OnionPayload::Invoice { _legacy_hop_data },
                                                                                        Some(payment_data), phantom_shared_secret, onion_fields)
                                                                        },
-                                                                       PendingHTLCRouting::ReceiveKeysend { payment_data, payment_preimage, payment_metadata, incoming_cltv_expiry } => {
+                                                                       PendingHTLCRouting::ReceiveKeysend { payment_data, payment_preimage, payment_metadata, incoming_cltv_expiry, custom_tlvs } => {
                                                                                let onion_fields = RecipientOnionFields {
                                                                                        payment_secret: payment_data.as_ref().map(|data| data.payment_secret),
-                                                                                       payment_metadata
+                                                                                       payment_metadata,
+                                                                                       custom_tlvs,
                                                                                };
                                                                                (incoming_cltv_expiry, OnionPayload::Spontaneous(payment_preimage),
                                                                                        payment_data, None, onion_fields)
@@ -4067,9 +4240,7 @@ where
                let best_block_height = self.best_block.read().unwrap().height();
                self.pending_outbound_payments.check_retry_payments(&self.router, || self.list_usable_channels(),
                        || self.compute_inflight_htlcs(), &self.entropy_source, &self.node_signer, best_block_height,
-                       &self.pending_events, &self.logger,
-                       |path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv|
-                       self.send_payment_along_path(path, payment_hash, recipient_onion, total_value, cur_height, payment_id, keysend_preimage, session_priv));
+                       &self.pending_events, &self.logger, |args| self.send_payment_along_path(args));
 
                for (htlc_source, payment_hash, failure_reason, destination) in failed_forwards.drain(..) {
                        self.fail_htlc_backwards_internal(&htlc_source, &payment_hash, &failure_reason, destination);
@@ -4103,7 +4274,7 @@ where
 
                for event in background_events.drain(..) {
                        match event {
-                               BackgroundEvent::ClosingMonitorUpdateRegeneratedOnStartup((funding_txo, update)) => {
+                               BackgroundEvent::ClosedMonitorUpdateRegeneratedOnStartup((funding_txo, update)) => {
                                        // The channel has already been closed, so no use bothering to care about the
                                        // monitor updating completing.
                                        let _ = self.chain_monitor.update_channel(funding_txo, &update);
@@ -4138,6 +4309,22 @@ where
                                        }
                                        let _ = handle_error!(self, res, counterparty_node_id);
                                },
+                               BackgroundEvent::MonitorUpdatesComplete { counterparty_node_id, channel_id } => {
+                                       let per_peer_state = self.per_peer_state.read().unwrap();
+                                       if let Some(peer_state_mutex) = per_peer_state.get(&counterparty_node_id) {
+                                               let mut peer_state_lock = peer_state_mutex.lock().unwrap();
+                                               let peer_state = &mut *peer_state_lock;
+                                               if let Some(chan) = peer_state.channel_by_id.get_mut(&channel_id) {
+                                                       handle_monitor_update_completion!(self, peer_state_lock, peer_state, per_peer_state, chan);
+                                               } else {
+                                                       let update_actions = peer_state.monitor_update_blocked_actions
+                                                               .remove(&channel_id).unwrap_or(Vec::new());
+                                                       mem::drop(peer_state_lock);
+                                                       mem::drop(per_peer_state);
+                                                       self.handle_monitor_update_completion_actions(update_actions);
+                                               }
+                                       }
+                               },
                        }
                }
                NotifyOption::DoPersist
@@ -4166,7 +4353,7 @@ where
                log_trace!(self.logger, "Channel {} qualifies for a feerate change from {} to {}.",
                        log_bytes!(chan_id[..]), chan.context.get_feerate_sat_per_1000_weight(), new_feerate);
 
-               chan.queue_update_fee(new_feerate, &self.logger);
+               chan.queue_update_fee(new_feerate, &self.fee_estimator, &self.logger);
                NotifyOption::DoPersist
        }
 
@@ -4179,13 +4366,19 @@ where
                PersistenceNotifierGuard::optionally_notify(&self.total_consistency_lock, &self.persistence_notifier, || {
                        let mut should_persist = self.process_background_events();
 
-                       let new_feerate = self.fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::Normal);
+                       let normal_feerate = self.fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::Normal);
+                       let min_mempool_feerate = self.fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::MempoolMinimum);
 
                        let per_peer_state = self.per_peer_state.read().unwrap();
                        for (_cp_id, peer_state_mutex) in per_peer_state.iter() {
                                let mut peer_state_lock = peer_state_mutex.lock().unwrap();
                                let peer_state = &mut *peer_state_lock;
                                for (chan_id, chan) in peer_state.channel_by_id.iter_mut() {
+                                       let new_feerate = if chan.context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
+                                               min_mempool_feerate
+                                       } else {
+                                               normal_feerate
+                                       };
                                        let chan_needs_persist = self.update_channel_fee(chan_id, chan, new_feerate);
                                        if chan_needs_persist == NotifyOption::DoPersist { should_persist = NotifyOption::DoPersist; }
                                }
@@ -4205,6 +4398,7 @@ where
        ///  * Expiring a channel's previous [`ChannelConfig`] if necessary to only allow forwarding HTLCs
        ///    with the current [`ChannelConfig`].
        ///  * Removing peers which have disconnected but and no longer have any channels.
+       ///  * Force-closing and removing channels which have not completed establishment in a timely manner.
        ///
        /// Note that this may cause reentrancy through [`chain::Watch::update_channel`] calls or feerate
        /// estimate fetches.
@@ -4215,7 +4409,8 @@ where
                PersistenceNotifierGuard::optionally_notify(&self.total_consistency_lock, &self.persistence_notifier, || {
                        let mut should_persist = self.process_background_events();
 
-                       let new_feerate = self.fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::Normal);
+                       let normal_feerate = self.fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::Normal);
+                       let min_mempool_feerate = self.fee_estimator.bounded_sat_per_1000_weight(ConfirmationTarget::MempoolMinimum);
 
                        let mut handle_errors: Vec<(Result<(), _>, _)> = Vec::new();
                        let mut timed_out_mpp_htlcs = Vec::new();
@@ -4228,6 +4423,11 @@ where
                                        let pending_msg_events = &mut peer_state.pending_msg_events;
                                        let counterparty_node_id = *counterparty_node_id;
                                        peer_state.channel_by_id.retain(|chan_id, chan| {
+                                               let new_feerate = if chan.context.get_channel_type().supports_anchors_zero_fee_htlc_tx() {
+                                                       min_mempool_feerate
+                                               } else {
+                                                       normal_feerate
+                                               };
                                                let chan_needs_persist = self.update_channel_fee(chan_id, chan, new_feerate);
                                                if chan_needs_persist == NotifyOption::DoPersist { should_persist = NotifyOption::DoPersist; }
 
@@ -4293,6 +4493,55 @@ where
 
                                                true
                                        });
+
+                                       let process_unfunded_channel_tick = |
+                                               chan_id: &[u8; 32],
+                                               chan_context: &mut ChannelContext<<SP::Target as SignerProvider>::Signer>,
+                                               unfunded_chan_context: &mut UnfundedChannelContext,
+                                               pending_msg_events: &mut Vec<MessageSendEvent>,
+                                       | {
+                                               chan_context.maybe_expire_prev_config();
+                                               if unfunded_chan_context.should_expire_unfunded_channel() {
+                                                       log_error!(self.logger,
+                                                               "Force-closing pending channel with ID {} for not establishing in a timely manner",
+                                                               log_bytes!(&chan_id[..]));
+                                                       update_maps_on_chan_removal!(self, &chan_context);
+                                                       self.issue_channel_close_events(&chan_context, ClosureReason::HolderForceClosed);
+                                                       self.finish_force_close_channel(chan_context.force_shutdown(false));
+                                                       pending_msg_events.push(MessageSendEvent::HandleError {
+                                                               node_id: counterparty_node_id,
+                                                               action: msgs::ErrorAction::SendErrorMessage {
+                                                                       msg: msgs::ErrorMessage {
+                                                                               channel_id: *chan_id,
+                                                                               data: "Force-closing pending channel due to timeout awaiting establishment handshake".to_owned(),
+                                                                       },
+                                                               },
+                                                       });
+                                                       false
+                                               } else {
+                                                       true
+                                               }
+                                       };
+                                       peer_state.outbound_v1_channel_by_id.retain(|chan_id, chan| process_unfunded_channel_tick(
+                                               chan_id, &mut chan.context, &mut chan.unfunded_context, pending_msg_events));
+                                       peer_state.inbound_v1_channel_by_id.retain(|chan_id, chan| process_unfunded_channel_tick(
+                                               chan_id, &mut chan.context, &mut chan.unfunded_context, pending_msg_events));
+
+                                       for (chan_id, req) in peer_state.inbound_channel_request_by_id.iter_mut() {
+                                               if { req.ticks_remaining -= 1 ; req.ticks_remaining } <= 0 {
+                                                       log_error!(self.logger, "Force-closing unaccepted inbound channel {} for not accepting in a timely manner", log_bytes!(&chan_id[..]));
+                                                       peer_state.pending_msg_events.push(
+                                                               events::MessageSendEvent::HandleError {
+                                                                       node_id: counterparty_node_id,
+                                                                       action: msgs::ErrorAction::SendErrorMessage {
+                                                                               msg: msgs::ErrorMessage { channel_id: chan_id.clone(), data: "Channel force-closed".to_owned() }
+                                                                       },
+                                                               }
+                                                       );
+                                               }
+                                       }
+                                       peer_state.inbound_channel_request_by_id.retain(|_, req| req.ticks_remaining > 0);
+
                                        if peer_state.ok_to_remove(true) {
                                                pending_peers_awaiting_removal.push(counterparty_node_id);
                                        }
@@ -4415,12 +4664,19 @@ where
        /// Gets error data to form an [`HTLCFailReason`] given a [`FailureCode`] and [`ClaimableHTLC`].
        fn get_htlc_fail_reason_from_failure_code(&self, failure_code: FailureCode, htlc: &ClaimableHTLC) -> HTLCFailReason {
                match failure_code {
-                       FailureCode::TemporaryNodeFailure => HTLCFailReason::from_failure_code(failure_code as u16),
-                       FailureCode::RequiredNodeFeatureMissing => HTLCFailReason::from_failure_code(failure_code as u16),
+                       FailureCode::TemporaryNodeFailure => HTLCFailReason::from_failure_code(failure_code.into()),
+                       FailureCode::RequiredNodeFeatureMissing => HTLCFailReason::from_failure_code(failure_code.into()),
                        FailureCode::IncorrectOrUnknownPaymentDetails => {
                                let mut htlc_msat_height_data = htlc.value.to_be_bytes().to_vec();
                                htlc_msat_height_data.extend_from_slice(&self.best_block.read().unwrap().height().to_be_bytes());
-                               HTLCFailReason::reason(failure_code as u16, htlc_msat_height_data)
+                               HTLCFailReason::reason(failure_code.into(), htlc_msat_height_data)
+                       },
+                       FailureCode::InvalidOnionPayload(data) => {
+                               let fail_data = match data {
+                                       Some((typ, offset)) => [BigSize(typ).encode(), offset.encode()].concat(),
+                                       None => Vec::new(),
+                               };
+                               HTLCFailReason::reason(failure_code.into(), fail_data)
                        }
                }
        }
@@ -4567,13 +4823,35 @@ where
        /// event matches your expectation. If you fail to do so and call this method, you may provide
        /// the sender "proof-of-payment" when they did not fulfill the full expected payment.
        ///
+       /// This function will fail the payment if it has custom TLVs with even type numbers, as we
+       /// will assume they are unknown. If you intend to accept even custom TLVs, you should use
+       /// [`claim_funds_with_known_custom_tlvs`].
+       ///
        /// [`Event::PaymentClaimable`]: crate::events::Event::PaymentClaimable
        /// [`Event::PaymentClaimable::claim_deadline`]: crate::events::Event::PaymentClaimable::claim_deadline
        /// [`Event::PaymentClaimed`]: crate::events::Event::PaymentClaimed
        /// [`process_pending_events`]: EventsProvider::process_pending_events
        /// [`create_inbound_payment`]: Self::create_inbound_payment
        /// [`create_inbound_payment_for_hash`]: Self::create_inbound_payment_for_hash
+       /// [`claim_funds_with_known_custom_tlvs`]: Self::claim_funds_with_known_custom_tlvs
        pub fn claim_funds(&self, payment_preimage: PaymentPreimage) {
+               self.claim_payment_internal(payment_preimage, false);
+       }
+
+       /// This is a variant of [`claim_funds`] that allows accepting a payment with custom TLVs with
+       /// even type numbers.
+       ///
+       /// # Note
+       ///
+       /// You MUST check you've understood all even TLVs before using this to
+       /// claim, otherwise you may unintentionally agree to some protocol you do not understand.
+       ///
+       /// [`claim_funds`]: Self::claim_funds
+       pub fn claim_funds_with_known_custom_tlvs(&self, payment_preimage: PaymentPreimage) {
+               self.claim_payment_internal(payment_preimage, true);
+       }
+
+       fn claim_payment_internal(&self, payment_preimage: PaymentPreimage, custom_tlvs_known: bool) {
                let payment_hash = PaymentHash(Sha256::hash(&payment_preimage.0).into_inner());
 
                let _persistence_guard = PersistenceNotifierGuard::notify_on_drop(self);
@@ -4600,6 +4878,23 @@ where
                                        log_error!(self.logger, "Got a duplicate pending claimable event on payment hash {}! Please report this bug",
                                                log_bytes!(payment_hash.0));
                                }
+
+                               if let Some(RecipientOnionFields { ref custom_tlvs, .. }) = payment.onion_fields {
+                                       if !custom_tlvs_known && custom_tlvs.iter().any(|(typ, _)| typ % 2 == 0) {
+                                               log_info!(self.logger, "Rejecting payment with payment hash {} as we cannot accept payment with unknown even TLVs: {}",
+                                                       log_bytes!(payment_hash.0), log_iter!(custom_tlvs.iter().map(|(typ, _)| typ).filter(|typ| *typ % 2 == 0)));
+                                               claimable_payments.pending_claiming_payments.remove(&payment_hash);
+                                               mem::drop(claimable_payments);
+                                               for htlc in payment.htlcs {
+                                                       let reason = self.get_htlc_fail_reason_from_failure_code(FailureCode::InvalidOnionPayload(None), &htlc);
+                                                       let source = HTLCSource::PreviousHopData(htlc.prev_hop);
+                                                       let receiver = HTLCDestination::FailedPayment { payment_hash };
+                                                       self.fail_htlc_backwards_internal(&source, &payment_hash, &reason, receiver);
+                                               }
+                                               return;
+                                       }
+                               }
+
                                payment.htlcs
                        } else { return; }
                };
@@ -4763,7 +5058,7 @@ where
                        // If we're running during init we cannot update a monitor directly - they probably
                        // haven't actually been loaded yet. Instead, push the monitor update as a background
                        // event.
-                       // Note that while its safe to use `ClosingMonitorUpdateRegeneratedOnStartup` here (the
+                       // Note that while it's safe to use `ClosedMonitorUpdateRegeneratedOnStartup` here (the
                        // channel is already closed) we need to ultimately handle the monitor update
                        // completion action only after we've completed the monitor update. This is the only
                        // way to guarantee this update *will* be regenerated on startup (otherwise if this was
@@ -4771,7 +5066,7 @@ where
                        // upstream). Thus, we need to transition to some new `BackgroundEvent` type which will
                        // complete the monitor update completion action from `completion_action`.
                        self.pending_background_events.lock().unwrap().push(
-                               BackgroundEvent::ClosingMonitorUpdateRegeneratedOnStartup((
+                               BackgroundEvent::ClosedMonitorUpdateRegeneratedOnStartup((
                                        prev_hop.outpoint, preimage_update,
                                )));
                }
@@ -4948,24 +5243,29 @@ where
                if peer_state_mutex_opt.is_none() { return }
                peer_state_lock = peer_state_mutex_opt.unwrap().lock().unwrap();
                let peer_state = &mut *peer_state_lock;
-               let mut channel = {
-                       match peer_state.channel_by_id.entry(funding_txo.to_channel_id()){
-                               hash_map::Entry::Occupied(chan) => chan,
-                               hash_map::Entry::Vacant(_) => return,
-                       }
-               };
+               let channel =
+                       if let Some(chan) = peer_state.channel_by_id.get_mut(&funding_txo.to_channel_id()) {
+                               chan
+                       } else {
+                               let update_actions = peer_state.monitor_update_blocked_actions
+                                       .remove(&funding_txo.to_channel_id()).unwrap_or(Vec::new());
+                               mem::drop(peer_state_lock);
+                               mem::drop(per_peer_state);
+                               self.handle_monitor_update_completion_actions(update_actions);
+                               return;
+                       };
                let remaining_in_flight =
                        if let Some(pending) = peer_state.in_flight_monitor_updates.get_mut(funding_txo) {
                                pending.retain(|upd| upd.update_id > highest_applied_update_id);
                                pending.len()
                        } else { 0 };
                log_trace!(self.logger, "ChannelMonitor updated to {}. Current highest is {}. {} pending in-flight updates.",
-                       highest_applied_update_id, channel.get().context.get_latest_monitor_update_id(),
+                       highest_applied_update_id, channel.context.get_latest_monitor_update_id(),
                        remaining_in_flight);
-               if !channel.get().is_awaiting_monitor_update() || channel.get().context.get_latest_monitor_update_id() != highest_applied_update_id {
+               if !channel.is_awaiting_monitor_update() || channel.context.get_latest_monitor_update_id() != highest_applied_update_id {
                        return;
                }
-               handle_monitor_update_completion!(self, peer_state_lock, peer_state, per_peer_state, channel.get_mut());
+               handle_monitor_update_completion!(self, peer_state_lock, peer_state, per_peer_state, channel);
        }
 
        /// Accepts a request to open a channel after a [`Event::OpenChannelRequest`].
@@ -5021,49 +5321,61 @@ where
                let mut peer_state_lock = peer_state_mutex.lock().unwrap();
                let peer_state = &mut *peer_state_lock;
                let is_only_peer_channel = peer_state.total_channel_count() == 1;
-               match peer_state.inbound_v1_channel_by_id.entry(temporary_channel_id.clone()) {
-                       hash_map::Entry::Occupied(mut channel) => {
-                               if !channel.get().is_awaiting_accept() {
-                                       return Err(APIError::APIMisuseError { err: "The channel isn't currently awaiting to be accepted.".to_owned() });
+
+               // Find (and remove) the channel in the unaccepted table. If it's not there, something weird is
+               // happening and return an error. N.B. that we create channel with an outbound SCID of zero so
+               // that we can delay allocating the SCID until after we're sure that the checks below will
+               // succeed.
+               let mut channel = match peer_state.inbound_channel_request_by_id.remove(temporary_channel_id) {
+                       Some(unaccepted_channel) => {
+                               let best_block_height = self.best_block.read().unwrap().height();
+                               InboundV1Channel::new(&self.fee_estimator, &self.entropy_source, &self.signer_provider,
+                                       counterparty_node_id.clone(), &self.channel_type_features(), &peer_state.latest_features,
+                                       &unaccepted_channel.open_channel_msg, user_channel_id, &self.default_configuration, best_block_height,
+                                       &self.logger, accept_0conf).map_err(|e| APIError::ChannelUnavailable { err: e.to_string() })
+                       }
+                       _ => Err(APIError::APIMisuseError { err: "No such channel awaiting to be accepted.".to_owned() })
+               }?;
+
+               if accept_0conf {
+                       // This should have been correctly configured by the call to InboundV1Channel::new.
+                       debug_assert!(channel.context.minimum_depth().unwrap() == 0);
+               } else if channel.context.get_channel_type().requires_zero_conf() {
+                       let send_msg_err_event = events::MessageSendEvent::HandleError {
+                               node_id: channel.context.get_counterparty_node_id(),
+                               action: msgs::ErrorAction::SendErrorMessage{
+                                       msg: msgs::ErrorMessage { channel_id: temporary_channel_id.clone(), data: "No zero confirmation channels accepted".to_owned(), }
                                }
-                               if accept_0conf {
-                                       channel.get_mut().set_0conf();
-                               } else if channel.get().context.get_channel_type().requires_zero_conf() {
-                                       let send_msg_err_event = events::MessageSendEvent::HandleError {
-                                               node_id: channel.get().context.get_counterparty_node_id(),
-                                               action: msgs::ErrorAction::SendErrorMessage{
-                                                       msg: msgs::ErrorMessage { channel_id: temporary_channel_id.clone(), data: "No zero confirmation channels accepted".to_owned(), }
-                                               }
-                                       };
-                                       peer_state.pending_msg_events.push(send_msg_err_event);
-                                       let _ = remove_channel!(self, channel);
-                                       return Err(APIError::APIMisuseError { err: "Please use accept_inbound_channel_from_trusted_peer_0conf to accept channels with zero confirmations.".to_owned() });
-                               } else {
-                                       // If this peer already has some channels, a new channel won't increase our number of peers
-                                       // with unfunded channels, so as long as we aren't over the maximum number of unfunded
-                                       // channels per-peer we can accept channels from a peer with existing ones.
-                                       if is_only_peer_channel && peers_without_funded_channels >= MAX_UNFUNDED_CHANNEL_PEERS {
-                                               let send_msg_err_event = events::MessageSendEvent::HandleError {
-                                                       node_id: channel.get().context.get_counterparty_node_id(),
-                                                       action: msgs::ErrorAction::SendErrorMessage{
-                                                               msg: msgs::ErrorMessage { channel_id: temporary_channel_id.clone(), data: "Have too many peers with unfunded channels, not accepting new ones".to_owned(), }
-                                                       }
-                                               };
-                                               peer_state.pending_msg_events.push(send_msg_err_event);
-                                               let _ = remove_channel!(self, channel);
-                                               return Err(APIError::APIMisuseError { err: "Too many peers with unfunded channels, refusing to accept new ones".to_owned() });
+                       };
+                       peer_state.pending_msg_events.push(send_msg_err_event);
+                       return Err(APIError::APIMisuseError { err: "Please use accept_inbound_channel_from_trusted_peer_0conf to accept channels with zero confirmations.".to_owned() });
+               } else {
+                       // If this peer already has some channels, a new channel won't increase our number of peers
+                       // with unfunded channels, so as long as we aren't over the maximum number of unfunded
+                       // channels per-peer we can accept channels from a peer with existing ones.
+                       if is_only_peer_channel && peers_without_funded_channels >= MAX_UNFUNDED_CHANNEL_PEERS {
+                               let send_msg_err_event = events::MessageSendEvent::HandleError {
+                                       node_id: channel.context.get_counterparty_node_id(),
+                                       action: msgs::ErrorAction::SendErrorMessage{
+                                               msg: msgs::ErrorMessage { channel_id: temporary_channel_id.clone(), data: "Have too many peers with unfunded channels, not accepting new ones".to_owned(), }
                                        }
-                               }
-
-                               peer_state.pending_msg_events.push(events::MessageSendEvent::SendAcceptChannel {
-                                       node_id: channel.get().context.get_counterparty_node_id(),
-                                       msg: channel.get_mut().accept_inbound_channel(user_channel_id),
-                               });
-                       }
-                       hash_map::Entry::Vacant(_) => {
-                               return Err(APIError::ChannelUnavailable { err: format!("Channel with id {} not found for the passed counterparty node_id {}", log_bytes!(*temporary_channel_id), counterparty_node_id) });
+                               };
+                               peer_state.pending_msg_events.push(send_msg_err_event);
+                               return Err(APIError::APIMisuseError { err: "Too many peers with unfunded channels, refusing to accept new ones".to_owned() });
                        }
                }
+
+               // Now that we know we have a channel, assign an outbound SCID alias.
+               let outbound_scid_alias = self.create_and_insert_outbound_scid_alias();
+               channel.context.set_outbound_scid_alias(outbound_scid_alias);
+
+               peer_state.pending_msg_events.push(events::MessageSendEvent::SendAcceptChannel {
+                       node_id: channel.context.get_counterparty_node_id(),
+                       msg: channel.accept_inbound_channel(),
+               });
+
+               peer_state.inbound_v1_channel_by_id.insert(temporary_channel_id.clone(), channel);
+
                Ok(())
        }
 
@@ -5108,7 +5420,7 @@ where
                                num_unfunded_channels += 1;
                        }
                }
-               num_unfunded_channels
+               num_unfunded_channels + peer.inbound_channel_request_by_id.len()
        }
 
        fn internal_open_channel(&self, counterparty_node_id: &PublicKey, msg: &msgs::OpenChannel) -> Result<(), MsgHandleErrInternal> {
@@ -5120,11 +5432,6 @@ where
                        return Err(MsgHandleErrInternal::send_err_msg_no_close("No inbound channels accepted".to_owned(), msg.temporary_channel_id.clone()));
                }
 
-               let mut random_bytes = [0u8; 16];
-               random_bytes.copy_from_slice(&self.entropy_source.get_secure_random_bytes()[..16]);
-               let user_channel_id = u128::from_be_bytes(random_bytes);
-               let outbound_scid_alias = self.create_and_insert_outbound_scid_alias();
-
                // Get the number of peers with channels, but without funded ones. We don't care too much
                // about peers that never open a channel, so we filter by peers that have at least one
                // channel, and then limit the number of those with unfunded channels.
@@ -5159,46 +5466,59 @@ where
                                msg.temporary_channel_id.clone()));
                }
 
+               let channel_id = msg.temporary_channel_id;
+               let channel_exists = peer_state.has_channel(&channel_id);
+               if channel_exists {
+                       return Err(MsgHandleErrInternal::send_err_msg_no_close("temporary_channel_id collision for the same peer!".to_owned(), msg.temporary_channel_id.clone()));
+               }
+
+               // If we're doing manual acceptance checks on the channel, then defer creation until we're sure we want to accept.
+               if self.default_configuration.manually_accept_inbound_channels {
+                       let mut pending_events = self.pending_events.lock().unwrap();
+                       pending_events.push_back((events::Event::OpenChannelRequest {
+                               temporary_channel_id: msg.temporary_channel_id.clone(),
+                               counterparty_node_id: counterparty_node_id.clone(),
+                               funding_satoshis: msg.funding_satoshis,
+                               push_msat: msg.push_msat,
+                               channel_type: msg.channel_type.clone().unwrap(),
+                       }, None));
+                       peer_state.inbound_channel_request_by_id.insert(channel_id, InboundChannelRequest {
+                               open_channel_msg: msg.clone(),
+                               ticks_remaining: UNACCEPTED_INBOUND_CHANNEL_AGE_LIMIT_TICKS,
+                       });
+                       return Ok(());
+               }
+
+               // Otherwise create the channel right now.
+               let mut random_bytes = [0u8; 16];
+               random_bytes.copy_from_slice(&self.entropy_source.get_secure_random_bytes()[..16]);
+               let user_channel_id = u128::from_be_bytes(random_bytes);
                let mut channel = match InboundV1Channel::new(&self.fee_estimator, &self.entropy_source, &self.signer_provider,
                        counterparty_node_id.clone(), &self.channel_type_features(), &peer_state.latest_features, msg, user_channel_id,
-                       &self.default_configuration, best_block_height, &self.logger, outbound_scid_alias)
+                       &self.default_configuration, best_block_height, &self.logger, /*is_0conf=*/false)
                {
                        Err(e) => {
-                               self.outbound_scid_aliases.lock().unwrap().remove(&outbound_scid_alias);
                                return Err(MsgHandleErrInternal::from_chan_no_close(e, msg.temporary_channel_id));
                        },
                        Ok(res) => res
                };
-               let channel_id = channel.context.channel_id();
-               let channel_exists = peer_state.has_channel(&channel_id);
-               if channel_exists {
-                       self.outbound_scid_aliases.lock().unwrap().remove(&outbound_scid_alias);
-                       return Err(MsgHandleErrInternal::send_err_msg_no_close("temporary_channel_id collision for the same peer!".to_owned(), msg.temporary_channel_id.clone()))
-               } else {
-                       if !self.default_configuration.manually_accept_inbound_channels {
-                               let channel_type = channel.context.get_channel_type();
-                               if channel_type.requires_zero_conf() {
-                                       return Err(MsgHandleErrInternal::send_err_msg_no_close("No zero confirmation channels accepted".to_owned(), msg.temporary_channel_id.clone()));
-                               }
-                               if channel_type.requires_anchors_zero_fee_htlc_tx() {
-                                       return Err(MsgHandleErrInternal::send_err_msg_no_close("No channels with anchor outputs accepted".to_owned(), msg.temporary_channel_id.clone()));
-                               }
-                               peer_state.pending_msg_events.push(events::MessageSendEvent::SendAcceptChannel {
-                                       node_id: counterparty_node_id.clone(),
-                                       msg: channel.accept_inbound_channel(user_channel_id),
-                               });
-                       } else {
-                               let mut pending_events = self.pending_events.lock().unwrap();
-                               pending_events.push_back((events::Event::OpenChannelRequest {
-                                       temporary_channel_id: msg.temporary_channel_id.clone(),
-                                       counterparty_node_id: counterparty_node_id.clone(),
-                                       funding_satoshis: msg.funding_satoshis,
-                                       push_msat: msg.push_msat,
-                                       channel_type: channel.context.get_channel_type().clone(),
-                               }, None));
-                       }
-                       peer_state.inbound_v1_channel_by_id.insert(channel_id, channel);
+
+               let channel_type = channel.context.get_channel_type();
+               if channel_type.requires_zero_conf() {
+                       return Err(MsgHandleErrInternal::send_err_msg_no_close("No zero confirmation channels accepted".to_owned(), msg.temporary_channel_id.clone()));
+               }
+               if channel_type.requires_anchors_zero_fee_htlc_tx() {
+                       return Err(MsgHandleErrInternal::send_err_msg_no_close("No channels with anchor outputs accepted".to_owned(), msg.temporary_channel_id.clone()));
                }
+
+               let outbound_scid_alias = self.create_and_insert_outbound_scid_alias();
+               channel.context.set_outbound_scid_alias(outbound_scid_alias);
+
+               peer_state.pending_msg_events.push(events::MessageSendEvent::SendAcceptChannel {
+                       node_id: counterparty_node_id.clone(),
+                       msg: channel.accept_inbound_channel(),
+               });
+               peer_state.inbound_v1_channel_by_id.insert(channel_id, channel);
                Ok(())
        }
 
@@ -5256,7 +5576,7 @@ where
                                                        let user_id = inbound_chan.context.get_user_id();
                                                        let shutdown_res = inbound_chan.context.force_shutdown(false);
                                                        return Err(MsgHandleErrInternal::from_finish_shutdown(format!("{}", err),
-                                                               msg.temporary_channel_id, user_id, shutdown_res, None));
+                                                               msg.temporary_channel_id, user_id, shutdown_res, None, inbound_chan.context.get_value_satoshis()));
                                                },
                                        }
                                },
@@ -5398,38 +5718,50 @@ where
                                })?;
                        let mut peer_state_lock = peer_state_mutex.lock().unwrap();
                        let peer_state = &mut *peer_state_lock;
-                       match peer_state.channel_by_id.entry(msg.channel_id.clone()) {
-                               hash_map::Entry::Occupied(mut chan_entry) => {
-
-                                       if !chan_entry.get().received_shutdown() {
-                                               log_info!(self.logger, "Received a shutdown message from our counterparty for channel {}{}.",
-                                                       log_bytes!(msg.channel_id),
-                                                       if chan_entry.get().sent_shutdown() { " after we initiated shutdown" } else { "" });
-                                       }
+                       // TODO(dunxen): Fix this duplication when we switch to a single map with enums as per
+                       // https://github.com/lightningdevkit/rust-lightning/issues/2422
+                       if let hash_map::Entry::Occupied(chan_entry) = peer_state.outbound_v1_channel_by_id.entry(msg.channel_id.clone()) {
+                               log_error!(self.logger, "Immediately closing unfunded channel {} as peer asked to cooperatively shut it down (which is unnecessary)", log_bytes!(&msg.channel_id[..]));
+                               self.issue_channel_close_events(&chan_entry.get().context, ClosureReason::CounterpartyCoopClosedUnfundedChannel);
+                               let mut chan = remove_channel!(self, chan_entry);
+                               self.finish_force_close_channel(chan.context.force_shutdown(false));
+                               return Ok(());
+                       } else if let hash_map::Entry::Occupied(chan_entry) = peer_state.inbound_v1_channel_by_id.entry(msg.channel_id.clone()) {
+                               log_error!(self.logger, "Immediately closing unfunded channel {} as peer asked to cooperatively shut it down (which is unnecessary)", log_bytes!(&msg.channel_id[..]));
+                               self.issue_channel_close_events(&chan_entry.get().context, ClosureReason::CounterpartyCoopClosedUnfundedChannel);
+                               let mut chan = remove_channel!(self, chan_entry);
+                               self.finish_force_close_channel(chan.context.force_shutdown(false));
+                               return Ok(());
+                       } else if let hash_map::Entry::Occupied(mut chan_entry) = peer_state.channel_by_id.entry(msg.channel_id.clone()) {
+                               if !chan_entry.get().received_shutdown() {
+                                       log_info!(self.logger, "Received a shutdown message from our counterparty for channel {}{}.",
+                                               log_bytes!(msg.channel_id),
+                                               if chan_entry.get().sent_shutdown() { " after we initiated shutdown" } else { "" });
+                               }
 
-                                       let funding_txo_opt = chan_entry.get().context.get_funding_txo();
-                                       let (shutdown, monitor_update_opt, htlcs) = try_chan_entry!(self,
-                                               chan_entry.get_mut().shutdown(&self.signer_provider, &peer_state.latest_features, &msg), chan_entry);
-                                       dropped_htlcs = htlcs;
+                               let funding_txo_opt = chan_entry.get().context.get_funding_txo();
+                               let (shutdown, monitor_update_opt, htlcs) = try_chan_entry!(self,
+                                       chan_entry.get_mut().shutdown(&self.signer_provider, &peer_state.latest_features, &msg), chan_entry);
+                               dropped_htlcs = htlcs;
 
-                                       if let Some(msg) = shutdown {
-                                               // We can send the `shutdown` message before updating the `ChannelMonitor`
-                                               // here as we don't need the monitor update to complete until we send a
-                                               // `shutdown_signed`, which we'll delay if we're pending a monitor update.
-                                               peer_state.pending_msg_events.push(events::MessageSendEvent::SendShutdown {
-                                                       node_id: *counterparty_node_id,
-                                                       msg,
-                                               });
-                                       }
+                               if let Some(msg) = shutdown {
+                                       // We can send the `shutdown` message before updating the `ChannelMonitor`
+                                       // here as we don't need the monitor update to complete until we send a
+                                       // `shutdown_signed`, which we'll delay if we're pending a monitor update.
+                                       peer_state.pending_msg_events.push(events::MessageSendEvent::SendShutdown {
+                                               node_id: *counterparty_node_id,
+                                               msg,
+                                       });
+                               }
 
-                                       // Update the monitor with the shutdown script if necessary.
-                                       if let Some(monitor_update) = monitor_update_opt {
-                                               break handle_new_monitor_update!(self, funding_txo_opt.unwrap(), monitor_update,
-                                                       peer_state_lock, peer_state, per_peer_state, chan_entry).map(|_| ());
-                                       }
-                                       break Ok(());
-                               },
-                               hash_map::Entry::Vacant(_) => return Err(MsgHandleErrInternal::send_err_msg_no_close(format!("Got a message for a channel from the wrong node! No such channel for the passed counterparty_node_id {}", counterparty_node_id), msg.channel_id))
+                               // Update the monitor with the shutdown script if necessary.
+                               if let Some(monitor_update) = monitor_update_opt {
+                                       break handle_new_monitor_update!(self, funding_txo_opt.unwrap(), monitor_update,
+                                               peer_state_lock, peer_state, per_peer_state, chan_entry).map(|_| ());
+                               }
+                               break Ok(());
+                       } else {
+                               return Err(MsgHandleErrInternal::send_err_msg_no_close(format!("Got a message for a channel from the wrong node! No such channel for the passed counterparty_node_id {}", counterparty_node_id), msg.channel_id))
                        }
                };
                for htlc_source in dropped_htlcs.drain(..) {
@@ -5539,7 +5871,7 @@ where
                                                _ => pending_forward_info
                                        }
                                };
-                               try_chan_entry!(self, chan.get_mut().update_add_htlc(&msg, pending_forward_info, create_pending_htlc_status, &self.logger), chan);
+                               try_chan_entry!(self, chan.get_mut().update_add_htlc(&msg, pending_forward_info, create_pending_htlc_status, &self.fee_estimator, &self.logger), chan);
                        },
                        hash_map::Entry::Vacant(_) => return Err(MsgHandleErrInternal::send_err_msg_no_close(format!("Got a message for a channel from the wrong node! No such channel for the passed counterparty_node_id {}", counterparty_node_id), msg.channel_id))
                }
@@ -5712,16 +6044,21 @@ where
                }
        }
 
-       // We only want to push a PendingHTLCsForwardable event if no others are queued.
        fn push_pending_forwards_ev(&self) {
                let mut pending_events = self.pending_events.lock().unwrap();
-               let forward_ev_exists = pending_events.iter()
-                       .find(|(ev, _)| if let events::Event::PendingHTLCsForwardable { .. } = ev { true } else { false })
-                       .is_some();
-               if !forward_ev_exists {
-                       pending_events.push_back((events::Event::PendingHTLCsForwardable {
-                               time_forwardable:
-                                       Duration::from_millis(MIN_HTLC_RELAY_HOLDING_CELL_MILLIS),
+               let is_processing_events = self.pending_events_processor.load(Ordering::Acquire);
+               let num_forward_events = pending_events.iter().filter(|(ev, _)|
+                       if let events::Event::PendingHTLCsForwardable { .. } = ev { true } else { false }
+               ).count();
+               // We only want to push a PendingHTLCsForwardable event if no others are queued. Processing
+               // events is done in batches and they are not removed until we're done processing each
+               // batch. Since handling a `PendingHTLCsForwardable` event will call back into the
+               // `ChannelManager`, we'll still see the original forwarding event not removed. Phantom
+               // payments will need an additional forwarding event before being claimed to make them look
+               // real by taking more time.
+               if (is_processing_events && num_forward_events <= 1) || num_forward_events < 1 {
+                       pending_events.push_back((Event::PendingHTLCsForwardable {
+                               time_forwardable: Duration::from_millis(MIN_HTLC_RELAY_HOLDING_CELL_MILLIS),
                        }, None));
                }
        }
@@ -5756,7 +6093,7 @@ where
                        match peer_state.channel_by_id.entry(msg.channel_id) {
                                hash_map::Entry::Occupied(mut chan) => {
                                        let funding_txo = chan.get().context.get_funding_txo();
-                                       let (htlcs_to_fail, monitor_update_opt) = try_chan_entry!(self, chan.get_mut().revoke_and_ack(&msg, &self.logger), chan);
+                                       let (htlcs_to_fail, monitor_update_opt) = try_chan_entry!(self, chan.get_mut().revoke_and_ack(&msg, &self.fee_estimator, &self.logger), chan);
                                        let res = if let Some(monitor_update) = monitor_update_opt {
                                                handle_new_monitor_update!(self, funding_txo.unwrap(), monitor_update,
                                                        peer_state_lock, peer_state, per_peer_state, chan).map(|_| ())
@@ -6027,7 +6364,7 @@ where
                                                let counterparty_node_id = chan.context.get_counterparty_node_id();
                                                let funding_txo = chan.context.get_funding_txo();
                                                let (monitor_opt, holding_cell_failed_htlcs) =
-                                                       chan.maybe_free_holding_cell_htlcs(&self.logger);
+                                                       chan.maybe_free_holding_cell_htlcs(&self.fee_estimator, &self.logger);
                                                if !holding_cell_failed_htlcs.is_empty() {
                                                        failed_htlcs.push((holding_cell_failed_htlcs, *channel_id, counterparty_node_id));
                                                }
@@ -6839,13 +7176,13 @@ where
                provided_node_features(&self.default_configuration)
        }
 
-       /// Fetches the set of [`InvoiceFeatures`] flags which are provided by or required by
+       /// Fetches the set of [`Bolt11InvoiceFeatures`] flags which are provided by or required by
        /// [`ChannelManager`].
        ///
        /// Note that the invoice feature flags can vary depending on if the invoice is a "phantom invoice"
        /// or not. Thus, this method is not public.
        #[cfg(any(feature = "_test_utils", test))]
-       pub fn invoice_features(&self) -> InvoiceFeatures {
+       pub fn invoice_features(&self) -> Bolt11InvoiceFeatures {
                provided_invoice_features(&self.default_configuration)
        }
 
@@ -7013,6 +7350,9 @@ where
                                        self.issue_channel_close_events(&chan.context, ClosureReason::DisconnectedPeer);
                                        false
                                });
+                               // Note that we don't bother generating any events for pre-accept channels -
+                               // they're not considered "channels" yet from the PoV of our events interface.
+                               peer_state.inbound_channel_request_by_id.clear();
                                pending_msg_events.retain(|msg| {
                                        match msg {
                                                // V1 Channel Establishment
@@ -7096,6 +7436,7 @@ where
                                                channel_by_id: HashMap::new(),
                                                outbound_v1_channel_by_id: HashMap::new(),
                                                inbound_v1_channel_by_id: HashMap::new(),
+                                               inbound_channel_request_by_id: HashMap::new(),
                                                latest_features: init_msg.features.clone(),
                                                pending_msg_events: Vec::new(),
                                                in_flight_monitor_updates: BTreeMap::new(),
@@ -7125,37 +7466,20 @@ where
                log_debug!(self.logger, "Generating channel_reestablish events for {}", log_pubkey!(counterparty_node_id));
 
                let per_peer_state = self.per_peer_state.read().unwrap();
-               for (_cp_id, peer_state_mutex) in per_peer_state.iter() {
+               if let Some(peer_state_mutex) = per_peer_state.get(counterparty_node_id) {
                        let mut peer_state_lock = peer_state_mutex.lock().unwrap();
                        let peer_state = &mut *peer_state_lock;
                        let pending_msg_events = &mut peer_state.pending_msg_events;
-                       peer_state.channel_by_id.retain(|_, chan| {
-                               let retain = if chan.context.get_counterparty_node_id() == *counterparty_node_id {
-                                       if !chan.context.have_received_message() {
-                                               // If we created this (outbound) channel while we were disconnected from the
-                                               // peer we probably failed to send the open_channel message, which is now
-                                               // lost. We can't have had anything pending related to this channel, so we just
-                                               // drop it.
-                                               false
-                                       } else {
-                                               pending_msg_events.push(events::MessageSendEvent::SendChannelReestablish {
-                                                       node_id: chan.context.get_counterparty_node_id(),
-                                                       msg: chan.get_channel_reestablish(&self.logger),
-                                               });
-                                               true
-                                       }
-                               } else { true };
-                               if retain && chan.context.get_counterparty_node_id() != *counterparty_node_id {
-                                       if let Some(msg) = chan.get_signed_channel_announcement(&self.node_signer, self.genesis_hash.clone(), self.best_block.read().unwrap().height(), &self.default_configuration) {
-                                               if let Ok(update_msg) = self.get_channel_update_for_broadcast(chan) {
-                                                       pending_msg_events.push(events::MessageSendEvent::SendChannelAnnouncement {
-                                                               node_id: *counterparty_node_id,
-                                                               msg, update_msg,
-                                                       });
-                                               }
-                                       }
-                               }
-                               retain
+
+                       // Since unfunded channel maps are cleared upon disconnecting a peer, and they're not persisted
+                       // (so won't be recovered after a crash) we don't need to bother closing unfunded channels and
+                       // clearing their maps here. Instead we can just send queue channel_reestablish messages for
+                       // channels in the channel_by_id map.
+                       peer_state.channel_by_id.iter_mut().for_each(|(_, chan)| {
+                               pending_msg_events.push(events::MessageSendEvent::SendChannelReestablish {
+                                       node_id: chan.context.get_counterparty_node_id(),
+                                       msg: chan.get_channel_reestablish(&self.logger),
+                               });
                        });
                }
                //TODO: Also re-broadcast announcement_signatures
@@ -7165,6 +7489,46 @@ where
        fn handle_error(&self, counterparty_node_id: &PublicKey, msg: &msgs::ErrorMessage) {
                let _persistence_guard = PersistenceNotifierGuard::notify_on_drop(self);
 
+               match &msg.data as &str {
+                       "cannot co-op close channel w/ active htlcs"|
+                       "link failed to shutdown" =>
+                       {
+                               // LND hasn't properly handled shutdown messages ever, and force-closes any time we
+                               // send one while HTLCs are still present. The issue is tracked at
+                               // https://github.com/lightningnetwork/lnd/issues/6039 and has had multiple patches
+                               // to fix it but none so far have managed to land upstream. The issue appears to be
+                               // very low priority for the LND team despite being marked "P1".
+                               // We're not going to bother handling this in a sensible way, instead simply
+                               // repeating the Shutdown message on repeat until morale improves.
+                               if msg.channel_id != [0; 32] {
+                                       let per_peer_state = self.per_peer_state.read().unwrap();
+                                       let peer_state_mutex_opt = per_peer_state.get(counterparty_node_id);
+                                       if peer_state_mutex_opt.is_none() { return; }
+                                       let mut peer_state = peer_state_mutex_opt.unwrap().lock().unwrap();
+                                       if let Some(chan) = peer_state.channel_by_id.get(&msg.channel_id) {
+                                               if let Some(msg) = chan.get_outbound_shutdown() {
+                                                       peer_state.pending_msg_events.push(events::MessageSendEvent::SendShutdown {
+                                                               node_id: *counterparty_node_id,
+                                                               msg,
+                                                       });
+                                               }
+                                               peer_state.pending_msg_events.push(events::MessageSendEvent::HandleError {
+                                                       node_id: *counterparty_node_id,
+                                                       action: msgs::ErrorAction::SendWarningMessage {
+                                                               msg: msgs::WarningMessage {
+                                                                       channel_id: msg.channel_id,
+                                                                       data: "You appear to be exhibiting LND bug 6039, we'll keep sending you shutdown messages until you handle them correctly".to_owned()
+                                                               },
+                                                               log_level: Level::Trace,
+                                                       }
+                                               });
+                                       }
+                               }
+                               return;
+                       }
+                       _ => {}
+               }
+
                if msg.channel_id == [0; 32] {
                        let channel_ids: Vec<[u8; 32]> = {
                                let per_peer_state = self.per_peer_state.read().unwrap();
@@ -7172,6 +7536,9 @@ where
                                if peer_state_mutex_opt.is_none() { return; }
                                let mut peer_state_lock = peer_state_mutex_opt.unwrap().lock().unwrap();
                                let peer_state = &mut *peer_state_lock;
+                               // Note that we don't bother generating any events for pre-accept channels -
+                               // they're not considered "channels" yet from the PoV of our events interface.
+                               peer_state.inbound_channel_request_by_id.clear();
                                peer_state.channel_by_id.keys().cloned()
                                        .chain(peer_state.outbound_v1_channel_by_id.keys().cloned())
                                        .chain(peer_state.inbound_v1_channel_by_id.keys().cloned()).collect()
@@ -7189,7 +7556,7 @@ where
                                let mut peer_state_lock = peer_state_mutex_opt.unwrap().lock().unwrap();
                                let peer_state = &mut *peer_state_lock;
                                if let Some(chan) = peer_state.outbound_v1_channel_by_id.get_mut(&msg.channel_id) {
-                                       if let Ok(msg) = chan.maybe_handle_error_without_close(self.genesis_hash) {
+                                       if let Ok(msg) = chan.maybe_handle_error_without_close(self.genesis_hash, &self.fee_estimator) {
                                                peer_state.pending_msg_events.push(events::MessageSendEvent::SendOpenChannel {
                                                        node_id: *counterparty_node_id,
                                                        msg,
@@ -7274,16 +7641,18 @@ where
 /// Fetches the set of [`NodeFeatures`] flags which are provided by or required by
 /// [`ChannelManager`].
 pub(crate) fn provided_node_features(config: &UserConfig) -> NodeFeatures {
-       provided_init_features(config).to_context()
+       let mut node_features = provided_init_features(config).to_context();
+       node_features.set_keysend_optional();
+       node_features
 }
 
-/// Fetches the set of [`InvoiceFeatures`] flags which are provided by or required by
+/// Fetches the set of [`Bolt11InvoiceFeatures`] flags which are provided by or required by
 /// [`ChannelManager`].
 ///
 /// Note that the invoice feature flags can vary depending on if the invoice is a "phantom invoice"
 /// or not. Thus, this method is not public.
 #[cfg(any(feature = "_test_utils", test))]
-pub(crate) fn provided_invoice_features(config: &UserConfig) -> InvoiceFeatures {
+pub(crate) fn provided_invoice_features(config: &UserConfig) -> Bolt11InvoiceFeatures {
        provided_init_features(config).to_context()
 }
 
@@ -7360,7 +7729,7 @@ impl Writeable for ChannelDetails {
                        (10, self.channel_value_satoshis, required),
                        (12, self.unspendable_punishment_reserve, option),
                        (14, user_channel_id_low, required),
-                       (16, self.balance_msat, required),
+                       (16, self.next_outbound_htlc_limit_msat, required),  // Forwards compatibility for removed balance_msat field.
                        (18, self.outbound_capacity_msat, required),
                        (19, self.next_outbound_htlc_limit_msat, required),
                        (20, self.inbound_capacity_msat, required),
@@ -7375,6 +7744,7 @@ impl Writeable for ChannelDetails {
                        (35, self.inbound_htlc_maximum_msat, option),
                        (37, user_channel_id_high_opt, option),
                        (39, self.feerate_sat_per_1000_weight, option),
+                       (41, self.channel_shutdown_state, option),
                });
                Ok(())
        }
@@ -7395,7 +7765,7 @@ impl Readable for ChannelDetails {
                        (10, channel_value_satoshis, required),
                        (12, unspendable_punishment_reserve, option),
                        (14, user_channel_id_low, required),
-                       (16, balance_msat, required),
+                       (16, _balance_msat, option),  // Backwards compatibility for removed balance_msat field.
                        (18, outbound_capacity_msat, required),
                        // Note that by the time we get past the required read above, outbound_capacity_msat will be
                        // filled in, so we can safely unwrap it here.
@@ -7412,6 +7782,7 @@ impl Readable for ChannelDetails {
                        (35, inbound_htlc_maximum_msat, option),
                        (37, user_channel_id_high_opt, option),
                        (39, feerate_sat_per_1000_weight, option),
+                       (41, channel_shutdown_state, option),
                });
 
                // `user_channel_id` used to be a single u64 value. In order to remain backwards compatible with
@@ -7420,6 +7791,8 @@ impl Readable for ChannelDetails {
                let user_channel_id = user_channel_id_low as u128 +
                        ((user_channel_id_high_opt.unwrap_or(0 as u64) as u128) << 64);
 
+               let _balance_msat: Option<u64> = _balance_msat;
+
                Ok(Self {
                        inbound_scid_alias,
                        channel_id: channel_id.0.unwrap(),
@@ -7432,7 +7805,6 @@ impl Readable for ChannelDetails {
                        channel_value_satoshis: channel_value_satoshis.0.unwrap(),
                        unspendable_punishment_reserve,
                        user_channel_id,
-                       balance_msat: balance_msat.0.unwrap(),
                        outbound_capacity_msat: outbound_capacity_msat.0.unwrap(),
                        next_outbound_htlc_limit_msat: next_outbound_htlc_limit_msat.0.unwrap(),
                        next_outbound_htlc_minimum_msat: next_outbound_htlc_minimum_msat.0.unwrap(),
@@ -7447,12 +7819,13 @@ impl Readable for ChannelDetails {
                        inbound_htlc_minimum_msat,
                        inbound_htlc_maximum_msat,
                        feerate_sat_per_1000_weight,
+                       channel_shutdown_state,
                })
        }
 }
 
 impl_writeable_tlv_based!(PhantomRouteHints, {
-       (2, channels, vec_type),
+       (2, channels, required_vec),
        (4, phantom_scid, required),
        (6, real_node_pubkey, required),
 });
@@ -7467,12 +7840,14 @@ impl_writeable_tlv_based_enum!(PendingHTLCRouting,
                (1, phantom_shared_secret, option),
                (2, incoming_cltv_expiry, required),
                (3, payment_metadata, option),
+               (5, custom_tlvs, optional_vec),
        },
        (2, ReceiveKeysend) => {
                (0, payment_preimage, required),
                (2, incoming_cltv_expiry, required),
                (3, payment_metadata, option),
                (4, payment_data, option), // Added in 0.0.116
+               (5, custom_tlvs, optional_vec),
        },
 ;);
 
@@ -7644,7 +8019,7 @@ impl Readable for HTLCSource {
                        0 => {
                                let mut session_priv: crate::util::ser::RequiredWrapper<SecretKey> = crate::util::ser::RequiredWrapper(None);
                                let mut first_hop_htlc_msat: u64 = 0;
-                               let mut path_hops: Option<Vec<RouteHop>> = Some(Vec::new());
+                               let mut path_hops = Vec::new();
                                let mut payment_id = None;
                                let mut payment_params: Option<PaymentParameters> = None;
                                let mut blinded_tail: Option<BlindedTail> = None;
@@ -7652,7 +8027,7 @@ impl Readable for HTLCSource {
                                        (0, session_priv, required),
                                        (1, payment_id, option),
                                        (2, first_hop_htlc_msat, required),
-                                       (4, path_hops, vec_type),
+                                       (4, path_hops, required_vec),
                                        (5, payment_params, (option: ReadableArgs, 0)),
                                        (6, blinded_tail, option),
                                });
@@ -7661,7 +8036,7 @@ impl Readable for HTLCSource {
                                        // instead.
                                        payment_id = Some(PaymentId(*session_priv.0.unwrap().as_ref()));
                                }
-                               let path = Path { hops: path_hops.ok_or(DecodeError::InvalidValue)?, blinded_tail };
+                               let path = Path { hops: path_hops, blinded_tail };
                                if path.hops.len() == 0 {
                                        return Err(DecodeError::InvalidValue);
                                }
@@ -7696,7 +8071,7 @@ impl Writeable for HTLCSource {
                                        (1, payment_id_opt, option),
                                        (2, first_hop_htlc_msat, required),
                                        // 3 was previously used to write a PaymentSecret for the payment.
-                                       (4, path.hops, vec_type),
+                                       (4, path.hops, required_vec),
                                        (5, None::<PaymentParameters>, option), // payment_params in LDK versions prior to 0.0.115
                                        (6, path.blinded_tail, option),
                                 });
@@ -7946,7 +8321,7 @@ where
                        (6, monitor_update_blocked_actions_per_peer, option),
                        (7, self.fake_scid_rand_bytes, required),
                        (8, if events_not_backwards_compatible { Some(&*events) } else { None }, option),
-                       (9, htlc_purposes, vec_type),
+                       (9, htlc_purposes, required_vec),
                        (10, in_flight_monitor_updates, option),
                        (11, self.probing_cookie_secret, required),
                        (13, htlc_onion_fields, optional_vec),
@@ -7997,6 +8372,14 @@ impl Readable for VecDeque<(Event, Option<EventCompletionAction>)> {
        }
 }
 
+impl_writeable_tlv_based_enum!(ChannelShutdownState,
+       (0, NotShuttingDown) => {},
+       (2, ShutdownInitiated) => {},
+       (4, ResolvingHTLCs) => {},
+       (6, NegotiatingClosingFee) => {},
+       (8, ShutdownComplete) => {}, ;
+);
+
 /// Arguments for the creation of a ChannelManager that are not deserialized.
 ///
 /// At a high-level, the process for deserializing a ChannelManager and resuming normal operation
@@ -8191,7 +8574,9 @@ where
                                        channel_closures.push_back((events::Event::ChannelClosed {
                                                channel_id: channel.context.channel_id(),
                                                user_channel_id: channel.context.get_user_id(),
-                                               reason: ClosureReason::OutdatedChannelManager
+                                               reason: ClosureReason::OutdatedChannelManager,
+                                               counterparty_node_id: Some(channel.context.get_counterparty_node_id()),
+                                               channel_capacity_sats: Some(channel.context.get_value_satoshis()),
                                        }, None));
                                        for (channel_htlc_source, payment_hash) in channel.inflight_htlc_sources() {
                                                let mut found_htlc = false;
@@ -8243,6 +8628,8 @@ where
                                        channel_id: channel.context.channel_id(),
                                        user_channel_id: channel.context.get_user_id(),
                                        reason: ClosureReason::DisconnectedPeer,
+                                       counterparty_node_id: Some(channel.context.get_counterparty_node_id()),
+                                       channel_capacity_sats: Some(channel.context.get_value_satoshis()),
                                }, None));
                        } else {
                                log_error!(args.logger, "Missing ChannelMonitor for channel {} needed by ChannelManager.", log_bytes!(channel.context.channel_id()));
@@ -8262,7 +8649,7 @@ where
                                        update_id: CLOSED_CHANNEL_UPDATE_ID,
                                        updates: vec![ChannelMonitorUpdateStep::ChannelForceClosed { should_broadcast: true }],
                                };
-                               close_background_events.push(BackgroundEvent::ClosingMonitorUpdateRegeneratedOnStartup((*funding_txo, monitor_update)));
+                               close_background_events.push(BackgroundEvent::ClosedMonitorUpdateRegeneratedOnStartup((*funding_txo, monitor_update)));
                        }
                }
 
@@ -8296,6 +8683,7 @@ where
                                channel_by_id,
                                outbound_v1_channel_by_id: HashMap::new(),
                                inbound_v1_channel_by_id: HashMap::new(),
+                               inbound_channel_request_by_id: HashMap::new(),
                                latest_features: InitFeatures::empty(),
                                pending_msg_events: Vec::new(),
                                in_flight_monitor_updates: BTreeMap::new(),
@@ -8385,7 +8773,7 @@ where
                        (6, monitor_update_blocked_actions_per_peer, option),
                        (7, fake_scid_rand_bytes, option),
                        (8, events_override, option),
-                       (9, claimable_htlc_purposes, vec_type),
+                       (9, claimable_htlc_purposes, optional_vec),
                        (10, in_flight_monitor_updates, option),
                        (11, probing_cookie_secret, option),
                        (13, claimable_htlc_onion_fields, optional_vec),
@@ -8449,6 +8837,16 @@ where
                                                        update: update.clone(),
                                                });
                                }
+                               if $chan_in_flight_upds.is_empty() {
+                                       // We had some updates to apply, but it turns out they had completed before we
+                                       // were serialized, we just weren't notified of that. Thus, we may have to run
+                                       // the completion actions for any monitor updates, but otherwise are done.
+                                       pending_background_events.push(
+                                               BackgroundEvent::MonitorUpdatesComplete {
+                                                       counterparty_node_id: $counterparty_node_id,
+                                                       channel_id: $funding_txo.to_channel_id(),
+                                               });
+                               }
                                if $peer_state.in_flight_monitor_updates.insert($funding_txo, $chan_in_flight_upds).is_some() {
                                        log_error!(args.logger, "Duplicate in-flight monitor update set for the same channel!");
                                        return Err(DecodeError::InvalidValue);
@@ -8561,6 +8959,7 @@ where
                                                                                payment_secret: None, // only used for retries, and we'll never retry on startup
                                                                                payment_metadata: None, // only used for retries, and we'll never retry on startup
                                                                                keysend_preimage: None, // only used for retries, and we'll never retry on startup
+                                                                               custom_tlvs: Vec::new(), // only used for retries, and we'll never retry on startup
                                                                                pending_amt_msat: path_amt,
                                                                                pending_fee_msat: Some(path_fee),
                                                                                total_msat: path_amt,
@@ -8841,6 +9240,12 @@ where
                                                                blocked_peer_state.lock().unwrap().actions_blocking_raa_monitor_updates
                                                                        .entry(blocked_channel_outpoint.to_channel_id())
                                                                        .or_insert_with(Vec::new).push(blocking_action.clone());
+                                                       } else {
+                                                               // If the channel we were blocking has closed, we don't need to
+                                                               // worry about it - the blocked monitor update should never have
+                                                               // been released from the `Channel` object so it can't have
+                                                               // completed, and if the channel closed there's no reason to bother
+                                                               // anymore.
                                                        }
                                                }
                                        }
@@ -9442,7 +9847,7 @@ mod tests {
                nodes[0].node.force_close_broadcasting_latest_txn(&chan.2, &nodes[1].node.get_our_node_id()).unwrap();
                check_closed_broadcast!(nodes[0], true);
                check_added_monitors!(nodes[0], 1);
-               check_closed_event!(nodes[0], 1, ClosureReason::HolderForceClosed);
+               check_closed_event!(nodes[0], 1, ClosureReason::HolderForceClosed, [nodes[1].node.get_our_node_id()], 100000);
 
                {
                        // Assert that nodes[1] is awaiting removal for nodes[0] once nodes[1] has been
@@ -9605,8 +10010,8 @@ mod tests {
                }
                let (_nodes_1_update, _none) = get_closing_signed_broadcast!(nodes[1].node, nodes[0].node.get_our_node_id());
 
-               check_closed_event!(nodes[0], 1, ClosureReason::CooperativeClosure);
-               check_closed_event!(nodes[1], 1, ClosureReason::CooperativeClosure);
+               check_closed_event!(nodes[0], 1, ClosureReason::CooperativeClosure, [nodes[1].node.get_our_node_id()], 1000000);
+               check_closed_event!(nodes[1], 1, ClosureReason::CooperativeClosure, [nodes[0].node.get_our_node_id()], 1000000);
        }
 
        fn check_not_connected_to_peer_error<T>(res_err: Result<T, APIError>, expected_public_key: PublicKey) {
@@ -9889,20 +10294,19 @@ mod tests {
                let node = create_network(1, &node_cfg, &node_chanmgr);
                let sender_intended_amt_msat = 100;
                let extra_fee_msat = 10;
-               let hop_data = msgs::OnionHopData {
-                       amt_to_forward: 100,
+               let hop_data = msgs::InboundOnionPayload::Receive {
+                       amt_msat: 100,
                        outgoing_cltv_value: 42,
-                       format: msgs::OnionHopDataFormat::FinalNode {
-                               keysend_preimage: None,
-                               payment_metadata: None,
-                               payment_data: Some(msgs::FinalOnionHopData {
-                                       payment_secret: PaymentSecret([0; 32]), total_msat: sender_intended_amt_msat,
-                               }),
-                       }
+                       payment_metadata: None,
+                       keysend_preimage: None,
+                       payment_data: Some(msgs::FinalOnionHopData {
+                               payment_secret: PaymentSecret([0; 32]), total_msat: sender_intended_amt_msat,
+                       }),
+                       custom_tlvs: Vec::new(),
                };
                // Check that if the amount we received + the penultimate hop extra fee is less than the sender
                // intended amount, we fail the payment.
-               if let Err(crate::ln::channelmanager::ReceiveError { err_code, .. }) =
+               if let Err(crate::ln::channelmanager::InboundOnionErr { err_code, .. }) =
                        node[0].node.construct_recv_pending_htlc_info(hop_data, [0; 32], PaymentHash([0; 32]),
                                sender_intended_amt_msat - extra_fee_msat - 1, 42, None, true, Some(extra_fee_msat))
                {
@@ -9910,16 +10314,15 @@ mod tests {
                } else { panic!(); }
 
                // If amt_received + extra_fee is equal to the sender intended amount, we're fine.
-               let hop_data = msgs::OnionHopData { // This is the same hop_data as above, OnionHopData doesn't implement Clone
-                       amt_to_forward: 100,
+               let hop_data = msgs::InboundOnionPayload::Receive { // This is the same payload as above, InboundOnionPayload doesn't implement Clone
+                       amt_msat: 100,
                        outgoing_cltv_value: 42,
-                       format: msgs::OnionHopDataFormat::FinalNode {
-                               keysend_preimage: None,
-                               payment_metadata: None,
-                               payment_data: Some(msgs::FinalOnionHopData {
-                                       payment_secret: PaymentSecret([0; 32]), total_msat: sender_intended_amt_msat,
-                               }),
-                       }
+                       payment_metadata: None,
+                       keysend_preimage: None,
+                       payment_data: Some(msgs::FinalOnionHopData {
+                               payment_secret: PaymentSecret([0; 32]), total_msat: sender_intended_amt_msat,
+                       }),
+                       custom_tlvs: Vec::new(),
                };
                assert!(node[0].node.construct_recv_pending_htlc_info(hop_data, [0; 32], PaymentHash([0; 32]),
                        sender_intended_amt_msat - extra_fee_msat, 42, None, true, Some(extra_fee_msat)).is_ok());
@@ -10001,7 +10404,9 @@ mod tests {
                let open_channel_msg = get_event_msg!(nodes[0], MessageSendEvent::SendOpenChannel, nodes[1].node.get_our_node_id());
                assert!(!open_channel_msg.channel_type.unwrap().supports_anchors_zero_fee_htlc_tx());
 
-               check_closed_event!(nodes[1], 1, ClosureReason::HolderForceClosed);
+               // Since nodes[1] should not have accepted the channel, it should
+               // not have generated any events.
+               assert!(nodes[1].node.get_and_clear_pending_events().is_empty());
        }
 
        #[test]
@@ -10058,6 +10463,25 @@ mod tests {
                        MessageSendEvent::BroadcastChannelUpdate { .. } => {},
                        _ => panic!("expected BroadcastChannelUpdate event"),
                }
+
+               // If we provide a channel_id not associated with the peer, we should get an error and no updates
+               // should be applied to ensure update atomicity as specified in the API docs.
+               let bad_channel_id = [10; 32];
+               let current_fee = nodes[0].node.list_channels()[0].config.unwrap().forwarding_fee_proportional_millionths;
+               let new_fee = current_fee + 100;
+               assert!(
+                       matches!(
+                               nodes[0].node.update_partial_channel_config(&channel.counterparty.node_id, &[channel.channel_id, bad_channel_id], &ChannelConfigUpdate {
+                                       forwarding_fee_proportional_millionths: Some(new_fee),
+                                       ..Default::default()
+                               }),
+                               Err(APIError::ChannelUnavailable { err: _ }),
+                       )
+               );
+               // Check that the fee hasn't changed for the channel that exists.
+               assert_eq!(nodes[0].node.list_channels()[0].config.unwrap().forwarding_fee_proportional_millionths, current_fee);
+               let events = nodes[0].node.get_and_clear_pending_msg_events();
+               assert_eq!(events.len(), 0);
        }
 }
 
@@ -10073,7 +10497,7 @@ pub mod bench {
        use crate::routing::gossip::NetworkGraph;
        use crate::routing::router::{PaymentParameters, RouteParameters};
        use crate::util::test_utils;
-       use crate::util::config::UserConfig;
+       use crate::util::config::{UserConfig, MaxDustHTLCExposure};
 
        use bitcoin::hashes::Hash;
        use bitcoin::hashes::sha256::Hash as Sha256;
@@ -10111,6 +10535,7 @@ pub mod bench {
                // Note that this is unrealistic as each payment send will require at least two fsync
                // calls per node.
                let network = bitcoin::Network::Testnet;
+               let genesis_block = bitcoin::blockdata::constants::genesis_block(network);
 
                let tx_broadcaster = test_utils::TestBroadcaster::new(network);
                let fee_estimator = test_utils::TestFeeEstimator { sat_per_kw: Mutex::new(253) };
@@ -10119,6 +10544,7 @@ pub mod bench {
                let router = test_utils::TestRouter::new(Arc::new(NetworkGraph::new(network, &logger_a)), &scorer);
 
                let mut config: UserConfig = Default::default();
+               config.channel_config.max_dust_htlc_exposure = MaxDustHTLCExposure::FeeRateMultiplier(5_000_000 / 253);
                config.channel_handshake_config.minimum_depth = 1;
 
                let chain_monitor_a = ChainMonitor::new(None, &tx_broadcaster, &logger_a, &fee_estimator, &persister_a);
@@ -10127,7 +10553,7 @@ pub mod bench {
                let node_a = ChannelManager::new(&fee_estimator, &chain_monitor_a, &tx_broadcaster, &router, &logger_a, &keys_manager_a, &keys_manager_a, &keys_manager_a, config.clone(), ChainParameters {
                        network,
                        best_block: BestBlock::from_network(network),
-               });
+               }, genesis_block.header.time);
                let node_a_holder = ANodeHolder { node: &node_a };
 
                let logger_b = test_utils::TestLogger::with_id("node a".to_owned());
@@ -10137,7 +10563,7 @@ pub mod bench {
                let node_b = ChannelManager::new(&fee_estimator, &chain_monitor_b, &tx_broadcaster, &router, &logger_b, &keys_manager_b, &keys_manager_b, &keys_manager_b, config.clone(), ChainParameters {
                        network,
                        best_block: BestBlock::from_network(network),
-               });
+               }, genesis_block.header.time);
                let node_b_holder = ANodeHolder { node: &node_b };
 
                node_a.peer_connected(&node_b.get_our_node_id(), &Init {