--------------- / (as EventsProvider) ^ | |
| PeerManager |- \ | | |
--------------- \ | (is-a) | |
- | ----------------- \ _---------------- / /
- | | chain::Access | \ / | ChainMonitor |---------------
- | ----------------- \ / ----------------
+ | -------------- \ _---------------- / /
+ | | UtxoLookup | \ / | ChainMonitor |---------------
+ | -------------- \ / ----------------
| ^ \ / |
(as RoutingMessageHandler) | v v
\ ----------------- --------- -----------------
use lightning::ln::msgs::{self, DecodeError};
use lightning::ln::script::ShutdownScript;
use lightning::routing::gossip::{P2PGossipSync, NetworkGraph};
+use lightning::routing::utxo::UtxoLookup;
use lightning::routing::router::{find_route, InFlightHtlcs, PaymentParameters, Route, RouteHop, RouteParameters, Router};
use lightning::routing::scoring::FixedPenaltyScorer;
use lightning::util::config::UserConfig;
type ChannelMan<'a> = ChannelManager<
Arc<chainmonitor::ChainMonitor<EnforcingSigner, Arc<dyn chain::Filter>, Arc<TestBroadcaster>, Arc<FuzzEstimator>, Arc<dyn Logger>, Arc<TestPersister>>>,
Arc<TestBroadcaster>, Arc<KeyProvider>, Arc<KeyProvider>, Arc<KeyProvider>, Arc<FuzzEstimator>, &'a FuzzRouter, Arc<dyn Logger>>;
-type PeerMan<'a> = PeerManager<Peer<'a>, Arc<ChannelMan<'a>>, Arc<P2PGossipSync<Arc<NetworkGraph<Arc<dyn Logger>>>, Arc<dyn chain::Access>, Arc<dyn Logger>>>, IgnoringMessageHandler, Arc<dyn Logger>, IgnoringMessageHandler, Arc<KeyProvider>>;
+type PeerMan<'a> = PeerManager<Peer<'a>, Arc<ChannelMan<'a>>, Arc<P2PGossipSync<Arc<NetworkGraph<Arc<dyn Logger>>>, Arc<dyn UtxoLookup>, Arc<dyn Logger>>>, IgnoringMessageHandler, Arc<dyn Logger>, IgnoringMessageHandler, Arc<KeyProvider>>;
struct MoneyLossDetector<'a> {
manager: Arc<ChannelMan<'a>>,
use bitcoin::blockdata::transaction::TxOut;
use bitcoin::hash_types::BlockHash;
-use lightning::chain;
use lightning::chain::transaction::OutPoint;
use lightning::ln::channelmanager::{self, ChannelDetails, ChannelCounterparty};
use lightning::ln::msgs;
use lightning::routing::gossip::{NetworkGraph, RoutingFees};
+use lightning::routing::utxo::{UtxoLookup, UtxoLookupError};
use lightning::routing::router::{find_route, PaymentParameters, RouteHint, RouteHintHop, RouteParameters};
use lightning::routing::scoring::FixedPenaltyScorer;
use lightning::util::config::UserConfig;
struct FuzzChainSource {
input: Arc<InputData>,
}
-impl chain::Access for FuzzChainSource {
- fn get_utxo(&self, _genesis_hash: &BlockHash, _short_channel_id: u64) -> Result<TxOut, chain::AccessError> {
+impl UtxoLookup for FuzzChainSource {
+ fn get_utxo(&self, _genesis_hash: &BlockHash, _short_channel_id: u64) -> Result<TxOut, UtxoLookupError> {
match self.input.get_slice(2) {
- Some(&[0, _]) => Err(chain::AccessError::UnknownChain),
- Some(&[1, _]) => Err(chain::AccessError::UnknownTx),
+ Some(&[0, _]) => Err(UtxoLookupError::UnknownChain),
+ Some(&[1, _]) => Err(UtxoLookupError::UnknownTx),
Some(&[_, x]) => Ok(TxOut { value: 0, script_pubkey: Builder::new().push_int(x as i64).into_script().to_v0_p2wsh() }),
- None => Err(chain::AccessError::UnknownTx),
+ None => Err(UtxoLookupError::UnknownTx),
_ => unreachable!(),
}
}
use lightning::ln::msgs::{ChannelMessageHandler, OnionMessageHandler, RoutingMessageHandler};
use lightning::ln::peer_handler::{CustomMessageHandler, PeerManager, SocketDescriptor};
use lightning::routing::gossip::{NetworkGraph, P2PGossipSync};
+use lightning::routing::utxo::UtxoLookup;
use lightning::routing::router::Router;
use lightning::routing::scoring::{Score, WriteableScore};
use lightning::util::events::{Event, EventHandler, EventsProvider};
/// Either [`P2PGossipSync`] or [`RapidGossipSync`].
pub enum GossipSync<
- P: Deref<Target = P2PGossipSync<G, A, L>>,
+ P: Deref<Target = P2PGossipSync<G, U, L>>,
R: Deref<Target = RapidGossipSync<G, L>>,
G: Deref<Target = NetworkGraph<L>>,
- A: Deref,
+ U: Deref,
L: Deref,
>
-where A::Target: chain::Access, L::Target: Logger {
+where U::Target: UtxoLookup, L::Target: Logger {
/// Gossip sync via the lightning peer-to-peer network as defined by BOLT 7.
P2P(P),
/// Rapid gossip sync from a trusted server.
}
impl<
- P: Deref<Target = P2PGossipSync<G, A, L>>,
+ P: Deref<Target = P2PGossipSync<G, U, L>>,
R: Deref<Target = RapidGossipSync<G, L>>,
G: Deref<Target = NetworkGraph<L>>,
- A: Deref,
+ U: Deref,
L: Deref,
-> GossipSync<P, R, G, A, L>
-where A::Target: chain::Access, L::Target: Logger {
+> GossipSync<P, R, G, U, L>
+where U::Target: UtxoLookup, L::Target: Logger {
fn network_graph(&self) -> Option<&G> {
match self {
GossipSync::P2P(gossip_sync) => Some(gossip_sync.network_graph()),
}
/// (C-not exported) as the bindings concretize everything and have constructors for us
-impl<P: Deref<Target = P2PGossipSync<G, A, L>>, G: Deref<Target = NetworkGraph<L>>, A: Deref, L: Deref>
- GossipSync<P, &RapidGossipSync<G, L>, G, A, L>
+impl<P: Deref<Target = P2PGossipSync<G, U, L>>, G: Deref<Target = NetworkGraph<L>>, U: Deref, L: Deref>
+ GossipSync<P, &RapidGossipSync<G, L>, G, U, L>
where
- A::Target: chain::Access,
+ U::Target: UtxoLookup,
L::Target: Logger,
{
/// Initializes a new [`GossipSync::P2P`] variant.
/// (C-not exported) as the bindings concretize everything and have constructors for us
impl<'a, R: Deref<Target = RapidGossipSync<G, L>>, G: Deref<Target = NetworkGraph<L>>, L: Deref>
GossipSync<
- &P2PGossipSync<G, &'a (dyn chain::Access + Send + Sync), L>,
+ &P2PGossipSync<G, &'a (dyn UtxoLookup + Send + Sync), L>,
R,
G,
- &'a (dyn chain::Access + Send + Sync),
+ &'a (dyn UtxoLookup + Send + Sync),
L,
>
where
/// (C-not exported) as the bindings concretize everything and have constructors for us
impl<'a, L: Deref>
GossipSync<
- &P2PGossipSync<&'a NetworkGraph<L>, &'a (dyn chain::Access + Send + Sync), L>,
+ &P2PGossipSync<&'a NetworkGraph<L>, &'a (dyn UtxoLookup + Send + Sync), L>,
&RapidGossipSync<&'a NetworkGraph<L>, L>,
&'a NetworkGraph<L>,
- &'a (dyn chain::Access + Send + Sync),
+ &'a (dyn UtxoLookup + Send + Sync),
L,
>
where
#[cfg(feature = "futures")]
pub async fn process_events_async<
'a,
- CA: 'static + Deref + Send + Sync,
+ UL: 'static + Deref + Send + Sync,
CF: 'static + Deref + Send + Sync,
CW: 'static + Deref + Send + Sync,
T: 'static + Deref + Send + Sync,
PS: 'static + Deref + Send,
M: 'static + Deref<Target = ChainMonitor<<SP::Target as SignerProvider>::Signer, CF, T, F, L, P>> + Send + Sync,
CM: 'static + Deref<Target = ChannelManager<CW, T, ES, NS, SP, F, R, L>> + Send + Sync,
- PGS: 'static + Deref<Target = P2PGossipSync<G, CA, L>> + Send + Sync,
+ PGS: 'static + Deref<Target = P2PGossipSync<G, UL, L>> + Send + Sync,
RGS: 'static + Deref<Target = RapidGossipSync<G, L>> + Send,
UMH: 'static + Deref + Send + Sync,
PM: 'static + Deref<Target = PeerManager<Descriptor, CMH, RMH, OMH, L, UMH, NS>> + Send + Sync,
Sleeper: Fn(Duration) -> SleepFuture
>(
persister: PS, event_handler: EventHandler, chain_monitor: M, channel_manager: CM,
- gossip_sync: GossipSync<PGS, RGS, G, CA, L>, peer_manager: PM, logger: L, scorer: Option<S>,
+ gossip_sync: GossipSync<PGS, RGS, G, UL, L>, peer_manager: PM, logger: L, scorer: Option<S>,
sleeper: Sleeper,
) -> Result<(), io::Error>
where
- CA::Target: 'static + chain::Access,
+ UL::Target: 'static + UtxoLookup,
CF::Target: 'static + chain::Filter,
CW::Target: 'static + chain::Watch<<SP::Target as SignerProvider>::Signer>,
T::Target: 'static + BroadcasterInterface,
/// [`NetworkGraph::write`]: lightning::routing::gossip::NetworkGraph#impl-Writeable
pub fn start<
'a,
- CA: 'static + Deref + Send + Sync,
+ UL: 'static + Deref + Send + Sync,
CF: 'static + Deref + Send + Sync,
CW: 'static + Deref + Send + Sync,
T: 'static + Deref + Send + Sync,
PS: 'static + Deref + Send,
M: 'static + Deref<Target = ChainMonitor<<SP::Target as SignerProvider>::Signer, CF, T, F, L, P>> + Send + Sync,
CM: 'static + Deref<Target = ChannelManager<CW, T, ES, NS, SP, F, R, L>> + Send + Sync,
- PGS: 'static + Deref<Target = P2PGossipSync<G, CA, L>> + Send + Sync,
+ PGS: 'static + Deref<Target = P2PGossipSync<G, UL, L>> + Send + Sync,
RGS: 'static + Deref<Target = RapidGossipSync<G, L>> + Send,
UMH: 'static + Deref + Send + Sync,
PM: 'static + Deref<Target = PeerManager<Descriptor, CMH, RMH, OMH, L, UMH, NS>> + Send + Sync,
SC: for <'b> WriteableScore<'b>,
>(
persister: PS, event_handler: EH, chain_monitor: M, channel_manager: CM,
- gossip_sync: GossipSync<PGS, RGS, G, CA, L>, peer_manager: PM, logger: L, scorer: Option<S>,
+ gossip_sync: GossipSync<PGS, RGS, G, UL, L>, peer_manager: PM, logger: L, scorer: Option<S>,
) -> Self
where
- CA::Target: 'static + chain::Access,
+ UL::Target: 'static + UtxoLookup,
CF::Target: 'static + chain::Filter,
CW::Target: 'static + chain::Watch<<SP::Target as SignerProvider>::Signer>,
T::Target: 'static + BroadcasterInterface,
//! type FeeEstimator = dyn lightning::chain::chaininterface::FeeEstimator + Send + Sync;
//! type Logger = dyn lightning::util::logger::Logger + Send + Sync;
//! type NodeSigner = dyn lightning::chain::keysinterface::NodeSigner + Send + Sync;
-//! type ChainAccess = dyn lightning::chain::Access + Send + Sync;
+//! type UtxoLookup = dyn lightning::routing::utxo::UtxoLookup + Send + Sync;
//! type ChainFilter = dyn lightning::chain::Filter + Send + Sync;
//! type DataPersister = dyn lightning::chain::chainmonitor::Persist<lightning::chain::keysinterface::InMemorySigner> + Send + Sync;
//! type ChainMonitor = lightning::chain::chainmonitor::ChainMonitor<lightning::chain::keysinterface::InMemorySigner, Arc<ChainFilter>, Arc<TxBroadcaster>, Arc<FeeEstimator>, Arc<Logger>, Arc<DataPersister>>;
//! type ChannelManager = Arc<lightning::ln::channelmanager::SimpleArcChannelManager<ChainMonitor, TxBroadcaster, FeeEstimator, Logger>>;
-//! type PeerManager = Arc<lightning::ln::peer_handler::SimpleArcPeerManager<lightning_net_tokio::SocketDescriptor, ChainMonitor, TxBroadcaster, FeeEstimator, ChainAccess, Logger>>;
+//! type PeerManager = Arc<lightning::ln::peer_handler::SimpleArcPeerManager<lightning_net_tokio::SocketDescriptor, ChainMonitor, TxBroadcaster, FeeEstimator, UtxoLookup, Logger>>;
//!
//! // Connect to node with pubkey their_node_id at addr:
//! async fn connect_to_node(peer_manager: PeerManager, chain_monitor: Arc<ChainMonitor>, channel_manager: ChannelManager, their_node_id: PublicKey, addr: SocketAddr) {
use bitcoin::blockdata::block::{Block, BlockHeader};
use bitcoin::blockdata::constants::genesis_block;
use bitcoin::blockdata::script::Script;
-use bitcoin::blockdata::transaction::TxOut;
use bitcoin::hash_types::{BlockHash, Txid};
use bitcoin::network::constants::Network;
use bitcoin::secp256k1::PublicKey;
pub fn height(&self) -> u32 { self.height }
}
-/// An error when accessing the chain via [`Access`].
-#[derive(Clone, Debug)]
-pub enum AccessError {
- /// The requested chain is unknown.
- UnknownChain,
-
- /// The requested transaction doesn't exist or hasn't confirmed.
- UnknownTx,
-}
-
-/// The `Access` trait defines behavior for accessing chain data and state, such as blocks and
-/// UTXOs.
-pub trait Access {
- /// Returns the transaction output of a funding transaction encoded by [`short_channel_id`].
- /// Returns an error if `genesis_hash` is for a different chain or if such a transaction output
- /// is unknown.
- ///
- /// [`short_channel_id`]: https://github.com/lightning/bolts/blob/master/07-routing-gossip.md#definition-of-short_channel_id
- fn get_utxo(&self, genesis_hash: &BlockHash, short_channel_id: u64) -> Result<TxOut, AccessError>;
-}
/// The `Listen` trait is used to notify when blocks have been connected or disconnected from the
/// chain.
use bitcoin::blockdata::transaction::TxOut;
use bitcoin::hash_types::BlockHash;
-use crate::chain;
-use crate::chain::Access;
use crate::ln::chan_utils::make_funding_redeemscript_from_slices;
use crate::ln::features::{ChannelFeatures, NodeFeatures, InitFeatures};
use crate::ln::msgs::{DecodeError, ErrorAction, Init, LightningError, RoutingMessageHandler, NetAddress, MAX_VALUE_MSAT};
use crate::ln::msgs::{ChannelAnnouncement, ChannelUpdate, NodeAnnouncement, GossipTimestampFilter};
use crate::ln::msgs::{QueryChannelRange, ReplyChannelRange, QueryShortChannelIds, ReplyShortChannelIdsEnd};
use crate::ln::msgs;
+use crate::routing::utxo::{UtxoLookup, UtxoLookupError};
use crate::util::ser::{Readable, ReadableArgs, Writeable, Writer, MaybeReadable};
use crate::util::logger::{Logger, Level};
use crate::util::events::{MessageSendEvent, MessageSendEventsProvider};
/// This network graph is then used for routing payments.
/// Provides interface to help with initial routing sync by
/// serving historical announcements.
-pub struct P2PGossipSync<G: Deref<Target=NetworkGraph<L>>, C: Deref, L: Deref>
-where C::Target: chain::Access, L::Target: Logger
+pub struct P2PGossipSync<G: Deref<Target=NetworkGraph<L>>, U: Deref, L: Deref>
+where U::Target: UtxoLookup, L::Target: Logger
{
network_graph: G,
- chain_access: Option<C>,
+ utxo_lookup: Option<U>,
#[cfg(feature = "std")]
full_syncs_requested: AtomicUsize,
pending_events: Mutex<Vec<MessageSendEvent>>,
logger: L,
}
-impl<G: Deref<Target=NetworkGraph<L>>, C: Deref, L: Deref> P2PGossipSync<G, C, L>
-where C::Target: chain::Access, L::Target: Logger
+impl<G: Deref<Target=NetworkGraph<L>>, U: Deref, L: Deref> P2PGossipSync<G, U, L>
+where U::Target: UtxoLookup, L::Target: Logger
{
/// Creates a new tracker of the actual state of the network of channels and nodes,
/// assuming an existing Network Graph.
- /// Chain monitor is used to make sure announced channels exist on-chain,
- /// channel data is correct, and that the announcement is signed with
- /// channel owners' keys.
- pub fn new(network_graph: G, chain_access: Option<C>, logger: L) -> Self {
+ /// UTXO lookup is used to make sure announced channels exist on-chain, channel data is
+ /// correct, and the announcement is signed with channel owners' keys.
+ pub fn new(network_graph: G, utxo_lookup: Option<U>, logger: L) -> Self {
P2PGossipSync {
network_graph,
#[cfg(feature = "std")]
full_syncs_requested: AtomicUsize::new(0),
- chain_access,
+ utxo_lookup,
pending_events: Mutex::new(vec![]),
logger,
}
/// Adds a provider used to check new announcements. Does not affect
/// existing announcements unless they are updated.
/// Add, update or remove the provider would replace the current one.
- pub fn add_chain_access(&mut self, chain_access: Option<C>) {
- self.chain_access = chain_access;
+ pub fn add_utxo_lookup(&mut self, utxo_lookup: Option<U>) {
+ self.utxo_lookup = utxo_lookup;
}
/// Gets a reference to the underlying [`NetworkGraph`] which was provided in
}
}
-impl<G: Deref<Target=NetworkGraph<L>>, C: Deref, L: Deref> RoutingMessageHandler for P2PGossipSync<G, C, L>
-where C::Target: chain::Access, L::Target: Logger
+impl<G: Deref<Target=NetworkGraph<L>>, U: Deref, L: Deref> RoutingMessageHandler for P2PGossipSync<G, U, L>
+where U::Target: UtxoLookup, L::Target: Logger
{
fn handle_node_announcement(&self, msg: &msgs::NodeAnnouncement) -> Result<bool, LightningError> {
self.network_graph.update_node_from_announcement(msg)?;
}
fn handle_channel_announcement(&self, msg: &msgs::ChannelAnnouncement) -> Result<bool, LightningError> {
- self.network_graph.update_channel_from_announcement(msg, &self.chain_access)?;
+ self.network_graph.update_channel_from_announcement(msg, &self.utxo_lookup)?;
log_gossip!(self.logger, "Added channel_announcement for {}{}", msg.contents.short_channel_id, if !msg.contents.excess_data.is_empty() { " with excess uninterpreted data!" } else { "" });
Ok(msg.contents.excess_data.len() <= MAX_EXCESS_BYTES_FOR_RELAY)
}
}
}
-impl<G: Deref<Target=NetworkGraph<L>>, C: Deref, L: Deref> MessageSendEventsProvider for P2PGossipSync<G, C, L>
+impl<G: Deref<Target=NetworkGraph<L>>, U: Deref, L: Deref> MessageSendEventsProvider for P2PGossipSync<G, U, L>
where
- C::Target: chain::Access,
+ U::Target: UtxoLookup,
L::Target: Logger,
{
fn get_and_clear_pending_msg_events(&self) -> Vec<MessageSendEvent> {
/// RoutingMessageHandler implementation to call it indirectly. This may be useful to accept
/// routing messages from a source using a protocol other than the lightning P2P protocol.
///
- /// If a `chain::Access` object is provided via `chain_access`, it will be called to verify
+ /// If a [`UtxoLookup`] object is provided via `utxo_lookup`, it will be called to verify
/// the corresponding UTXO exists on chain and is correctly-formatted.
- pub fn update_channel_from_announcement<C: Deref>(
- &self, msg: &msgs::ChannelAnnouncement, chain_access: &Option<C>,
+ pub fn update_channel_from_announcement<U: Deref>(
+ &self, msg: &msgs::ChannelAnnouncement, utxo_lookup: &Option<U>,
) -> Result<(), LightningError>
where
- C::Target: chain::Access,
+ U::Target: UtxoLookup,
{
let msg_hash = hash_to_message!(&Sha256dHash::hash(&msg.contents.encode()[..])[..]);
secp_verify_sig!(self.secp_ctx, &msg_hash, &msg.node_signature_1, &get_pubkey_from_node_id!(msg.contents.node_id_1, "channel_announcement"), "channel_announcement");
secp_verify_sig!(self.secp_ctx, &msg_hash, &msg.node_signature_2, &get_pubkey_from_node_id!(msg.contents.node_id_2, "channel_announcement"), "channel_announcement");
secp_verify_sig!(self.secp_ctx, &msg_hash, &msg.bitcoin_signature_1, &get_pubkey_from_node_id!(msg.contents.bitcoin_key_1, "channel_announcement"), "channel_announcement");
secp_verify_sig!(self.secp_ctx, &msg_hash, &msg.bitcoin_signature_2, &get_pubkey_from_node_id!(msg.contents.bitcoin_key_2, "channel_announcement"), "channel_announcement");
- self.update_channel_from_unsigned_announcement_intern(&msg.contents, Some(msg), chain_access)
+ self.update_channel_from_unsigned_announcement_intern(&msg.contents, Some(msg), utxo_lookup)
}
/// Store or update channel info from a channel announcement without verifying the associated
/// signatures. Because we aren't given the associated signatures here we cannot relay the
/// channel announcement to any of our peers.
///
- /// If a `chain::Access` object is provided via `chain_access`, it will be called to verify
+ /// If a [`UtxoLookup`] object is provided via `utxo_lookup`, it will be called to verify
/// the corresponding UTXO exists on chain and is correctly-formatted.
- pub fn update_channel_from_unsigned_announcement<C: Deref>(
- &self, msg: &msgs::UnsignedChannelAnnouncement, chain_access: &Option<C>
+ pub fn update_channel_from_unsigned_announcement<U: Deref>(
+ &self, msg: &msgs::UnsignedChannelAnnouncement, utxo_lookup: &Option<U>
) -> Result<(), LightningError>
where
- C::Target: chain::Access,
+ U::Target: UtxoLookup,
{
- self.update_channel_from_unsigned_announcement_intern(msg, None, chain_access)
+ self.update_channel_from_unsigned_announcement_intern(msg, None, utxo_lookup)
}
/// Update channel from partial announcement data received via rapid gossip sync
Ok(())
}
- fn update_channel_from_unsigned_announcement_intern<C: Deref>(
- &self, msg: &msgs::UnsignedChannelAnnouncement, full_msg: Option<&msgs::ChannelAnnouncement>, chain_access: &Option<C>
+ fn update_channel_from_unsigned_announcement_intern<U: Deref>(
+ &self, msg: &msgs::UnsignedChannelAnnouncement, full_msg: Option<&msgs::ChannelAnnouncement>, utxo_lookup: &Option<U>
) -> Result<(), LightningError>
where
- C::Target: chain::Access,
+ U::Target: UtxoLookup,
{
if msg.node_id_1 == msg.node_id_2 || msg.bitcoin_key_1 == msg.bitcoin_key_2 {
return Err(LightningError{err: "Channel announcement node had a channel with itself".to_owned(), action: ErrorAction::IgnoreError});
action: ErrorAction::IgnoreDuplicateGossip
});
}
- } else if chain_access.is_none() {
+ } else if utxo_lookup.is_none() {
// Similarly, if we can't check the chain right now anyway, ignore the
// duplicate announcement without bothering to take the channels write lock.
return Err(LightningError {
}
}
- let utxo_value = match &chain_access {
+ let utxo_value = match &utxo_lookup {
&None => {
// Tentatively accept, potentially exposing us to DoS attacks
None
},
- &Some(ref chain_access) => {
- match chain_access.get_utxo(&msg.chain_hash, msg.short_channel_id) {
+ &Some(ref utxo_lookup) => {
+ match utxo_lookup.get_utxo(&msg.chain_hash, msg.short_channel_id) {
Ok(TxOut { value, script_pubkey }) => {
let expected_script =
make_funding_redeemscript_from_slices(msg.bitcoin_key_1.as_slice(), msg.bitcoin_key_2.as_slice()).to_v0_p2wsh();
//to the new HTLC max field in channel_update
Some(value)
},
- Err(chain::AccessError::UnknownChain) => {
+ Err(UtxoLookupError::UnknownChain) => {
return Err(LightningError{err: format!("Channel announced on an unknown chain ({})", msg.chain_hash.encode().to_hex()), action: ErrorAction::IgnoreError});
},
- Err(chain::AccessError::UnknownTx) => {
+ Err(UtxoLookupError::UnknownTx) => {
return Err(LightningError{err: "Channel announced without corresponding UTXO entry".to_owned(), action: ErrorAction::IgnoreError});
},
}
#[cfg(test)]
mod tests {
- use crate::chain;
use crate::ln::channelmanager;
use crate::ln::chan_utils::make_funding_redeemscript;
#[cfg(feature = "std")]
use crate::ln::features::InitFeatures;
use crate::routing::gossip::{P2PGossipSync, NetworkGraph, NetworkUpdate, NodeAlias, MAX_EXCESS_BYTES_FOR_RELAY, NodeId, RoutingFees, ChannelUpdateInfo, ChannelInfo, NodeAnnouncementInfo, NodeInfo};
+ use crate::routing::utxo::UtxoLookupError;
use crate::ln::msgs::{RoutingMessageHandler, UnsignedNodeAnnouncement, NodeAnnouncement,
UnsignedChannelAnnouncement, ChannelAnnouncement, UnsignedChannelUpdate, ChannelUpdate,
ReplyChannelRange, QueryChannelRange, QueryShortChannelIds, MAX_VALUE_MSAT};
// Test if an associated transaction were not on-chain (or not confirmed).
let chain_source = test_utils::TestChainSource::new(Network::Testnet);
- *chain_source.utxo_ret.lock().unwrap() = Err(chain::AccessError::UnknownTx);
+ *chain_source.utxo_ret.lock().unwrap() = Err(UtxoLookupError::UnknownTx);
let network_graph = NetworkGraph::new(genesis_hash, &logger);
gossip_sync = P2PGossipSync::new(&network_graph, Some(&chain_source), &logger);
//! Structs and impls for receiving messages about the network and storing the topology live here.
+pub mod utxo;
pub mod gossip;
pub mod router;
pub mod scoring;
.push_opcode(opcodes::all::OP_CHECKMULTISIG).into_script().to_v0_p2wsh();
*chain_monitor.utxo_ret.lock().unwrap() = Ok(TxOut { value: 15, script_pubkey: good_script.clone() });
- gossip_sync.add_chain_access(Some(chain_monitor));
+ gossip_sync.add_utxo_lookup(Some(chain_monitor));
add_channel(&gossip_sync, &secp_ctx, &privkeys[0], &privkeys[2], ChannelFeatures::from_le_bytes(id_to_feature_flags(3)), 333);
update_channel(&gossip_sync, &secp_ctx, &privkeys[0], UnsignedChannelUpdate {
--- /dev/null
+// This file is Copyright its original authors, visible in version control
+// history.
+//
+// This file is licensed under the Apache License, Version 2.0 <LICENSE-APACHE
+// or http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
+// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your option.
+// You may not use this file except in accordance with one or both of these
+// licenses.
+
+//! This module contains traits for LDK to access UTXOs to check gossip data is correct.
+//!
+//! When lightning nodes gossip channel information, they resist DoS attacks by checking that each
+//! channel matches a UTXO on-chain, requiring at least some marginal on-chain transacting in
+//! order to announce a channel. This module handles that checking.
+
+use bitcoin::{BlockHash, TxOut};
+
+/// An error when accessing the chain via [`UtxoLookup`].
+#[derive(Clone, Debug)]
+pub enum UtxoLookupError {
+ /// The requested chain is unknown.
+ UnknownChain,
+
+ /// The requested transaction doesn't exist or hasn't confirmed.
+ UnknownTx,
+}
+
+/// The `UtxoLookup` trait defines behavior for accessing on-chain UTXOs.
+pub trait UtxoLookup {
+ /// Returns the transaction output of a funding transaction encoded by [`short_channel_id`].
+ /// Returns an error if `genesis_hash` is for a different chain or if such a transaction output
+ /// is unknown.
+ ///
+ /// [`short_channel_id`]: https://github.com/lightning/bolts/blob/master/07-routing-gossip.md#definition-of-short_channel_id
+ fn get_utxo(&self, genesis_hash: &BlockHash, short_channel_id: u64) -> Result<TxOut, UtxoLookupError>;
+}
use crate::ln::{msgs, wire};
use crate::ln::msgs::LightningError;
use crate::ln::script::ShutdownScript;
-use crate::routing::gossip::NetworkGraph;
-use crate::routing::gossip::NodeId;
+use crate::routing::gossip::{NetworkGraph, NodeId};
+use crate::routing::utxo::{UtxoLookup, UtxoLookupError};
use crate::routing::router::{find_route, InFlightHtlcs, Route, RouteHop, RouteParameters, Router, ScorerAccountingForInFlightHtlcs};
use crate::routing::scoring::FixedPenaltyScorer;
use crate::util::config::UserConfig;
pub struct TestChainSource {
pub genesis_hash: BlockHash,
- pub utxo_ret: Mutex<Result<TxOut, chain::AccessError>>,
+ pub utxo_ret: Mutex<Result<TxOut, UtxoLookupError>>,
pub watched_txn: Mutex<HashSet<(Txid, Script)>>,
pub watched_outputs: Mutex<HashSet<(OutPoint, Script)>>,
}
}
}
-impl chain::Access for TestChainSource {
- fn get_utxo(&self, genesis_hash: &BlockHash, _short_channel_id: u64) -> Result<TxOut, chain::AccessError> {
+impl UtxoLookup for TestChainSource {
+ fn get_utxo(&self, genesis_hash: &BlockHash, _short_channel_id: u64) -> Result<TxOut, UtxoLookupError> {
if self.genesis_hash != *genesis_hash {
- return Err(chain::AccessError::UnknownChain);
+ return Err(UtxoLookupError::UnknownChain);
}
self.utxo_ret.lock().unwrap().clone()