// Alternatively, we can fill an outbound HTLC with a HTLCSource::OutboundRoute indicating this is
// our payment, which we can use to decode errors or inform the user that the payment was sent.
+/// Routing info for an inbound HTLC onion.
#[derive(Clone)] // See Channel::revoke_and_ack for why, tl;dr: Rust bug
-pub(super) enum PendingHTLCRouting {
+pub enum PendingHTLCRouting {
+ /// A forwarded HTLC.
Forward {
+ /// BOLT 4 onion packet.
onion_packet: msgs::OnionPacket,
/// The SCID from the onion that we should forward to. This could be a real SCID or a fake one
/// generated using `get_fake_scid` from the scid_utils::fake_scid module.
short_channel_id: u64, // This should be NonZero<u64> eventually when we bump MSRV
},
+ /// An HTLC paid to an invoice we generated.
Receive {
+ /// Payment secret and total msat received.
payment_data: msgs::FinalOnionHopData,
+ /// See [`RecipientOnionFields::payment_metadata`] for more info.
payment_metadata: Option<Vec<u8>>,
- incoming_cltv_expiry: u32, // Used to track when we should expire pending HTLCs that go unclaimed
+ /// Used to track when we should expire pending HTLCs that go unclaimed.
+ incoming_cltv_expiry: u32,
+ /// Optional shared secret for phantom node.
phantom_shared_secret: Option<[u8; 32]>,
/// See [`RecipientOnionFields::custom_tlvs`] for more info.
custom_tlvs: Vec<(u64, Vec<u8>)>,
},
+ /// Incoming keysend (sender provided the preimage in a TLV).
ReceiveKeysend {
/// This was added in 0.0.116 and will break deserialization on downgrades.
payment_data: Option<msgs::FinalOnionHopData>,
+ /// Preimage for this onion payment.
payment_preimage: PaymentPreimage,
+ /// See [`RecipientOnionFields::payment_metadata`] for more info.
payment_metadata: Option<Vec<u8>>,
+ /// CLTV expiry of the incoming HTLC.
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>)>,
},
}
+/// Full details of an incoming HTLC, including routing info.
#[derive(Clone)] // See Channel::revoke_and_ack for why, tl;dr: Rust bug
-pub(super) struct PendingHTLCInfo {
- pub(super) routing: PendingHTLCRouting,
- pub(super) incoming_shared_secret: [u8; 32],
+pub struct PendingHTLCInfo {
+ /// Further routing details based on whether the HTLC is being forwarded or received.
+ pub routing: PendingHTLCRouting,
+ /// Shared secret from the previous hop.
+ pub incoming_shared_secret: [u8; 32],
payment_hash: PaymentHash,
/// Amount received
- pub(super) incoming_amt_msat: Option<u64>, // Added in 0.0.113
+ pub incoming_amt_msat: Option<u64>, // Added in 0.0.113
/// Sender intended amount to forward or receive (actual amount received
/// may overshoot this in either case)
- pub(super) outgoing_amt_msat: u64,
- pub(super) outgoing_cltv_value: u32,
+ pub outgoing_amt_msat: u64,
+ /// Outgoing CLTV height.
+ pub outgoing_cltv_value: u32,
/// The fee being skimmed off the top of this HTLC. If this is a forward, it'll be the fee we are
/// skimming. If we're receiving this HTLC, it's the fee that our counterparty skimmed.
- pub(super) skimmed_fee_msat: Option<u64>,
+ pub skimmed_fee_msat: Option<u64>,
}
#[derive(Clone)] // See Channel::revoke_and_ack for why, tl;dr: Rust bug
}
}
-struct InboundOnionErr {
+/// Invalid inbound onion payment.
+pub struct InboundOnionErr {
err_code: u16,
err_data: Vec<u8>,
msg: &'static str,
}
}
- 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 { .. } | msgs::InboundOnionPayload::BlindedReceive { .. } =>
- 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::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, 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),
- msgs::InboundOnionPayload::BlindedReceive {
- amt_msat, total_msat, outgoing_cltv_value, payment_secret, ..
- } => {
- let payment_data = msgs::FinalOnionHopData { payment_secret, total_msat };
- (Some(payment_data), None, Vec::new(), amt_msat, outgoing_cltv_value, None)
- }
- msgs::InboundOnionPayload::Forward { .. } => {
- 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 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()
- })
- }
- // final_expiry_too_soon
- // We have to have some headroom to broadcast on chain if we have the preimage, so make sure
- // we have at least HTLC_FAIL_BACK_BUFFER blocks to go.
- //
- // Also, ensure that, in the case of an unknown preimage for the received payment hash, our
- // 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 cltv_expiry <= current_height + HTLC_FAIL_BACK_BUFFER + 1 {
- let mut err_data = Vec::with_capacity(12);
- err_data.extend_from_slice(&amt_msat.to_be_bytes());
- err_data.extend_from_slice(¤t_height.to_be_bytes());
- return Err(InboundOnionErr {
- err_code: 0x4000 | 15, err_data,
- msg: "The final CLTV expiry is too soon to handle",
- });
- }
- 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(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 = 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: "Payment preimage didn't match payment hash",
- });
- }
- 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: onion_amt_msat,
- outgoing_cltv_value,
- skimmed_fee_msat: counterparty_skimmed_fee_msat,
- })
- }
-
fn decode_update_add_htlc_onion(
&self, msg: &msgs::UpdateAddHTLC
- ) -> Result<(onion_utils::Hop, [u8; 32], Option<Result<PublicKey, secp256k1::Error>>), HTLCFailureMsg> {
- macro_rules! return_malformed_err {
- ($msg: expr, $err_code: expr) => {
- {
- log_info!(self.logger, "Failed to accept/forward incoming HTLC: {}", $msg);
- return Err(HTLCFailureMsg::Malformed(msgs::UpdateFailMalformedHTLC {
- channel_id: msg.channel_id,
- htlc_id: msg.htlc_id,
- sha256_of_onion: Sha256::hash(&msg.onion_routing_packet.hop_data).into_inner(),
- failure_code: $err_code,
- }));
- }
- }
- }
-
- if let Err(_) = msg.onion_routing_packet.public_key {
- return_malformed_err!("invalid ephemeral pubkey", 0x8000 | 0x4000 | 6);
- }
+ ) -> Result<
+ (onion_utils::Hop, [u8; 32], Option<Result<PublicKey, secp256k1::Error>>), HTLCFailureMsg
+ > {
+ let (next_hop, shared_secret, next_packet_details_opt) = decode_incoming_update_add_htlc_onion(
+ msg, &self.node_signer, &self.logger, &self.secp_ctx
+ )?;
- let shared_secret = self.node_signer.ecdh(
- Recipient::Node, &msg.onion_routing_packet.public_key.unwrap(), None
- ).unwrap().secret_bytes();
-
- if msg.onion_routing_packet.version != 0 {
- //TODO: Spec doesn't indicate if we should only hash hop_data here (and in other
- //sha256_of_onion error data packets), or the entire onion_routing_packet. Either way,
- //the hash doesn't really serve any purpose - in the case of hashing all data, the
- //receiving node would have to brute force to figure out which version was put in the
- //packet by the node that send us the message, in the case of hashing the hop_data, the
- //node knows the HMAC matched, so they already know what is there...
- return_malformed_err!("Unknown onion packet version", 0x8000 | 0x4000 | 4);
- }
macro_rules! return_err {
($msg: expr, $err_code: expr, $data: expr) => {
{
}
}
- let next_hop = match onion_utils::decode_next_payment_hop(
- shared_secret, &msg.onion_routing_packet.hop_data[..], msg.onion_routing_packet.hmac,
- msg.payment_hash, &self.node_signer
- ) {
- Ok(res) => res,
- Err(onion_utils::OnionDecodeErr::Malformed { err_msg, err_code }) => {
- return_malformed_err!(err_msg, err_code);
- },
- Err(onion_utils::OnionDecodeErr::Relay { err_msg, err_code }) => {
- return_err!(err_msg, err_code, &[0; 0]);
- },
- };
- let (outgoing_scid, outgoing_amt_msat, outgoing_cltv_value, next_packet_pk_opt) = match next_hop {
- onion_utils::Hop::Forward {
- next_hop_data: msgs::InboundOnionPayload::Forward {
- short_channel_id, amt_to_forward, outgoing_cltv_value
- }, ..
- } => {
- let next_packet_pk = onion_utils::next_hop_pubkey(&self.secp_ctx,
- msg.onion_routing_packet.public_key.unwrap(), &shared_secret);
- (short_channel_id, amt_to_forward, outgoing_cltv_value, Some(next_packet_pk))
- },
- // 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::InboundOnionPayload::Receive { .. }, .. } |
- onion_utils::Hop::Forward { next_hop_data: msgs::InboundOnionPayload::BlindedReceive { .. }, .. } =>
- {
- return_err!("Final Node OnionHopData provided for us as an intermediary node", 0x4000 | 22, &[0; 0]);
- }
+ let NextPacketDetails {
+ next_packet_pubkey, outgoing_amt_msat, outgoing_scid, outgoing_cltv_value
+ } = match next_packet_details_opt {
+ Some(next_packet_details) => next_packet_details,
+ // it is a receive, so no need for outbound checks
+ None => return Ok((next_hop, shared_secret, None)),
};
// Perform outbound checks here instead of in [`Self::construct_pending_htlc_info`] because we
}
chan_update_opt
} else {
- if (msg.cltv_expiry as u64) < (outgoing_cltv_value) as u64 + MIN_CLTV_EXPIRY_DELTA as u64 {
- // We really should set `incorrect_cltv_expiry` here but as we're not
- // forwarding over a real channel we can't generate a channel_update
- // for it. Instead we just return a generic temporary_node_failure.
- break Some((
- "Forwarding node has tampered with the intended HTLC values or origin node has an obsolete cltv_expiry_delta",
- 0x2000 | 2, None,
- ));
- }
None
};
let cur_height = self.best_block.read().unwrap().height() + 1;
- // Theoretically, channel counterparty shouldn't send us a HTLC expiring now,
- // but we want to be robust wrt to counterparty packet sanitization (see
- // HTLC_FAIL_BACK_BUFFER rationale).
- if msg.cltv_expiry <= cur_height + HTLC_FAIL_BACK_BUFFER as u32 { // expiry_too_soon
- break Some(("CLTV expiry is too close", 0x1000 | 14, chan_update_opt));
- }
- if msg.cltv_expiry > cur_height + CLTV_FAR_FAR_AWAY as u32 { // expiry_too_far
- break Some(("CLTV expiry is too far in the future", 21, None));
- }
- // If the HTLC expires ~now, don't bother trying to forward it to our
- // counterparty. They should fail it anyway, but we don't want to bother with
- // the round-trips or risk them deciding they definitely want the HTLC and
- // force-closing to ensure they get it if we're offline.
- // We previously had a much more aggressive check here which tried to ensure
- // our counterparty receives an HTLC which has *our* risk threshold met on it,
- // but there is no need to do that, and since we're a bit conservative with our
- // risk threshold it just results in failing to forward payments.
- if (outgoing_cltv_value) as u64 <= (cur_height + LATENCY_GRACE_PERIOD_BLOCKS) as u64 {
- break Some(("Outgoing CLTV value is too soon", 0x1000 | 14, chan_update_opt));
+
+ if let Err((err_msg, code)) = check_incoming_htlc_cltv(
+ cur_height, outgoing_cltv_value, msg.cltv_expiry
+ ) {
+ if code & 0x1000 != 0 && chan_update_opt.is_none() {
+ // We really should set `incorrect_cltv_expiry` here but as we're not
+ // forwarding over a real channel we can't generate a channel_update
+ // for it. Instead we just return a generic temporary_node_failure.
+ break Some((err_msg, 0x2000 | 2, None))
+ }
+ let chan_update_opt = if code & 0x1000 != 0 { chan_update_opt } else { None };
+ break Some((err_msg, code, chan_update_opt));
}
break None;
}
return_err!(err, code, &res.0[..]);
}
- Ok((next_hop, shared_secret, next_packet_pk_opt))
+ Ok((next_hop, shared_secret, Some(next_packet_pubkey)))
}
fn construct_pending_htlc_status<'a>(
match decoded_hop {
onion_utils::Hop::Receive(next_hop_data) => {
// OUR PAYMENT!
- match self.construct_recv_pending_htlc_info(next_hop_data, shared_secret, msg.payment_hash,
- msg.amount_msat, msg.cltv_expiry, None, allow_underpay, msg.skimmed_fee_msat)
+ let current_height: u32 = self.best_block.read().unwrap().height();
+ match create_recv_pending_htlc_info(next_hop_data, shared_secret, msg.payment_hash,
+ msg.amount_msat, msg.cltv_expiry, None, allow_underpay, msg.skimmed_fee_msat,
+ current_height, self.default_configuration.accept_mpp_keysend)
{
Ok(info) => {
// Note that we could obviously respond immediately with an update_fulfill_htlc
}
},
onion_utils::Hop::Forward { next_hop_data, next_hop_hmac, new_packet_bytes } => {
- match self.construct_fwd_pending_htlc_info(msg, next_hop_data, next_hop_hmac,
+ match create_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)
};
match next_hop {
onion_utils::Hop::Receive(hop_data) => {
- match self.construct_recv_pending_htlc_info(hop_data,
+ let current_height: u32 = self.best_block.read().unwrap().height();
+ match create_recv_pending_htlc_info(hop_data,
incoming_shared_secret, payment_hash, outgoing_amt_msat,
- outgoing_cltv_value, Some(phantom_shared_secret), false, None)
+ outgoing_cltv_value, Some(phantom_shared_secret), false, None,
+ current_height, self.default_configuration.accept_mpp_keysend)
{
Ok(info) => phantom_receives.push((prev_short_channel_id, prev_funding_outpoint, prev_user_channel_id, vec![(info, prev_htlc_id)])),
Err(InboundOnionErr { err_code, err_data, msg }) => failed_payment!(msg, err_code, err_data, Some(phantom_shared_secret))
}
}
+fn create_fwd_pending_htlc_info(
+ 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 { .. } | msgs::InboundOnionPayload::BlindedReceive { .. } =>
+ 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 create_recv_pending_htlc_info(
+ 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>, current_height: u32, accept_mpp_keysend: bool,
+) -> 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),
+ msgs::InboundOnionPayload::BlindedReceive {
+ amt_msat, total_msat, outgoing_cltv_value, payment_secret, ..
+ } => {
+ let payment_data = msgs::FinalOnionHopData { payment_secret, total_msat };
+ (Some(payment_data), None, Vec::new(), amt_msat, outgoing_cltv_value, None)
+ }
+ msgs::InboundOnionPayload::Forward { .. } => {
+ 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 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()
+ })
+ }
+ // final_expiry_too_soon
+ // We have to have some headroom to broadcast on chain if we have the preimage, so make sure
+ // we have at least HTLC_FAIL_BACK_BUFFER blocks to go.
+ //
+ // Also, ensure that, in the case of an unknown preimage for the received payment hash, our
+ // payment logic has enough time to fail the HTLC backward before our onchain logic triggers a
+ // channel closure (see HTLC_FAIL_BACK_BUFFER rationale).
+ if cltv_expiry <= current_height + HTLC_FAIL_BACK_BUFFER + 1 {
+ let mut err_data = Vec::with_capacity(12);
+ err_data.extend_from_slice(&amt_msat.to_be_bytes());
+ err_data.extend_from_slice(¤t_height.to_be_bytes());
+ return Err(InboundOnionErr {
+ err_code: 0x4000 | 15, err_data,
+ msg: "The final CLTV expiry is too soon to handle",
+ });
+ }
+ 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(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 = 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: "Payment preimage didn't match payment hash",
+ });
+ }
+ if !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: onion_amt_msat,
+ outgoing_cltv_value,
+ skimmed_fee_msat: counterparty_skimmed_fee_msat,
+ })
+}
+
+/// Peel one layer off an incoming onion, returning [`PendingHTLCInfo`] (either Forward or Receive).
+/// This does all the relevant context-free checks that LDK requires for payment relay or
+/// acceptance. If the payment is to be received, and the amount matches the expected amount for
+/// a given invoice, this indicates the [`msgs::UpdateAddHTLC`], once fully committed in the
+/// channel, will generate an [`Event::PaymentClaimable`].
+pub fn peel_payment_onion<NS: Deref, L: Deref, T: secp256k1::Verification>(
+ msg: &msgs::UpdateAddHTLC, node_signer: &NS, logger: &L, secp_ctx: &Secp256k1<T>,
+ cur_height: u32, accept_mpp_keysend: bool,
+) -> Result<PendingHTLCInfo, InboundOnionErr>
+where
+ NS::Target: NodeSigner,
+ L::Target: Logger,
+{
+ let (hop, shared_secret, next_packet_details_opt) =
+ decode_incoming_update_add_htlc_onion(msg, node_signer, logger, secp_ctx
+ ).map_err(|e| {
+ let (err_code, err_data) = match e {
+ HTLCFailureMsg::Malformed(m) => (m.failure_code, Vec::new()),
+ HTLCFailureMsg::Relay(r) => (0x4000 | 22, r.reason.data),
+ };
+ let msg = "Failed to decode update add htlc onion";
+ InboundOnionErr { msg, err_code, err_data }
+ })?;
+ Ok(match hop {
+ onion_utils::Hop::Forward { next_hop_data, next_hop_hmac, new_packet_bytes } => {
+ let NextPacketDetails {
+ next_packet_pubkey, outgoing_amt_msat: _, outgoing_scid: _, outgoing_cltv_value
+ } = match next_packet_details_opt {
+ Some(next_packet_details) => next_packet_details,
+ // Forward should always include the next hop details
+ None => return Err(InboundOnionErr {
+ msg: "Failed to decode update add htlc onion",
+ err_code: 0x4000 | 22,
+ err_data: Vec::new(),
+ }),
+ };
+
+ if let Err((err_msg, code)) = check_incoming_htlc_cltv(
+ cur_height, outgoing_cltv_value, msg.cltv_expiry
+ ) {
+ return Err(InboundOnionErr {
+ msg: err_msg,
+ err_code: code,
+ err_data: Vec::new(),
+ });
+ }
+ create_fwd_pending_htlc_info(
+ msg, next_hop_data, next_hop_hmac, new_packet_bytes, shared_secret,
+ Some(next_packet_pubkey)
+ )?
+ },
+ onion_utils::Hop::Receive(received_data) => {
+ create_recv_pending_htlc_info(
+ received_data, shared_secret, msg.payment_hash, msg.amount_msat, msg.cltv_expiry,
+ None, false, msg.skimmed_fee_msat, cur_height, accept_mpp_keysend,
+ )?
+ }
+ })
+}
+
+struct NextPacketDetails {
+ next_packet_pubkey: Result<PublicKey, secp256k1::Error>,
+ outgoing_scid: u64,
+ outgoing_amt_msat: u64,
+ outgoing_cltv_value: u32,
+}
+
+fn decode_incoming_update_add_htlc_onion<NS: Deref, L: Deref, T: secp256k1::Verification>(
+ msg: &msgs::UpdateAddHTLC, node_signer: &NS, logger: &L, secp_ctx: &Secp256k1<T>,
+) -> Result<(onion_utils::Hop, [u8; 32], Option<NextPacketDetails>), HTLCFailureMsg>
+where
+ NS::Target: NodeSigner,
+ L::Target: Logger,
+{
+ macro_rules! return_malformed_err {
+ ($msg: expr, $err_code: expr) => {
+ {
+ log_info!(logger, "Failed to accept/forward incoming HTLC: {}", $msg);
+ return Err(HTLCFailureMsg::Malformed(msgs::UpdateFailMalformedHTLC {
+ channel_id: msg.channel_id,
+ htlc_id: msg.htlc_id,
+ sha256_of_onion: Sha256::hash(&msg.onion_routing_packet.hop_data).into_inner(),
+ failure_code: $err_code,
+ }));
+ }
+ }
+ }
+
+ if let Err(_) = msg.onion_routing_packet.public_key {
+ return_malformed_err!("invalid ephemeral pubkey", 0x8000 | 0x4000 | 6);
+ }
+
+ let shared_secret = node_signer.ecdh(
+ Recipient::Node, &msg.onion_routing_packet.public_key.unwrap(), None
+ ).unwrap().secret_bytes();
+
+ if msg.onion_routing_packet.version != 0 {
+ //TODO: Spec doesn't indicate if we should only hash hop_data here (and in other
+ //sha256_of_onion error data packets), or the entire onion_routing_packet. Either way,
+ //the hash doesn't really serve any purpose - in the case of hashing all data, the
+ //receiving node would have to brute force to figure out which version was put in the
+ //packet by the node that send us the message, in the case of hashing the hop_data, the
+ //node knows the HMAC matched, so they already know what is there...
+ return_malformed_err!("Unknown onion packet version", 0x8000 | 0x4000 | 4);
+ }
+ macro_rules! return_err {
+ ($msg: expr, $err_code: expr, $data: expr) => {
+ {
+ log_info!(logger, "Failed to accept/forward incoming HTLC: {}", $msg);
+ return Err(HTLCFailureMsg::Relay(msgs::UpdateFailHTLC {
+ channel_id: msg.channel_id,
+ htlc_id: msg.htlc_id,
+ reason: HTLCFailReason::reason($err_code, $data.to_vec())
+ .get_encrypted_failure_packet(&shared_secret, &None),
+ }));
+ }
+ }
+ }
+
+ let next_hop = match onion_utils::decode_next_payment_hop(
+ shared_secret, &msg.onion_routing_packet.hop_data[..], msg.onion_routing_packet.hmac,
+ msg.payment_hash, node_signer
+ ) {
+ Ok(res) => res,
+ Err(onion_utils::OnionDecodeErr::Malformed { err_msg, err_code }) => {
+ return_malformed_err!(err_msg, err_code);
+ },
+ Err(onion_utils::OnionDecodeErr::Relay { err_msg, err_code }) => {
+ return_err!(err_msg, err_code, &[0; 0]);
+ },
+ };
+
+ let next_packet_details = match next_hop {
+ onion_utils::Hop::Forward {
+ next_hop_data: msgs::InboundOnionPayload::Forward {
+ short_channel_id, amt_to_forward, outgoing_cltv_value
+ }, ..
+ } => {
+ let next_packet_pubkey = onion_utils::next_hop_pubkey(secp_ctx,
+ msg.onion_routing_packet.public_key.unwrap(), &shared_secret);
+ NextPacketDetails {
+ next_packet_pubkey, outgoing_scid: short_channel_id,
+ outgoing_amt_msat: amt_to_forward, outgoing_cltv_value
+ }
+ },
+ // 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::InboundOnionPayload::Receive { .. }, .. } |
+ onion_utils::Hop::Forward { next_hop_data: msgs::InboundOnionPayload::BlindedReceive { .. }, .. } =>
+ {
+ return_err!("Final Node OnionHopData provided for us as an intermediary node", 0x4000 | 22, &[0; 0]);
+ }
+ };
+
+ Ok((next_hop, shared_secret, Some(next_packet_details)))
+}
+
+fn check_incoming_htlc_cltv(
+ cur_height: u32, outgoing_cltv_value: u32, cltv_expiry: u32
+) -> Result<(), (&'static str, u16)> {
+ if (cltv_expiry as u64) < (outgoing_cltv_value) as u64 + MIN_CLTV_EXPIRY_DELTA as u64 {
+ return Err((
+ "Forwarding node has tampered with the intended HTLC values or origin node has an obsolete cltv_expiry_delta",
+ 0x1000 | 13, // incorrect_cltv_expiry
+ ));
+ }
+ // Theoretically, channel counterparty shouldn't send us a HTLC expiring now,
+ // but we want to be robust wrt to counterparty packet sanitization (see
+ // HTLC_FAIL_BACK_BUFFER rationale).
+ if cltv_expiry <= cur_height + HTLC_FAIL_BACK_BUFFER as u32 { // expiry_too_soon
+ return Err(("CLTV expiry is too close", 0x1000 | 14));
+ }
+ if cltv_expiry > cur_height + CLTV_FAR_FAR_AWAY as u32 { // expiry_too_far
+ return Err(("CLTV expiry is too far in the future", 21));
+ }
+ // If the HTLC expires ~now, don't bother trying to forward it to our
+ // counterparty. They should fail it anyway, but we don't want to bother with
+ // the round-trips or risk them deciding they definitely want the HTLC and
+ // force-closing to ensure they get it if we're offline.
+ // We previously had a much more aggressive check here which tried to ensure
+ // our counterparty receives an HTLC which has *our* risk threshold met on it,
+ // but there is no need to do that, and since we're a bit conservative with our
+ // risk threshold it just results in failing to forward payments.
+ if (outgoing_cltv_value) as u64 <= (cur_height + LATENCY_GRACE_PERIOD_BLOCKS) as u64 {
+ return Err(("Outgoing CLTV value is too soon", 0x1000 | 14));
+ }
+
+ Ok(())
+}
+
impl<M: Deref, T: Deref, ES: Deref, NS: Deref, SP: Deref, F: Deref, R: Deref, L: Deref> MessageSendEventsProvider for ChannelManager<M, T, ES, NS, SP, F, R, L>
where
M::Target: chain::Watch<<SP::Target as SignerProvider>::Signer>,
use crate::events::{Event, HTLCDestination, MessageSendEvent, MessageSendEventsProvider, ClosureReason};
use crate::ln::{PaymentPreimage, PaymentHash, PaymentSecret};
use crate::ln::ChannelId;
- use crate::ln::channelmanager::{inbound_payment, PaymentId, PaymentSendFailure, RecipientOnionFields, InterceptId};
+ use crate::ln::channelmanager::{create_recv_pending_htlc_info, inbound_payment, PaymentId, PaymentSendFailure, RecipientOnionFields, InterceptId};
+ use crate::ln::features::{ChannelFeatures, NodeFeatures};
use crate::ln::functional_test_utils::*;
use crate::ln::msgs::{self, ErrorAction};
use crate::ln::msgs::ChannelMessageHandler;
- use crate::routing::router::{PaymentParameters, RouteParameters, find_route};
+ use crate::routing::router::{Path, PaymentParameters, RouteHop, RouteParameters, find_route};
use crate::util::errors::APIError;
use crate::util::test_utils;
use crate::util::config::{ChannelConfig, ChannelConfigUpdate};
};
// Check that if the amount we received + the penultimate hop extra fee is less than the sender
// intended amount, we fail the payment.
+ let current_height: u32 = node[0].node.best_block.read().unwrap().height();
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))
+ create_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),
+ current_height, node[0].node.default_configuration.accept_mpp_keysend)
{
assert_eq!(err_code, 19);
} else { panic!(); }
}),
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());
+ let current_height: u32 = node[0].node.best_block.read().unwrap().height();
+ assert!(create_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),
+ current_height, node[0].node.default_configuration.accept_mpp_keysend).is_ok());
}
#[test]
let node_chanmgr = create_node_chanmgrs(1, &node_cfg, &[None]);
let node = create_network(1, &node_cfg, &node_chanmgr);
- let result = node[0].node.construct_recv_pending_htlc_info(msgs::InboundOnionPayload::Receive {
+ let current_height: u32 = node[0].node.best_block.read().unwrap().height();
+ let result = create_recv_pending_htlc_info(msgs::InboundOnionPayload::Receive {
amt_msat: 100,
outgoing_cltv_value: 22,
payment_metadata: None,
payment_secret: PaymentSecret([0; 32]), total_msat: 100,
}),
custom_tlvs: Vec::new(),
- }, [0; 32], PaymentHash([0; 32]), 100, 23, None, true, None);
+ }, [0; 32], PaymentHash([0; 32]), 100, 23, None, true, None, current_height,
+ node[0].node.default_configuration.accept_mpp_keysend);
// Should not return an error as this condition:
// https://github.com/lightning/bolts/blob/4dcc377209509b13cf89a4b91fde7d478f5b46d8/04-onion-routing.md?plain=1#L334
check_spends!(txn[0], funding_tx);
}
}
+
+ #[test]
+ fn test_peel_payment_onion() {
+ use super::*;
+ let secp_ctx = Secp256k1::new();
+
+ let bob = crate::sign::KeysManager::new(&[2; 32], 42, 42);
+ let bob_pk = PublicKey::from_secret_key(&secp_ctx, &bob.get_node_secret_key());
+ let charlie = crate::sign::KeysManager::new(&[3; 32], 42, 42);
+ let charlie_pk = PublicKey::from_secret_key(&secp_ctx, &charlie.get_node_secret_key());
+
+ let (session_priv, total_amt_msat, cur_height, recipient_onion, preimage, payment_hash,
+ prng_seed, hops, recipient_amount, pay_secret) = payment_onion_args(bob_pk, charlie_pk);
+
+ let path = Path {
+ hops: hops,
+ blinded_tail: None,
+ };
+
+ let (amount_msat, cltv_expiry, onion) = create_payment_onion(
+ &secp_ctx, &path, &session_priv, total_amt_msat, recipient_onion, cur_height,
+ payment_hash, Some(preimage), prng_seed
+ ).unwrap();
+
+ let msg = make_update_add_msg(amount_msat, cltv_expiry, payment_hash, onion);
+ let logger = test_utils::TestLogger::with_id("bob".to_string());
+
+ let peeled = peel_payment_onion(&msg, &&bob, &&logger, &secp_ctx, cur_height, true)
+ .map_err(|e| e.msg).unwrap();
+
+ let next_onion = match peeled.routing {
+ PendingHTLCRouting::Forward { onion_packet, short_channel_id: _ } => {
+ onion_packet
+ },
+ _ => panic!("expected a forwarded onion"),
+ };
+
+ let msg2 = make_update_add_msg(amount_msat, cltv_expiry, payment_hash, next_onion);
+ let peeled2 = peel_payment_onion(&msg2, &&charlie, &&logger, &secp_ctx, cur_height, true)
+ .map_err(|e| e.msg).unwrap();
+
+ match peeled2.routing {
+ PendingHTLCRouting::ReceiveKeysend { payment_preimage, payment_data, incoming_cltv_expiry, .. } => {
+ assert_eq!(payment_preimage, preimage);
+ assert_eq!(peeled2.outgoing_amt_msat, recipient_amount);
+ assert_eq!(incoming_cltv_expiry, peeled2.outgoing_cltv_value);
+ let msgs::FinalOnionHopData{total_msat, payment_secret} = payment_data.unwrap();
+ assert_eq!(total_msat, total_amt_msat);
+ assert_eq!(payment_secret, pay_secret);
+ },
+ _ => panic!("expected a received keysend"),
+ };
+ }
+
+ fn make_update_add_msg(
+ amount_msat: u64, cltv_expiry: u32, payment_hash: PaymentHash,
+ onion_routing_packet: msgs::OnionPacket
+ ) -> msgs::UpdateAddHTLC {
+ msgs::UpdateAddHTLC {
+ channel_id: ChannelId::from_bytes([0; 32]),
+ htlc_id: 0,
+ amount_msat,
+ cltv_expiry,
+ payment_hash,
+ onion_routing_packet,
+ skimmed_fee_msat: None,
+ }
+ }
+
+ fn payment_onion_args(hop_pk: PublicKey, recipient_pk: PublicKey) -> (
+ SecretKey, u64, u32, RecipientOnionFields, PaymentPreimage, PaymentHash, [u8; 32],
+ Vec<RouteHop>, u64, PaymentSecret,
+ ) {
+ let session_priv_bytes = [42; 32];
+ let session_priv = SecretKey::from_slice(&session_priv_bytes).unwrap();
+ let total_amt_msat = 1000;
+ let cur_height = 1000;
+ let pay_secret = PaymentSecret([99; 32]);
+ let recipient_onion = RecipientOnionFields::secret_only(pay_secret);
+ let preimage_bytes = [43; 32];
+ let preimage = PaymentPreimage(preimage_bytes);
+ let rhash_bytes = Sha256::hash(&preimage_bytes).into_inner();
+ let payment_hash = PaymentHash(rhash_bytes);
+ let prng_seed = [44; 32];
+
+ // make a route alice -> bob -> charlie
+ let hop_fee = 1;
+ let recipient_amount = total_amt_msat - hop_fee;
+ let hops = vec![
+ RouteHop {
+ pubkey: hop_pk,
+ fee_msat: hop_fee,
+ cltv_expiry_delta: 42,
+ short_channel_id: 1,
+ node_features: NodeFeatures::empty(),
+ channel_features: ChannelFeatures::empty(),
+ maybe_announced_channel: false,
+ },
+ RouteHop {
+ pubkey: recipient_pk,
+ fee_msat: recipient_amount,
+ cltv_expiry_delta: 42,
+ short_channel_id: 2,
+ node_features: NodeFeatures::empty(),
+ channel_features: ChannelFeatures::empty(),
+ maybe_announced_channel: false,
+ }
+ ];
+
+ (session_priv, total_amt_msat, cur_height, recipient_onion, preimage, payment_hash,
+ prng_seed, hops, recipient_amount, pay_secret)
+ }
+
+ pub fn create_payment_onion<T: bitcoin::secp256k1::Signing>(
+ secp_ctx: &Secp256k1<T>, path: &Path, session_priv: &SecretKey, total_msat: u64,
+ recipient_onion: RecipientOnionFields, best_block_height: u32, payment_hash: PaymentHash,
+ keysend_preimage: Option<PaymentPreimage>, prng_seed: [u8; 32]
+ ) -> Result<(u64, u32, msgs::OnionPacket), ()> {
+ let onion_keys = super::onion_utils::construct_onion_keys(&secp_ctx, &path, &session_priv).map_err(|_| ())?;
+ let (onion_payloads, htlc_msat, htlc_cltv) = super::onion_utils::build_onion_payloads(
+ &path,
+ total_msat,
+ recipient_onion,
+ best_block_height + 1,
+ &keysend_preimage,
+ ).map_err(|_| ())?;
+ let onion_packet = super::onion_utils::construct_onion_packet(
+ onion_payloads, onion_keys, prng_seed, &payment_hash
+ )?;
+ Ok((htlc_msat, htlc_cltv, onion_packet))
+ }
}
#[cfg(ldk_bench)]