Skip to content

Commit

Permalink
Parameterize ChannelManager by a Router trait
Browse files Browse the repository at this point in the history
This will be used in upcoming work to fetch routes on-the-fly for trampoline
payments.
  • Loading branch information
valentinewallace committed Oct 28, 2022
1 parent edd9358 commit 8c2e3f0
Show file tree
Hide file tree
Showing 17 changed files with 257 additions and 138 deletions.
28 changes: 24 additions & 4 deletions fuzz/src/chanmon_consistency.rs
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ use lightning::chain::keysinterface::{KeyMaterial, KeysInterface, InMemorySigner
use lightning::ln::{PaymentHash, PaymentPreimage, PaymentSecret};
use lightning::ln::channelmanager::{self, ChainParameters, ChannelManager, PaymentSendFailure, ChannelManagerReadArgs};
use lightning::ln::channel::FEE_SPIKE_BUFFER_FEE_INCREASE_MULTIPLE;
use lightning::ln::msgs::{CommitmentUpdate, ChannelMessageHandler, DecodeError, UpdateAddHTLC, Init};
use lightning::ln::msgs::{self, CommitmentUpdate, ChannelMessageHandler, DecodeError, UpdateAddHTLC, Init};
use lightning::ln::script::ShutdownScript;
use lightning::util::enforcing_trait_impls::{EnforcingSigner, EnforcementState};
use lightning::util::errors::APIError;
Expand All @@ -49,7 +49,7 @@ use lightning::util::logger::Logger;
use lightning::util::config::UserConfig;
use lightning::util::events::MessageSendEventsProvider;
use lightning::util::ser::{Readable, ReadableArgs, Writeable, Writer};
use lightning::routing::router::{Route, RouteHop};
use lightning::routing::router::{Route, RouteHop, RouteParameters};

use crate::utils::test_logger::{self, Output};
use crate::utils::test_persister::TestPersister;
Expand Down Expand Up @@ -85,6 +85,24 @@ impl FeeEstimator for FuzzEstimator {
}
}

struct FuzzRouter {}

impl channelmanager::Router for FuzzRouter {
fn find_route(
&self, _payer: &PublicKey, _params: &RouteParameters, _first_hops: Option<&[&channelmanager::ChannelDetails]>,
_inflight_htlcs: channelmanager::InFlightHtlcs
) -> Result<Route, msgs::LightningError> {
Err(msgs::LightningError {
err: String::from("Not implemented"),
action: msgs::ErrorAction::IgnoreError
})
}

fn notify_payment_path_failed(&self, _path: &[&RouteHop], _short_channel_id: u64) {}

fn notify_payment_path_successful(&self, _path: &[&RouteHop]) {}
}

pub struct TestBroadcaster {}
impl BroadcasterInterface for TestBroadcaster {
fn broadcast_transaction(&self, _tx: &Transaction) { }
Expand Down Expand Up @@ -292,7 +310,7 @@ fn check_payment_err(send_err: PaymentSendFailure) {
}
}

type ChanMan = ChannelManager<Arc<TestChainMonitor>, Arc<TestBroadcaster>, Arc<KeyProvider>, Arc<FuzzEstimator>, Arc<dyn Logger>>;
type ChanMan = ChannelManager<Arc<TestChainMonitor>, Arc<TestBroadcaster>, Arc<KeyProvider>, Arc<FuzzEstimator>, Arc<FuzzRouter>, Arc<dyn Logger>>;

#[inline]
fn get_payment_secret_hash(dest: &ChanMan, payment_id: &mut u8) -> Option<(PaymentSecret, PaymentHash)> {
Expand Down Expand Up @@ -357,6 +375,7 @@ fn send_hop_payment(source: &ChanMan, middle: &ChanMan, middle_chan_id: u64, des
pub fn do_test<Out: Output>(data: &[u8], underlying_out: Out) {
let out = SearchingOutput::new(underlying_out);
let broadcast = Arc::new(TestBroadcaster{});
let router = Arc::new(FuzzRouter {});

macro_rules! make_node {
($node_id: expr, $fee_estimator: expr) => { {
Expand All @@ -375,7 +394,7 @@ pub fn do_test<Out: Output>(data: &[u8], underlying_out: Out) {
network,
best_block: BestBlock::from_genesis(network),
};
(ChannelManager::new($fee_estimator.clone(), monitor.clone(), broadcast.clone(), Arc::clone(&logger), keys_manager.clone(), config, params),
(ChannelManager::new($fee_estimator.clone(), monitor.clone(), broadcast.clone(), router.clone(), Arc::clone(&logger), keys_manager.clone(), config, params),
monitor, keys_manager)
} }
}
Expand Down Expand Up @@ -409,6 +428,7 @@ pub fn do_test<Out: Output>(data: &[u8], underlying_out: Out) {
fee_estimator: $fee_estimator.clone(),
chain_monitor: chain_monitor.clone(),
tx_broadcaster: broadcast.clone(),
router: router.clone(),
logger,
default_config: config,
channel_monitors: monitor_refs,
Expand Down
29 changes: 24 additions & 5 deletions fuzz/src/full_stack.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,12 +35,12 @@ use lightning::chain::chainmonitor;
use lightning::chain::transaction::OutPoint;
use lightning::chain::keysinterface::{InMemorySigner, Recipient, KeyMaterial, KeysInterface};
use lightning::ln::{PaymentHash, PaymentPreimage, PaymentSecret};
use lightning::ln::channelmanager::{ChainParameters, ChannelManager};
use lightning::ln::channelmanager::{self, ChainParameters, ChannelManager};
use lightning::ln::peer_handler::{MessageHandler,PeerManager,SocketDescriptor,IgnoringMessageHandler};
use lightning::ln::msgs::DecodeError;
use lightning::ln::msgs::{self, DecodeError};
use lightning::ln::script::ShutdownScript;
use lightning::routing::gossip::{P2PGossipSync, NetworkGraph};
use lightning::routing::router::{find_route, PaymentParameters, RouteParameters};
use lightning::routing::router::{find_route, PaymentParameters, Route, RouteHop, RouteParameters};
use lightning::routing::scoring::FixedPenaltyScorer;
use lightning::util::config::UserConfig;
use lightning::util::errors::APIError;
Expand Down Expand Up @@ -127,6 +127,24 @@ impl FeeEstimator for FuzzEstimator {
}
}

struct FuzzRouter {}

impl channelmanager::Router for FuzzRouter {
fn find_route(
&self, _payer: &PublicKey, _params: &RouteParameters, _first_hops: Option<&[&channelmanager::ChannelDetails]>,
_inflight_htlcs: channelmanager::InFlightHtlcs
) -> Result<Route, msgs::LightningError> {
Err(msgs::LightningError {
err: String::from("Not implemented"),
action: msgs::ErrorAction::IgnoreError
})
}

fn notify_payment_path_failed(&self, _path: &[&RouteHop], _short_channel_id: u64) {}

fn notify_payment_path_successful(&self, _path: &[&RouteHop]) {}
}

struct TestBroadcaster {
txn_broadcasted: Mutex<Vec<Transaction>>,
}
Expand Down Expand Up @@ -164,7 +182,7 @@ impl<'a> std::hash::Hash for Peer<'a> {

type ChannelMan = ChannelManager<
Arc<chainmonitor::ChainMonitor<EnforcingSigner, Arc<dyn chain::Filter>, Arc<TestBroadcaster>, Arc<FuzzEstimator>, Arc<dyn Logger>, Arc<TestPersister>>>,
Arc<TestBroadcaster>, Arc<KeyProvider>, Arc<FuzzEstimator>, Arc<dyn Logger>>;
Arc<TestBroadcaster>, Arc<KeyProvider>, Arc<FuzzEstimator>, Arc<FuzzRouter>, Arc<dyn Logger>>;
type PeerMan<'a> = PeerManager<Peer<'a>, Arc<ChannelMan>, Arc<P2PGossipSync<Arc<NetworkGraph<Arc<dyn Logger>>>, Arc<dyn chain::Access>, Arc<dyn Logger>>>, IgnoringMessageHandler, Arc<dyn Logger>, IgnoringMessageHandler>;

struct MoneyLossDetector<'a> {
Expand Down Expand Up @@ -360,6 +378,7 @@ pub fn do_test(data: &[u8], logger: &Arc<dyn Logger>) {
let fee_est = Arc::new(FuzzEstimator {
input: input.clone(),
});
let router = Arc::new(FuzzRouter {});

macro_rules! get_slice {
($len: expr) => {
Expand Down Expand Up @@ -399,7 +418,7 @@ pub fn do_test(data: &[u8], logger: &Arc<dyn Logger>) {
network,
best_block: BestBlock::from_genesis(network),
};
let channelmanager = Arc::new(ChannelManager::new(fee_est.clone(), monitor.clone(), broadcast.clone(), Arc::clone(&logger), keys_manager.clone(), config, params));
let channelmanager = Arc::new(ChannelManager::new(fee_est.clone(), monitor.clone(), broadcast.clone(), router, Arc::clone(&logger), keys_manager.clone(), config, params));
// Adding new calls to `KeysInterface::get_secure_random_bytes` during startup can change all the
// keys subsequently generated in this test. Rather than regenerating all the messages manually,
// it's easier to just increment the counter here so the keys don't change.
Expand Down
13 changes: 8 additions & 5 deletions lightning-background-processor/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ use lightning::chain;
use lightning::chain::chaininterface::{BroadcasterInterface, FeeEstimator};
use lightning::chain::chainmonitor::{ChainMonitor, Persist};
use lightning::chain::keysinterface::{Sign, KeysInterface};
use lightning::ln::channelmanager::ChannelManager;
use lightning::ln::channelmanager::{ChannelManager, Router};
use lightning::ln::msgs::{ChannelMessageHandler, OnionMessageHandler, RoutingMessageHandler};
use lightning::ln::peer_handler::{CustomMessageHandler, PeerManager, SocketDescriptor};
use lightning::routing::gossip::{NetworkGraph, P2PGossipSync};
Expand Down Expand Up @@ -478,6 +478,7 @@ impl BackgroundProcessor {
T: 'static + Deref + Send + Sync,
K: 'static + Deref + Send + Sync,
F: 'static + Deref + Send + Sync,
R: 'static + Deref + Send + Sync,
G: 'static + Deref<Target = NetworkGraph<L>> + Send + Sync,
L: 'static + Deref + Send + Sync,
P: 'static + Deref + Send + Sync,
Expand All @@ -488,7 +489,7 @@ impl BackgroundProcessor {
EH: 'static + EventHandler + Send,
PS: 'static + Deref + Send,
M: 'static + Deref<Target = ChainMonitor<Signer, CF, T, F, L, P>> + Send + Sync,
CM: 'static + Deref<Target = ChannelManager<CW, T, K, F, L>> + Send + Sync,
CM: 'static + Deref<Target = ChannelManager<CW, T, K, F, R, L>> + Send + Sync,
PGS: 'static + Deref<Target = P2PGossipSync<G, CA, L>> + Send + Sync,
RGS: 'static + Deref<Target = RapidGossipSync<G, L>> + Send,
UMH: 'static + Deref + Send + Sync,
Expand All @@ -506,13 +507,14 @@ impl BackgroundProcessor {
T::Target: 'static + BroadcasterInterface,
K::Target: 'static + KeysInterface<Signer = Signer>,
F::Target: 'static + FeeEstimator,
R::Target: 'static + Router,
L::Target: 'static + Logger,
P::Target: 'static + Persist<Signer>,
CMH::Target: 'static + ChannelMessageHandler,
OMH::Target: 'static + OnionMessageHandler,
RMH::Target: 'static + RoutingMessageHandler,
UMH::Target: 'static + CustomMessageHandler,
PS::Target: 'static + Persister<'a, Signer, CW, T, K, F, L, SC>,
PS::Target: 'static + Persister<'a, Signer, CW, T, K, F, R, L, SC>,
{
let stop_thread = Arc::new(AtomicBool::new(false));
let stop_thread_clone = stop_thread.clone();
Expand Down Expand Up @@ -625,7 +627,7 @@ mod tests {
type RGS = Arc<RapidGossipSync<Arc<NetworkGraph<Arc<test_utils::TestLogger>>>, Arc<test_utils::TestLogger>>>;

struct Node {
node: Arc<SimpleArcChannelManager<ChainMonitor, test_utils::TestBroadcaster, test_utils::TestFeeEstimator, test_utils::TestLogger>>,
node: Arc<SimpleArcChannelManager<ChainMonitor, test_utils::TestBroadcaster, test_utils::TestFeeEstimator, test_utils::TestRouter, test_utils::TestLogger>>,
p2p_gossip_sync: PGS,
rapid_gossip_sync: RGS,
peer_manager: Arc<PeerManager<TestDescriptor, Arc<test_utils::TestChannelMessageHandler>, Arc<test_utils::TestRoutingMessageHandler>, IgnoringMessageHandler, Arc<test_utils::TestLogger>, IgnoringMessageHandler>>,
Expand Down Expand Up @@ -732,6 +734,7 @@ mod tests {
for i in 0..num_nodes {
let tx_broadcaster = Arc::new(test_utils::TestBroadcaster{txn_broadcasted: Mutex::new(Vec::new()), blocks: Arc::new(Mutex::new(Vec::new()))});
let fee_estimator = Arc::new(test_utils::TestFeeEstimator { sat_per_kw: Mutex::new(253) });
let router = Arc::new(test_utils::TestRouter { });
let chain_source = Arc::new(test_utils::TestChainSource::new(Network::Testnet));
let logger = Arc::new(test_utils::TestLogger::with_id(format!("node {}", i)));
let persister = Arc::new(FilesystemPersister::new(format!("{}_persister_{}", persist_dir, i)));
Expand All @@ -743,7 +746,7 @@ mod tests {
let chain_monitor = Arc::new(chainmonitor::ChainMonitor::new(Some(chain_source.clone()), tx_broadcaster.clone(), logger.clone(), fee_estimator.clone(), persister.clone()));
let best_block = BestBlock::from_genesis(network);
let params = ChainParameters { network, best_block };
let manager = Arc::new(ChannelManager::new(fee_estimator.clone(), chain_monitor.clone(), tx_broadcaster.clone(), logger.clone(), keys_manager.clone(), UserConfig::default(), params));
let manager = Arc::new(ChannelManager::new(fee_estimator.clone(), chain_monitor.clone(), tx_broadcaster.clone(), router.clone(), logger.clone(), keys_manager.clone(), UserConfig::default(), params));
let network_graph = Arc::new(NetworkGraph::new(genesis_block.header.block_hash(), logger.clone()));
let p2p_gossip_sync = Arc::new(P2PGossipSync::new(network_graph.clone(), Some(chain_source.clone()), logger.clone()));
let rapid_gossip_sync = Arc::new(RapidGossipSync::new(network_graph.clone()));
Expand Down
8 changes: 5 additions & 3 deletions lightning-block-sync/src/init.rs
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,7 @@ BlockSourceResult<ValidatedBlockHeader> where B::Target: BlockSource {
/// use lightning::chain::chaininterface::FeeEstimator;
/// use lightning::chain::keysinterface;
/// use lightning::chain::keysinterface::KeysInterface;
/// use lightning::ln::channelmanager::ChannelManager;
/// use lightning::ln::channelmanager::ChannelManagerReadArgs;
/// use lightning::ln::channelmanager::{ChannelManager, ChannelManagerReadArgs, Router};
/// use lightning::util::config::UserConfig;
/// use lightning::util::logger::Logger;
/// use lightning::util::ser::ReadableArgs;
Expand All @@ -65,6 +64,7 @@ BlockSourceResult<ValidatedBlockHeader> where B::Target: BlockSource {
/// S: keysinterface::Sign,
/// T: BroadcasterInterface,
/// F: FeeEstimator,
/// R: Router,
/// L: Logger,
/// C: chain::Filter,
/// P: chainmonitor::Persist<S>,
Expand All @@ -75,6 +75,7 @@ BlockSourceResult<ValidatedBlockHeader> where B::Target: BlockSource {
/// keys_manager: &K,
/// tx_broadcaster: &T,
/// fee_estimator: &F,
/// router: &R,
/// logger: &L,
/// persister: &P,
/// ) {
Expand All @@ -91,11 +92,12 @@ BlockSourceResult<ValidatedBlockHeader> where B::Target: BlockSource {
/// fee_estimator,
/// chain_monitor,
/// tx_broadcaster,
/// router,
/// logger,
/// config,
/// vec![&mut monitor],
/// );
/// <(BlockHash, ChannelManager<&ChainMonitor<S, &C, &T, &F, &L, &P>, &T, &K, &F, &L>)>::read(
/// <(BlockHash, ChannelManager<&ChainMonitor<S, &C, &T, &F, &L, &P>, &T, &K, &F, &R, &L>)>::read(
/// &mut Cursor::new(&serialized_manager), read_args).unwrap()
/// };
///
Expand Down
28 changes: 17 additions & 11 deletions lightning-invoice/src/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -235,15 +235,16 @@ where
///
/// `invoice_expiry_delta_secs` describes the number of seconds that the invoice is valid for
/// in excess of the current time.
pub fn create_invoice_from_channelmanager<M: Deref, T: Deref, K: Deref, F: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, L>, keys_manager: K, logger: L,
pub fn create_invoice_from_channelmanager<M: Deref, T: Deref, K: Deref, F: Deref, R: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, R, L>, keys_manager: K, logger: L,
network: Currency, amt_msat: Option<u64>, description: String, invoice_expiry_delta_secs: u32
) -> Result<Invoice, SignOrCreationError<()>>
where
M::Target: chain::Watch<<K::Target as KeysInterface>::Signer>,
T::Target: BroadcasterInterface,
K::Target: KeysInterface,
F::Target: FeeEstimator,
R::Target: Router,
L::Target: Logger,
{
use std::time::SystemTime;
Expand All @@ -265,8 +266,8 @@ where
///
/// `invoice_expiry_delta_secs` describes the number of seconds that the invoice is valid for
/// in excess of the current time.
pub fn create_invoice_from_channelmanager_with_description_hash<M: Deref, T: Deref, K: Deref, F: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, L>, keys_manager: K, logger: L,
pub fn create_invoice_from_channelmanager_with_description_hash<M: Deref, T: Deref, K: Deref, F: Deref, R: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, R, L>, keys_manager: K, logger: L,
network: Currency, amt_msat: Option<u64>, description_hash: Sha256,
invoice_expiry_delta_secs: u32
) -> Result<Invoice, SignOrCreationError<()>>
Expand All @@ -275,6 +276,7 @@ where
T::Target: BroadcasterInterface,
K::Target: KeysInterface,
F::Target: FeeEstimator,
R::Target: Router,
L::Target: Logger,
{
use std::time::SystemTime;
Expand All @@ -292,8 +294,8 @@ where
/// See [`create_invoice_from_channelmanager_with_description_hash`]
/// This version can be used in a `no_std` environment, where [`std::time::SystemTime`] is not
/// available and the current time is supplied by the caller.
pub fn create_invoice_from_channelmanager_with_description_hash_and_duration_since_epoch<M: Deref, T: Deref, K: Deref, F: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, L>, keys_manager: K, logger: L,
pub fn create_invoice_from_channelmanager_with_description_hash_and_duration_since_epoch<M: Deref, T: Deref, K: Deref, F: Deref, R: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, R, L>, keys_manager: K, logger: L,
network: Currency, amt_msat: Option<u64>, description_hash: Sha256,
duration_since_epoch: Duration, invoice_expiry_delta_secs: u32
) -> Result<Invoice, SignOrCreationError<()>>
Expand All @@ -302,6 +304,7 @@ where
T::Target: BroadcasterInterface,
K::Target: KeysInterface,
F::Target: FeeEstimator,
R::Target: Router,
L::Target: Logger,
{
_create_invoice_from_channelmanager_and_duration_since_epoch(
Expand All @@ -314,8 +317,8 @@ where
/// See [`create_invoice_from_channelmanager`]
/// This version can be used in a `no_std` environment, where [`std::time::SystemTime`] is not
/// available and the current time is supplied by the caller.
pub fn create_invoice_from_channelmanager_and_duration_since_epoch<M: Deref, T: Deref, K: Deref, F: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, L>, keys_manager: K, logger: L,
pub fn create_invoice_from_channelmanager_and_duration_since_epoch<M: Deref, T: Deref, K: Deref, F: Deref, R: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, R, L>, keys_manager: K, logger: L,
network: Currency, amt_msat: Option<u64>, description: String, duration_since_epoch: Duration,
invoice_expiry_delta_secs: u32
) -> Result<Invoice, SignOrCreationError<()>>
Expand All @@ -324,6 +327,7 @@ where
T::Target: BroadcasterInterface,
K::Target: KeysInterface,
F::Target: FeeEstimator,
R::Target: Router,
L::Target: Logger,
{
_create_invoice_from_channelmanager_and_duration_since_epoch(
Expand All @@ -335,8 +339,8 @@ where
)
}

fn _create_invoice_from_channelmanager_and_duration_since_epoch<M: Deref, T: Deref, K: Deref, F: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, L>, keys_manager: K, logger: L,
fn _create_invoice_from_channelmanager_and_duration_since_epoch<M: Deref, T: Deref, K: Deref, F: Deref, R: Deref, L: Deref>(
channelmanager: &ChannelManager<M, T, K, F, R, L>, keys_manager: K, logger: L,
network: Currency, amt_msat: Option<u64>, description: InvoiceDescription,
duration_since_epoch: Duration, invoice_expiry_delta_secs: u32
) -> Result<Invoice, SignOrCreationError<()>>
Expand All @@ -345,6 +349,7 @@ where
T::Target: BroadcasterInterface,
K::Target: KeysInterface,
F::Target: FeeEstimator,
R::Target: Router,
L::Target: Logger,
{
// `create_inbound_payment` only returns an error if the amount is greater than the total bitcoin
Expand Down Expand Up @@ -590,12 +595,13 @@ impl<G: Deref<Target = NetworkGraph<L>>, L: Deref, S: Deref> ProbingRouter for D
}
}

impl<M: Deref, T: Deref, K: Deref, F: Deref, L: Deref> Payer for ChannelManager<M, T, K, F, L>
impl<M: Deref, T: Deref, K: Deref, F: Deref, R: Deref, L: Deref> Payer for ChannelManager<M, T, K, F, R, L>
where
M::Target: chain::Watch<<K::Target as KeysInterface>::Signer>,
T::Target: BroadcasterInterface,
K::Target: KeysInterface,
F::Target: FeeEstimator,
R::Target: Router,
L::Target: Logger,
{
fn node_id(&self) -> PublicKey {
Expand Down
5 changes: 3 additions & 2 deletions lightning-net-tokio/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,13 +31,14 @@
//! // Define concrete types for our high-level objects:
//! type TxBroadcaster = dyn lightning::chain::chaininterface::BroadcasterInterface + Send + Sync;
//! type FeeEstimator = dyn lightning::chain::chaininterface::FeeEstimator + Send + Sync;
//! type Router = dyn lightning::ln::channelmanager::Router + Send + Sync;
//! type Logger = dyn lightning::util::logger::Logger + Send + Sync;
//! type ChainAccess = dyn lightning::chain::Access + 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 ChannelManager = Arc<lightning::ln::channelmanager::SimpleArcChannelManager<ChainMonitor, TxBroadcaster, FeeEstimator, Router, Logger>>;
//! type PeerManager = Arc<lightning::ln::peer_handler::SimpleArcPeerManager<lightning_net_tokio::SocketDescriptor, ChainMonitor, TxBroadcaster, FeeEstimator, ChainAccess, Router, 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) {
Expand Down
Loading

0 comments on commit 8c2e3f0

Please sign in to comment.