channel_monitors: &mut monitor_refs,
};
- (<(Sha256d, ChannelManager<EnforcingChannelKeys, Arc<TestChannelMonitor>, Arc<TestBroadcaster>>)>::read(&mut Cursor::new(&$ser.0), read_args).expect("Failed to read manager").1, monitor)
+ (<(Sha256d, ChannelManager<EnforcingChannelKeys, Arc<TestChannelMonitor>, Arc<TestBroadcaster>, Arc<KeyProvider>>)>::read(&mut Cursor::new(&$ser.0), read_args).expect("Failed to read manager").1, monitor)
} }
}
}
struct MoneyLossDetector<'a> {
- manager: Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>>>,
+ manager: Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>, Arc<KeyProvider>>>,
monitor: Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>,
- handler: PeerManager<Peer<'a>, Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>>>>,
+ handler: PeerManager<Peer<'a>, Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>, Arc<KeyProvider>>>>,
peers: &'a RefCell<[bool; 256]>,
funding_txn: Vec<Transaction>,
}
impl<'a> MoneyLossDetector<'a> {
pub fn new(peers: &'a RefCell<[bool; 256]>,
- manager: Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>>>,
+ manager: Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>, Arc<KeyProvider>>>,
monitor: Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>,
- handler: PeerManager<Peer<'a>, Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>>>>) -> Self {
+ handler: PeerManager<Peer<'a>, Arc<ChannelManager<EnforcingChannelKeys, Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>>>, Arc<TestBroadcaster>, Arc<KeyProvider>>>>) -> Self {
MoneyLossDetector {
manager,
monitor,
use std::default::Default;
use std::{cmp,mem,fmt};
use std::sync::{Arc};
+use std::ops::Deref;
#[cfg(test)]
pub struct ChannelValueStat {
}
// Constructors:
- pub fn new_outbound(fee_estimator: &FeeEstimator, keys_provider: &Arc<KeysInterface<ChanKeySigner = ChanSigner>>, their_node_id: PublicKey, channel_value_satoshis: u64, push_msat: u64, user_id: u64, logger: Arc<Logger>, config: &UserConfig) -> Result<Channel<ChanSigner>, APIError> {
+ pub fn new_outbound<K: Deref>(fee_estimator: &FeeEstimator, keys_provider: &K, their_node_id: PublicKey, channel_value_satoshis: u64, push_msat: u64, user_id: u64, logger: Arc<Logger>, config: &UserConfig) -> Result<Channel<ChanSigner>, APIError>
+ where K::Target: KeysInterface<ChanKeySigner = ChanSigner>
+ {
let chan_keys = keys_provider.get_channel_keys(false, channel_value_satoshis);
if channel_value_satoshis >= MAX_FUNDING_SATOSHIS {
/// Creates a new channel from a remote sides' request for one.
/// Assumes chain_hash has already been checked and corresponds with what we expect!
- pub fn new_from_req(fee_estimator: &FeeEstimator, keys_provider: &Arc<KeysInterface<ChanKeySigner = ChanSigner>>, their_node_id: PublicKey, their_features: InitFeatures, msg: &msgs::OpenChannel, user_id: u64, logger: Arc<Logger>, config: &UserConfig) -> Result<Channel<ChanSigner>, ChannelError> {
+ pub fn new_from_req<K: Deref>(fee_estimator: &FeeEstimator, keys_provider: &K, their_node_id: PublicKey, their_features: InitFeatures, msg: &msgs::OpenChannel, user_id: u64, logger: Arc<Logger>, config: &UserConfig) -> Result<Channel<ChanSigner>, ChannelError>
+ where K::Target: KeysInterface<ChanKeySigner = ChanSigner>
+ {
let mut chan_keys = keys_provider.get_channel_keys(true, msg.funding_satoshis);
let their_pubkeys = ChannelPublicKeys {
funding_pubkey: msg.funding_pubkey,
use ln::msgs;
use ln::onion_utils;
use ln::msgs::{ChannelMessageHandler, DecodeError, LightningError};
-use chain::keysinterface::{ChannelKeys, KeysInterface, InMemoryChannelKeys};
+use chain::keysinterface::{ChannelKeys, KeysInterface, KeysManager, InMemoryChannelKeys};
use util::config::UserConfig;
use util::{byte_utils, events};
use util::ser::{Readable, ReadableArgs, Writeable, Writer};
/// when you're using lightning-net-tokio (since tokio::spawn requires parameters with static
/// lifetimes). Other times you can afford a reference, which is more efficient, in which case
/// SimpleRefChannelManager is the more appropriate type. Defining these type aliases prevents
-/// issues such as overly long function definitions.
-pub type SimpleArcChannelManager<M, T> = Arc<ChannelManager<InMemoryChannelKeys, Arc<M>, Arc<T>>>;
+/// issues such as overly long function definitions. Note that the ChannelManager can take any
+/// type that implements KeysInterface for its keys manager, but this type alias chooses the
+/// concrete type of the KeysManager.
+pub type SimpleArcChannelManager<M, T> = Arc<ChannelManager<InMemoryChannelKeys, Arc<M>, Arc<T>, Arc<KeysManager>>>;
/// SimpleRefChannelManager is a type alias for a ChannelManager reference, and is the reference
/// counterpart to the SimpleArcChannelManager type alias. Use this type by default when you don't
/// need a ChannelManager with a static lifetime. You'll need a static lifetime in cases such as
/// usage of lightning-net-tokio (since tokio::spawn requires parameters with static lifetimes).
/// But if this is not necessary, using a reference is more efficient. Defining these type aliases
-/// helps with issues such as long function definitions.
-pub type SimpleRefChannelManager<'a, 'b, M, T> = ChannelManager<InMemoryChannelKeys, &'a M, &'b T>;
+/// helps with issues such as long function definitions. Note that the ChannelManager can take any
+/// type that implements KeysInterface for its keys manager, but this type alias chooses the
+/// concrete type of the KeysManager.
+pub type SimpleRefChannelManager<'a, 'b, 'c, M, T> = ChannelManager<InMemoryChannelKeys, &'a M, &'b T, &'c KeysManager>;
/// Manager which keeps track of a number of channels and sends messages to the appropriate
/// channel, also tracking HTLC preimages and forwarding onion packets appropriately.
/// essentially you should default to using a SimpleRefChannelManager, and use a
/// SimpleArcChannelManager when you require a ChannelManager with a static lifetime, such as when
/// you're using lightning-net-tokio.
-pub struct ChannelManager<ChanSigner: ChannelKeys, M: Deref, T: Deref>
+pub struct ChannelManager<ChanSigner: ChannelKeys, M: Deref, T: Deref, K: Deref>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
default_configuration: UserConfig,
genesis_hash: Sha256dHash,
/// Taken first everywhere where we are making changes before any other locks.
total_consistency_lock: RwLock<()>,
- keys_manager: Arc<KeysInterface<ChanKeySigner = ChanSigner>>,
+ keys_manager: K,
logger: Arc<Logger>,
}
}
}
-impl<ChanSigner: ChannelKeys, M: Deref, T: Deref> ChannelManager<ChanSigner, M, T>
+impl<ChanSigner: ChannelKeys, M: Deref, T: Deref, K: Deref> ChannelManager<ChanSigner, M, T, K>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
/// Constructs a new ChannelManager to hold several channels and route between them.
///
/// the ChannelManager as a listener to the BlockNotifier and call the BlockNotifier's
/// `block_(dis)connected` methods, which will notify all registered listeners in one
/// go.
- pub fn new(network: Network, feeest: Arc<FeeEstimator>, monitor: M, tx_broadcaster: T, logger: Arc<Logger>,keys_manager: Arc<KeysInterface<ChanKeySigner = ChanSigner>>, config: UserConfig, current_blockchain_height: usize) -> Result<ChannelManager<ChanSigner, M, T>, secp256k1::Error> {
+ pub fn new(network: Network, feeest: Arc<FeeEstimator>, monitor: M, tx_broadcaster: T, logger: Arc<Logger>, keys_manager: K, config: UserConfig, current_blockchain_height: usize) -> Result<ChannelManager<ChanSigner, M, T, K>, secp256k1::Error> {
let secp_ctx = Secp256k1::new();
let res = ChannelManager {
}
}
-impl<ChanSigner: ChannelKeys, M: Deref, T: Deref> events::MessageSendEventsProvider for ChannelManager<ChanSigner, M, T>
+impl<ChanSigner: ChannelKeys, M: Deref, T: Deref, K: Deref> events::MessageSendEventsProvider for ChannelManager<ChanSigner, M, T, K>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
fn get_and_clear_pending_msg_events(&self) -> Vec<events::MessageSendEvent> {
// TODO: Event release to users and serialization is currently race-y: it's very easy for a
}
}
-impl<ChanSigner: ChannelKeys, M: Deref, T: Deref> events::EventsProvider for ChannelManager<ChanSigner, M, T>
+impl<ChanSigner: ChannelKeys, M: Deref, T: Deref, K: Deref> events::EventsProvider for ChannelManager<ChanSigner, M, T, K>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
fn get_and_clear_pending_events(&self) -> Vec<events::Event> {
// TODO: Event release to users and serialization is currently race-y: it's very easy for a
}
}
-impl<ChanSigner: ChannelKeys, M: Deref + Sync + Send, T: Deref + Sync + Send> ChainListener for ChannelManager<ChanSigner, M, T>
+impl<ChanSigner: ChannelKeys, M: Deref + Sync + Send, T: Deref + Sync + Send, K: Deref + Sync + Send>
+ ChainListener for ChannelManager<ChanSigner, M, T, K>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
fn block_connected(&self, header: &BlockHeader, height: u32, txn_matched: &[&Transaction], indexes_of_txn_matched: &[u32]) {
let header_hash = header.bitcoin_hash();
}
}
-impl<ChanSigner: ChannelKeys, M: Deref + Sync + Send, T: Deref + Sync + Send> ChannelMessageHandler for ChannelManager<ChanSigner, M, T>
+impl<ChanSigner: ChannelKeys, M: Deref + Sync + Send, T: Deref + Sync + Send, K: Deref + Sync + Send>
+ ChannelMessageHandler for ChannelManager<ChanSigner, M, T, K>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
fn handle_open_channel(&self, their_node_id: &PublicKey, their_features: InitFeatures, msg: &msgs::OpenChannel) {
let _ = self.total_consistency_lock.read().unwrap();
}
}
-impl<ChanSigner: ChannelKeys + Writeable, M: Deref, T: Deref> Writeable for ChannelManager<ChanSigner, M, T>
+impl<ChanSigner: ChannelKeys + Writeable, M: Deref, T: Deref, K: Deref> Writeable for ChannelManager<ChanSigner, M, T, K>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
fn write<W: Writer>(&self, writer: &mut W) -> Result<(), ::std::io::Error> {
let _ = self.total_consistency_lock.write().unwrap();
/// 5) Move the ChannelMonitors into your local ManyChannelMonitor.
/// 6) Disconnect/connect blocks on the ChannelManager.
/// 7) Register the new ChannelManager with your ChainWatchInterface.
-pub struct ChannelManagerReadArgs<'a, ChanSigner: 'a + ChannelKeys, M: Deref, T: Deref>
+pub struct ChannelManagerReadArgs<'a, ChanSigner: 'a + ChannelKeys, M: Deref, T: Deref, K: Deref>
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
/// The keys provider which will give us relevant keys. Some keys will be loaded during
/// deserialization.
- pub keys_manager: Arc<KeysInterface<ChanKeySigner = ChanSigner>>,
+ pub keys_manager: K,
/// The fee_estimator for use in the ChannelManager in the future.
///
pub channel_monitors: &'a mut HashMap<OutPoint, &'a mut ChannelMonitor<ChanSigner>>,
}
-impl<'a, R : ::std::io::Read, ChanSigner: ChannelKeys + Readable<R>, M: Deref, T: Deref> ReadableArgs<R, ChannelManagerReadArgs<'a, ChanSigner, M, T>> for (Sha256dHash, ChannelManager<ChanSigner, M, T>)
+impl<'a, R : ::std::io::Read, ChanSigner: ChannelKeys + Readable<R>, M: Deref, T: Deref, K: Deref>
+ ReadableArgs<R, ChannelManagerReadArgs<'a, ChanSigner, M, T, K>> for (Sha256dHash, ChannelManager<ChanSigner, M, T, K>)
where M::Target: ManyChannelMonitor<ChanSigner>,
T::Target: BroadcasterInterface,
+ K::Target: KeysInterface<ChanKeySigner = ChanSigner>,
{
- fn read(reader: &mut R, args: ChannelManagerReadArgs<'a, ChanSigner, M, T>) -> Result<Self, DecodeError> {
+ fn read(reader: &mut R, args: ChannelManagerReadArgs<'a, ChanSigner, M, T, K>) -> Result<Self, DecodeError> {
let _ver: u8 = Readable::read(reader)?;
let min_ver: u8 = Readable::read(reader)?;
if min_ver > SERIALIZATION_VERSION {
pub tx_broadcaster: &'a test_utils::TestBroadcaster,
pub fee_estimator: Arc<test_utils::TestFeeEstimator>,
pub chan_monitor: test_utils::TestChannelMonitor<'a>,
- pub keys_manager: Arc<test_utils::TestKeysInterface>,
+ pub keys_manager: test_utils::TestKeysInterface,
pub logger: Arc<test_utils::TestLogger>,
pub node_seed: [u8; 32],
}
pub chain_monitor: Arc<chaininterface::ChainWatchInterfaceUtil>,
pub tx_broadcaster: &'c test_utils::TestBroadcaster,
pub chan_monitor: &'b test_utils::TestChannelMonitor<'c>,
- pub keys_manager: Arc<test_utils::TestKeysInterface>,
- pub node: &'a ChannelManager<EnforcingChannelKeys, &'b TestChannelMonitor<'c>, &'c test_utils::TestBroadcaster>,
+ pub keys_manager: &'b test_utils::TestKeysInterface,
+ pub node: &'a ChannelManager<EnforcingChannelKeys, &'b TestChannelMonitor<'c>, &'c test_utils::TestBroadcaster, &'b test_utils::TestKeysInterface>,
pub router: Router,
pub node_seed: [u8; 32],
pub network_payment_count: Rc<RefCell<u8>>,
let mut w = test_utils::TestVecWriter(Vec::new());
self.node.write(&mut w).unwrap();
- <(Sha256d, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>)>::read(&mut ::std::io::Cursor::new(w.0), ChannelManagerReadArgs {
+ <(Sha256d, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>)>::read(&mut ::std::io::Cursor::new(w.0), ChannelManagerReadArgs {
default_config: UserConfig::default(),
- keys_manager: self.keys_manager.clone(),
+ keys_manager: self.keys_manager,
fee_estimator: Arc::new(test_utils::TestFeeEstimator { sat_per_kw: 253 }),
monitor: self.chan_monitor,
tx_broadcaster: self.tx_broadcaster.clone(),
pub fn create_chanmon_cfgs(node_count: usize) -> Vec<TestChanMonCfg> {
let mut chan_mon_cfgs = Vec::new();
- for i in 0..node_count {
+ for _ in 0..node_count {
let tx_broadcaster = test_utils::TestBroadcaster{txn_broadcasted: Mutex::new(Vec::new()), broadcasted_txn: Mutex::new(HashSet::new())};
chan_mon_cfgs.push(TestChanMonCfg{ tx_broadcaster });
}
let chain_monitor = Arc::new(chaininterface::ChainWatchInterfaceUtil::new(Network::Testnet, logger.clone() as Arc<Logger>));
let mut seed = [0; 32];
rng.fill_bytes(&mut seed);
- let keys_manager = Arc::new(test_utils::TestKeysInterface::new(&seed, Network::Testnet, logger.clone() as Arc<Logger>));
+ let keys_manager = test_utils::TestKeysInterface::new(&seed, Network::Testnet, logger.clone() as Arc<Logger>);
let chan_monitor = test_utils::TestChannelMonitor::new(chain_monitor.clone(), &chanmon_cfgs[i].tx_broadcaster, logger.clone(), fee_estimator.clone());
nodes.push(NodeCfg { chain_monitor, logger, tx_broadcaster: &chanmon_cfgs[i].tx_broadcaster, fee_estimator, chan_monitor, keys_manager, node_seed: seed });
}
nodes
}
-pub fn create_node_chanmgrs<'a, 'b>(node_count: usize, cfgs: &'a Vec<NodeCfg<'b>>, node_config: &[Option<UserConfig>]) -> Vec<ChannelManager<EnforcingChannelKeys, &'a TestChannelMonitor<'b>, &'b test_utils::TestBroadcaster>> {
+pub fn create_node_chanmgrs<'a, 'b>(node_count: usize, cfgs: &'a Vec<NodeCfg<'b>>, node_config: &[Option<UserConfig>]) -> Vec<ChannelManager<EnforcingChannelKeys, &'a TestChannelMonitor<'b>, &'b test_utils::TestBroadcaster, &'a test_utils::TestKeysInterface>> {
let mut chanmgrs = Vec::new();
for i in 0..node_count {
let mut default_config = UserConfig::default();
default_config.channel_options.announced_channel = true;
default_config.peer_channel_config_limits.force_announced_channel_preference = false;
- let node = ChannelManager::new(Network::Testnet, cfgs[i].fee_estimator.clone(), &cfgs[i].chan_monitor, cfgs[i].tx_broadcaster, cfgs[i].logger.clone(), cfgs[i].keys_manager.clone(), if node_config[i].is_some() { node_config[i].clone().unwrap() } else { default_config }, 0).unwrap();
+ let node = ChannelManager::new(Network::Testnet, cfgs[i].fee_estimator.clone(), &cfgs[i].chan_monitor, cfgs[i].tx_broadcaster, cfgs[i].logger.clone(), &cfgs[i].keys_manager, if node_config[i].is_some() { node_config[i].clone().unwrap() } else { default_config }, 0).unwrap();
chanmgrs.push(node);
}
chanmgrs
}
-pub fn create_network<'a, 'b: 'a, 'c: 'b>(node_count: usize, cfgs: &'b Vec<NodeCfg<'c>>, chan_mgrs: &'a Vec<ChannelManager<EnforcingChannelKeys, &'b TestChannelMonitor<'c>, &'c test_utils::TestBroadcaster>>) -> Vec<Node<'a, 'b, 'c>> {
+pub fn create_network<'a, 'b: 'a, 'c: 'b>(node_count: usize, cfgs: &'b Vec<NodeCfg<'c>>, chan_mgrs: &'a Vec<ChannelManager<EnforcingChannelKeys, &'b TestChannelMonitor<'c>, &'c test_utils::TestBroadcaster, &'b test_utils::TestKeysInterface>>) -> Vec<Node<'a, 'b, 'c>> {
let secp_ctx = Secp256k1::new();
let mut nodes = Vec::new();
let chan_count = Rc::new(RefCell::new(0));
let router = Router::new(PublicKey::from_secret_key(&secp_ctx, &cfgs[i].keys_manager.get_node_secret()), cfgs[i].chain_monitor.clone(), cfgs[i].logger.clone() as Arc<Logger>);
nodes.push(Node{ chain_monitor: cfgs[i].chain_monitor.clone(), block_notifier,
tx_broadcaster: cfgs[i].tx_broadcaster, chan_monitor: &cfgs[i].chan_monitor,
- keys_manager: cfgs[i].keys_manager.clone(), node: &chan_mgrs[i], router,
+ keys_manager: &cfgs[i].keys_manager, node: &chan_mgrs[i], router,
node_seed: cfgs[i].node_seed, network_chan_count: chan_count.clone(),
network_payment_count: payment_count.clone(), logger: cfgs[i].logger.clone(),
})
let node_cfgs = create_node_cfgs(2, &chanmon_cfgs);
let node_chanmgrs = create_node_chanmgrs(2, &node_cfgs, &[None, None]);
let new_chan_monitor: test_utils::TestChannelMonitor;
- let nodes_0_deserialized: ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>;
+ let keys_manager: test_utils::TestKeysInterface;
+ let nodes_0_deserialized: ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>;
let mut nodes = create_network(2, &node_cfgs, &node_chanmgrs);
let tx = create_chan_between_nodes_with_value_init(&nodes[0], &nodes[1], 100000, 10001, InitFeatures::supported(), InitFeatures::supported());
let mut nodes_0_read = &nodes_0_serialized[..];
let config = UserConfig::default();
- let keys_manager = Arc::new(test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::new(test_utils::TestLogger::new())));
+ keys_manager = test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::new(test_utils::TestLogger::new()));
let (_, nodes_0_deserialized_tmp) = {
let mut channel_monitors = HashMap::new();
channel_monitors.insert(chan_0_monitor.get_funding_txo().unwrap(), &mut chan_0_monitor);
- <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>)>::read(&mut nodes_0_read, ChannelManagerReadArgs {
+ <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>)>::read(&mut nodes_0_read, ChannelManagerReadArgs {
default_config: config,
- keys_manager,
+ keys_manager: &keys_manager,
fee_estimator: Arc::new(test_utils::TestFeeEstimator { sat_per_kw: 253 }),
monitor: nodes[0].chan_monitor,
tx_broadcaster: nodes[0].tx_broadcaster.clone(),
let node_cfgs = create_node_cfgs(2, &chanmon_cfgs);
let node_chanmgrs = create_node_chanmgrs(2, &node_cfgs, &[None, None]);
let new_chan_monitor: test_utils::TestChannelMonitor;
- let nodes_0_deserialized: ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>;
+ let keys_manager: test_utils::TestKeysInterface;
+ let nodes_0_deserialized: ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>;
let mut nodes = create_network(2, &node_cfgs, &node_chanmgrs);
create_announced_chan_between_nodes(&nodes, 0, 1, InitFeatures::supported(), InitFeatures::supported());
assert!(chan_0_monitor_read.is_empty());
let mut nodes_0_read = &nodes_0_serialized[..];
- let keys_manager = Arc::new(test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::new(test_utils::TestLogger::new())));
+ keys_manager = test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::new(test_utils::TestLogger::new()));
let (_, nodes_0_deserialized_tmp) = {
let mut channel_monitors = HashMap::new();
channel_monitors.insert(chan_0_monitor.get_funding_txo().unwrap(), &mut chan_0_monitor);
- <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>)>::read(&mut nodes_0_read, ChannelManagerReadArgs {
+ <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>)>::read(&mut nodes_0_read, ChannelManagerReadArgs {
default_config: UserConfig::default(),
- keys_manager,
+ keys_manager: &keys_manager,
fee_estimator: Arc::new(test_utils::TestFeeEstimator { sat_per_kw: 253 }),
monitor: nodes[0].chan_monitor,
tx_broadcaster: nodes[0].tx_broadcaster.clone(),
let node_cfgs = create_node_cfgs(4, &chanmon_cfgs);
let node_chanmgrs = create_node_chanmgrs(4, &node_cfgs, &[None, None, None, None]);
let new_chan_monitor: test_utils::TestChannelMonitor;
- let nodes_0_deserialized: ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>;
+ let keys_manager: test_utils::TestKeysInterface;
+ let nodes_0_deserialized: ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>;
let mut nodes = create_network(4, &node_cfgs, &node_chanmgrs);
create_announced_chan_between_nodes(&nodes, 0, 1, InitFeatures::supported(), InitFeatures::supported());
create_announced_chan_between_nodes(&nodes, 2, 0, InitFeatures::supported(), InitFeatures::supported());
}
let mut nodes_0_read = &nodes_0_serialized[..];
- let keys_manager = Arc::new(test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::new(test_utils::TestLogger::new())));
- let (_, nodes_0_deserialized_tmp) = <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>)>::read(&mut nodes_0_read, ChannelManagerReadArgs {
+ keys_manager = test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::new(test_utils::TestLogger::new()));
+ let (_, nodes_0_deserialized_tmp) = <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>)>::read(&mut nodes_0_read, ChannelManagerReadArgs {
default_config: UserConfig::default(),
- keys_manager,
+ keys_manager: &keys_manager,
fee_estimator: Arc::new(test_utils::TestFeeEstimator { sat_per_kw: 253 }),
monitor: nodes[0].chan_monitor,
tx_broadcaster: nodes[0].tx_broadcaster.clone(),
// * we don't broadcast our Local Commitment Tx in case of fallen behind
// * we close channel in case of detecting other being fallen behind
// * we are able to claim our own outputs thanks to remote my_current_per_commitment_point
+ let keys_manager;
let tx_broadcaster;
let monitor;
let node_state_0;
let chain_monitor = Arc::new(ChainWatchInterfaceUtil::new(Network::Testnet, Arc::clone(&logger)));
tx_broadcaster = test_utils::TestBroadcaster{txn_broadcasted: Mutex::new(Vec::new()), broadcasted_txn: Mutex::new(HashSet::new())};
let feeest = Arc::new(test_utils::TestFeeEstimator { sat_per_kw: 253 });
+ keys_manager = test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::clone(&logger));
monitor = test_utils::TestChannelMonitor::new(chain_monitor.clone(), &tx_broadcaster, logger.clone(), feeest.clone());
node_state_0 = {
let mut channel_monitors = HashMap::new();
channel_monitors.insert(OutPoint { txid: chan.3.txid(), index: 0 }, &mut chan_monitor);
- <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster>)>::read(&mut ::std::io::Cursor::new(previous_node_state), ChannelManagerReadArgs {
- keys_manager: Arc::new(test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet, Arc::clone(&logger))),
+ <(Sha256dHash, ChannelManager<EnforcingChannelKeys, &test_utils::TestChannelMonitor, &test_utils::TestBroadcaster, &test_utils::TestKeysInterface>)>::read(&mut ::std::io::Cursor::new(previous_node_state), ChannelManagerReadArgs {
+ keys_manager: &keys_manager,
fee_estimator: feeest.clone(),
monitor: &monitor,
logger: Arc::clone(&logger),
/// usage of lightning-net-tokio (since tokio::spawn requires parameters with static lifetimes).
/// But if this is not necessary, using a reference is more efficient. Defining these type aliases
/// helps with issues such as long function definitions.
-pub type SimpleRefPeerManager<'a, 'b, SD, M, T> = PeerManager<SD, SimpleRefChannelManager<'a, 'b, M, T>>;
+pub type SimpleRefPeerManager<'a, 'b, 'c, SD, M, T> = PeerManager<SD, SimpleRefChannelManager<'a, 'b, 'c, M, T>>;
/// A PeerManager manages a set of peers, described by their SocketDescriptor and marshalls socket
/// events into messages which it passes on to its MessageHandlers.