Merge pull request #2248 from TheBlueMatt/2023-04-gossip-check
authorElias Rohrer <dev@tnull.de>
Fri, 25 Aug 2023 12:10:39 +0000 (14:10 +0200)
committerGitHub <noreply@github.com>
Fri, 25 Aug 2023 12:10:39 +0000 (14:10 +0200)
Implement the UtxoSource interface for REST/RPC clients

lightning-block-sync/src/convert.rs
lightning-block-sync/src/gossip.rs [new file with mode: 0644]
lightning-block-sync/src/lib.rs
lightning-block-sync/src/rest.rs
lightning-block-sync/src/rpc.rs
lightning/src/ln/functional_test_utils.rs
lightning/src/routing/gossip.rs
lightning/src/routing/router.rs

index d6294e1d2a79518c05a674cae5ea86d41fc1faf7..bf9e9577619a3fd0a1a7fecc3d79cd070bd706e1 100644 (file)
@@ -13,8 +13,14 @@ use serde_json;
 use std::convert::From;
 use std::convert::TryFrom;
 use std::convert::TryInto;
+use std::str::FromStr;
 use bitcoin::hashes::Hash;
 
+impl TryInto<serde_json::Value> for JsonResponse {
+       type Error = std::io::Error;
+       fn try_into(self) -> Result<serde_json::Value, std::io::Error> { Ok(self.0) }
+}
+
 /// Conversion from `std::io::Error` into `BlockSourceError`.
 impl From<std::io::Error> for BlockSourceError {
        fn from(e: std::io::Error) -> BlockSourceError {
@@ -38,6 +44,17 @@ impl TryInto<Block> for BinaryResponse {
        }
 }
 
+/// Parses binary data as a block hash.
+impl TryInto<BlockHash> for BinaryResponse {
+       type Error = std::io::Error;
+
+       fn try_into(self) -> std::io::Result<BlockHash> {
+               BlockHash::from_slice(&self.0).map_err(|_|
+                       std::io::Error::new(std::io::ErrorKind::InvalidData, "bad block hash length")
+               )
+       }
+}
+
 /// Converts a JSON value into block header data. The JSON value may be an object representing a
 /// block header or an array of such objects. In the latter case, the first object is converted.
 impl TryInto<BlockHeaderData> for JsonResponse {
@@ -226,6 +243,46 @@ impl TryInto<Transaction> for JsonResponse {
        }
 }
 
+impl TryInto<BlockHash> for JsonResponse {
+       type Error = std::io::Error;
+
+       fn try_into(self) -> std::io::Result<BlockHash> {
+               match self.0.as_str() {
+                       None => Err(std::io::Error::new(std::io::ErrorKind::InvalidData, "expected JSON string")),
+                       Some(hex_data) if hex_data.len() != 64 =>
+                               Err(std::io::Error::new(std::io::ErrorKind::InvalidData, "invalid hash length")),
+                       Some(hex_data) => BlockHash::from_str(hex_data)
+                               .map_err(|_| std::io::Error::new(std::io::ErrorKind::InvalidData, "invalid hex data")),
+               }
+       }
+}
+
+/// The REST `getutxos` endpoint retuns a whole pile of data we don't care about and one bit we do
+/// - whether the `hit bitmap` field had any entries. Thus we condense the result down into only
+/// that.
+pub(crate) struct GetUtxosResponse {
+       pub(crate) hit_bitmap_nonempty: bool
+}
+
+impl TryInto<GetUtxosResponse> for JsonResponse {
+       type Error = std::io::Error;
+
+       fn try_into(self) -> std::io::Result<GetUtxosResponse> {
+               let bitmap_str =
+                       self.0.as_object().ok_or(std::io::Error::new(std::io::ErrorKind::InvalidData, "expected an object"))?
+                       .get("bitmap").ok_or(std::io::Error::new(std::io::ErrorKind::InvalidData, "missing bitmap field"))?
+                       .as_str().ok_or(std::io::Error::new(std::io::ErrorKind::InvalidData, "bitmap should be an str"))?;
+                       let mut hit_bitmap_nonempty = false;
+                       for c in bitmap_str.chars() {
+                               if c < '0' || c > '9' {
+                                       return Err(std::io::Error::new(std::io::ErrorKind::InvalidData, "invalid byte"));
+                               }
+                               if c > '0' { hit_bitmap_nonempty = true; }
+                       }
+                       Ok(GetUtxosResponse { hit_bitmap_nonempty })
+       }
+}
+
 #[cfg(test)]
 pub(crate) mod tests {
        use super::*;
diff --git a/lightning-block-sync/src/gossip.rs b/lightning-block-sync/src/gossip.rs
new file mode 100644 (file)
index 0000000..37f4268
--- /dev/null
@@ -0,0 +1,319 @@
+//! When fetching gossip from peers, lightning nodes need to validate that gossip against the
+//! current UTXO set. This module defines an implementation of the LDK API required to do so
+//! against a [`BlockSource`] which implements a few additional methods for accessing the UTXO set.
+
+use crate::{AsyncBlockSourceResult, BlockData, BlockSource, BlockSourceError};
+
+use bitcoin::blockdata::block::Block;
+use bitcoin::blockdata::transaction::{TxOut, OutPoint};
+use bitcoin::hash_types::BlockHash;
+
+use lightning::sign::NodeSigner;
+
+use lightning::ln::peer_handler::{CustomMessageHandler, PeerManager, SocketDescriptor};
+use lightning::ln::msgs::{ChannelMessageHandler, OnionMessageHandler};
+
+use lightning::routing::gossip::{NetworkGraph, P2PGossipSync};
+use lightning::routing::utxo::{UtxoFuture, UtxoLookup, UtxoResult, UtxoLookupError};
+
+use lightning::util::logger::Logger;
+
+use std::sync::{Arc, Mutex};
+use std::collections::VecDeque;
+use std::future::Future;
+use std::ops::Deref;
+use std::pin::Pin;
+use std::task::Poll;
+
+/// A trait which extends [`BlockSource`] and can be queried to fetch the block at a given height
+/// as well as whether a given output is unspent (i.e. a member of the current UTXO set).
+///
+/// Note that while this is implementable for a [`BlockSource`] which returns filtered block data
+/// (i.e. [`BlockData::HeaderOnly`] for [`BlockSource::get_block`] requests), such an
+/// implementation will reject all gossip as it is not fully able to verify the UTXOs referenced.
+pub trait UtxoSource : BlockSource + 'static {
+       /// Fetches the block hash of the block at the given height.
+       ///
+       /// This will, in turn, be passed to to [`BlockSource::get_block`] to fetch the block needed
+       /// for gossip validation.
+       fn get_block_hash_by_height<'a>(&'a self, block_height: u32) -> AsyncBlockSourceResult<'a, BlockHash>;
+
+       /// Returns true if the given output has *not* been spent, i.e. is a member of the current UTXO
+       /// set.
+       fn is_output_unspent<'a>(&'a self, outpoint: OutPoint) -> AsyncBlockSourceResult<'a, bool>;
+}
+
+/// A generic trait which is able to spawn futures in the background.
+///
+/// If the `tokio` feature is enabled, this is implemented on `TokioSpawner` struct which
+/// delegates to `tokio::spawn()`.
+pub trait FutureSpawner : Send + Sync + 'static {
+       /// Spawns the given future as a background task.
+       ///
+       /// This method MUST NOT block on the given future immediately.
+       fn spawn<T: Future<Output = ()> + Send + 'static>(&self, future: T);
+}
+
+#[cfg(feature = "tokio")]
+/// A trivial [`FutureSpawner`] which delegates to `tokio::spawn`.
+pub struct TokioSpawner;
+#[cfg(feature = "tokio")]
+impl FutureSpawner for TokioSpawner {
+       fn spawn<T: Future<Output = ()> + Send + 'static>(&self, future: T) {
+               tokio::spawn(future);
+       }
+}
+
+/// A trivial future which joins two other futures and polls them at the same time, returning only
+/// once both complete.
+pub(crate) struct Joiner<
+       A: Future<Output=Result<(BlockHash, Option<u32>), BlockSourceError>> + Unpin,
+       B: Future<Output=Result<BlockHash, BlockSourceError>> + Unpin,
+> {
+       pub a: A,
+       pub b: B,
+       a_res: Option<(BlockHash, Option<u32>)>,
+       b_res: Option<BlockHash>,
+}
+
+impl<
+       A: Future<Output=Result<(BlockHash, Option<u32>), BlockSourceError>> + Unpin,
+       B: Future<Output=Result<BlockHash, BlockSourceError>> + Unpin,
+> Joiner<A, B> {
+       fn new(a: A, b: B) -> Self { Self { a, b, a_res: None, b_res: None } }
+}
+
+impl<
+       A: Future<Output=Result<(BlockHash, Option<u32>), BlockSourceError>> + Unpin,
+       B: Future<Output=Result<BlockHash, BlockSourceError>> + Unpin,
+> Future for Joiner<A, B> {
+       type Output = Result<((BlockHash, Option<u32>), BlockHash), BlockSourceError>;
+       fn poll(mut self: Pin<&mut Self>, ctx: &mut core::task::Context<'_>) -> Poll<Self::Output> {
+               if self.a_res.is_none() {
+                       match Pin::new(&mut self.a).poll(ctx) {
+                               Poll::Ready(res) => {
+                                       if let Ok(ok) = res {
+                                               self.a_res = Some(ok);
+                                       } else {
+                                               return Poll::Ready(Err(res.unwrap_err()));
+                                       }
+                               },
+                               Poll::Pending => {},
+                       }
+               }
+               if self.b_res.is_none() {
+                       match Pin::new(&mut self.b).poll(ctx) {
+                               Poll::Ready(res) => {
+                                       if let Ok(ok) = res {
+                                               self.b_res = Some(ok);
+                                       } else {
+                                               return Poll::Ready(Err(res.unwrap_err()));
+                                       }
+
+                               },
+                               Poll::Pending => {},
+                       }
+               }
+               if let Some(b_res) = self.b_res {
+                       if let Some(a_res) = self.a_res {
+                               return Poll::Ready(Ok((a_res, b_res)))
+                       }
+               }
+               Poll::Pending
+       }
+}
+
+/// A struct which wraps a [`UtxoSource`] and a few LDK objects and implements the LDK
+/// [`UtxoLookup`] trait.
+///
+/// Note that if you're using this against a Bitcoin Core REST or RPC server, you likely wish to
+/// increase the `rpcworkqueue` setting in Bitcoin Core as LDK attempts to parallelize requests (a
+/// value of 1024 should more than suffice), and ensure you have sufficient file descriptors
+/// available on both Bitcoin Core and your LDK application for each request to hold its own
+/// connection.
+pub struct GossipVerifier<S: FutureSpawner,
+       Blocks: Deref + Send + Sync + 'static + Clone,
+       L: Deref + Send + Sync + 'static,
+       Descriptor: SocketDescriptor + Send + Sync + 'static,
+       CM: Deref + Send + Sync + 'static,
+       OM: Deref + Send + Sync + 'static,
+       CMH: Deref + Send + Sync + 'static,
+       NS: Deref + Send + Sync + 'static,
+> where
+       Blocks::Target: UtxoSource,
+       L::Target: Logger,
+       CM::Target: ChannelMessageHandler,
+       OM::Target: OnionMessageHandler,
+       CMH::Target: CustomMessageHandler,
+       NS::Target: NodeSigner,
+{
+       source: Blocks,
+       peer_manager: Arc<PeerManager<Descriptor, CM, Arc<P2PGossipSync<Arc<NetworkGraph<L>>, Self, L>>, OM, L, CMH, NS>>,
+       gossiper: Arc<P2PGossipSync<Arc<NetworkGraph<L>>, Self, L>>,
+       spawn: S,
+       block_cache: Arc<Mutex<VecDeque<(u32, Block)>>>,
+}
+
+const BLOCK_CACHE_SIZE: usize = 5;
+
+impl<S: FutureSpawner,
+       Blocks: Deref + Send + Sync + Clone,
+       L: Deref + Send + Sync,
+       Descriptor: SocketDescriptor + Send + Sync,
+       CM: Deref + Send + Sync,
+       OM: Deref + Send + Sync,
+       CMH: Deref + Send + Sync,
+       NS: Deref + Send + Sync,
+> GossipVerifier<S, Blocks, L, Descriptor, CM, OM, CMH, NS> where
+       Blocks::Target: UtxoSource,
+       L::Target: Logger,
+       CM::Target: ChannelMessageHandler,
+       OM::Target: OnionMessageHandler,
+       CMH::Target: CustomMessageHandler,
+       NS::Target: NodeSigner,
+{
+       /// Constructs a new [`GossipVerifier`].
+       ///
+       /// This is expected to be given to a [`P2PGossipSync`] (initially constructed with `None` for
+       /// the UTXO lookup) via [`P2PGossipSync::add_utxo_lookup`].
+       pub fn new(source: Blocks, spawn: S, gossiper: Arc<P2PGossipSync<Arc<NetworkGraph<L>>, Self, L>>, peer_manager: Arc<PeerManager<Descriptor, CM, Arc<P2PGossipSync<Arc<NetworkGraph<L>>, Self, L>>, OM, L, CMH, NS>>) -> Self {
+               Self {
+                       source, spawn, gossiper, peer_manager,
+                       block_cache: Arc::new(Mutex::new(VecDeque::with_capacity(BLOCK_CACHE_SIZE))),
+               }
+       }
+
+       async fn retrieve_utxo(
+               source: Blocks, block_cache: Arc<Mutex<VecDeque<(u32, Block)>>>, short_channel_id: u64
+       ) -> Result<TxOut, UtxoLookupError> {
+               let block_height = (short_channel_id >> 5 * 8) as u32; // block height is most significant three bytes
+               let transaction_index = ((short_channel_id >> 2 * 8) & 0xffffff) as u32;
+               let output_index = (short_channel_id & 0xffff) as u16;
+
+               let (outpoint, output);
+
+               'tx_found: loop { // Used as a simple goto
+                       macro_rules! process_block {
+                               ($block: expr) => { {
+                                       if transaction_index as usize >= $block.txdata.len() {
+                                               return Err(UtxoLookupError::UnknownTx);
+                                       }
+                                       let transaction = &$block.txdata[transaction_index as usize];
+                                       if output_index as usize >= transaction.output.len() {
+                                               return Err(UtxoLookupError::UnknownTx);
+                                       }
+
+                                       outpoint = OutPoint::new(transaction.txid(), output_index.into());
+                                       output = transaction.output[output_index as usize].clone();
+                               } }
+                       }
+                       {
+                               let recent_blocks = block_cache.lock().unwrap();
+                               for (height, block) in recent_blocks.iter() {
+                                       if *height == block_height {
+                                               process_block!(block);
+                                               break 'tx_found;
+                                       }
+                               }
+                       }
+
+                       let ((_, tip_height_opt), block_hash) =
+                               Joiner::new(source.get_best_block(), source.get_block_hash_by_height(block_height))
+                               .await
+                               .map_err(|_| UtxoLookupError::UnknownTx)?;
+                       if let Some(tip_height) = tip_height_opt {
+                               // If the block doesn't yet have five confirmations, error out.
+                               //
+                               // The BOLT spec requires nodes wait for six confirmations before announcing a
+                               // channel, and we give them one block of headroom in case we're delayed seeing a
+                               // block.
+                               if block_height + 5 > tip_height {
+                                       return Err(UtxoLookupError::UnknownTx);
+                               }
+                       }
+                       let block_data = source.get_block(&block_hash).await
+                               .map_err(|_| UtxoLookupError::UnknownTx)?;
+                       let block = match block_data {
+                               BlockData::HeaderOnly(_) => return Err(UtxoLookupError::UnknownTx),
+                               BlockData::FullBlock(block) => block,
+                       };
+                       process_block!(block);
+                       {
+                               let mut recent_blocks = block_cache.lock().unwrap();
+                               let mut insert = true;
+                               for (height, _) in recent_blocks.iter() {
+                                       if *height == block_height {
+                                               insert = false;
+                                       }
+                               }
+                               if insert {
+                                       if recent_blocks.len() >= BLOCK_CACHE_SIZE {
+                                               recent_blocks.pop_front();
+                                       }
+                                       recent_blocks.push_back((block_height, block));
+                               }
+                       }
+                       break 'tx_found;
+               };
+               let outpoint_unspent =
+                       source.is_output_unspent(outpoint).await.map_err(|_| UtxoLookupError::UnknownTx)?;
+               if outpoint_unspent {
+                       Ok(output)
+               } else {
+                       Err(UtxoLookupError::UnknownTx)
+               }
+       }
+}
+
+impl<S: FutureSpawner,
+       Blocks: Deref + Send + Sync + Clone,
+       L: Deref + Send + Sync,
+       Descriptor: SocketDescriptor + Send + Sync,
+       CM: Deref + Send + Sync,
+       OM: Deref + Send + Sync,
+       CMH: Deref + Send + Sync,
+       NS: Deref + Send + Sync,
+> Deref for GossipVerifier<S, Blocks, L, Descriptor, CM, OM, CMH, NS> where
+       Blocks::Target: UtxoSource,
+       L::Target: Logger,
+       CM::Target: ChannelMessageHandler,
+       OM::Target: OnionMessageHandler,
+       CMH::Target: CustomMessageHandler,
+       NS::Target: NodeSigner,
+{
+       type Target = Self;
+       fn deref(&self) -> &Self { self }
+}
+
+
+impl<S: FutureSpawner,
+       Blocks: Deref + Send + Sync + Clone,
+       L: Deref + Send + Sync,
+       Descriptor: SocketDescriptor + Send + Sync,
+       CM: Deref + Send + Sync,
+       OM: Deref + Send + Sync,
+       CMH: Deref + Send + Sync,
+       NS: Deref + Send + Sync,
+> UtxoLookup for GossipVerifier<S, Blocks, L, Descriptor, CM, OM, CMH, NS> where
+       Blocks::Target: UtxoSource,
+       L::Target: Logger,
+       CM::Target: ChannelMessageHandler,
+       OM::Target: OnionMessageHandler,
+       CMH::Target: CustomMessageHandler,
+       NS::Target: NodeSigner,
+{
+       fn get_utxo(&self, _genesis_hash: &BlockHash, short_channel_id: u64) -> UtxoResult {
+               let res = UtxoFuture::new();
+               let fut = res.clone();
+               let source = self.source.clone();
+               let gossiper = Arc::clone(&self.gossiper);
+               let block_cache = Arc::clone(&self.block_cache);
+               let pm = Arc::clone(&self.peer_manager);
+               self.spawn.spawn(async move {
+                       let res = Self::retrieve_utxo(source, block_cache, short_channel_id).await;
+                       fut.resolve(gossiper.network_graph(), &*gossiper, res);
+                       pm.process_events();
+               });
+               UtxoResult::Async(res)
+       }
+}
index 5c7c0dee8c10c4dddaf3ea630238db5e59387650..3561a1b5d769f19381dfdf1bec1e7749cdf8c2d8 100644 (file)
@@ -28,6 +28,8 @@ pub mod http;
 pub mod init;
 pub mod poll;
 
+pub mod gossip;
+
 #[cfg(feature = "rest-client")]
 pub mod rest;
 
index 4300893013c06d7a3ee81c715b256066f4d28333..5690da12ea0495c1c8062655a95b27e83611cfa8 100644 (file)
@@ -3,7 +3,10 @@
 
 use crate::{BlockData, BlockHeaderData, BlockSource, AsyncBlockSourceResult};
 use crate::http::{BinaryResponse, HttpEndpoint, HttpClient, JsonResponse};
+use crate::gossip::UtxoSource;
+use crate::convert::GetUtxosResponse;
 
+use bitcoin::OutPoint;
 use bitcoin::hash_types::BlockHash;
 use bitcoin::hashes::hex::ToHex;
 
@@ -60,11 +63,30 @@ impl BlockSource for RestClient {
        }
 }
 
+impl UtxoSource for RestClient {
+       fn get_block_hash_by_height<'a>(&'a self, block_height: u32) -> AsyncBlockSourceResult<'a, BlockHash> {
+               Box::pin(async move {
+                       let resource_path = format!("blockhashbyheight/{}.bin", block_height);
+                       Ok(self.request_resource::<BinaryResponse, _>(&resource_path).await?)
+               })
+       }
+
+       fn is_output_unspent<'a>(&'a self, outpoint: OutPoint) -> AsyncBlockSourceResult<'a, bool> {
+               Box::pin(async move {
+                       let resource_path = format!("getutxos/{}-{}.json", outpoint.txid.to_hex(), outpoint.vout);
+                       let utxo_result =
+                               self.request_resource::<JsonResponse, GetUtxosResponse>(&resource_path).await?;
+                       Ok(utxo_result.hit_bitmap_nonempty)
+               })
+       }
+}
+
 #[cfg(test)]
 mod tests {
        use super::*;
        use crate::http::BinaryResponse;
        use crate::http::client_tests::{HttpServer, MessageBody};
+       use bitcoin::hashes::Hash;
 
        /// Parses binary data as a string-encoded `u32`.
        impl TryInto<u32> for BinaryResponse {
@@ -113,4 +135,32 @@ mod tests {
                        Ok(n) => assert_eq!(n, 42),
                }
        }
+
+       #[tokio::test]
+       async fn parses_negative_getutxos() {
+               let server = HttpServer::responding_with_ok(MessageBody::Content(
+                       // A real response contains a few more fields, but we actually only look at the
+                       // "bitmap" field, so this should suffice for testing
+                       "{\"chainHeight\": 1, \"bitmap\":\"0\",\"utxos\":[]}"
+               ));
+               let client = RestClient::new(server.endpoint()).unwrap();
+
+               let outpoint = OutPoint::new(bitcoin::Txid::from_inner([0; 32]), 0);
+               let unspent_output = client.is_output_unspent(outpoint).await.unwrap();
+               assert_eq!(unspent_output, false);
+       }
+
+       #[tokio::test]
+       async fn parses_positive_getutxos() {
+               let server = HttpServer::responding_with_ok(MessageBody::Content(
+                       // A real response contains lots more data, but we actually only look at the "bitmap"
+                       // field, so this should suffice for testing
+                       "{\"chainHeight\": 1, \"bitmap\":\"1\",\"utxos\":[]}"
+               ));
+               let client = RestClient::new(server.endpoint()).unwrap();
+
+               let outpoint = OutPoint::new(bitcoin::Txid::from_inner([0; 32]), 0);
+               let unspent_output = client.is_output_unspent(outpoint).await.unwrap();
+               assert_eq!(unspent_output, true);
+       }
 }
index 4c4706cb1cd584e201fe1464d20abe5015dbe7bc..0ad94040acaf0cfcfd326588ecedc35ed1a00a19 100644 (file)
@@ -3,9 +3,11 @@
 
 use crate::{BlockData, BlockHeaderData, BlockSource, AsyncBlockSourceResult};
 use crate::http::{HttpClient, HttpEndpoint, HttpError, JsonResponse};
+use crate::gossip::UtxoSource;
 
 use bitcoin::hash_types::BlockHash;
 use bitcoin::hashes::hex::ToHex;
+use bitcoin::OutPoint;
 
 use std::sync::Mutex;
 
@@ -105,12 +107,13 @@ impl RpcClient {
                        return Err(std::io::Error::new(std::io::ErrorKind::Other, rpc_error));
                }
 
-               let result = &mut response["result"];
-               if result.is_null() {
-                       return Err(std::io::Error::new(std::io::ErrorKind::InvalidData, "expected JSON result"));
-               }
+               let result = match response.get_mut("result") {
+                       Some(result) => result.take(),
+                       None =>
+                               return Err(std::io::Error::new(std::io::ErrorKind::InvalidData, "expected JSON result")),
+               };
 
-               JsonResponse(result.take()).try_into()
+               JsonResponse(result).try_into()
        }
 }
 
@@ -137,11 +140,33 @@ impl BlockSource for RpcClient {
        }
 }
 
+impl UtxoSource for RpcClient {
+       fn get_block_hash_by_height<'a>(&'a self, block_height: u32) -> AsyncBlockSourceResult<'a, BlockHash> {
+               Box::pin(async move {
+                       let height_param = serde_json::json!(block_height);
+                       Ok(self.call_method("getblockhash", &[height_param]).await?)
+               })
+       }
+
+       fn is_output_unspent<'a>(&'a self, outpoint: OutPoint) -> AsyncBlockSourceResult<'a, bool> {
+               Box::pin(async move {
+                       let txid_param = serde_json::json!(outpoint.txid.to_hex());
+                       let vout_param = serde_json::json!(outpoint.vout);
+                       let include_mempool = serde_json::json!(false);
+                       let utxo_opt: serde_json::Value = self.call_method(
+                               "gettxout", &[txid_param, vout_param, include_mempool]).await?;
+                       Ok(!utxo_opt.is_null())
+               })
+       }
+}
+
 #[cfg(test)]
 mod tests {
        use super::*;
        use crate::http::client_tests::{HttpServer, MessageBody};
 
+       use bitcoin::hashes::Hash;
+
        /// Credentials encoded in base64.
        const CREDENTIALS: &'static str = "dXNlcjpwYXNzd29yZA==";
 
@@ -205,7 +230,7 @@ mod tests {
 
        #[tokio::test]
        async fn call_method_returning_missing_result() {
-               let response = serde_json::json!({ "result": null });
+               let response = serde_json::json!({  });
                let server = HttpServer::responding_with_ok(MessageBody::Content(response));
                let client = RpcClient::new(CREDENTIALS, server.endpoint()).unwrap();
 
@@ -244,4 +269,24 @@ mod tests {
                        Ok(count) => assert_eq!(count, 654470),
                }
        }
+
+       #[tokio::test]
+       async fn fails_to_fetch_spent_utxo() {
+               let response = serde_json::json!({ "result": null });
+               let server = HttpServer::responding_with_ok(MessageBody::Content(response));
+               let client = RpcClient::new(CREDENTIALS, server.endpoint()).unwrap();
+               let outpoint = OutPoint::new(bitcoin::Txid::from_inner([0; 32]), 0);
+               let unspent_output = client.is_output_unspent(outpoint).await.unwrap();
+               assert_eq!(unspent_output, false);
+       }
+
+       #[tokio::test]
+       async fn fetches_utxo() {
+               let response = serde_json::json!({ "result": {"bestblock": 1, "confirmations": 42}});
+               let server = HttpServer::responding_with_ok(MessageBody::Content(response));
+               let client = RpcClient::new(CREDENTIALS, server.endpoint()).unwrap();
+               let outpoint = OutPoint::new(bitcoin::Txid::from_inner([0; 32]), 0);
+               let unspent_output = client.is_output_unspent(outpoint).await.unwrap();
+               assert_eq!(unspent_output, true);
+       }
 }
index 34568b07cad5e4d5dd5f13bb4404e17b6c8948d0..7f1ded0323ecd3804ac3cb3c90cf6b5062fbe411 100644 (file)
@@ -570,11 +570,11 @@ impl<'a, 'b, 'c> Drop for Node<'a, 'b, 'c> {
        }
 }
 
-pub fn create_chan_between_nodes<'a, 'b, 'c, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>) -> (msgs::ChannelAnnouncement, msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
+pub fn create_chan_between_nodes<'a, 'b, 'c: 'd, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>) -> (msgs::ChannelAnnouncement, msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
        create_chan_between_nodes_with_value(node_a, node_b, 100000, 10001)
 }
 
-pub fn create_chan_between_nodes_with_value<'a, 'b, 'c, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, channel_value: u64, push_msat: u64) -> (msgs::ChannelAnnouncement, msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
+pub fn create_chan_between_nodes_with_value<'a, 'b, 'c: 'd, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, channel_value: u64, push_msat: u64) -> (msgs::ChannelAnnouncement, msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
        let (channel_ready, channel_id, tx) = create_chan_between_nodes_with_value_a(node_a, node_b, channel_value, push_msat);
        let (announcement, as_update, bs_update) = create_chan_between_nodes_with_value_b(node_a, node_b, &channel_ready);
        (announcement, as_update, bs_update, channel_id, tx)
@@ -1193,7 +1193,7 @@ pub fn create_chan_between_nodes_with_value_confirm_second<'a, 'b, 'c>(node_recv
        }), channel_id)
 }
 
-pub fn create_chan_between_nodes_with_value_confirm<'a, 'b, 'c, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, tx: &Transaction) -> ((msgs::ChannelReady, msgs::AnnouncementSignatures), [u8; 32]) {
+pub fn create_chan_between_nodes_with_value_confirm<'a, 'b, 'c: 'd, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, tx: &Transaction) -> ((msgs::ChannelReady, msgs::AnnouncementSignatures), [u8; 32]) {
        let conf_height = core::cmp::max(node_a.best_block_info().1 + 1, node_b.best_block_info().1 + 1);
        create_chan_between_nodes_with_value_confirm_first(node_a, node_b, tx, conf_height);
        confirm_transaction_at(node_a, tx, conf_height);
@@ -1202,7 +1202,7 @@ pub fn create_chan_between_nodes_with_value_confirm<'a, 'b, 'c, 'd>(node_a: &'a
        create_chan_between_nodes_with_value_confirm_second(node_b, node_a)
 }
 
-pub fn create_chan_between_nodes_with_value_a<'a, 'b, 'c, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, channel_value: u64, push_msat: u64) -> ((msgs::ChannelReady, msgs::AnnouncementSignatures), [u8; 32], Transaction) {
+pub fn create_chan_between_nodes_with_value_a<'a, 'b, 'c: 'd, 'd>(node_a: &'a Node<'b, 'c, 'd>, node_b: &'a Node<'b, 'c, 'd>, channel_value: u64, push_msat: u64) -> ((msgs::ChannelReady, msgs::AnnouncementSignatures), [u8; 32], Transaction) {
        let tx = create_chan_between_nodes_with_value_init(node_a, node_b, channel_value, push_msat);
        let (msgs, chan_id) = create_chan_between_nodes_with_value_confirm(node_a, node_b, &tx);
        (msgs, chan_id, tx)
@@ -1242,11 +1242,11 @@ pub fn create_chan_between_nodes_with_value_b<'a, 'b, 'c>(node_a: &Node<'a, 'b,
        ((*announcement).clone(), as_update, bs_update)
 }
 
-pub fn create_announced_chan_between_nodes<'a, 'b, 'c, 'd>(nodes: &'a Vec<Node<'b, 'c, 'd>>, a: usize, b: usize) -> (msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
+pub fn create_announced_chan_between_nodes<'a, 'b, 'c: 'd, 'd>(nodes: &'a Vec<Node<'b, 'c, 'd>>, a: usize, b: usize) -> (msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
        create_announced_chan_between_nodes_with_value(nodes, a, b, 100000, 10001)
 }
 
-pub fn create_announced_chan_between_nodes_with_value<'a, 'b, 'c, 'd>(nodes: &'a Vec<Node<'b, 'c, 'd>>, a: usize, b: usize, channel_value: u64, push_msat: u64) -> (msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
+pub fn create_announced_chan_between_nodes_with_value<'a, 'b, 'c: 'd, 'd>(nodes: &'a Vec<Node<'b, 'c, 'd>>, a: usize, b: usize, channel_value: u64, push_msat: u64) -> (msgs::ChannelUpdate, msgs::ChannelUpdate, [u8; 32], Transaction) {
        let chan_announcement = create_chan_between_nodes_with_value(&nodes[a], &nodes[b], channel_value, push_msat);
        update_nodes_with_chan_announce(nodes, a, b, &chan_announcement.0, &chan_announcement.1, &chan_announcement.2);
        (chan_announcement.1, chan_announcement.2, chan_announcement.3, chan_announcement.4)
index dc02206db24efbece6f434832d0061c113e563bf..bb5531242188ba28028952b47aaeb3899a6e722c 100644 (file)
@@ -254,7 +254,7 @@ pub struct P2PGossipSync<G: Deref<Target=NetworkGraph<L>>, U: Deref, L: Deref>
 where U::Target: UtxoLookup, L::Target: Logger
 {
        network_graph: G,
-       utxo_lookup: Option<U>,
+       utxo_lookup: RwLock<Option<U>>,
        #[cfg(feature = "std")]
        full_syncs_requested: AtomicUsize,
        pending_events: Mutex<Vec<MessageSendEvent>>,
@@ -273,7 +273,7 @@ where U::Target: UtxoLookup, L::Target: Logger
                        network_graph,
                        #[cfg(feature = "std")]
                        full_syncs_requested: AtomicUsize::new(0),
-                       utxo_lookup,
+                       utxo_lookup: RwLock::new(utxo_lookup),
                        pending_events: Mutex::new(vec![]),
                        logger,
                }
@@ -282,8 +282,8 @@ where U::Target: UtxoLookup, L::Target: 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_utxo_lookup(&mut self, utxo_lookup: Option<U>) {
-               self.utxo_lookup = utxo_lookup;
+       pub fn add_utxo_lookup(&self, utxo_lookup: Option<U>) {
+               *self.utxo_lookup.write().unwrap() = utxo_lookup;
        }
 
        /// Gets a reference to the underlying [`NetworkGraph`] which was provided in
@@ -443,7 +443,7 @@ where U::Target: UtxoLookup, L::Target: Logger
        }
 
        fn handle_channel_announcement(&self, msg: &msgs::ChannelAnnouncement) -> Result<bool, LightningError> {
-               self.network_graph.update_channel_from_announcement(msg, &self.utxo_lookup)?;
+               self.network_graph.update_channel_from_announcement(msg, &*self.utxo_lookup.read().unwrap())?;
                Ok(msg.contents.excess_data.len() <= MAX_EXCESS_BYTES_FOR_RELAY)
        }
 
index 2973878a4d4db239b0a6d8d9f96ff1fc71ba54d5..1f143ed938dcf8435334c3c646057cee7163e200 100644 (file)
@@ -3903,7 +3903,7 @@ mod tests {
        fn available_amount_while_routing_test() {
                // Tests whether we choose the correct available channel amount while routing.
 
-               let (secp_ctx, network_graph, mut gossip_sync, chain_monitor, logger) = build_graph();
+               let (secp_ctx, network_graph, gossip_sync, chain_monitor, logger) = build_graph();
                let (our_privkey, our_id, privkeys, nodes) = get_nodes(&secp_ctx);
                let scorer = ln_test_utils::TestScorer::new();
                let keys_manager = ln_test_utils::TestKeysInterface::new(&[0u8; 32], Network::Testnet);