use bitcoin::hashes::Hash as TraitImport;
use bitcoin::WPubkeyHash;
-use lightning::blinded_path::message::MessageContext;
+use lightning::blinded_path::message::{BlindedMessagePath, MessageContext};
use lightning::blinded_path::payment::{BlindedPaymentPath, ReceiveTlvs};
-use lightning::blinded_path::BlindedPath;
use lightning::chain;
use lightning::chain::chaininterface::{BroadcasterInterface, ConfirmationTarget, FeeEstimator};
use lightning::chain::channelmonitor::{ChannelMonitor, MonitorEvent};
fn create_blinded_paths<T: secp256k1::Signing + secp256k1::Verification>(
&self, _recipient: PublicKey, _context: MessageContext, _peers: Vec<PublicKey>,
_secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
unreachable!()
}
}
use bitcoin::hex::FromHex;
use bitcoin::WPubkeyHash;
-use lightning::blinded_path::message::MessageContext;
+use lightning::blinded_path::message::{BlindedMessagePath, MessageContext};
use lightning::blinded_path::payment::{BlindedPaymentPath, ReceiveTlvs};
-use lightning::blinded_path::BlindedPath;
use lightning::chain;
use lightning::chain::chaininterface::{BroadcasterInterface, ConfirmationTarget, FeeEstimator};
use lightning::chain::chainmonitor;
fn create_blinded_paths<T: secp256k1::Signing + secp256k1::Verification>(
&self, _recipient: PublicKey, _context: MessageContext, _peers: Vec<PublicKey>,
_secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
unreachable!()
}
}
use bitcoin::secp256k1::schnorr;
use bitcoin::secp256k1::{self, PublicKey, Scalar, Secp256k1, SecretKey};
-use lightning::blinded_path::message::{MessageContext, OffersContext};
-use lightning::blinded_path::{BlindedPath, EmptyNodeIdLookUp};
+use lightning::blinded_path::message::{BlindedMessagePath, MessageContext, OffersContext};
+use lightning::blinded_path::EmptyNodeIdLookUp;
use lightning::ln::features::InitFeatures;
use lightning::ln::msgs::{self, DecodeError, OnionMessageHandler};
use lightning::ln::script::ShutdownScript;
fn create_blinded_paths<T: secp256k1::Signing + secp256k1::Verification>(
&self, _recipient: PublicKey, _context: MessageContext, _peers: Vec<PublicKey>,
_secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
unreachable!()
}
}
// You may not use this file except in accordance with one or both of these
// licenses.
-//! Data structures and methods for constructing [`BlindedPath`]s to send a message over.
-//!
-//! [`BlindedPath`]: crate::blinded_path::BlindedPath
+//! Data structures and methods for constructing [`BlindedMessagePath`]s to send a message over.
use bitcoin::secp256k1::{self, PublicKey, Secp256k1, SecretKey};
use crate::io;
use crate::io::Cursor;
use crate::ln::channelmanager::PaymentId;
+use crate::ln::msgs::DecodeError;
use crate::ln::{PaymentHash, onion_utils};
use crate::offers::nonce::Nonce;
use crate::onion_message::packet::ControlTlvs;
-use crate::sign::{NodeSigner, Recipient};
+use crate::sign::{EntropySource, NodeSigner, Recipient};
use crate::crypto::streams::ChaChaPolyReadAdapter;
-use crate::util::ser::{FixedLengthReader, LengthReadableArgs, Writeable, Writer};
+use crate::util::ser::{FixedLengthReader, LengthReadableArgs, Readable, Writeable, Writer};
use core::mem;
use core::ops::Deref;
+/// A [`BlindedPath`] to be used for sending or receiving a message, hiding the identity of the
+/// recipient.
+#[derive(Clone, Debug, Hash, PartialEq, Eq)]
+pub struct BlindedMessagePath(pub BlindedPath);
+
+impl Writeable for BlindedMessagePath {
+ fn write<W: Writer>(&self, w: &mut W) -> Result<(), io::Error> {
+ self.0.write(w)
+ }
+}
+
+impl Readable for BlindedMessagePath {
+ fn read<R: io::Read>(r: &mut R) -> Result<Self, DecodeError> {
+ Ok(Self(BlindedPath::read(r)?))
+ }
+}
+
+impl BlindedMessagePath {
+ /// Create a one-hop blinded path for a message.
+ pub fn one_hop<ES: Deref, T: secp256k1::Signing + secp256k1::Verification>(
+ recipient_node_id: PublicKey, context: MessageContext, entropy_source: ES, secp_ctx: &Secp256k1<T>
+ ) -> Result<Self, ()> where ES::Target: EntropySource {
+ Self::new(&[], recipient_node_id, context, entropy_source, secp_ctx)
+ }
+
+ /// Create a path for an onion message, to be forwarded along `node_pks`. The last node
+ /// pubkey in `node_pks` will be the destination node.
+ ///
+ /// Errors if no hops are provided or if `node_pk`(s) are invalid.
+ // TODO: make all payloads the same size with padding + add dummy hops
+ pub fn new<ES: Deref, T: secp256k1::Signing + secp256k1::Verification>(
+ intermediate_nodes: &[ForwardNode], recipient_node_id: PublicKey, context: MessageContext,
+ entropy_source: ES, secp_ctx: &Secp256k1<T>
+ ) -> Result<Self, ()> where ES::Target: EntropySource {
+ let introduction_node = IntroductionNode::NodeId(
+ intermediate_nodes.first().map_or(recipient_node_id, |n| n.node_id)
+ );
+ let blinding_secret_bytes = entropy_source.get_secure_random_bytes();
+ let blinding_secret = SecretKey::from_slice(&blinding_secret_bytes[..]).expect("RNG is busted");
+
+ Ok(Self(BlindedPath {
+ introduction_node,
+ blinding_point: PublicKey::from_secret_key(secp_ctx, &blinding_secret),
+ blinded_hops: blinded_hops(
+ secp_ctx, intermediate_nodes, recipient_node_id,
+ context, &blinding_secret,
+ ).map_err(|_| ())?,
+ }))
+ }
+}
+
/// An intermediate node, and possibly a short channel id leading to the next node.
#[derive(Clone, Copy, Debug, Hash, PartialEq, Eq)]
pub struct ForwardNode {
}
}
-/// Additional data included by the recipient in a [`BlindedPath`].
+/// Additional data included by the recipient in a [`BlindedMessagePath`].
///
/// This data is encrypted by the recipient and will be given to the corresponding message handler
-/// when handling a message sent over the [`BlindedPath`]. The recipient can use this data to
+/// when handling a message sent over the [`BlindedMessagePath`]. The recipient can use this data to
/// authenticate the message or for further processing if needed.
#[derive(Clone, Debug)]
pub enum MessageContext {
/// [`OffersMessage`]: crate::onion_message::offers::OffersMessage
#[derive(Clone, Debug, Eq, PartialEq)]
pub enum OffersContext {
- /// Context used by a [`BlindedPath`] within an [`Offer`].
+ /// Context used by a [`BlindedMessagePath`] within an [`Offer`].
///
/// This variant is intended to be received when handling an [`InvoiceRequest`].
///
/// [`Offer`]: crate::offers::offer::Offer
nonce: Nonce,
},
- /// Context used by a [`BlindedPath`] within a [`Refund`] or as a reply path for an
+ /// Context used by a [`BlindedMessagePath`] within a [`Refund`] or as a reply path for an
/// [`InvoiceRequest`].
///
/// This variant is intended to be received when handling a [`Bolt12Invoice`] or an
/// [`InvoiceError`]: crate::offers::invoice_error::InvoiceError
hmac: Option<Hmac<Sha256>>,
},
- /// Context used by a [`BlindedPath`] as a reply path for a [`Bolt12Invoice`].
+ /// Context used by a [`BlindedMessagePath`] as a reply path for a [`Bolt12Invoice`].
///
/// This variant is intended to be received when handling an [`InvoiceError`].
///
//
// Will only modify `path` when returning `Ok`.
pub(crate) fn advance_path_by_one<NS: Deref, NL: Deref, T>(
- path: &mut BlindedPath, node_signer: &NS, node_id_lookup: &NL, secp_ctx: &Secp256k1<T>
+ path: &mut BlindedMessagePath, node_signer: &NS, node_id_lookup: &NL, secp_ctx: &Secp256k1<T>
) -> Result<(), ()>
where
NS::Target: NodeSigner,
NL::Target: NodeIdLookUp,
T: secp256k1::Signing + secp256k1::Verification,
{
- let control_tlvs_ss = node_signer.ecdh(Recipient::Node, &path.blinding_point, None)?;
+ let control_tlvs_ss = node_signer.ecdh(Recipient::Node, &path.0.blinding_point, None)?;
let rho = onion_utils::gen_rho_from_shared_secret(&control_tlvs_ss.secret_bytes());
- let encrypted_control_tlvs = &path.blinded_hops.get(0).ok_or(())?.encrypted_payload;
+ let encrypted_control_tlvs = &path.0.blinded_hops.get(0).ok_or(())?.encrypted_payload;
let mut s = Cursor::new(encrypted_control_tlvs);
let mut reader = FixedLengthReader::new(&mut s, encrypted_control_tlvs.len() as u64);
match ChaChaPolyReadAdapter::read(&mut reader, rho) {
let mut new_blinding_point = match next_blinding_override {
Some(blinding_point) => blinding_point,
None => {
- onion_utils::next_hop_pubkey(secp_ctx, path.blinding_point,
+ onion_utils::next_hop_pubkey(secp_ctx, path.0.blinding_point,
control_tlvs_ss.as_ref()).map_err(|_| ())?
}
};
- mem::swap(&mut path.blinding_point, &mut new_blinding_point);
- path.introduction_node = IntroductionNode::NodeId(next_node_id);
- path.blinded_hops.remove(0);
+ mem::swap(&mut path.0.blinding_point, &mut new_blinding_point);
+ path.0.introduction_node = IntroductionNode::NodeId(next_node_id);
+ path.0.blinded_hops.remove(0);
Ok(())
},
_ => Err(())
pub mod message;
pub(crate) mod utils;
-use bitcoin::secp256k1::{self, PublicKey, Secp256k1, SecretKey};
-use message::MessageContext;
+use bitcoin::secp256k1::PublicKey;
use core::ops::Deref;
use crate::ln::msgs::DecodeError;
use crate::routing::gossip::{NodeId, ReadOnlyNetworkGraph};
-use crate::sign::EntropySource;
use crate::util::ser::{Readable, Writeable, Writer};
use crate::util::scid_utils;
}
impl BlindedPath {
- /// Create a one-hop blinded path for a message.
- pub fn one_hop_for_message<ES: Deref, T: secp256k1::Signing + secp256k1::Verification>(
- recipient_node_id: PublicKey, context: MessageContext, entropy_source: ES, secp_ctx: &Secp256k1<T>
- ) -> Result<Self, ()> where ES::Target: EntropySource {
- Self::new_for_message(&[], recipient_node_id, context, entropy_source, secp_ctx)
- }
-
- /// Create a blinded path for an onion message, to be forwarded along `node_pks`. The last node
- /// pubkey in `node_pks` will be the destination node.
- ///
- /// Errors if no hops are provided or if `node_pk`(s) are invalid.
- // TODO: make all payloads the same size with padding + add dummy hops
- pub fn new_for_message<ES: Deref, T: secp256k1::Signing + secp256k1::Verification>(
- intermediate_nodes: &[message::ForwardNode], recipient_node_id: PublicKey,
- context: MessageContext, entropy_source: ES, secp_ctx: &Secp256k1<T>
- ) -> Result<Self, ()> where ES::Target: EntropySource {
- let introduction_node = IntroductionNode::NodeId(
- intermediate_nodes.first().map_or(recipient_node_id, |n| n.node_id)
- );
- let blinding_secret_bytes = entropy_source.get_secure_random_bytes();
- let blinding_secret = SecretKey::from_slice(&blinding_secret_bytes[..]).expect("RNG is busted");
-
- Ok(BlindedPath {
- introduction_node,
- blinding_point: PublicKey::from_secret_key(secp_ctx, &blinding_secret),
- blinded_hops: message::blinded_hops(
- secp_ctx, intermediate_nodes, recipient_node_id,
- context, &blinding_secret,
- ).map_err(|_| ())?,
- })
- }
-
/// Returns the introduction [`NodeId`] of the blinded path, if it is publicly reachable (i.e.,
/// it is found in the network graph).
pub fn public_introduction_node_id<'a>(
use bitcoin::secp256k1::ecdh::SharedSecret;
use super::{BlindedHop, BlindedPath};
+use super::message::BlindedMessagePath;
use crate::ln::msgs::DecodeError;
use crate::ln::onion_utils;
use crate::onion_message::messenger::Destination;
Destination::Node(pk) => {
build_keys!(pk, false, None);
},
- Destination::BlindedPath(BlindedPath { blinded_hops, .. }) => {
+ Destination::BlindedPath(BlindedMessagePath(BlindedPath { blinded_hops, .. })) => {
for hop in blinded_hops {
build_keys_in_loop!(hop.blinded_node_id, true, Some(hop.encrypted_payload));
}
payment_id: PaymentId,
/// The invoice to pay.
invoice: Bolt12Invoice,
- /// The context of the [`BlindedPath`] used to send the invoice.
+ /// The context of the [`BlindedMessagePath`] used to send the invoice.
///
- /// [`BlindedPath`]: crate::blinded_path::BlindedPath
+ /// [`BlindedMessagePath`]: crate::blinded_path::message::BlindedMessagePath
context: Option<OffersContext>,
/// A responder for replying with an [`InvoiceError`] if needed.
///
use bitcoin::{secp256k1, Sequence};
use crate::blinded_path::message::{MessageContext, OffersContext};
-use crate::blinded_path::{BlindedPath, NodeIdLookUp};
-use crate::blinded_path::message::ForwardNode;
+use crate::blinded_path::NodeIdLookUp;
+use crate::blinded_path::message::{BlindedMessagePath, ForwardNode};
use crate::blinded_path::payment::{BlindedPaymentPath, Bolt12OfferContext, Bolt12RefundContext, PaymentConstraints, PaymentContext, ReceiveTlvs};
use crate::chain;
use crate::chain::{Confirm, ChannelMonitorUpdateStatus, Watch, BestBlock};
/// short-lived, while anything with a greater expiration is considered long-lived.
///
/// Using [`ChannelManager::create_offer_builder`] or [`ChannelManager::create_refund_builder`],
-/// will included a [`BlindedPath`] created using:
+/// will included a [`BlindedMessagePath`] created using:
/// - [`MessageRouter::create_compact_blinded_paths`] when short-lived, and
/// - [`MessageRouter::create_blinded_paths`] when long-lived.
///
-/// Using compact [`BlindedPath`]s may provide better privacy as the [`MessageRouter`] could select
+/// Using compact [`BlindedMessagePath`]s may provide better privacy as the [`MessageRouter`] could select
/// more hops. However, since they use short channel ids instead of pubkeys, they are more likely to
/// become invalid over time as channels are closed. Thus, they are only suitable for short-term use.
pub const MAX_SHORT_LIVED_RELATIVE_EXPIRY: Duration = Duration::from_secs(60 * 60 * 24);
///
/// # Privacy
///
- /// Uses [`MessageRouter`] to construct a [`BlindedPath`] for the offer based on the given
+ /// Uses [`MessageRouter`] to construct a [`BlindedMessagePath`] for the offer based on the given
/// `absolute_expiry` according to [`MAX_SHORT_LIVED_RELATIVE_EXPIRY`]. See those docs for
/// privacy implications as well as those of the parameterized [`Router`], which implements
/// [`MessageRouter`].
///
/// # Privacy
///
- /// Uses [`MessageRouter`] to construct a [`BlindedPath`] for the refund based on the given
+ /// Uses [`MessageRouter`] to construct a [`BlindedMessagePath`] for the refund based on the given
/// `absolute_expiry` according to [`MAX_SHORT_LIVED_RELATIVE_EXPIRY`]. See those docs for
/// privacy implications as well as those of the parameterized [`Router`], which implements
/// [`MessageRouter`].
/// # Privacy
///
/// For payer privacy, uses a derived payer id and uses [`MessageRouter::create_blinded_paths`]
- /// to construct a [`BlindedPath`] for the reply path. For further privacy implications, see the
+ /// to construct a [`BlindedMessagePath`] for the reply path. For further privacy implications, see the
/// docs of the parameterized [`Router`], which implements [`MessageRouter`].
///
/// # Limitations
/// [`MAX_SHORT_LIVED_RELATIVE_EXPIRY`].
fn create_blinded_paths_using_absolute_expiry(
&self, context: OffersContext, absolute_expiry: Option<Duration>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
let now = self.duration_since_epoch();
let max_short_lived_absolute_expiry = now.saturating_add(MAX_SHORT_LIVED_RELATIVE_EXPIRY);
/// [`MessageRouter::create_blinded_paths`].
///
/// Errors if the `MessageRouter` errors.
- fn create_blinded_paths(&self, context: OffersContext) -> Result<Vec<BlindedPath>, ()> {
+ fn create_blinded_paths(&self, context: OffersContext) -> Result<Vec<BlindedMessagePath>, ()> {
let recipient = self.get_our_node_id();
let secp_ctx = &self.secp_ctx;
/// [`MessageRouter::create_compact_blinded_paths`].
///
/// Errors if the `MessageRouter` errors.
- fn create_compact_blinded_paths(&self, context: OffersContext) -> Result<Vec<BlindedPath>, ()> {
+ fn create_compact_blinded_paths(&self, context: OffersContext) -> Result<Vec<BlindedMessagePath>, ()> {
let recipient = self.get_our_node_id();
let secp_ctx = &self.secp_ctx;
use bitcoin::network::Network;
use bitcoin::secp256k1::{PublicKey, Secp256k1};
use core::time::Duration;
-use crate::blinded_path::{BlindedPath, IntroductionNode};
+use crate::blinded_path::IntroductionNode;
+use crate::blinded_path::message::BlindedMessagePath;
use crate::blinded_path::payment::{Bolt12OfferContext, Bolt12RefundContext, PaymentContext};
use crate::blinded_path::message::{MessageContext, OffersContext};
use crate::events::{Event, MessageSendEventsProvider, PaymentFailureReason, PaymentPurpose};
}
}
-fn resolve_introduction_node<'a, 'b, 'c>(node: &Node<'a, 'b, 'c>, path: &BlindedPath) -> PublicKey {
- path.public_introduction_node_id(&node.network_graph.read_only())
+fn resolve_introduction_node<'a, 'b, 'c>(node: &Node<'a, 'b, 'c>, path: &BlindedMessagePath) -> PublicKey {
+ path.0.public_introduction_node_id(&node.network_graph.read_only())
.and_then(|node_id| node_id.as_pubkey().ok())
.unwrap()
}
fn extract_invoice_request<'a, 'b, 'c>(
node: &Node<'a, 'b, 'c>, message: &OnionMessage
-) -> (InvoiceRequest, BlindedPath) {
+) -> (InvoiceRequest, BlindedMessagePath) {
match node.onion_messenger.peel_onion_message(message) {
Ok(PeeledOnion::Receive(message, _, reply_path)) => match message {
ParsedOnionMessageContents::Offers(offers_message) => match offers_message {
}
}
-fn extract_invoice<'a, 'b, 'c>(node: &Node<'a, 'b, 'c>, message: &OnionMessage) -> (Bolt12Invoice, Option<BlindedPath>) {
+fn extract_invoice<'a, 'b, 'c>(node: &Node<'a, 'b, 'c>, message: &OnionMessage) -> (Bolt12Invoice, Option<BlindedMessagePath>) {
match node.onion_messenger.peel_onion_message(message) {
Ok(PeeledOnion::Receive(message, _, reply_path)) => match message {
ParsedOnionMessageContents::Offers(offers_message) => match offers_message {
for path in offer.paths() {
let introduction_node_id = resolve_introduction_node(bob, &path);
assert_eq!(introduction_node_id, alice_id);
- assert!(matches!(path.introduction_node, IntroductionNode::DirectedShortChannelId(..)));
+ assert!(matches!(path.0.introduction_node, IntroductionNode::DirectedShortChannelId(..)));
}
}
assert_eq!(offer.absolute_expiry(), Some(absolute_expiry));
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(alice_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(alice_id));
}
let offer = alice.node
assert_eq!(offer.absolute_expiry(), None);
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(alice_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(alice_id));
}
}
for path in refund.paths() {
let introduction_node_id = resolve_introduction_node(alice, &path);
assert_eq!(introduction_node_id, bob_id);
- assert!(matches!(path.introduction_node, IntroductionNode::DirectedShortChannelId(..)));
+ assert!(matches!(path.0.introduction_node, IntroductionNode::DirectedShortChannelId(..)));
}
}
assert_eq!(refund.absolute_expiry(), Some(absolute_expiry));
assert!(!refund.paths().is_empty());
for path in refund.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
}
assert_ne!(offer.signing_pubkey(), Some(alice_id));
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
let payment_id = PaymentId([1; 32]);
});
assert_eq!(invoice_request.amount_msats(), None);
assert_ne!(invoice_request.payer_id(), david_id);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
let onion_message = alice.onion_messenger.next_onion_message_for_peer(charlie_id).unwrap();
charlie.onion_messenger.handle_onion_message(&alice_id, &onion_message);
assert_ne!(refund.payer_id(), david_id);
assert!(!refund.paths().is_empty());
for path in refund.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
}
expect_recent_payment!(david, RecentPaymentDetails::AwaitingInvoice, payment_id);
assert_ne!(offer.signing_pubkey(), Some(alice_id));
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(alice_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(alice_id));
}
let payment_id = PaymentId([1; 32]);
});
assert_eq!(invoice_request.amount_msats(), None);
assert_ne!(invoice_request.payer_id(), bob_id);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(bob_id));
let onion_message = alice.onion_messenger.next_onion_message_for_peer(bob_id).unwrap();
bob.onion_messenger.handle_onion_message(&alice_id, &onion_message);
assert_ne!(refund.payer_id(), bob_id);
assert!(!refund.paths().is_empty());
for path in refund.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
expect_recent_payment!(bob, RecentPaymentDetails::AwaitingInvoice, payment_id);
assert_ne!(offer.signing_pubkey(), Some(alice_id));
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
let payment_id = PaymentId([1; 32]);
alice.onion_messenger.handle_onion_message(&bob_id, &onion_message);
let (_, reply_path) = extract_invoice_request(alice, &onion_message);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
// Send, extract and verify the second Invoice Request message
let onion_message = david.onion_messenger.next_onion_message_for_peer(bob_id).unwrap();
alice.onion_messenger.handle_onion_message(&bob_id, &onion_message);
let (_, reply_path) = extract_invoice_request(alice, &onion_message);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(nodes[6].node.get_our_node_id()));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(nodes[6].node.get_our_node_id()));
}
/// This test checks that when multiple potential introduction nodes are available for the payee,
.build().unwrap();
assert_ne!(refund.payer_id(), alice_id);
for path in refund.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
expect_recent_payment!(alice, RecentPaymentDetails::AwaitingInvoice, payment_id);
let onion_message = bob.onion_messenger.next_onion_message_for_peer(alice_id).unwrap();
let (_, reply_path) = extract_invoice(alice, &onion_message);
- assert_eq!(reply_path.unwrap().introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(reply_path.unwrap().0.introduction_node, IntroductionNode::NodeId(charlie_id));
// Send, extract and verify the second Invoice Request message
let onion_message = david.onion_messenger.next_onion_message_for_peer(bob_id).unwrap();
let onion_message = bob.onion_messenger.next_onion_message_for_peer(alice_id).unwrap();
let (_, reply_path) = extract_invoice(alice, &onion_message);
- assert_eq!(reply_path.unwrap().introduction_node, IntroductionNode::NodeId(nodes[6].node.get_our_node_id()));
+ assert_eq!(reply_path.unwrap().0.introduction_node, IntroductionNode::NodeId(nodes[6].node.get_our_node_id()));
}
/// Checks that a deferred invoice can be paid asynchronously from an Event::InvoiceReceived.
assert_ne!(offer.signing_pubkey(), Some(alice_id));
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
let payment_id = PaymentId([1; 32]);
},
});
assert_ne!(invoice_request.payer_id(), bob_id);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(alice_id));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(alice_id));
let onion_message = alice.onion_messenger.next_onion_message_for_peer(bob_id).unwrap();
bob.onion_messenger.handle_onion_message(&alice_id, &onion_message);
assert_ne!(refund.payer_id(), bob_id);
assert!(!refund.paths().is_empty());
for path in refund.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(alice_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(alice_id));
}
expect_recent_payment!(bob, RecentPaymentDetails::AwaitingInvoice, payment_id);
assert_ne!(offer.signing_pubkey(), Some(alice_id));
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
let invalid_path = alice.node
.build().unwrap()
.paths().first().unwrap()
.clone();
- assert_eq!(invalid_path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(invalid_path.0.introduction_node, IntroductionNode::NodeId(bob_id));
// Send the invoice request directly to Alice instead of using a blinded path.
let payment_id = PaymentId([1; 32]);
let (invoice_request, reply_path) = extract_invoice_request(alice, &onion_message);
assert_eq!(invoice_request.amount_msats(), None);
assert_ne!(invoice_request.payer_id(), david_id);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
assert_eq!(alice.onion_messenger.next_onion_message_for_peer(charlie_id), None);
let (invoice_request, reply_path) = extract_invoice_request(alice, &onion_message);
assert_eq!(invoice_request.amount_msats(), None);
assert_ne!(invoice_request.payer_id(), david_id);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
assert_eq!(alice.onion_messenger.next_onion_message_for_peer(charlie_id), None);
}
assert_ne!(offer.signing_pubkey(), Some(alice_id));
assert!(!offer.paths().is_empty());
for path in offer.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(bob_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(bob_id));
}
// Initiate an invoice request, but abandon tracking it.
let (invoice_request, reply_path) = extract_invoice_request(alice, &onion_message);
assert_eq!(invoice_request.amount_msats(), None);
assert_ne!(invoice_request.payer_id(), david_id);
- assert_eq!(reply_path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(reply_path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
let onion_message = alice.onion_messenger.next_onion_message_for_peer(charlie_id).unwrap();
charlie.onion_messenger.handle_onion_message(&alice_id, &onion_message);
assert_ne!(refund.payer_id(), david_id);
assert!(!refund.paths().is_empty());
for path in refund.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
}
expect_recent_payment!(david, RecentPaymentDetails::AwaitingInvoice, payment_id);
assert_ne!(refund.payer_id(), david_id);
assert!(!refund.paths().is_empty());
for path in refund.paths() {
- assert_eq!(path.introduction_node, IntroductionNode::NodeId(charlie_id));
+ assert_eq!(path.0.introduction_node, IntroductionNode::NodeId(charlie_id));
}
let expected_invoice = alice.node.request_refund_payment(&refund).unwrap();
use core::time::Duration;
use core::hash::{Hash, Hasher};
use crate::io;
-use crate::blinded_path::BlindedPath;
+use crate::blinded_path::message::BlindedMessagePath;
use crate::blinded_path::payment::BlindedPaymentPath;
use crate::ln::types::PaymentHash;
use crate::ln::channelmanager::PaymentId;
/// From [`Offer::paths`] or [`Refund::paths`].
///
/// [`Offer::paths`]: crate::offers::offer::Offer::paths
- pub fn message_paths(&$self) -> &[BlindedPath] {
+ pub fn message_paths(&$self) -> &[BlindedMessagePath] {
$contents.message_paths()
}
}
/// Verifies that the invoice was for a request or refund created using the given key by
- /// checking a payment id and nonce included with the [`BlindedPath`] for which the invoice was
+ /// checking a payment id and nonce included with the [`BlindedMessagePath`] for which the invoice was
/// sent through.
pub fn verify_using_payer_data<T: secp256k1::Signing>(
&self, payment_id: PaymentId, nonce: Nonce, key: &ExpandedKey, secp_ctx: &Secp256k1<T>
}
}
- fn message_paths(&self) -> &[BlindedPath] {
+ fn message_paths(&self) -> &[BlindedMessagePath] {
match self {
InvoiceContents::ForOffer { invoice_request, .. } => {
invoice_request.inner.offer.paths()
(174, features: (Bolt12InvoiceFeatures, WithoutLength)),
(176, node_id: PublicKey),
// Only present in `StaticInvoice`s.
- (238, message_paths: (Vec<BlindedPath>, WithoutLength)),
+ (238, message_paths: (Vec<BlindedMessagePath>, WithoutLength)),
});
pub(super) type BlindedPathIter<'a> = core::iter::Map<
(None, Some(paths)) => {
if !paths
.iter()
- .filter_map(|path| path.blinded_hops.last())
+ .filter_map(|path| path.0.blinded_hops.last())
.any(|last_hop| invoice_signing_pubkey == &last_hop.blinded_node_id)
{
return Err(Bolt12SemanticError::InvalidSigningPubkey);
use core::time::Duration;
use crate::blinded_path::{BlindedHop, BlindedPath, IntroductionNode};
+ use crate::blinded_path::message::BlindedMessagePath;
use crate::sign::KeyMaterial;
use crate::ln::features::{Bolt12InvoiceFeatures, InvoiceRequestFeatures, OfferFeatures};
use crate::ln::inbound_payment::ExpandedKey;
let nonce = Nonce::from_entropy_source(&entropy);
let secp_ctx = Secp256k1::new();
- let blinded_path = BlindedPath {
+ let blinded_path = BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(42), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: node_id, encrypted_payload: vec![0; 44] },
],
- };
+ });
#[cfg(c_bindings)]
use crate::offers::offer::OfferWithDerivedMetadataBuilder as OfferBuilder;
let entropy = FixedEntropy {};
let secp_ctx = Secp256k1::new();
- let blinded_path = BlindedPath {
+ let blinded_path = BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(42), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: node_id, encrypted_payload: vec![0; 44] },
],
- };
+ });
let refund = RefundBuilder::new(vec![1; 32], payer_pubkey(), 1000).unwrap()
.path(blinded_path)
#[test]
fn parses_invoice_with_node_id_from_blinded_path() {
let paths = vec![
- BlindedPath {
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(44), encrypted_payload: vec![0; 44] },
],
- },
- BlindedPath {
+ }),
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(45), encrypted_payload: vec![0; 45] },
BlindedHop { blinded_node_id: pubkey(46), encrypted_payload: vec![0; 46] },
],
- },
+ }),
];
let blinded_node_id_sign = |message: &UnsignedBolt12Invoice| {
.build().unwrap()
.sign(recipient_sign).unwrap();
- let blinded_path = BlindedPath {
+ let blinded_path = BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(42), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 44] },
],
- };
+ });
let mut tlv_stream = invoice.as_tlv_stream();
let message_paths = vec![blinded_path];
use bitcoin::secp256k1::{Keypair, PublicKey, Secp256k1, self};
use bitcoin::secp256k1::schnorr::Signature;
use crate::io;
-use crate::blinded_path::BlindedPath;
+use crate::blinded_path::message::BlindedMessagePath;
use crate::blinded_path::payment::BlindedPaymentPath;
use crate::ln::types::PaymentHash;
use crate::ln::channelmanager::PaymentId;
}
/// Verifies that the request was for an offer created using the given key by checking a nonce
- /// included with the [`BlindedPath`] for which the request was sent through.
+ /// included with the [`BlindedMessagePath`] for which the request was sent through.
///
/// Returns the verified request which contains the derived keys needed to sign a
/// [`Bolt12Invoice`] for the request if they could be extracted from the metadata.
(INVOICE_REQUEST_PAYER_ID_TYPE, payer_id: PublicKey),
(89, payer_note: (String, WithoutLength)),
// Only used for Refund since the onion message of an InvoiceRequest has a reply path.
- (90, paths: (Vec<BlindedPath>, WithoutLength)),
+ (90, paths: (Vec<BlindedMessagePath>, WithoutLength)),
});
type FullInvoiceRequestTlvStream =
//! use lightning::offers::parse::Bolt12ParseError;
//! use lightning::util::ser::{Readable, Writeable};
//!
-//! # use lightning::blinded_path::BlindedPath;
+//! # use lightning::blinded_path::message::BlindedMessagePath;
//! # #[cfg(feature = "std")]
//! # use std::time::SystemTime;
//! #
-//! # fn create_blinded_path() -> BlindedPath { unimplemented!() }
-//! # fn create_another_blinded_path() -> BlindedPath { unimplemented!() }
+//! # fn create_blinded_path() -> BlindedMessagePath { unimplemented!() }
+//! # fn create_another_blinded_path() -> BlindedMessagePath { unimplemented!() }
//! #
//! # #[cfg(feature = "std")]
//! # fn build() -> Result<(), Bolt12ParseError> {
use core::str::FromStr;
use core::time::Duration;
use crate::io;
-use crate::blinded_path::BlindedPath;
+use crate::blinded_path::message::BlindedMessagePath;
use crate::ln::channelmanager::PaymentId;
use crate::ln::features::OfferFeatures;
use crate::ln::inbound_payment::{ExpandedKey, IV_LEN};
/// Also, sets the metadata when [`OfferBuilder::build`] is called such that it can be used by
/// [`InvoiceRequest::verify_using_metadata`] to determine if the request was produced for the
/// offer given an [`ExpandedKey`]. However, if [`OfferBuilder::path`] is called, then the
- /// metadata will not be set and must be included in each [`BlindedPath`] instead. In this case,
+ /// metadata will not be set and must be included in each [`BlindedMessagePath`] instead. In this case,
/// use [`InvoiceRequest::verify_using_recipient_data`].
///
/// [`InvoiceRequest::verify_using_metadata`]: crate::offers::invoice_request::InvoiceRequest::verify_using_metadata
///
/// Successive calls to this method will add another blinded path. Caller is responsible for not
/// adding duplicate paths.
- pub fn path($($self_mut)* $self: $self_type, path: BlindedPath) -> $return_type {
+ pub fn path($($self_mut)* $self: $self_type, path: BlindedMessagePath) -> $return_type {
$self.offer.paths.get_or_insert_with(Vec::new).push(path);
$return_value
}
/// Offers may be denominated in currency other than bitcoin but are ultimately paid using the
/// latter.
///
-/// Through the use of [`BlindedPath`]s, offers provide recipient privacy.
+/// Through the use of [`BlindedMessagePath`]s, offers provide recipient privacy.
///
/// [`InvoiceRequest`]: crate::offers::invoice_request::InvoiceRequest
/// [`Bolt12Invoice`]: crate::offers::invoice::Bolt12Invoice
features: OfferFeatures,
absolute_expiry: Option<Duration>,
issuer: Option<String>,
- paths: Option<Vec<BlindedPath>>,
+ paths: Option<Vec<BlindedMessagePath>>,
supported_quantity: Quantity,
signing_pubkey: Option<PublicKey>,
}
/// Paths to the recipient originating from publicly reachable nodes. Blinded paths provide
/// recipient privacy by obfuscating its node id.
- pub fn paths(&$self) -> &[$crate::blinded_path::BlindedPath] {
+ pub fn paths(&$self) -> &[$crate::blinded_path::message::BlindedMessagePath] {
$contents.paths()
}
self.issuer.as_ref().map(|issuer| PrintableString(issuer.as_str()))
}
- pub fn paths(&self) -> &[BlindedPath] {
+ pub fn paths(&self) -> &[BlindedMessagePath] {
self.paths.as_ref().map(|paths| paths.as_slice()).unwrap_or(&[])
}
(10, description: (String, WithoutLength)),
(12, features: (OfferFeatures, WithoutLength)),
(14, absolute_expiry: (u64, HighZeroBytesDroppedBigSize)),
- (16, paths: (Vec<BlindedPath>, WithoutLength)),
+ (16, paths: (Vec<BlindedMessagePath>, WithoutLength)),
(18, issuer: (String, WithoutLength)),
(20, quantity_max: (u64, HighZeroBytesDroppedBigSize)),
(OFFER_NODE_ID_TYPE, node_id: PublicKey),
use core::num::NonZeroU64;
use core::time::Duration;
use crate::blinded_path::{BlindedHop, BlindedPath, IntroductionNode};
+ use crate::blinded_path::message::BlindedMessagePath;
use crate::sign::KeyMaterial;
use crate::ln::features::OfferFeatures;
use crate::ln::inbound_payment::ExpandedKey;
let nonce = Nonce::from_entropy_source(&entropy);
let secp_ctx = Secp256k1::new();
- let blinded_path = BlindedPath {
+ let blinded_path = BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(42), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: node_id, encrypted_payload: vec![0; 44] },
],
- };
+ });
#[cfg(c_bindings)]
use super::OfferWithDerivedMetadataBuilder as OfferBuilder;
#[test]
fn builds_offer_with_paths() {
let paths = vec![
- BlindedPath {
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(44), encrypted_payload: vec![0; 44] },
],
- },
- BlindedPath {
+ }),
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(45), encrypted_payload: vec![0; 45] },
BlindedHop { blinded_node_id: pubkey(46), encrypted_payload: vec![0; 46] },
],
- },
+ }),
];
let offer = OfferBuilder::new(pubkey(42))
#[test]
fn parses_offer_with_paths() {
let offer = OfferBuilder::new(pubkey(42))
- .path(BlindedPath {
+ .path(BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(44), encrypted_payload: vec![0; 44] },
],
- })
- .path(BlindedPath {
+ }))
+ .path(BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(45), encrypted_payload: vec![0; 45] },
BlindedHop { blinded_node_id: pubkey(46), encrypted_payload: vec![0; 46] },
],
- })
+ }))
.build()
.unwrap();
if let Err(e) = offer.to_string().parse::<Offer>() {
}
let offer = OfferBuilder::new(pubkey(42))
- .path(BlindedPath {
+ .path(BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(44), encrypted_payload: vec![0; 44] },
],
- })
+ }))
.clear_signing_pubkey()
.build()
.unwrap();
//! use lightning::offers::refund::{Refund, RefundBuilder};
//! use lightning::util::ser::{Readable, Writeable};
//!
-//! # use lightning::blinded_path::BlindedPath;
+//! # use lightning::blinded_path::message::BlindedMessagePath;
//! # #[cfg(feature = "std")]
//! # use std::time::SystemTime;
//! #
-//! # fn create_blinded_path() -> BlindedPath { unimplemented!() }
-//! # fn create_another_blinded_path() -> BlindedPath { unimplemented!() }
+//! # fn create_blinded_path() -> BlindedMessagePath { unimplemented!() }
+//! # fn create_another_blinded_path() -> BlindedMessagePath { unimplemented!() }
//! #
//! # #[cfg(feature = "std")]
//! # fn build() -> Result<(), Bolt12ParseError> {
use core::time::Duration;
use crate::sign::EntropySource;
use crate::io;
-use crate::blinded_path::BlindedPath;
+use crate::blinded_path::message::BlindedMessagePath;
use crate::blinded_path::payment::BlindedPaymentPath;
use crate::ln::types::PaymentHash;
use crate::ln::channelmanager::PaymentId;
/// Also, sets the metadata when [`RefundBuilder::build`] is called such that it can be used by
/// [`Bolt12Invoice::verify_using_metadata`] to determine if the invoice was produced for the
/// refund given an [`ExpandedKey`]. However, if [`RefundBuilder::path`] is called, then the
- /// metadata must be included in each [`BlindedPath`] instead. In this case, use
+ /// metadata must be included in each [`BlindedMessagePath`] instead. In this case, use
/// [`Bolt12Invoice::verify_using_payer_data`].
///
/// The `payment_id` is encrypted in the metadata and should be unique. This ensures that only
///
/// Successive calls to this method will add another blinded path. Caller is responsible for not
/// adding duplicate paths.
- pub fn path($($self_mut)* $self: $self_type, path: BlindedPath) -> $return_type {
+ pub fn path($($self_mut)* $self: $self_type, path: BlindedMessagePath) -> $return_type {
$self.refund.paths.get_or_insert_with(Vec::new).push(path);
$return_value
}
quantity: Option<u64>,
payer_id: PublicKey,
payer_note: Option<String>,
- paths: Option<Vec<BlindedPath>>,
+ paths: Option<Vec<BlindedMessagePath>>,
}
impl Refund {
/// Paths to the sender originating from publicly reachable nodes. Blinded paths provide sender
/// privacy by obfuscating its node id.
- pub fn paths(&self) -> &[BlindedPath] {
+ pub fn paths(&self) -> &[BlindedMessagePath] {
self.contents.paths()
}
self.issuer.as_ref().map(|issuer| PrintableString(issuer.as_str()))
}
- pub fn paths(&self) -> &[BlindedPath] {
+ pub fn paths(&self) -> &[BlindedMessagePath] {
self.paths.as_ref().map(|paths| paths.as_slice()).unwrap_or(&[])
}
use core::time::Duration;
use crate::blinded_path::{BlindedHop, BlindedPath, IntroductionNode};
+ use crate::blinded_path::message::BlindedMessagePath;
use crate::sign::KeyMaterial;
use crate::ln::channelmanager::PaymentId;
use crate::ln::features::{InvoiceRequestFeatures, OfferFeatures};
let secp_ctx = Secp256k1::new();
let payment_id = PaymentId([1; 32]);
- let blinded_path = BlindedPath {
+ let blinded_path = BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: node_id, encrypted_payload: vec![0; 44] },
],
- };
+ });
let refund = RefundBuilder
::deriving_payer_id(node_id, &expanded_key, nonce, &secp_ctx, 1000, payment_id)
#[test]
fn builds_refund_with_paths() {
let paths = vec![
- BlindedPath {
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(44), encrypted_payload: vec![0; 44] },
],
- },
- BlindedPath {
+ }),
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(45), encrypted_payload: vec![0; 45] },
BlindedHop { blinded_node_id: pubkey(46), encrypted_payload: vec![0; 46] },
],
- },
+ }),
];
let refund = RefundBuilder::new(vec![1; 32], payer_pubkey(), 1000).unwrap()
fn parses_refund_with_optional_fields() {
let past_expiry = Duration::from_secs(0);
let paths = vec![
- BlindedPath {
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(44), encrypted_payload: vec![0; 44] },
],
- },
- BlindedPath {
+ }),
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(45), encrypted_payload: vec![0; 45] },
BlindedHop { blinded_node_id: pubkey(46), encrypted_payload: vec![0; 46] },
],
- },
+ }),
];
let refund = RefundBuilder::new(vec![1; 32], payer_pubkey(), 1000).unwrap()
//! Data structures and encoding for static BOLT 12 invoices.
+use crate::blinded_path::message::BlindedMessagePath;
use crate::blinded_path::payment::BlindedPaymentPath;
-use crate::blinded_path::BlindedPath;
use crate::io;
use crate::ln::features::{Bolt12InvoiceFeatures, OfferFeatures};
use crate::ln::inbound_payment::ExpandedKey;
fallbacks: Option<Vec<FallbackAddress>>,
features: Bolt12InvoiceFeatures,
signing_pubkey: PublicKey,
- message_paths: Vec<BlindedPath>,
+ message_paths: Vec<BlindedMessagePath>,
}
/// Builds a [`StaticInvoice`] from an [`Offer`].
/// after `created_at`.
pub fn for_offer_using_derived_keys<T: secp256k1::Signing>(
offer: &'a Offer, payment_paths: Vec<(BlindedPayInfo, BlindedPaymentPath)>,
- message_paths: Vec<BlindedPath>, created_at: Duration, expanded_key: &ExpandedKey,
+ message_paths: Vec<BlindedMessagePath>, created_at: Duration, expanded_key: &ExpandedKey,
nonce: Nonce, secp_ctx: &Secp256k1<T>,
) -> Result<Self, Bolt12SemanticError> {
if offer.chains().len() > 1 {
/// publicly reachable nodes. Taken from [`Offer::paths`].
///
/// [`Offer::paths`]: crate::offers::offer::Offer::paths
- pub fn offer_message_paths(&$self) -> &[BlindedPath] {
+ pub fn offer_message_paths(&$self) -> &[BlindedMessagePath] {
$contents.offer_message_paths()
}
/// Paths to the recipient for indicating that a held HTLC is available to claim when they next
/// come online.
- pub fn message_paths(&$self) -> &[BlindedPath] {
+ pub fn message_paths(&$self) -> &[BlindedMessagePath] {
$contents.message_paths()
}
fn new(
offer: &Offer, payment_paths: Vec<(BlindedPayInfo, BlindedPaymentPath)>,
- message_paths: Vec<BlindedPath>, created_at: Duration, signing_pubkey: PublicKey,
+ message_paths: Vec<BlindedMessagePath>, created_at: Duration, signing_pubkey: PublicKey,
) -> Self {
Self {
offer: offer.contents.clone(),
self.offer.issuer()
}
- fn offer_message_paths(&self) -> &[BlindedPath] {
+ fn offer_message_paths(&self) -> &[BlindedMessagePath] {
self.offer.paths()
}
- fn message_paths(&self) -> &[BlindedPath] {
+ fn message_paths(&self) -> &[BlindedMessagePath] {
&self.message_paths[..]
}
#[cfg(test)]
mod tests {
+ use crate::blinded_path::message::BlindedMessagePath;
use crate::blinded_path::{BlindedHop, BlindedPath, IntroductionNode};
use crate::ln::features::{Bolt12InvoiceFeatures, OfferFeatures};
use crate::ln::inbound_payment::ExpandedKey;
.unwrap()
}
- fn blinded_path() -> BlindedPath {
- BlindedPath {
+ fn blinded_path() -> BlindedMessagePath {
+ BlindedMessagePath(BlindedPath {
introduction_node: IntroductionNode::NodeId(pubkey(40)),
blinding_point: pubkey(41),
blinded_hops: vec![
BlindedHop { blinded_node_id: pubkey(42), encrypted_payload: vec![0; 43] },
BlindedHop { blinded_node_id: pubkey(43), encrypted_payload: vec![0; 44] },
],
- }
+ })
}
#[test]
) -> Vec<(
AsyncPaymentsMessage,
crate::onion_message::messenger::Destination,
- Option<crate::blinded_path::BlindedPath>,
+ Option<crate::blinded_path::message::BlindedMessagePath>,
)> {
vec![]
}
//! Onion message testing and test utilities live here.
-use crate::blinded_path::{BlindedPath, EmptyNodeIdLookUp};
-use crate::blinded_path::message::{ForwardNode, MessageContext, OffersContext};
+use crate::blinded_path::EmptyNodeIdLookUp;
+use crate::blinded_path::message::{BlindedMessagePath, ForwardNode, MessageContext, OffersContext};
use crate::events::{Event, EventsProvider};
use crate::ln::features::{ChannelFeatures, InitFeatures};
use crate::ln::msgs::{self, DecodeError, OnionMessageHandler};
let secp_ctx = Secp256k1::new();
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(&[], nodes[1].node_id, context, &*nodes[1].entropy_source, &secp_ctx).unwrap();
+ let blinded_path = BlindedMessagePath::new(&[], nodes[1].node_id, context, &*nodes[1].entropy_source, &secp_ctx).unwrap();
let destination = Destination::BlindedPath(blinded_path);
nodes[0].messenger.send_onion_message(test_msg, destination, None).unwrap();
nodes[1].custom_message_handler.expect_message(TestCustomMessage::Pong);
let secp_ctx = Secp256k1::new();
let intermediate_nodes = [ForwardNode { node_id: nodes[3].node_id, short_channel_id: None }];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[4].node_id, context, &*nodes[4].entropy_source, &secp_ctx).unwrap();
+ let blinded_path = BlindedMessagePath::new(&intermediate_nodes, nodes[4].node_id, context, &*nodes[4].entropy_source, &secp_ctx).unwrap();
let path = OnionMessagePath {
intermediate_nodes: vec![nodes[1].node_id, nodes[2].node_id],
destination: Destination::BlindedPath(blinded_path),
ForwardNode { node_id: nodes[2].node_id, short_channel_id: None },
];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[3].node_id, context, &*nodes[3].entropy_source, &secp_ctx).unwrap();
+ let blinded_path = BlindedMessagePath::new(&intermediate_nodes, nodes[3].node_id, context, &*nodes[3].entropy_source, &secp_ctx).unwrap();
let destination = Destination::BlindedPath(blinded_path);
nodes[0].messenger.send_onion_message(test_msg, destination, None).unwrap();
// 3. Simulate the creation of a Blinded Reply path provided by Bob.
let secp_ctx = Secp256k1::new();
let context = MessageContext::Custom(Vec::new());
- let reply_path = BlindedPath::new_for_message(&[], nodes[1].node_id, context, &*nodes[1].entropy_source, &secp_ctx).unwrap();
+ let reply_path = BlindedMessagePath::new(&[], nodes[1].node_id, context, &*nodes[1].entropy_source, &secp_ctx).unwrap();
// 4. Create a responder using the reply path for Alice.
let responder = Some(Responder::new(reply_path));
// Alice receives a message from Bob with an added reply_path for responding back.
let message = TestCustomMessage::Ping;
let context = MessageContext::Custom(Vec::new());
- let reply_path = BlindedPath::new_for_message(&[], bob.node_id, context, &*bob.entropy_source, &secp_ctx).unwrap();
+ let reply_path = BlindedMessagePath::new(&[], bob.node_id, context, &*bob.entropy_source, &secp_ctx).unwrap();
// Alice asynchronously responds to Bob, expecting a response back from him.
let responder = Responder::new(reply_path);
// Alice receives a message from Bob with an added reply_path for responding back.
let message = TestCustomMessage::Ping;
let context = MessageContext::Custom(Vec::new());
- let reply_path = BlindedPath::new_for_message(&[], bob.node_id, context, &*bob.entropy_source, &secp_ctx).unwrap();
+ let reply_path = BlindedMessagePath::new(&[], bob.node_id, context, &*bob.entropy_source, &secp_ctx).unwrap();
// Alice tries to asynchronously respond to Bob, but fails because the nodes are unannounced and
// disconnected. Thus, a reply path could no be created for the response.
ForwardNode { node_id: nodes[1].node_id, short_channel_id: None },
];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[2].node_id, context, &*nodes[2].entropy_source, &secp_ctx).unwrap();
+ let blinded_path = BlindedMessagePath::new(&intermediate_nodes, nodes[2].node_id, context, &*nodes[2].entropy_source, &secp_ctx).unwrap();
let destination = Destination::BlindedPath(blinded_path);
nodes[0].messenger.send_onion_message(test_msg.clone(), destination, None).unwrap();
// Try with a two-hop blinded path where we are the introduction node.
let intermediate_nodes = [ForwardNode { node_id: nodes[0].node_id, short_channel_id: None }];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[1].node_id, context, &*nodes[1].entropy_source, &secp_ctx).unwrap();
+ let blinded_path = BlindedMessagePath::new(&intermediate_nodes, nodes[1].node_id, context, &*nodes[1].entropy_source, &secp_ctx).unwrap();
let destination = Destination::BlindedPath(blinded_path);
nodes[0].messenger.send_onion_message(test_msg, destination, None).unwrap();
nodes[1].custom_message_handler.expect_message(TestCustomMessage::Pong);
let secp_ctx = Secp256k1::new();
let intermediate_nodes = [ForwardNode { node_id: nodes[1].node_id, short_channel_id: None }];
let context = MessageContext::Custom(Vec::new());
- let mut blinded_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[2].node_id, context, &*nodes[2].entropy_source, &secp_ctx).unwrap();
- blinded_path.blinded_hops.clear();
+ let mut blinded_path = BlindedMessagePath::new(&intermediate_nodes, nodes[2].node_id, context, &*nodes[2].entropy_source, &secp_ctx).unwrap();
+ blinded_path.0.blinded_hops.clear();
let destination = Destination::BlindedPath(blinded_path);
let err = nodes[0].messenger.send_onion_message(test_msg, destination, None).unwrap_err();
assert_eq!(err, SendError::TooFewBlindedHops);
ForwardNode { node_id: nodes[1].node_id, short_channel_id: None },
];
let context = MessageContext::Custom(Vec::new());
- let reply_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[0].node_id, context, &*nodes[0].entropy_source, &secp_ctx).unwrap();
+ let reply_path = BlindedMessagePath::new(&intermediate_nodes, nodes[0].node_id, context, &*nodes[0].entropy_source, &secp_ctx).unwrap();
nodes[0].messenger.send_onion_message_using_path(path, test_msg.clone(), Some(reply_path)).unwrap();
nodes[3].custom_message_handler.expect_message(TestCustomMessage::Ping);
pass_along_path(&nodes);
ForwardNode { node_id: nodes[2].node_id, short_channel_id: None },
];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[3].node_id, context, &*nodes[3].entropy_source, &secp_ctx).unwrap();
+ let blinded_path = BlindedMessagePath::new(&intermediate_nodes, nodes[3].node_id, context, &*nodes[3].entropy_source, &secp_ctx).unwrap();
let destination = Destination::BlindedPath(blinded_path);
let intermediate_nodes = [
ForwardNode { node_id: nodes[2].node_id, short_channel_id: None },
ForwardNode { node_id: nodes[1].node_id, short_channel_id: None },
];
let context = MessageContext::Custom(Vec::new());
- let reply_path = BlindedPath::new_for_message(&intermediate_nodes, nodes[0].node_id, context, &*nodes[0].entropy_source, &secp_ctx).unwrap();
+ let reply_path = BlindedMessagePath::new(&intermediate_nodes, nodes[0].node_id, context, &*nodes[0].entropy_source, &secp_ctx).unwrap();
nodes[0].messenger.send_onion_message(test_msg, destination, Some(reply_path)).unwrap();
nodes[3].custom_message_handler.expect_message(TestCustomMessage::Ping);
let intermediate_nodes = [ForwardNode { node_id: nodes[1].node_id, short_channel_id: None }];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(
+ let blinded_path = BlindedMessagePath::new(
&intermediate_nodes, nodes[2].node_id, context, &*nodes[0].entropy_source, &secp_ctx
).unwrap();
let destination = Destination::BlindedPath(blinded_path);
let intermediate_nodes = [ForwardNode { node_id: nodes[1].node_id, short_channel_id: None }];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(
+ let blinded_path = BlindedMessagePath::new(
&intermediate_nodes, nodes[2].node_id, context, &*nodes[0].entropy_source, &secp_ctx
).unwrap();
let destination = Destination::BlindedPath(blinded_path);
let secp_ctx = Secp256k1::new();
let intermediate_nodes = [ForwardNode { node_id: nodes[1].node_id, short_channel_id: None }];
let context = MessageContext::Custom(Vec::new());
- let blinded_path = BlindedPath::new_for_message(
+ let blinded_path = BlindedMessagePath::new(
&intermediate_nodes, nodes[2].node_id, context, &*nodes[2].entropy_source, &secp_ctx
).unwrap();
let destination = Destination::BlindedPath(blinded_path);
use bitcoin::secp256k1::{self, PublicKey, Scalar, Secp256k1, SecretKey};
use crate::blinded_path::{BlindedPath, IntroductionNode, NextMessageHop, NodeIdLookUp};
-use crate::blinded_path::message::{advance_path_by_one, ForwardNode, ForwardTlvs, MessageContext, ReceiveTlvs};
+use crate::blinded_path::message::{advance_path_by_one, BlindedMessagePath, ForwardNode, ForwardTlvs, MessageContext, ReceiveTlvs};
use crate::blinded_path::utils;
use crate::events::{Event, EventHandler, EventsProvider, ReplayEvent};
use crate::sign::{EntropySource, NodeSigner, Recipient};
/// # use bitcoin::hashes::_export::_core::time::Duration;
/// # use bitcoin::hex::FromHex;
/// # use bitcoin::secp256k1::{PublicKey, Secp256k1, SecretKey, self};
-/// # use lightning::blinded_path::{BlindedPath, EmptyNodeIdLookUp};
-/// # use lightning::blinded_path::message::{ForwardNode, MessageContext};
+/// # use lightning::blinded_path::EmptyNodeIdLookUp;
+/// # use lightning::blinded_path::message::{BlindedMessagePath, ForwardNode, MessageContext};
/// # use lightning::sign::{EntropySource, KeysManager};
/// # use lightning::ln::peer_handler::IgnoringMessageHandler;
/// # use lightning::onion_message::messenger::{Destination, MessageRouter, OnionMessagePath, OnionMessenger};
/// # }
/// # fn create_blinded_paths<T: secp256k1::Signing + secp256k1::Verification>(
/// # &self, _recipient: PublicKey, _context: MessageContext, _peers: Vec<PublicKey>, _secp_ctx: &Secp256k1<T>
-/// # ) -> Result<Vec<BlindedPath>, ()> {
+/// # ) -> Result<Vec<BlindedMessagePath>, ()> {
/// # unreachable!()
/// # }
/// # }
/// ForwardNode { node_id: hop_node_id4, short_channel_id: None },
/// ];
/// let context = MessageContext::Custom(Vec::new());
-/// let blinded_path = BlindedPath::new_for_message(&hops, your_node_id, context, &keys_manager, &secp_ctx).unwrap();
+/// let blinded_path = BlindedMessagePath::new(&hops, your_node_id, context, &keys_manager, &secp_ctx).unwrap();
///
/// // Send a custom onion message to a blinded path.
/// let destination = Destination::BlindedPath(blinded_path);
#[derive(Clone, Debug, Eq, PartialEq)]
pub struct Responder {
/// The path along which a response can be sent.
- reply_path: BlindedPath,
+ reply_path: BlindedMessagePath,
}
impl_writeable_tlv_based!(Responder, {
impl Responder {
/// Creates a new [`Responder`] instance with the provided reply path.
- pub(super) fn new(reply_path: BlindedPath) -> Self {
+ pub(super) fn new(reply_path: BlindedMessagePath) -> Self {
Responder {
reply_path,
}
/// This struct contains the information needed to reply to a received message.
pub struct OnionMessageResponse<T: OnionMessageContents> {
message: T,
- reply_path: BlindedPath,
+ reply_path: BlindedMessagePath,
}
/// `ResponseInstruction` represents instructions for responding to received messages.
pub destination: Destination,
/// A reply path to include in the [`OnionMessage`] for a response.
- pub reply_path: Option<BlindedPath>,
+ pub reply_path: Option<BlindedMessagePath>,
}
#[cfg(c_bindings)]
///
/// These are obtained when released from [`OnionMessenger`]'s handlers after which they are
/// enqueued for sending.
-pub type PendingOnionMessage<T> = (T, Destination, Option<BlindedPath>);
+pub type PendingOnionMessage<T> = (T, Destination, Option<BlindedMessagePath>);
pub(crate) fn new_pending_onion_message<T: OnionMessageContents>(
- contents: T, destination: Destination, reply_path: Option<BlindedPath>
+ contents: T, destination: Destination, reply_path: Option<BlindedMessagePath>
) -> PendingOnionMessage<T> {
#[cfg(not(c_bindings))]
return PendingOnionMessage { contents, destination, reply_path };
&self, sender: PublicKey, peers: Vec<PublicKey>, destination: Destination
) -> Result<OnionMessagePath, ()>;
- /// Creates [`BlindedPath`]s to the `recipient` node. The nodes in `peers` are assumed to be
- /// direct peers with the `recipient`.
+ /// Creates [`BlindedMessagePath`]s to the `recipient` node. The nodes in `peers` are assumed to
+ /// be direct peers with the `recipient`.
fn create_blinded_paths<
T: secp256k1::Signing + secp256k1::Verification
>(
&self, recipient: PublicKey, context: MessageContext, peers: Vec<PublicKey>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()>;
+ ) -> Result<Vec<BlindedMessagePath>, ()>;
- /// Creates compact [`BlindedPath`]s to the `recipient` node. The nodes in `peers` are assumed
- /// to be direct peers with the `recipient`.
+ /// Creates compact [`BlindedMessagePath`]s to the `recipient` node. The nodes in `peers` are
+ /// assumed to be direct peers with the `recipient`.
///
/// Compact blinded paths use short channel ids instead of pubkeys for a smaller serialization,
/// which is beneficial when a QR code is used to transport the data. The SCID is passed using a
>(
&self, recipient: PublicKey, context: MessageContext,
peers: Vec<ForwardNode>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
let peers = peers
.into_iter()
.map(|ForwardNode { node_id, short_channel_id: _ }| node_id)
///
/// # Privacy
///
-/// Creating [`BlindedPath`]s may affect privacy since, if a suitable path cannot be found, it will
-/// create a one-hop path using the recipient as the introduction node if it is a announced node.
-/// Otherwise, there is no way to find a path to the introduction node in order to send a message,
-/// and thus an `Err` is returned.
+/// Creating [`BlindedMessagePath`]s may affect privacy since, if a suitable path cannot be found,
+/// it will create a one-hop path using the recipient as the introduction node if it is a announced
+/// node. Otherwise, there is no way to find a path to the introduction node in order to send a
+/// message, and thus an `Err` is returned.
pub struct DefaultMessageRouter<G: Deref<Target=NetworkGraph<L>>, L: Deref, ES: Deref>
where
L::Target: Logger,
>(
network_graph: &G, recipient: PublicKey, context: MessageContext, peers: I,
entropy_source: &ES, secp_ctx: &Secp256k1<T>, compact_paths: bool,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
// Limit the number of blinded paths that are computed.
const MAX_PATHS: usize = 3;
let paths = peer_info.into_iter()
.map(|(peer, _, _)| {
- BlindedPath::new_for_message(&[peer], recipient, context.clone(), &**entropy_source, secp_ctx)
+ BlindedMessagePath::new(&[peer], recipient, context.clone(), &**entropy_source, secp_ctx)
})
.take(MAX_PATHS)
.collect::<Result<Vec<_>, _>>();
Ok(paths) if !paths.is_empty() => Ok(paths),
_ => {
if is_recipient_announced {
- BlindedPath::one_hop_for_message(recipient, context, &**entropy_source, secp_ctx)
+ BlindedMessagePath::new(&[], recipient, context, &**entropy_source, secp_ctx)
.map(|path| vec![path])
} else {
Err(())
if compact_paths {
for path in &mut paths {
- path.use_compact_introduction_node(&network_graph);
+ path.0.use_compact_introduction_node(&network_graph);
}
}
>(
network_graph: &G, recipient: PublicKey, context: MessageContext,
peers: Vec<PublicKey>, entropy_source: &ES, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
let peers = peers
.into_iter()
.map(|node_id| ForwardNode { node_id, short_channel_id: None });
>(
network_graph: &G, recipient: PublicKey, context: MessageContext,
peers: Vec<ForwardNode>, entropy_source: &ES, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
Self::create_blinded_paths_from_iter(network_graph, recipient, context, peers.into_iter(), entropy_source, secp_ctx, true)
}
}
T: secp256k1::Signing + secp256k1::Verification
>(
&self, recipient: PublicKey, context: MessageContext, peers: Vec<PublicKey>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
Self::create_blinded_paths(&self.network_graph, recipient, context, peers, &self.entropy_source, secp_ctx)
}
T: secp256k1::Signing + secp256k1::Verification
>(
&self, recipient: PublicKey, context: MessageContext, peers: Vec<ForwardNode>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
Self::create_compact_blinded_paths(&self.network_graph, recipient, context, peers, &self.entropy_source, secp_ctx)
}
/// We're sending this onion message to a node.
Node(PublicKey),
/// We're sending this onion message to a blinded path.
- BlindedPath(BlindedPath),
+ BlindedPath(BlindedMessagePath),
}
impl Destination {
/// provided [`ReadOnlyNetworkGraph`].
pub fn resolve(&mut self, network_graph: &ReadOnlyNetworkGraph) {
if let Destination::BlindedPath(path) = self {
- if let IntroductionNode::DirectedShortChannelId(..) = path.introduction_node {
+ if let IntroductionNode::DirectedShortChannelId(..) = path.0.introduction_node {
if let Some(pubkey) = path
+ .0
.public_introduction_node_id(network_graph)
.and_then(|node_id| node_id.as_pubkey().ok())
{
- path.introduction_node = IntroductionNode::NodeId(pubkey);
+ path.0.introduction_node = IntroductionNode::NodeId(pubkey);
}
}
}
pub(super) fn num_hops(&self) -> usize {
match self {
Destination::Node(_) => 1,
- Destination::BlindedPath(BlindedPath { blinded_hops, .. }) => blinded_hops.len(),
+ Destination::BlindedPath(BlindedMessagePath(BlindedPath { blinded_hops, .. })) => blinded_hops.len(),
}
}
fn first_node(&self) -> Option<PublicKey> {
match self {
Destination::Node(node_id) => Some(*node_id),
- Destination::BlindedPath(BlindedPath { introduction_node, .. }) => {
+ Destination::BlindedPath(BlindedMessagePath(BlindedPath { introduction_node, .. })) => {
match introduction_node {
IntroductionNode::NodeId(pubkey) => Some(*pubkey),
IntroductionNode::DirectedShortChannelId(..) => None,
/// Because implementations such as Eclair will drop onion messages where the message packet
/// exceeds 32834 bytes, we refuse to send messages where the packet exceeds this size.
TooBigPacket,
- /// The provided [`Destination`] was an invalid [`BlindedPath`] due to not having any blinded
- /// hops.
+ /// The provided [`Destination`] was an invalid [`BlindedMessagePath`] due to not having any
+ /// blinded hops.
TooFewBlindedHops,
/// The first hop is not a peer and doesn't have a known [`SocketAddress`].
InvalidFirstHop(PublicKey),
/// Typically, this is used for messages initiating a message flow rather than in response to
/// another message. The latter should use the return value of [`Self::handle_custom_message`].
#[cfg(c_bindings)]
- fn release_pending_custom_messages(&self) -> Vec<(Self::CustomMessage, Destination, Option<BlindedPath>)>;
+ fn release_pending_custom_messages(&self) -> Vec<(Self::CustomMessage, Destination, Option<BlindedMessagePath>)>;
}
/// A processed incoming onion message, containing either a Forward (another onion message)
/// Forwarded onion, with the next node id and a new onion
Forward(NextMessageHop, OnionMessage),
/// Received onion message, with decrypted contents, context, and reply path
- Receive(ParsedOnionMessageContents<T>, Option<MessageContext>, Option<BlindedPath>)
+ Receive(ParsedOnionMessageContents<T>, Option<MessageContext>, Option<BlindedMessagePath>)
}
>(
entropy_source: &ES, node_signer: &NS, node_id_lookup: &NL,
network_graph: &ReadOnlyNetworkGraph, secp_ctx: &Secp256k1<secp256k1::All>,
- mut path: OnionMessagePath, contents: T, reply_path: Option<BlindedPath>,
+ mut path: OnionMessagePath, contents: T, reply_path: Option<BlindedMessagePath>,
) -> Result<(PublicKey, OnionMessage, Option<Vec<SocketAddress>>), SendError>
where
ES::Target: EntropySource,
pub fn create_onion_message<ES: Deref, NS: Deref, NL: Deref, T: OnionMessageContents>(
entropy_source: &ES, node_signer: &NS, node_id_lookup: &NL,
secp_ctx: &Secp256k1<secp256k1::All>, path: OnionMessagePath, contents: T,
- reply_path: Option<BlindedPath>,
+ reply_path: Option<BlindedMessagePath>,
) -> Result<(PublicKey, OnionMessage, Option<Vec<SocketAddress>>), SendError>
where
ES::Target: EntropySource,
NL::Target: NodeIdLookUp,
{
let OnionMessagePath { intermediate_nodes, mut destination, first_node_addresses } = path;
- if let Destination::BlindedPath(BlindedPath { ref blinded_hops, .. }) = destination {
+ if let Destination::BlindedPath(BlindedMessagePath(BlindedPath { ref blinded_hops, .. })) = destination {
if blinded_hops.is_empty() {
return Err(SendError::TooFewBlindedHops);
}
if let Destination::BlindedPath(ref mut blinded_path) = destination {
let our_node_id = node_signer.get_node_id(Recipient::Node)
.map_err(|()| SendError::GetNodeIdFailed)?;
- let introduction_node_id = match blinded_path.introduction_node {
+ let introduction_node_id = match blinded_path.0.introduction_node {
IntroductionNode::NodeId(pubkey) => pubkey,
IntroductionNode::DirectedShortChannelId(direction, scid) => {
match node_id_lookup.next_node_id(scid) {
} else {
match &destination {
Destination::Node(pk) => (*pk, PublicKey::from_secret_key(&secp_ctx, &blinding_secret)),
- Destination::BlindedPath(BlindedPath { introduction_node, blinding_point, .. }) => {
+ Destination::BlindedPath(BlindedMessagePath(BlindedPath { introduction_node, blinding_point, .. })) => {
match introduction_node {
IntroductionNode::NodeId(pubkey) => (*pubkey, *blinding_point),
IntroductionNode::DirectedShortChannelId(..) => {
///
/// See [`OnionMessenger`] for example usage.
pub fn send_onion_message<T: OnionMessageContents>(
- &self, contents: T, destination: Destination, reply_path: Option<BlindedPath>
+ &self, contents: T, destination: Destination, reply_path: Option<BlindedMessagePath>
) -> Result<SendSuccess, SendError> {
self.find_path_and_enqueue_onion_message(
contents, destination, reply_path, format_args!("")
}
fn find_path_and_enqueue_onion_message<T: OnionMessageContents>(
- &self, contents: T, destination: Destination, reply_path: Option<BlindedPath>,
+ &self, contents: T, destination: Destination, reply_path: Option<BlindedMessagePath>,
log_suffix: fmt::Arguments
) -> Result<SendSuccess, SendError> {
let mut logger = WithContext::from(&self.logger, None, None, None);
.map_err(|_| SendError::PathNotFound)
}
- fn create_blinded_path(&self, context: MessageContext) -> Result<BlindedPath, SendError> {
+ fn create_blinded_path(&self, context: MessageContext) -> Result<BlindedMessagePath, SendError> {
let recipient = self.node_signer
.get_node_id(Recipient::Node)
.map_err(|_| SendError::GetNodeIdFailed)?;
}
fn enqueue_onion_message<T: OnionMessageContents>(
- &self, path: OnionMessagePath, contents: T, reply_path: Option<BlindedPath>,
+ &self, path: OnionMessagePath, contents: T, reply_path: Option<BlindedMessagePath>,
log_suffix: fmt::Arguments
) -> Result<SendSuccess, SendError> {
log_trace!(self.logger, "Constructing onion message {}: {:?}", log_suffix, contents);
#[cfg(any(test, feature = "_test_utils"))]
pub fn send_onion_message_using_path<T: OnionMessageContents>(
- &self, path: OnionMessagePath, contents: T, reply_path: Option<BlindedPath>
+ &self, path: OnionMessagePath, contents: T, reply_path: Option<BlindedMessagePath>
) -> Result<SendSuccess, SendError> {
self.enqueue_onion_message(path, contents, reply_path, format_args!(""))
}
/// `unblinded_path` to the given `destination`.
fn packet_payloads_and_keys<T: OnionMessageContents, S: secp256k1::Signing + secp256k1::Verification>(
secp_ctx: &Secp256k1<S>, unblinded_path: &[PublicKey], destination: Destination, message: T,
- mut reply_path: Option<BlindedPath>, session_priv: &SecretKey
+ mut reply_path: Option<BlindedMessagePath>, session_priv: &SecretKey
) -> Result<(Vec<(Payload<T>, [u8; 32])>, Vec<onion_utils::OnionKeys>), SendError> {
let num_hops = unblinded_path.len() + destination.num_hops();
let mut payloads = Vec::with_capacity(num_hops);
let (mut intro_node_id_blinding_pt, num_blinded_hops) = match &destination {
Destination::Node(_) => (None, 0),
- Destination::BlindedPath(BlindedPath { introduction_node, blinding_point, blinded_hops }) => {
+ Destination::BlindedPath(BlindedMessagePath(BlindedPath { introduction_node, blinding_point, blinded_hops })) => {
let introduction_node_id = match introduction_node {
IntroductionNode::NodeId(pubkey) => pubkey,
IntroductionNode::DirectedShortChannelId(..) => {
//! information on its usage.
//!
//! [offers]: <https://github.com/lightning/bolts/pull/798>
-//! [blinded paths]: crate::blinded_path::BlindedPath
+//! [blinded paths]: crate::blinded_path::message::BlindedMessagePath
//! [`OnionMessenger`]: self::messenger::OnionMessenger
pub mod async_payments;
/// Typically, this is used for messages initiating a payment flow rather than in response to
/// another message. The latter should use the return value of [`Self::handle_message`].
#[cfg(c_bindings)]
- fn release_pending_messages(&self) -> Vec<(OffersMessage, crate::onion_message::messenger::Destination, Option<crate::blinded_path::BlindedPath>)> { vec![] }
+ fn release_pending_messages(&self) -> Vec<(OffersMessage, crate::onion_message::messenger::Destination, Option<crate::blinded_path::message::BlindedMessagePath>)> { vec![] }
}
/// Possible BOLT 12 Offers messages sent and received via an [`OnionMessage`].
use bitcoin::secp256k1::PublicKey;
use bitcoin::secp256k1::ecdh::SharedSecret;
-use crate::blinded_path::{BlindedPath, NextMessageHop};
-use crate::blinded_path::message::{ForwardTlvs, ReceiveTlvs};
+use crate::blinded_path::NextMessageHop;
+use crate::blinded_path::message::{BlindedMessagePath, ForwardTlvs, ReceiveTlvs};
use crate::blinded_path::utils::Padding;
use crate::ln::msgs::DecodeError;
use crate::ln::onion_utils;
/// This payload is for the final hop.
Receive {
control_tlvs: ReceiveControlTlvs,
- reply_path: Option<BlindedPath>,
+ reply_path: Option<BlindedMessagePath>,
message: T,
}
}
let v: BigSize = Readable::read(r)?;
let mut rd = FixedLengthReader::new(r, v.0);
- let mut reply_path: Option<BlindedPath> = None;
+ let mut reply_path: Option<BlindedMessagePath> = None;
let mut read_adapter: Option<ChaChaPolyReadAdapter<ControlTlvs>> = None;
let rho = onion_utils::gen_rho_from_shared_secret(&encrypted_tlvs_ss.secret_bytes());
let mut message_type: Option<u64> = None;
use bitcoin::secp256k1::{PublicKey, Secp256k1, self};
-use crate::blinded_path::{BlindedHop, BlindedPath, Direction, IntroductionNode};
-use crate::blinded_path::message::{self, MessageContext};
+use crate::blinded_path::{BlindedHop, Direction, IntroductionNode};
+use crate::blinded_path::message::{self, BlindedMessagePath, MessageContext};
use crate::blinded_path::payment::{BlindedPaymentPath, ForwardTlvs, PaymentConstraints, PaymentRelay, ReceiveTlvs, self};
use crate::ln::{PaymentHash, PaymentPreimage};
use crate::ln::channel_state::ChannelDetails;
T: secp256k1::Signing + secp256k1::Verification
> (
&self, recipient: PublicKey, context: MessageContext, peers: Vec<PublicKey>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
DefaultMessageRouter::create_blinded_paths(&self.network_graph, recipient, context, peers, &self.entropy_source, secp_ctx)
}
T: secp256k1::Signing + secp256k1::Verification
> (
&self, recipient: PublicKey, context: MessageContext, peers: Vec<message::ForwardNode>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
DefaultMessageRouter::create_compact_blinded_paths(&self.network_graph, recipient, context, peers, &self.entropy_source, secp_ctx)
}
}
// licenses.
use crate::blinded_path::message::MessageContext;
-use crate::blinded_path::BlindedPath;
-use crate::blinded_path::message::ForwardNode;
+use crate::blinded_path::message::{BlindedMessagePath, ForwardNode};
use crate::blinded_path::payment::{BlindedPaymentPath, ReceiveTlvs};
use crate::chain;
use crate::chain::WatchedOutput;
>(
&self, recipient: PublicKey, context: MessageContext,
peers: Vec<PublicKey>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
self.router.create_blinded_paths(recipient, context, peers, secp_ctx)
}
>(
&self, recipient: PublicKey, context: MessageContext,
peers: Vec<ForwardNode>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
self.router.create_compact_blinded_paths(recipient, context, peers, secp_ctx)
}
}
fn create_blinded_paths<T: secp256k1::Signing + secp256k1::Verification>(
&self, recipient: PublicKey, context: MessageContext,
peers: Vec<PublicKey>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
self.inner.create_blinded_paths(recipient, context, peers, secp_ctx)
}
fn create_compact_blinded_paths<T: secp256k1::Signing + secp256k1::Verification>(
&self, recipient: PublicKey, context: MessageContext,
peers: Vec<ForwardNode>, secp_ctx: &Secp256k1<T>,
- ) -> Result<Vec<BlindedPath>, ()> {
+ ) -> Result<Vec<BlindedMessagePath>, ()> {
self.inner.create_compact_blinded_paths(recipient, context, peers, secp_ctx)
}
}