--------------- / (as EventsProvider)
| PeerManager |- \ /
--------------- \ /
- | ----------------------- \ /
- | | ChainWatchInterface | v
- | ----------------------- ---------
+ | ----------------- \ /
+ | | chain::Access | v
+ | ----------------- ---------
| | | Event |
(as RoutingMessageHandler) v ---------
\ --------------------
use bitcoin::hashes::sha256::Hash as Sha256;
use bitcoin::hash_types::{Txid, BlockHash, WPubkeyHash};
-use lightning::chain::chaininterface::{BroadcasterInterface,ConfirmationTarget,ChainListener,FeeEstimator,ChainWatchInterfaceUtil};
+use lightning::chain;
+use lightning::chain::chaininterface::{BroadcasterInterface,ConfirmationTarget,ChainListener,FeeEstimator};
use lightning::chain::transaction::OutPoint;
use lightning::chain::keysinterface::{InMemoryChannelKeys, KeysInterface};
use lightning::ln::channelmonitor;
EnforcingChannelKeys,
Arc<channelmonitor::SimpleManyChannelMonitor<OutPoint, EnforcingChannelKeys, Arc<TestBroadcaster>, Arc<FuzzEstimator>, Arc<dyn Logger>>>,
Arc<TestBroadcaster>, Arc<KeyProvider>, Arc<FuzzEstimator>, Arc<dyn Logger>>;
-type PeerMan<'a> = PeerManager<Peer<'a>, Arc<ChannelMan>, Arc<NetGraphMsgHandler<Arc<ChainWatchInterfaceUtil>, Arc<dyn Logger>>>, Arc<dyn Logger>>;
+type PeerMan<'a> = PeerManager<Peer<'a>, Arc<ChannelMan>, Arc<NetGraphMsgHandler<Arc<dyn chain::Access>, Arc<dyn Logger>>>, Arc<dyn Logger>>;
struct MoneyLossDetector<'a> {
manager: Arc<ChannelMan>,
Err(_) => return,
};
- let watch = Arc::new(ChainWatchInterfaceUtil::new(Network::Bitcoin));
let broadcast = Arc::new(TestBroadcaster{});
let monitor = Arc::new(channelmonitor::SimpleManyChannelMonitor::new(broadcast.clone(), Arc::clone(&logger), fee_est.clone()));
config.peer_channel_config_limits.min_dust_limit_satoshis = 0;
let channelmanager = Arc::new(ChannelManager::new(Network::Bitcoin, fee_est.clone(), monitor.clone(), broadcast.clone(), Arc::clone(&logger), keys_manager.clone(), config, 0));
let our_id = PublicKey::from_secret_key(&Secp256k1::signing_only(), &keys_manager.get_node_secret());
- let net_graph_msg_handler = Arc::new(NetGraphMsgHandler::new(watch.clone(), Arc::clone(&logger)));
+ let net_graph_msg_handler = Arc::new(NetGraphMsgHandler::new(None, Arc::clone(&logger)));
let peers = RefCell::new([false; 256]);
let mut loss_detector = MoneyLossDetector::new(&peers, channelmanager.clone(), monitor.clone(), PeerManager::new(MessageHandler {
-use bitcoin::blockdata::script::{Script, Builder};
-use bitcoin::blockdata::block::BlockHeader;
-use bitcoin::blockdata::transaction::Transaction;
-use bitcoin::hash_types::{Txid, BlockHash};
+use bitcoin::blockdata::script::Builder;
+use bitcoin::blockdata::transaction::TxOut;
+use bitcoin::hash_types::BlockHash;
-use lightning::chain::chaininterface::{ChainError,ChainWatchInterface};
+use lightning::chain;
use lightning::ln::channelmanager::ChannelDetails;
use lightning::ln::features::InitFeatures;
use lightning::ln::msgs;
}
}
-struct DummyChainWatcher {
+struct FuzzChainSource {
input: Arc<InputData>,
}
-
-impl ChainWatchInterface for DummyChainWatcher {
- fn install_watch_tx(&self, _txid: &Txid, _script_pub_key: &Script) { }
- fn install_watch_outpoint(&self, _outpoint: (Txid, u32), _out_script: &Script) { }
- fn watch_all_txn(&self) { }
- fn filter_block(&self, _header: &BlockHeader, _txdata: &[(usize, &Transaction)]) -> Vec<usize> {
- Vec::new()
- }
- fn reentered(&self) -> usize { 0 }
-
- fn get_chain_utxo(&self, _genesis_hash: BlockHash, _unspent_tx_output_identifier: u64) -> Result<(Script, u64), ChainError> {
+impl chain::Access for FuzzChainSource {
+ fn get_utxo(&self, _genesis_hash: &BlockHash, _short_channel_id: u64) -> Result<TxOut, chain::AccessError> {
match self.input.get_slice(2) {
- Some(&[0, _]) => Err(ChainError::NotSupported),
- Some(&[1, _]) => Err(ChainError::NotWatched),
- Some(&[2, _]) => Err(ChainError::UnknownTx),
- Some(&[_, x]) => Ok((Builder::new().push_int(x as i64).into_script().to_v0_p2wsh(), 0)),
- None => Err(ChainError::UnknownTx),
+ Some(&[0, _]) => Err(chain::AccessError::UnknownChain),
+ Some(&[1, _]) => Err(chain::AccessError::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),
_ => unreachable!(),
}
}
}
let logger: Arc<dyn Logger> = Arc::new(test_logger::TestLogger::new("".to_owned(), out));
- let chain_monitor = Arc::new(DummyChainWatcher {
+ let chain_source = Arc::new(FuzzChainSource {
input: Arc::clone(&input),
});
let our_pubkey = get_pubkey!();
- let net_graph_msg_handler = NetGraphMsgHandler::new(chain_monitor, Arc::clone(&logger));
+ let net_graph_msg_handler = NetGraphMsgHandler::new(Some(chain_source), Arc::clone(&logger));
loop {
match get_slice!(1)[0] {
//! type TxBroadcaster = dyn lightning::chain::chaininterface::BroadcasterInterface;
//! type FeeEstimator = dyn lightning::chain::chaininterface::FeeEstimator;
//! type Logger = dyn lightning::util::logger::Logger;
-//! type ChainWatchInterface = dyn lightning::chain::chaininterface::ChainWatchInterface;
+//! type ChainAccess = dyn lightning::chain::Access;
//! type ChannelMonitor = lightning::ln::channelmonitor::SimpleManyChannelMonitor<lightning::chain::transaction::OutPoint, lightning::chain::keysinterface::InMemoryChannelKeys, Arc<TxBroadcaster>, Arc<FeeEstimator>, Arc<Logger>>;
//! type ChannelManager = lightning::ln::channelmanager::SimpleArcChannelManager<ChannelMonitor, TxBroadcaster, FeeEstimator, Logger>;
-//! type PeerManager = lightning::ln::peer_handler::SimpleArcPeerManager<lightning_net_tokio::SocketDescriptor, ChannelMonitor, TxBroadcaster, FeeEstimator, ChainWatchInterface, Logger>;
+//! type PeerManager = lightning::ln::peer_handler::SimpleArcPeerManager<lightning_net_tokio::SocketDescriptor, ChannelMonitor, TxBroadcaster, FeeEstimator, ChainAccess, Logger>;
//!
//! // Connect to node with pubkey their_node_id at addr:
//! async fn connect_to_node(peer_manager: PeerManager, channel_monitor: Arc<ChannelMonitor>, channel_manager: ChannelManager, their_node_id: PublicKey, addr: SocketAddr) {
//! Structs and traits which allow other parts of rust-lightning to interact with the blockchain.
use bitcoin::blockdata::script::Script;
-use bitcoin::hash_types::Txid;
+use bitcoin::blockdata::transaction::TxOut;
+use bitcoin::hash_types::{BlockHash, Txid};
use chain::transaction::OutPoint;
pub mod transaction;
pub mod keysinterface;
+/// The `Access` trait defines behavior for accessing chain data and state, such as blocks and UTXO.
+pub trait Access: Send + Sync {
+ /// 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/lightningnetwork/lightning-rfc/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>;
+}
+
+/// An error when accessing the chain via [`Access`].
+///
+/// [`Access`]: trait.Access.html
+#[derive(Clone)]
+pub enum AccessError {
+ /// The requested chain is unknown.
+ UnknownChain,
+
+ /// The requested transaction doesn't exist or hasn't confirmed.
+ UnknownTx,
+}
+
/// An interface for providing [`WatchEvent`]s.
///
/// [`WatchEvent`]: enum.WatchEvent.html
pub struct TestChanMonCfg {
pub tx_broadcaster: test_utils::TestBroadcaster,
pub fee_estimator: test_utils::TestFeeEstimator,
- pub chain_monitor: chaininterface::ChainWatchInterfaceUtil,
+ pub chain_source: test_utils::TestChainSource,
pub logger: test_utils::TestLogger,
}
pub struct NodeCfg<'a> {
- pub chain_monitor: &'a chaininterface::ChainWatchInterfaceUtil,
+ pub chain_source: &'a test_utils::TestChainSource,
pub tx_broadcaster: &'a test_utils::TestBroadcaster,
pub fee_estimator: &'a test_utils::TestFeeEstimator,
pub chan_monitor: test_utils::TestChannelMonitor<'a>,
pub struct Node<'a, 'b: 'a, 'c: 'b> {
pub block_notifier: chaininterface::BlockNotifierRef<'a>,
- pub chain_monitor: &'c chaininterface::ChainWatchInterfaceUtil,
+ pub chain_source: &'c test_utils::TestChainSource,
pub tx_broadcaster: &'c test_utils::TestBroadcaster,
pub chan_monitor: &'b test_utils::TestChannelMonitor<'c>,
pub keys_manager: &'b test_utils::TestKeysInterface,
pub node: &'a ChannelManager<EnforcingChannelKeys, &'b TestChannelMonitor<'c>, &'c test_utils::TestBroadcaster, &'b test_utils::TestKeysInterface, &'c test_utils::TestFeeEstimator, &'c test_utils::TestLogger>,
- pub net_graph_msg_handler: NetGraphMsgHandler<&'c chaininterface::ChainWatchInterfaceUtil, &'c test_utils::TestLogger>,
+ pub net_graph_msg_handler: NetGraphMsgHandler<&'c test_utils::TestChainSource, &'c test_utils::TestLogger>,
pub node_seed: [u8; 32],
pub network_payment_count: Rc<RefCell<u8>>,
pub network_chan_count: Rc<RefCell<u32>>,
let network_graph_deser = <NetworkGraph>::read(&mut ::std::io::Cursor::new(&w.0)).unwrap();
assert!(network_graph_deser == *self.net_graph_msg_handler.network_graph.read().unwrap());
let net_graph_msg_handler = NetGraphMsgHandler::from_net_graph(
- self.chain_monitor, self.logger, network_graph_deser
+ Some(self.chain_source), self.logger, network_graph_deser
);
let mut chan_progress = 0;
loop {
for i in 0..node_count {
let tx_broadcaster = test_utils::TestBroadcaster{txn_broadcasted: Mutex::new(Vec::new())};
let fee_estimator = test_utils::TestFeeEstimator { sat_per_kw: 253 };
- let chain_monitor = chaininterface::ChainWatchInterfaceUtil::new(Network::Testnet);
+ let chain_source = test_utils::TestChainSource::new(Network::Testnet);
let logger = test_utils::TestLogger::with_id(format!("node {}", i));
- chan_mon_cfgs.push(TestChanMonCfg{ tx_broadcaster, fee_estimator, chain_monitor, logger });
+ chan_mon_cfgs.push(TestChanMonCfg{ tx_broadcaster, fee_estimator, chain_source, logger });
}
chan_mon_cfgs
let seed = [i as u8; 32];
let keys_manager = test_utils::TestKeysInterface::new(&seed, Network::Testnet);
let chan_monitor = test_utils::TestChannelMonitor::new(&chanmon_cfgs[i].tx_broadcaster, &chanmon_cfgs[i].logger, &chanmon_cfgs[i].fee_estimator);
- nodes.push(NodeCfg { chain_monitor: &chanmon_cfgs[i].chain_monitor, logger: &chanmon_cfgs[i].logger, tx_broadcaster: &chanmon_cfgs[i].tx_broadcaster, fee_estimator: &chanmon_cfgs[i].fee_estimator, chan_monitor, keys_manager, node_seed: seed });
+ nodes.push(NodeCfg { chain_source: &chanmon_cfgs[i].chain_source, logger: &chanmon_cfgs[i].logger, tx_broadcaster: &chanmon_cfgs[i].tx_broadcaster, fee_estimator: &chanmon_cfgs[i].fee_estimator, chan_monitor, keys_manager, node_seed: seed });
}
nodes
let block_notifier = chaininterface::BlockNotifier::new();
block_notifier.register_listener(&cfgs[i].chan_monitor.simple_monitor as &chaininterface::ChainListener);
block_notifier.register_listener(&chan_mgrs[i] as &chaininterface::ChainListener);
- let net_graph_msg_handler = NetGraphMsgHandler::new(cfgs[i].chain_monitor, cfgs[i].logger);
- nodes.push(Node{ chain_monitor: &cfgs[i].chain_monitor, block_notifier,
+ let net_graph_msg_handler = NetGraphMsgHandler::new(None, cfgs[i].logger);
+ nodes.push(Node{ chain_source: cfgs[i].chain_source, block_notifier,
tx_broadcaster: cfgs[i].tx_broadcaster, chan_monitor: &cfgs[i].chan_monitor,
keys_manager: &cfgs[i].keys_manager, node: &chan_mgrs[i], net_graph_msg_handler,
node_seed: cfgs[i].node_seed, network_chan_count: chan_count.clone(),
use chain::transaction::OutPoint;
use chain::keysinterface::{ChannelKeys, KeysInterface, SpendableOutputDescriptor};
-use chain::chaininterface::{ChainListener, ChainWatchInterfaceUtil, BlockNotifier};
+use chain::chaininterface::{ChainListener, BlockNotifier};
use ln::channel::{COMMITMENT_TX_BASE_WEIGHT, COMMITMENT_TX_WEIGHT_PER_HTLC};
use ln::channelmanager::{ChannelManager,ChannelManagerReadArgs,HTLCForwardInfo,RAACommitmentOrder, PaymentPreimage, PaymentHash, PaymentSecret, PaymentSendFailure, BREAKDOWN_TIMEOUT};
use ln::channelmonitor::{ChannelMonitor, CLTV_CLAIM_BUFFER, LATENCY_GRACE_PERIOD_BLOCKS, ManyChannelMonitor, ANTI_REORG_DELAY};
let seed = [42; 32];
let keys_manager = test_utils::TestKeysInterface::new(&seed, Network::Testnet);
let chan_monitor = test_utils::TestChannelMonitor::new(&chanmon_cfgs[0].tx_broadcaster, &chanmon_cfgs[0].logger, &chanmon_cfgs[0].fee_estimator);
- let node = NodeCfg { chain_monitor: &chanmon_cfgs[0].chain_monitor, logger: &chanmon_cfgs[0].logger, tx_broadcaster: &chanmon_cfgs[0].tx_broadcaster, fee_estimator: &chanmon_cfgs[0].fee_estimator, chan_monitor, keys_manager, node_seed: seed };
+ let node = NodeCfg { chain_source: &chanmon_cfgs[0].chain_source, logger: &chanmon_cfgs[0].logger, tx_broadcaster: &chanmon_cfgs[0].tx_broadcaster, fee_estimator: &chanmon_cfgs[0].fee_estimator, chan_monitor, keys_manager, node_seed: seed };
let mut node_cfgs = create_node_cfgs(3, &chanmon_cfgs);
node_cfgs.remove(0);
node_cfgs.insert(0, node);
let logger;
let fee_estimator;
let tx_broadcaster;
- let chain_monitor;
+ let chain_source;
let monitor;
let node_state_0;
let chanmon_cfgs = create_chanmon_cfgs(2);
// Restore node A from previous state
logger = test_utils::TestLogger::with_id(format!("node {}", 0));
let mut chan_monitor = <(BlockHash, ChannelMonitor<EnforcingChannelKeys>)>::read(&mut ::std::io::Cursor::new(previous_chan_monitor_state.0)).unwrap().1;
- chain_monitor = ChainWatchInterfaceUtil::new(Network::Testnet);
+ chain_source = test_utils::TestChainSource::new(Network::Testnet);
tx_broadcaster = test_utils::TestBroadcaster{txn_broadcasted: Mutex::new(Vec::new())};
fee_estimator = test_utils::TestFeeEstimator { sat_per_kw: 253 };
keys_manager = test_utils::TestKeysInterface::new(&nodes[0].node_seed, Network::Testnet);
nodes[0].node = &node_state_0;
assert!(monitor.add_monitor(OutPoint { txid: chan.3.txid(), index: 0 }, chan_monitor).is_ok());
nodes[0].chan_monitor = &monitor;
- nodes[0].chain_monitor = &chain_monitor;
+ nodes[0].chain_source = &chain_source;
nodes[0].block_notifier = BlockNotifier::new();
nodes[0].block_notifier.register_listener(&nodes[0].chan_monitor.simple_monitor);
use bitcoin::hashes::sha256d::Hash as Sha256dHash;
use bitcoin::hashes::Hash;
use bitcoin::blockdata::script::Builder;
+use bitcoin::blockdata::transaction::TxOut;
use bitcoin::blockdata::opcodes;
-use chain::chaininterface::{ChainError, ChainWatchInterface};
+use chain;
+use chain::Access;
use ln::features::{ChannelFeatures, NodeFeatures};
use ln::msgs::{DecodeError, ErrorAction, LightningError, RoutingMessageHandler, NetAddress, OptionalField, MAX_VALUE_MSAT};
use ln::msgs;
/// This network graph is then used for routing payments.
/// Provides interface to help with initial routing sync by
/// serving historical announcements.
-pub struct NetGraphMsgHandler<C: Deref, L: Deref> where C::Target: ChainWatchInterface, L::Target: Logger {
+pub struct NetGraphMsgHandler<C: Deref, L: Deref> where C::Target: chain::Access, L::Target: Logger {
secp_ctx: Secp256k1<secp256k1::VerifyOnly>,
/// Representation of the payment channel network
pub network_graph: RwLock<NetworkGraph>,
- chain_monitor: C,
+ chain_access: Option<C>,
full_syncs_requested: AtomicUsize,
logger: L,
}
-impl<C: Deref, L: Deref> NetGraphMsgHandler<C, L> where C::Target: ChainWatchInterface, L::Target: Logger {
+impl<C: Deref, L: Deref> NetGraphMsgHandler<C, L> where C::Target: chain::Access, L::Target: Logger {
/// Creates a new tracker of the actual state of the network of channels and nodes,
/// assuming a fresh 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(chain_monitor: C, logger: L) -> Self {
+ pub fn new(chain_access: Option<C>, logger: L) -> Self {
NetGraphMsgHandler {
secp_ctx: Secp256k1::verification_only(),
network_graph: RwLock::new(NetworkGraph {
nodes: BTreeMap::new(),
}),
full_syncs_requested: AtomicUsize::new(0),
- chain_monitor,
+ chain_access,
logger,
}
}
/// Creates a new tracker of the actual state of the network of channels and nodes,
/// assuming an existing Network Graph.
- pub fn from_net_graph(chain_monitor: C, logger: L, network_graph: NetworkGraph) -> Self {
+ pub fn from_net_graph(chain_access: Option<C>, logger: L, network_graph: NetworkGraph) -> Self {
NetGraphMsgHandler {
secp_ctx: Secp256k1::verification_only(),
network_graph: RwLock::new(network_graph),
full_syncs_requested: AtomicUsize::new(0),
- chain_monitor,
+ chain_access,
logger,
}
}
};
}
-impl<C: Deref + Sync + Send, L: Deref + Sync + Send> RoutingMessageHandler for NetGraphMsgHandler<C, L> where C::Target: ChainWatchInterface, L::Target: Logger {
+impl<C: Deref + Sync + Send, L: Deref + Sync + Send> RoutingMessageHandler for NetGraphMsgHandler<C, L> where C::Target: chain::Access, L::Target: Logger {
fn handle_node_announcement(&self, msg: &msgs::NodeAnnouncement) -> Result<bool, LightningError> {
self.network_graph.write().unwrap().update_node_from_announcement(msg, Some(&self.secp_ctx))
}
return Err(LightningError{err: "Channel announcement node had a channel with itself".to_owned(), action: ErrorAction::IgnoreError});
}
- let utxo_value = match self.chain_monitor.get_chain_utxo(msg.contents.chain_hash, msg.contents.short_channel_id) {
- Ok((script_pubkey, value)) => {
- let expected_script = Builder::new().push_opcode(opcodes::all::OP_PUSHNUM_2)
- .push_slice(&msg.contents.bitcoin_key_1.serialize())
- .push_slice(&msg.contents.bitcoin_key_2.serialize())
- .push_opcode(opcodes::all::OP_PUSHNUM_2)
- .push_opcode(opcodes::all::OP_CHECKMULTISIG).into_script().to_v0_p2wsh();
- if script_pubkey != expected_script {
- return Err(LightningError{err: format!("Channel announcement key ({}) didn't match on-chain script ({})", script_pubkey.to_hex(), expected_script.to_hex()), action: ErrorAction::IgnoreError});
- }
- //TODO: Check if value is worth storing, use it to inform routing, and compare it
- //to the new HTLC max field in channel_update
- Some(value)
- },
- Err(ChainError::NotSupported) => {
+ let utxo_value = match &self.chain_access {
+ &None => {
// Tentatively accept, potentially exposing us to DoS attacks
None
},
- Err(ChainError::NotWatched) => {
- return Err(LightningError{err: format!("Channel announced on an unknown chain ({})", msg.contents.chain_hash.encode().to_hex()), action: ErrorAction::IgnoreError});
- },
- Err(ChainError::UnknownTx) => {
- return Err(LightningError{err: "Channel announced without corresponding UTXO entry".to_owned(), action: ErrorAction::IgnoreError});
+ &Some(ref chain_access) => {
+ match chain_access.get_utxo(&msg.contents.chain_hash, msg.contents.short_channel_id) {
+ Ok(TxOut { value, script_pubkey }) => {
+ let expected_script = Builder::new().push_opcode(opcodes::all::OP_PUSHNUM_2)
+ .push_slice(&msg.contents.bitcoin_key_1.serialize())
+ .push_slice(&msg.contents.bitcoin_key_2.serialize())
+ .push_opcode(opcodes::all::OP_PUSHNUM_2)
+ .push_opcode(opcodes::all::OP_CHECKMULTISIG).into_script().to_v0_p2wsh();
+ if script_pubkey != expected_script {
+ return Err(LightningError{err: format!("Channel announcement key ({}) didn't match on-chain script ({})", script_pubkey.to_hex(), expected_script.to_hex()), action: ErrorAction::IgnoreError});
+ }
+ //TODO: Check if value is worth storing, use it to inform routing, and compare it
+ //to the new HTLC max field in channel_update
+ Some(value)
+ },
+ Err(chain::AccessError::UnknownChain) => {
+ return Err(LightningError{err: format!("Channel announced on an unknown chain ({})", msg.contents.chain_hash.encode().to_hex()), action: ErrorAction::IgnoreError});
+ },
+ Err(chain::AccessError::UnknownTx) => {
+ return Err(LightningError{err: "Channel announced without corresponding UTXO entry".to_owned(), action: ErrorAction::IgnoreError});
+ },
+ // format!("Channel announced on an unknown chain ({})", msg.contents.chain_hash.encode().to_hex())
+ }
},
};
let result = self.network_graph.write().unwrap().update_channel_from_announcement(msg, utxo_value, Some(&self.secp_ctx));
#[cfg(test)]
mod tests {
- use chain::chaininterface;
+ use chain;
use ln::features::{ChannelFeatures, NodeFeatures};
use routing::network_graph::{NetGraphMsgHandler, NetworkGraph};
use ln::msgs::{OptionalField, RoutingMessageHandler, UnsignedNodeAnnouncement, NodeAnnouncement,
use bitcoin::network::constants::Network;
use bitcoin::blockdata::constants::genesis_block;
use bitcoin::blockdata::script::Builder;
+ use bitcoin::blockdata::transaction::TxOut;
use bitcoin::blockdata::opcodes;
use bitcoin::util::hash::BitcoinHash;
use std::sync::Arc;
- fn create_net_graph_msg_handler() -> (Secp256k1<All>, NetGraphMsgHandler<Arc<chaininterface::ChainWatchInterfaceUtil>, Arc<test_utils::TestLogger>>) {
+ fn create_net_graph_msg_handler() -> (Secp256k1<All>, NetGraphMsgHandler<Arc<test_utils::TestChainSource>, Arc<test_utils::TestLogger>>) {
let secp_ctx = Secp256k1::new();
let logger = Arc::new(test_utils::TestLogger::new());
- let chain_monitor = Arc::new(chaininterface::ChainWatchInterfaceUtil::new(Network::Testnet));
- let net_graph_msg_handler = NetGraphMsgHandler::new(chain_monitor, Arc::clone(&logger));
+ let net_graph_msg_handler = NetGraphMsgHandler::new(None, Arc::clone(&logger));
(secp_ctx, net_graph_msg_handler)
}
fn handling_channel_announcements() {
let secp_ctx = Secp256k1::new();
let logger: Arc<Logger> = Arc::new(test_utils::TestLogger::new());
- let chain_monitor = Arc::new(test_utils::TestChainWatcher::new());
- let net_graph_msg_handler = NetGraphMsgHandler::new(chain_monitor.clone(), Arc::clone(&logger));
-
let node_1_privkey = &SecretKey::from_slice(&[42; 32]).unwrap();
let node_2_privkey = &SecretKey::from_slice(&[41; 32]).unwrap();
};
// Test if the UTXO lookups were not supported
- *chain_monitor.utxo_ret.lock().unwrap() = Err(chaininterface::ChainError::NotSupported);
-
+ let mut net_graph_msg_handler = NetGraphMsgHandler::new(None, Arc::clone(&logger));
match net_graph_msg_handler.handle_channel_announcement(&valid_announcement) {
Ok(res) => assert!(res),
_ => panic!()
}
}
-
// If we receive announcement for the same channel (with UTXO lookups disabled),
// drop new one on the floor, since we can't see any changes.
match net_graph_msg_handler.handle_channel_announcement(&valid_announcement) {
Err(e) => assert_eq!(e.err, "Already have knowledge of channel")
};
-
// Test if an associated transaction were not on-chain (or not confirmed).
- *chain_monitor.utxo_ret.lock().unwrap() = Err(chaininterface::ChainError::UnknownTx);
+ let chain_source = Arc::new(test_utils::TestChainSource::new(Network::Testnet));
+ *chain_source.utxo_ret.lock().unwrap() = Err(chain::AccessError::UnknownTx);
+ net_graph_msg_handler = NetGraphMsgHandler::new(Some(chain_source.clone()), Arc::clone(&logger));
unsigned_announcement.short_channel_id += 1;
msghash = hash_to_message!(&Sha256dHash::hash(&unsigned_announcement.encode()[..])[..]);
Err(e) => assert_eq!(e.err, "Channel announced without corresponding UTXO entry")
};
-
// Now test if the transaction is found in the UTXO set and the script is correct.
unsigned_announcement.short_channel_id += 1;
- *chain_monitor.utxo_ret.lock().unwrap() = Ok((good_script.clone(), 0));
+ *chain_source.utxo_ret.lock().unwrap() = Ok(TxOut { value: 0, script_pubkey: good_script.clone() });
msghash = hash_to_message!(&Sha256dHash::hash(&unsigned_announcement.encode()[..])[..]);
let valid_announcement = ChannelAnnouncement {
// If we receive announcement for the same channel (but TX is not confirmed),
// drop new one on the floor, since we can't see any changes.
- *chain_monitor.utxo_ret.lock().unwrap() = Err(chaininterface::ChainError::UnknownTx);
+ *chain_source.utxo_ret.lock().unwrap() = Err(chain::AccessError::UnknownTx);
match net_graph_msg_handler.handle_channel_announcement(&valid_announcement) {
Ok(_) => panic!(),
Err(e) => assert_eq!(e.err, "Channel announced without corresponding UTXO entry")
};
// But if it is confirmed, replace the channel
- *chain_monitor.utxo_ret.lock().unwrap() = Ok((good_script, 0));
+ *chain_source.utxo_ret.lock().unwrap() = Ok(TxOut { value: 0, script_pubkey: good_script });
unsigned_announcement.features = ChannelFeatures::empty();
msghash = hash_to_message!(&Sha256dHash::hash(&unsigned_announcement.encode()[..])[..]);
let valid_announcement = ChannelAnnouncement {
fn handling_channel_update() {
let secp_ctx = Secp256k1::new();
let logger: Arc<Logger> = Arc::new(test_utils::TestLogger::new());
- let chain_monitor = Arc::new(test_utils::TestChainWatcher::new());
- let net_graph_msg_handler = NetGraphMsgHandler::new(chain_monitor.clone(), Arc::clone(&logger));
+ let chain_source = Arc::new(test_utils::TestChainSource::new(Network::Testnet));
+ let net_graph_msg_handler = NetGraphMsgHandler::new(Some(chain_source.clone()), Arc::clone(&logger));
let node_1_privkey = &SecretKey::from_slice(&[42; 32]).unwrap();
let node_2_privkey = &SecretKey::from_slice(&[41; 32]).unwrap();
.push_slice(&PublicKey::from_secret_key(&secp_ctx, node_2_btckey).serialize())
.push_opcode(opcodes::all::OP_PUSHNUM_2)
.push_opcode(opcodes::all::OP_CHECKMULTISIG).into_script().to_v0_p2wsh();
- *chain_monitor.utxo_ret.lock().unwrap() = Ok((good_script.clone(), amount_sats));
+ *chain_source.utxo_ret.lock().unwrap() = Ok(TxOut { value: amount_sats, script_pubkey: good_script.clone() });
let unsigned_announcement = UnsignedChannelAnnouncement {
features: ChannelFeatures::empty(),
chain_hash,
#[cfg(test)]
mod tests {
- use chain::chaininterface;
use routing::router::{get_route, RouteHint, RoutingFees};
use routing::network_graph::NetGraphMsgHandler;
use ln::features::{ChannelFeatures, InitFeatures, NodeFeatures};
use std::sync::Arc;
// Using the same keys for LN and BTC ids
- fn add_channel(net_graph_msg_handler: &NetGraphMsgHandler<Arc<chaininterface::ChainWatchInterfaceUtil>, Arc<test_utils::TestLogger>>, secp_ctx: &Secp256k1<All>, node_1_privkey: &SecretKey,
+ fn add_channel(net_graph_msg_handler: &NetGraphMsgHandler<Arc<test_utils::TestChainSource>, Arc<test_utils::TestLogger>>, secp_ctx: &Secp256k1<All>, node_1_privkey: &SecretKey,
node_2_privkey: &SecretKey, features: ChannelFeatures, short_channel_id: u64) {
let node_id_1 = PublicKey::from_secret_key(&secp_ctx, node_1_privkey);
let node_id_2 = PublicKey::from_secret_key(&secp_ctx, node_2_privkey);
};
}
- fn update_channel(net_graph_msg_handler: &NetGraphMsgHandler<Arc<chaininterface::ChainWatchInterfaceUtil>, Arc<test_utils::TestLogger>>, secp_ctx: &Secp256k1<All>, node_privkey: &SecretKey, update: UnsignedChannelUpdate) {
+ fn update_channel(net_graph_msg_handler: &NetGraphMsgHandler<Arc<test_utils::TestChainSource>, Arc<test_utils::TestLogger>>, secp_ctx: &Secp256k1<All>, node_privkey: &SecretKey, update: UnsignedChannelUpdate) {
let msghash = hash_to_message!(&Sha256dHash::hash(&update.encode()[..])[..]);
let valid_channel_update = ChannelUpdate {
signature: secp_ctx.sign(&msghash, node_privkey),
}
- fn add_or_update_node(net_graph_msg_handler: &NetGraphMsgHandler<Arc<chaininterface::ChainWatchInterfaceUtil>, Arc<test_utils::TestLogger>>, secp_ctx: &Secp256k1<All>, node_privkey: &SecretKey,
+ fn add_or_update_node(net_graph_msg_handler: &NetGraphMsgHandler<Arc<test_utils::TestChainSource>, Arc<test_utils::TestLogger>>, secp_ctx: &Secp256k1<All>, node_privkey: &SecretKey,
features: NodeFeatures, timestamp: u32) {
let node_id = PublicKey::from_secret_key(&secp_ctx, node_privkey);
let unsigned_announcement = UnsignedNodeAnnouncement {
let our_privkey = &SecretKey::from_slice(&hex::decode("0101010101010101010101010101010101010101010101010101010101010101").unwrap()[..]).unwrap();
let our_id = PublicKey::from_secret_key(&secp_ctx, our_privkey);
let logger = Arc::new(test_utils::TestLogger::new());
- let chain_monitor = Arc::new(chaininterface::ChainWatchInterfaceUtil::new(Network::Testnet));
- let net_graph_msg_handler = NetGraphMsgHandler::new(chain_monitor, Arc::clone(&logger));
+ let net_graph_msg_handler = NetGraphMsgHandler::new(None, Arc::clone(&logger));
// Build network from our_id to node8:
//
// -1(1)2- node1 -1(3)2-
+use chain;
use chain::chaininterface;
-use chain::chaininterface::{ConfirmationTarget, ChainError, ChainWatchInterface};
+use chain::chaininterface::ConfirmationTarget;
use chain::transaction::OutPoint;
use chain::keysinterface;
use ln::channelmonitor;
use bitcoin::BitcoinHash;
use bitcoin::blockdata::constants::genesis_block;
-use bitcoin::blockdata::transaction::Transaction;
+use bitcoin::blockdata::transaction::{Transaction, TxOut};
use bitcoin::blockdata::script::{Builder, Script};
-use bitcoin::blockdata::block::BlockHeader;
use bitcoin::blockdata::opcodes;
use bitcoin::network::constants::Network;
-use bitcoin::hash_types::{Txid, BlockHash};
+use bitcoin::hash_types::BlockHash;
use bitcoin::secp256k1::{SecretKey, PublicKey, Secp256k1, Signature};
}
}
-pub struct TestChainWatcher {
- pub utxo_ret: Mutex<Result<(Script, u64), ChainError>>,
+pub struct TestChainSource {
+ pub genesis_hash: BlockHash,
+ pub utxo_ret: Mutex<Result<TxOut, chain::AccessError>>,
}
-impl TestChainWatcher {
- pub fn new() -> Self {
- let script = Builder::new().push_opcode(opcodes::OP_TRUE).into_script();
- Self { utxo_ret: Mutex::new(Ok((script, u64::max_value()))) }
+impl TestChainSource {
+ pub fn new(network: Network) -> Self {
+ let script_pubkey = Builder::new().push_opcode(opcodes::OP_TRUE).into_script();
+ Self {
+ genesis_hash: genesis_block(network).bitcoin_hash(),
+ utxo_ret: Mutex::new(Ok(TxOut { value: u64::max_value(), script_pubkey })),
+ }
}
}
-impl ChainWatchInterface for TestChainWatcher {
- fn install_watch_tx(&self, _txid: &Txid, _script_pub_key: &Script) { }
- fn install_watch_outpoint(&self, _outpoint: (Txid, u32), _out_script: &Script) { }
- fn watch_all_txn(&self) { }
- fn filter_block(&self, _header: &BlockHeader, _txdata: &[(usize, &Transaction)]) -> Vec<usize> {
- Vec::new()
- }
- fn reentered(&self) -> usize { 0 }
+impl chain::Access for TestChainSource {
+ fn get_utxo(&self, genesis_hash: &BlockHash, _short_channel_id: u64) -> Result<TxOut, chain::AccessError> {
+ if self.genesis_hash != *genesis_hash {
+ return Err(chain::AccessError::UnknownChain);
+ }
- fn get_chain_utxo(&self, _genesis_hash: BlockHash, _unspent_tx_output_identifier: u64) -> Result<(Script, u64), ChainError> {
self.utxo_ret.lock().unwrap().clone()
}
}