change(rpc): Add getpeerinfo RPC method (#5951)

* adds ValidateBlock request to state

* adds `Request` enum in block verifier

skips solution check for BlockProposal requests

calls CheckBlockValidity instead of Commit block for BlockProposal requests

* uses new Request in references to chain verifier

* adds getblocktemplate proposal mode response type

* makes getblocktemplate-rpcs feature in zebra-consensus select getblocktemplate-rpcs in zebra-state

* Adds PR review revisions

* adds info log in CheckBlockProposalValidity

* Reverts replacement of match statement

* adds `GetBlockTemplate::capabilities` fn

* conditions calling checkpoint verifier on !request.is_proposal

* updates references to validate_and_commit_non_finalized

* adds snapshot test, updates test vectors

* adds `should_count_metrics` to NonFinalizedState

* Returns an error from chain verifier for block proposal requests below checkpoint height

adds feature flags

* adds "proposal" to GET_BLOCK_TEMPLATE_CAPABILITIES_FIELD

* adds back block::Request to zebra-consensus lib

* updates snapshots

* Removes unnecessary network arg

* skips req in tracing intstrument for read state

* Moves out block proposal validation to its own fn

* corrects `difficulty_threshold_is_valid` docs

adds/fixes some comments, adds TODOs

general cleanup from a self-review.

* Update zebra-state/src/service.rs

* Apply suggestions from code review

Co-authored-by: teor <teor@riseup.net>

* Update zebra-rpc/src/methods/get_block_template_rpcs.rs

Co-authored-by: teor <teor@riseup.net>

* check best chain tip

* Update zebra-state/src/service.rs

Co-authored-by: teor <teor@riseup.net>

* Applies cleanup suggestions from code review

* updates gbt acceptance test to make a block proposal

* fixes json parsing mistake

* adds retries

* returns reject reason if there are no retries left

* moves result deserialization to RPCRequestClient method, adds docs, moves jsonrpc_core to dev-dependencies

* moves sleep(EXPECTED_TX_TIME) out of loop

* updates/adds info logs in retry loop

* Revert "moves sleep(EXPECTED_TX_TIME) out of loop"

This reverts commit f7f0926f4050519687a79afc16656c3f345c004b.

* adds `allow(dead_code)`

* tests with curtime, mintime, & maxtime

* Fixes doc comment

* Logs error responses from chain_verifier CheckProposal requests

* Removes retry loop, adds num_txs log

* removes verbose info log

* sorts mempool_txs before generating merkle root

* Make imports conditional on a feature

* Disable new CI tests until bugs are fixed

* adds support for getpeerinfo RPC

* adds vector test

* Always passes address_book to RpcServer

* Update zebra-network/src/address_book.rs

Co-authored-by: teor <teor@riseup.net>

* Renames PeerObserver to AddressBookPeers

* adds getpeerinfo acceptance test

* Asserts that addresses parse into SocketAddr

* adds launches_lightwalletd field to TestType::LaunchWithEmptyState and uses it from the get_peer_info test

* renames peer_observer mod

* uses SocketAddr as `addr` field type in PeerInfo

Co-authored-by: teor <teor@riseup.net>
Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
This commit is contained in:
Arya 2023-01-17 02:09:07 -05:00 committed by GitHub
parent dcf30679bf
commit 1bb8a9c924
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 370 additions and 46 deletions

View File

@ -1,7 +1,13 @@
//! The `AddressBook` manages information about what peers exist, when they were
//! seen, and what services they provide.
use std::{cmp::Reverse, iter::Extend, net::SocketAddr, time::Instant};
use std::{
cmp::Reverse,
iter::Extend,
net::SocketAddr,
sync::{Arc, Mutex},
time::Instant,
};
use chrono::Utc;
use ordered_map::OrderedMap;
@ -12,9 +18,8 @@ use zebra_chain::parameters::Network;
use crate::{
constants, meta_addr::MetaAddrChange, protocol::external::canonical_socket_addr,
types::MetaAddr, PeerAddrState,
types::MetaAddr, AddressBookPeers, PeerAddrState,
};
#[cfg(test)]
mod tests;
@ -288,7 +293,7 @@ impl AddressBook {
?updated,
?previous,
total_peers = self.by_addr.len(),
recent_peers = self.recently_live_peers(chrono_now).count(),
recent_peers = self.recently_live_peers(chrono_now).len(),
"calculated updated address book entry",
);
@ -317,7 +322,7 @@ impl AddressBook {
?updated,
?previous,
total_peers = self.by_addr.len(),
recent_peers = self.recently_live_peers(chrono_now).count(),
recent_peers = self.recently_live_peers(chrono_now).len(),
"updated address book entry",
);
@ -340,7 +345,7 @@ impl AddressBook {
surplus = ?surplus_peer,
?updated,
total_peers = self.by_addr.len(),
recent_peers = self.recently_live_peers(chrono_now).count(),
recent_peers = self.recently_live_peers(chrono_now).len(),
"removed surplus address book entry",
);
}
@ -370,7 +375,7 @@ impl AddressBook {
trace!(
?removed_addr,
total_peers = self.by_addr.len(),
recent_peers = self.recently_live_peers(chrono_now).count(),
recent_peers = self.recently_live_peers(chrono_now).len(),
);
if let Some(entry) = self.by_addr.remove(&removed_addr) {
@ -452,20 +457,6 @@ impl AddressBook {
.cloned()
}
/// Return an iterator over peers we've seen recently,
/// in reconnection attempt order.
pub fn recently_live_peers(
&'_ self,
now: chrono::DateTime<Utc>,
) -> impl Iterator<Item = MetaAddr> + DoubleEndedIterator + '_ {
let _guard = self.span.enter();
self.by_addr
.descending_values()
.filter(move |peer| peer.was_recently_live(now))
.cloned()
}
/// Returns the number of entries in this address book.
pub fn len(&self) -> usize {
self.by_addr.len()
@ -501,7 +492,7 @@ impl AddressBook {
let failed = self.state_peers(PeerAddrState::Failed).count();
let attempt_pending = self.state_peers(PeerAddrState::AttemptPending).count();
let recently_live = self.recently_live_peers(now).count();
let recently_live = self.recently_live_peers(now).len();
let recently_stopped_responding = responded
.checked_sub(recently_live)
.expect("all recently live peers must have responded");
@ -597,6 +588,26 @@ impl AddressBook {
}
}
impl AddressBookPeers for AddressBook {
fn recently_live_peers(&self, now: chrono::DateTime<Utc>) -> Vec<MetaAddr> {
let _guard = self.span.enter();
self.by_addr
.descending_values()
.filter(|peer| peer.was_recently_live(now))
.cloned()
.collect()
}
}
impl AddressBookPeers for Arc<Mutex<AddressBook>> {
fn recently_live_peers(&self, now: chrono::DateTime<Utc>) -> Vec<MetaAddr> {
self.lock()
.expect("panic in a previous thread that was holding the mutex")
.recently_live_peers(now)
}
}
impl Extend<MetaAddrChange> for AddressBook {
fn extend<T>(&mut self, iter: T)
where

View File

@ -0,0 +1,17 @@
//! A AddressBookPeers trait for getting the [`MetaAddr`] of recently live peers.
use chrono::Utc;
use crate::meta_addr::MetaAddr;
#[cfg(any(test, feature = "proptest-impl"))]
pub mod mock;
#[cfg(any(test, feature = "proptest-impl"))]
pub use mock::MockAddressBookPeers;
/// Method signatures for getting [`MetaAddr`]s of recently live peers.
pub trait AddressBookPeers {
/// Return an Vec of peers we've seen recently, in reconnection attempt order.
fn recently_live_peers(&self, now: chrono::DateTime<Utc>) -> Vec<MetaAddr>;
}

View File

@ -0,0 +1,25 @@
//! Mock [`AddressBookPeers`] for use in tests.
use crate::{meta_addr::MetaAddr, AddressBookPeers};
/// A mock [`AddressBookPeers`] implementation that's always empty.
#[derive(Default, Clone)]
pub struct MockAddressBookPeers {
/// Return value for mock `recently_live_peers` method.
recently_live_peers: Vec<MetaAddr>,
}
impl MockAddressBookPeers {
/// Creates a new [`MockAddressBookPeers`]
pub fn new(recently_live_peers: Vec<MetaAddr>) -> Self {
Self {
recently_live_peers,
}
}
}
impl AddressBookPeers for MockAddressBookPeers {
fn recently_live_peers(&self, _now: chrono::DateTime<chrono::Utc>) -> Vec<MetaAddr> {
self.recently_live_peers.clone()
}
}

View File

@ -144,6 +144,7 @@ extern crate bitflags;
pub type BoxError = Box<dyn std::error::Error + Send + Sync + 'static>;
mod address_book;
pub mod address_book_peers;
mod address_book_updater;
mod config;
pub mod constants;
@ -171,6 +172,7 @@ pub use crate::isolated::{
pub use crate::{
address_book::AddressBook,
address_book_peers::AddressBookPeers,
config::Config,
isolated::{connect_isolated, connect_isolated_tcp_direct},
meta_addr::PeerAddrState,

View File

@ -16,6 +16,7 @@ use zebra_chain::{
transparent,
};
use zebra_consensus::VerifyChainError;
use zebra_network::AddressBookPeers;
use zebra_node_services::mempool;
use zebra_state::{ReadRequest, ReadResponse};
@ -31,7 +32,7 @@ use crate::methods::{
},
types::{
get_block_template::GetBlockTemplate, get_mining_info, hex_data::HexData,
long_poll::LongPollInput, submit_block,
long_poll::LongPollInput, peer_info::PeerInfo, submit_block,
},
},
height_from_signed_int, GetBlockHash, MISSING_BLOCK_ERROR_CODE,
@ -149,10 +150,16 @@ pub trait GetBlockTemplateRpc {
) -> BoxFuture<Result<u64>> {
self.get_network_sol_ps(num_blocks, height)
}
/// Returns data about each connected network node.
///
/// zcashd reference: [`getpeerinfo`](https://zcash.github.io/rpc/getpeerinfo.html)
#[rpc(name = "getpeerinfo")]
fn get_peer_info(&self) -> BoxFuture<Result<Vec<PeerInfo>>>;
}
/// RPC method implementations.
pub struct GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier, SyncStatus>
pub struct GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier, SyncStatus, AddressBook>
where
Mempool: Service<
mempool::Request,
@ -170,6 +177,7 @@ where
+ Sync
+ 'static,
SyncStatus: ChainSyncStatus + Clone + Send + Sync + 'static,
AddressBook: AddressBookPeers,
{
// Configuration
//
@ -197,10 +205,13 @@ where
/// The chain sync status, used for checking if Zebra is likely close to the network chain tip.
sync_status: SyncStatus,
/// Address book of peers, used for `getpeerinfo`.
address_book: AddressBook,
}
impl<Mempool, State, Tip, ChainVerifier, SyncStatus>
GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier, SyncStatus>
impl<Mempool, State, Tip, ChainVerifier, SyncStatus, AddressBook>
GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier, SyncStatus, AddressBook>
where
Mempool: Service<
mempool::Request,
@ -222,8 +233,10 @@ where
+ Sync
+ 'static,
SyncStatus: ChainSyncStatus + Clone + Send + Sync + 'static,
AddressBook: AddressBookPeers + Clone + Send + Sync + 'static,
{
/// Create a new instance of the handler for getblocktemplate RPCs.
#[allow(clippy::too_many_arguments)]
pub fn new(
network: Network,
mining_config: config::Config,
@ -232,6 +245,7 @@ where
latest_chain_tip: Tip,
chain_verifier: ChainVerifier,
sync_status: SyncStatus,
address_book: AddressBook,
) -> Self {
Self {
network,
@ -241,12 +255,13 @@ where
latest_chain_tip,
chain_verifier,
sync_status,
address_book,
}
}
}
impl<Mempool, State, Tip, ChainVerifier, SyncStatus> GetBlockTemplateRpc
for GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier, SyncStatus>
impl<Mempool, State, Tip, ChainVerifier, SyncStatus, AddressBook> GetBlockTemplateRpc
for GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier, SyncStatus, AddressBook>
where
Mempool: Service<
mempool::Request,
@ -271,6 +286,7 @@ where
+ 'static,
<ChainVerifier as Service<zebra_consensus::Request>>::Future: Send,
SyncStatus: ChainSyncStatus + Clone + Send + Sync + 'static,
AddressBook: AddressBookPeers + Clone + Send + Sync + 'static,
{
fn get_block_count(&self) -> Result<u32> {
best_chain_tip_height(&self.latest_chain_tip).map(|height| height.0)
@ -704,6 +720,18 @@ where
}
.boxed()
}
fn get_peer_info(&self) -> BoxFuture<Result<Vec<PeerInfo>>> {
let address_book = self.address_book.clone();
async move {
Ok(address_book
.recently_live_peers(chrono::Utc::now())
.into_iter()
.map(PeerInfo::from)
.collect())
}
.boxed()
}
}
// Put support functions in a submodule, to keep this file small.

View File

@ -5,5 +5,6 @@ pub mod get_block_template;
pub mod get_mining_info;
pub mod hex_data;
pub mod long_poll;
pub mod peer_info;
pub mod submit_block;
pub mod transaction;

View File

@ -0,0 +1,20 @@
//! An array of [`PeerInfo`] is the output of the `getpeerinfo` RPC method.
use std::net::SocketAddr;
use zebra_network::types::MetaAddr;
/// Item of the `getpeerinfo` response
#[derive(Clone, Debug, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
pub struct PeerInfo {
/// The IP address and port of the peer
pub addr: SocketAddr,
}
impl From<MetaAddr> for PeerInfo {
fn from(meta_addr: MetaAddr) -> Self {
Self {
addr: meta_addr.addr(),
}
}
}

View File

@ -5,6 +5,8 @@
//! cargo insta test --review --features getblocktemplate-rpcs --delete-unreferenced-snapshots
//! ```
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
use hex::FromHex;
use insta::Settings;
use tower::{buffer::Buffer, Service};
@ -19,6 +21,7 @@ use zebra_chain::{
transparent,
work::difficulty::{CompactDifficulty, ExpandedDifficulty, U256},
};
use zebra_network::{address_book_peers::MockAddressBookPeers, types::MetaAddr};
use zebra_node_services::mempool;
use zebra_state::{GetBlockTemplateChainInfo, ReadRequest, ReadResponse};
@ -36,6 +39,7 @@ use crate::methods::{
get_mining_info,
hex_data::HexData,
long_poll::{LongPollId, LONG_POLL_ID_LENGTH},
peer_info::PeerInfo,
submit_block,
},
},
@ -113,6 +117,14 @@ pub async fn test_responses<State, ReadState>(
mock_chain_tip_sender.send_best_tip_hash(fake_tip_hash);
mock_chain_tip_sender.send_estimated_distance_to_network_chain_tip(Some(0));
let mock_address_book =
MockAddressBookPeers::new(vec![MetaAddr::new_initial_peer(SocketAddr::new(
IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)),
network.default_port(),
))
.into_new_meta_addr()
.unwrap()]);
// get an rpc instance with continuous blockchain state
let get_block_template_rpc = GetBlockTemplateRpcImpl::new(
network,
@ -122,6 +134,7 @@ pub async fn test_responses<State, ReadState>(
mock_chain_tip.clone(),
chain_verifier.clone(),
mock_sync_status.clone(),
mock_address_book,
);
// `getblockcount`
@ -145,6 +158,13 @@ pub async fn test_responses<State, ReadState>(
.expect("We should have a success response");
snapshot_rpc_getmininginfo(get_mining_info, &settings);
// `getpeerinfo`
let get_peer_info = get_block_template_rpc
.get_peer_info()
.await
.expect("We should have a success response");
snapshot_rpc_getpeerinfo(get_peer_info, &settings);
// `getnetworksolps` (and `getnetworkhashps`)
let get_network_sol_ps = get_block_template_rpc
.get_network_sol_ps(None, None)
@ -169,6 +189,7 @@ pub async fn test_responses<State, ReadState>(
mock_chain_tip.clone(),
chain_verifier,
mock_sync_status.clone(),
MockAddressBookPeers::default(),
);
// Basic variant (default mode and no extra features)
@ -308,6 +329,7 @@ pub async fn test_responses<State, ReadState>(
mock_chain_tip,
mock_chain_verifier.clone(),
mock_sync_status,
MockAddressBookPeers::default(),
);
let get_block_template = tokio::spawn(get_block_template_rpc.get_block_template(Some(
@ -391,6 +413,11 @@ fn snapshot_rpc_getmininginfo(
settings.bind(|| insta::assert_json_snapshot!("get_mining_info", get_mining_info));
}
/// Snapshot `getpeerinfo` response, using `cargo insta` and JSON serialization.
fn snapshot_rpc_getpeerinfo(get_peer_info: Vec<PeerInfo>, settings: &insta::Settings) {
settings.bind(|| insta::assert_json_snapshot!("get_peer_info", get_peer_info));
}
/// Snapshot `getnetworksolps` response, using `cargo insta` and JSON serialization.
fn snapshot_rpc_getnetworksolps(get_network_sol_ps: u64, settings: &insta::Settings) {
settings.bind(|| insta::assert_json_snapshot!("get_network_sol_ps", get_network_sol_ps));

View File

@ -0,0 +1,9 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: get_peer_info
---
[
{
"addr": "127.0.0.1:8233"
}
]

View File

@ -0,0 +1,9 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: get_peer_info
---
[
{
"addr": "127.0.0.1:18233"
}
]

View File

@ -625,6 +625,7 @@ async fn rpc_getaddressutxos_response() {
#[cfg(feature = "getblocktemplate-rpcs")]
async fn rpc_getblockcount() {
use zebra_chain::chain_sync_status::MockSyncStatus;
use zebra_network::address_book_peers::MockAddressBookPeers;
let _init_guard = zebra_test::init();
@ -667,6 +668,7 @@ async fn rpc_getblockcount() {
latest_chain_tip.clone(),
chain_verifier,
MockSyncStatus::default(),
MockAddressBookPeers::default(),
);
// Get the tip height using RPC method `get_block_count`
@ -684,6 +686,7 @@ async fn rpc_getblockcount() {
#[tokio::test(flavor = "multi_thread")]
async fn rpc_getblockcount_empty_state() {
use zebra_chain::chain_sync_status::MockSyncStatus;
use zebra_network::address_book_peers::MockAddressBookPeers;
let _init_guard = zebra_test::init();
@ -715,6 +718,7 @@ async fn rpc_getblockcount_empty_state() {
latest_chain_tip.clone(),
chain_verifier,
MockSyncStatus::default(),
MockAddressBookPeers::default(),
);
// Get the tip height using RPC method `get_block_count
@ -729,10 +733,78 @@ async fn rpc_getblockcount_empty_state() {
mempool.expect_no_requests().await;
}
#[cfg(feature = "getblocktemplate-rpcs")]
#[tokio::test(flavor = "multi_thread")]
async fn rpc_getpeerinfo() {
use zebra_chain::chain_sync_status::MockSyncStatus;
use zebra_network::address_book_peers::MockAddressBookPeers;
let _init_guard = zebra_test::init();
let network = Mainnet;
// Get a mempool handle
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
// Create an empty state
let (state, read_state, latest_chain_tip, _chain_tip_change) =
zebra_state::init_test_services(Mainnet);
let (
chain_verifier,
_transaction_verifier,
_parameter_download_task_handle,
_max_checkpoint_height,
) = zebra_consensus::chain::init(
zebra_consensus::Config::default(),
network,
state.clone(),
true,
)
.await;
let mock_peer_address =
zebra_network::types::MetaAddr::new_initial_peer(std::net::SocketAddr::new(
std::net::IpAddr::V4(std::net::Ipv4Addr::new(127, 0, 0, 1)),
network.default_port(),
))
.into_new_meta_addr()
.unwrap();
let mock_address_book = MockAddressBookPeers::new(vec![mock_peer_address]);
// Init RPC
let get_block_template_rpc = get_block_template_rpcs::GetBlockTemplateRpcImpl::new(
network,
Default::default(),
Buffer::new(mempool.clone(), 1),
read_state,
latest_chain_tip.clone(),
chain_verifier,
MockSyncStatus::default(),
mock_address_book,
);
// Call `get_peer_info`
let get_peer_info = get_block_template_rpc
.get_peer_info()
.await
.expect("We should have an array of addresses");
assert_eq!(
get_peer_info
.into_iter()
.next()
.expect("there should be a mock peer address"),
mock_peer_address.into()
);
mempool.expect_no_requests().await;
}
#[cfg(feature = "getblocktemplate-rpcs")]
#[tokio::test(flavor = "multi_thread")]
async fn rpc_getblockhash() {
use zebra_chain::chain_sync_status::MockSyncStatus;
use zebra_network::address_book_peers::MockAddressBookPeers;
let _init_guard = zebra_test::init();
@ -769,6 +841,7 @@ async fn rpc_getblockhash() {
latest_chain_tip.clone(),
tower::ServiceBuilder::new().service(chain_verifier),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
);
// Query the hashes using positive indexes
@ -801,6 +874,7 @@ async fn rpc_getblockhash() {
#[tokio::test(flavor = "multi_thread")]
async fn rpc_getmininginfo() {
use zebra_chain::chain_sync_status::MockSyncStatus;
use zebra_network::address_book_peers::MockAddressBookPeers;
let _init_guard = zebra_test::init();
@ -823,6 +897,7 @@ async fn rpc_getmininginfo() {
latest_chain_tip.clone(),
MockService::build().for_unit_tests(),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
);
get_block_template_rpc
@ -835,6 +910,7 @@ async fn rpc_getmininginfo() {
#[tokio::test(flavor = "multi_thread")]
async fn rpc_getnetworksolps() {
use zebra_chain::chain_sync_status::MockSyncStatus;
use zebra_network::address_book_peers::MockAddressBookPeers;
let _init_guard = zebra_test::init();
@ -857,6 +933,7 @@ async fn rpc_getnetworksolps() {
latest_chain_tip.clone(),
MockService::build().for_unit_tests(),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
);
let get_network_sol_ps_inputs = [
@ -910,6 +987,7 @@ async fn rpc_getblocktemplate_mining_address(use_p2pkh: bool) {
work::difficulty::{CompactDifficulty, ExpandedDifficulty, U256},
};
use zebra_consensus::MAX_BLOCK_SIGOPS;
use zebra_network::address_book_peers::MockAddressBookPeers;
use zebra_state::{GetBlockTemplateChainInfo, ReadRequest, ReadResponse};
use crate::methods::{
@ -969,6 +1047,7 @@ async fn rpc_getblocktemplate_mining_address(use_p2pkh: bool) {
mock_chain_tip,
chain_verifier,
mock_sync_status.clone(),
MockAddressBookPeers::default(),
);
// Fake the ChainInfo response
@ -1140,6 +1219,7 @@ async fn rpc_getblocktemplate_mining_address(use_p2pkh: bool) {
#[tokio::test(flavor = "multi_thread")]
async fn rpc_submitblock_errors() {
use zebra_chain::chain_sync_status::MockSyncStatus;
use zebra_network::address_book_peers::MockAddressBookPeers;
use crate::methods::get_block_template_rpcs::types::{hex_data::HexData, submit_block};
@ -1179,6 +1259,7 @@ async fn rpc_submitblock_errors() {
latest_chain_tip.clone(),
chain_verifier,
MockSyncStatus::default(),
MockAddressBookPeers::default(),
);
// Try to submit pre-populated blocks and assert that it responds with duplicate.

View File

@ -19,6 +19,7 @@ use tracing::{Instrument, *};
use zebra_chain::{
block, chain_sync_status::ChainSyncStatus, chain_tip::ChainTip, parameters::Network,
};
use zebra_network::AddressBookPeers;
use zebra_node_services::mempool;
use crate::{
@ -69,7 +70,7 @@ impl RpcServer {
//
// TODO: put some of the configs or services in their own struct?
#[allow(clippy::too_many_arguments)]
pub fn spawn<Version, Mempool, State, Tip, ChainVerifier, SyncStatus>(
pub fn spawn<Version, Mempool, State, Tip, ChainVerifier, SyncStatus, AddressBook>(
config: Config,
#[cfg(feature = "getblocktemplate-rpcs")]
mining_config: get_block_template_rpcs::config::Config,
@ -83,6 +84,8 @@ impl RpcServer {
chain_verifier: ChainVerifier,
#[cfg_attr(not(feature = "getblocktemplate-rpcs"), allow(unused_variables))]
sync_status: SyncStatus,
#[cfg_attr(not(feature = "getblocktemplate-rpcs"), allow(unused_variables))]
address_book: AddressBook,
latest_chain_tip: Tip,
network: Network,
) -> (JoinHandle<()>, JoinHandle<()>, Option<Self>)
@ -114,6 +117,7 @@ impl RpcServer {
+ 'static,
<ChainVerifier as Service<zebra_consensus::Request>>::Future: Send,
SyncStatus: ChainSyncStatus + Clone + Send + Sync + 'static,
AddressBook: AddressBookPeers + Clone + Send + Sync + 'static,
{
if let Some(listen_addr) = config.listen_addr {
info!("Trying to open RPC endpoint at {}...", listen_addr,);
@ -144,6 +148,7 @@ impl RpcServer {
latest_chain_tip.clone(),
chain_verifier,
sync_status,
address_book,
);
io.extend_with(get_block_template_rpc_impl.to_delegate());

View File

@ -11,6 +11,7 @@ use tower::buffer::Buffer;
use zebra_chain::{
chain_sync_status::MockSyncStatus, chain_tip::NoChainTip, parameters::Network::*,
};
use zebra_network::address_book_peers::MockAddressBookPeers;
use zebra_node_services::BoxError;
use zebra_test::mock_service::MockService;
@ -61,6 +62,7 @@ fn rpc_server_spawn(parallel_cpu_threads: bool) {
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
NoChainTip,
Mainnet,
);
@ -146,6 +148,7 @@ fn rpc_server_spawn_unallocated_port(parallel_cpu_threads: bool, do_shutdown: bo
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
NoChainTip,
Mainnet,
);
@ -225,6 +228,7 @@ fn rpc_server_spawn_port_conflict() {
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
NoChainTip,
Mainnet,
);
@ -241,6 +245,7 @@ fn rpc_server_spawn_port_conflict() {
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
NoChainTip,
Mainnet,
);
@ -331,6 +336,7 @@ fn rpc_server_spawn_port_conflict_parallel_auto() {
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
NoChainTip,
Mainnet,
);
@ -347,6 +353,7 @@ fn rpc_server_spawn_port_conflict_parallel_auto() {
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
MockSyncStatus::default(),
MockAddressBookPeers::default(),
NoChainTip,
Mainnet,
);

View File

@ -186,6 +186,7 @@ impl StartCmd {
read_only_state_service,
chain_verifier.clone(),
sync_status.clone(),
address_book.clone(),
latest_chain_tip.clone(),
config.network.network,
);

View File

@ -1483,7 +1483,9 @@ fn non_blocking_logger() -> Result<()> {
#[test]
#[cfg(not(target_os = "windows"))]
fn lightwalletd_integration() -> Result<()> {
lightwalletd_integration_test(LaunchWithEmptyState)
lightwalletd_integration_test(LaunchWithEmptyState {
launches_lightwalletd: true,
})
}
/// Make sure `zebrad` can sync from peers, but don't actually launch `lightwalletd`.
@ -1547,7 +1549,9 @@ fn lightwalletd_full_sync() -> Result<()> {
#[ignore]
#[cfg(not(target_os = "windows"))]
async fn lightwalletd_test_suite() -> Result<()> {
lightwalletd_integration_test(LaunchWithEmptyState)?;
lightwalletd_integration_test(LaunchWithEmptyState {
launches_lightwalletd: true,
})?;
// Only runs when ZEBRA_CACHED_STATE_DIR is set.
lightwalletd_integration_test(UpdateZebraCachedStateNoRpc)?;
@ -2174,6 +2178,15 @@ async fn lightwalletd_wallet_grpc_tests() -> Result<()> {
common::lightwalletd::wallet_grpc_test::run().await
}
/// Test successful getpeerinfo rpc call
///
/// See [`common::get_block_template_rpcs::get_peer_info`] for more information.
#[tokio::test]
#[cfg(feature = "getblocktemplate-rpcs")]
async fn get_peer_info() -> Result<()> {
common::get_block_template_rpcs::get_peer_info::run().await
}
/// Test successful getblocktemplate rpc call
///
/// See [`common::get_block_template_rpcs::get_block_template`] for more information.

View File

@ -1,4 +1,5 @@
//! Acceptance tests for getblocktemplate RPC methods in Zebra.
pub(crate) mod get_block_template;
pub(crate) mod get_peer_info;
pub(crate) mod submit_block;

View File

@ -0,0 +1,62 @@
//! Tests that `getpeerinfo` RPC method responds with info about at least 1 peer.
use color_eyre::eyre::{Context, Result};
use zebra_chain::parameters::Network;
use zebra_rpc::methods::get_block_template_rpcs::types::peer_info::PeerInfo;
use crate::common::{
launch::{can_spawn_zebrad_for_rpc, spawn_zebrad_for_rpc},
rpc_client::RPCRequestClient,
test_type::TestType,
};
pub(crate) async fn run() -> Result<()> {
let _init_guard = zebra_test::init();
let test_type = TestType::LaunchWithEmptyState {
launches_lightwalletd: false,
};
let test_name = "get_peer_info_test";
let network = Network::Mainnet;
// Skip the test unless the user specifically asked for it and there is a zebrad_state_path
if !can_spawn_zebrad_for_rpc(test_name, test_type) {
return Ok(());
}
tracing::info!(?network, "running getpeerinfo test using zebrad",);
let (mut zebrad, zebra_rpc_address) =
spawn_zebrad_for_rpc(network, test_name, test_type, true)?
.expect("Already checked zebra state path with can_spawn_zebrad_for_rpc");
let rpc_address = zebra_rpc_address.expect("getpeerinfo test must have RPC port");
// Wait until port is open.
zebrad.expect_stdout_line_matches(&format!("Opened RPC endpoint at {rpc_address}"))?;
tracing::info!(?rpc_address, "zebrad opened its RPC port",);
// call `getpeerinfo` RPC method
let peer_info_result: Vec<PeerInfo> = RPCRequestClient::new(rpc_address)
.json_result_from_call("getpeerinfo", "[]".to_string())
.await?;
assert!(
!peer_info_result.is_empty(),
"getpeerinfo should return info for at least 1 peer"
);
zebrad.kill(false)?;
let output = zebrad.wait_with_output()?;
let output = output.assert_failure()?;
// [Note on port conflict](#Note on port conflict)
output
.assert_was_killed()
.wrap_err("Possible port conflict. Are there other acceptance tests running?")?;
Ok(())
}

View File

@ -23,7 +23,10 @@ use TestType::*;
#[derive(Copy, Clone, Debug, Eq, PartialEq)]
pub enum TestType {
/// Launch with an empty Zebra and lightwalletd state.
LaunchWithEmptyState,
LaunchWithEmptyState {
/// Configures whether the test uses lightwalletd.
launches_lightwalletd: bool,
},
/// Do a full sync from an empty lightwalletd state.
///
@ -66,7 +69,7 @@ impl TestType {
// - FullSyncFromGenesis, UpdateCachedState, UpdateZebraCachedStateNoRpc:
// skip the test if it is not available
match self {
LaunchWithEmptyState => false,
LaunchWithEmptyState { .. } => false,
FullSyncFromGenesis { .. }
| UpdateCachedState
| UpdateZebraCachedStateNoRpc
@ -77,11 +80,10 @@ impl TestType {
/// Does this test need a Zebra rpc server?
pub fn needs_zebra_rpc_server(&self) -> bool {
match self {
UpdateZebraCachedStateWithRpc => true,
UpdateZebraCachedStateNoRpc
| LaunchWithEmptyState
| FullSyncFromGenesis { .. }
| UpdateCachedState => self.launches_lightwalletd(),
UpdateZebraCachedStateWithRpc | LaunchWithEmptyState { .. } => true,
UpdateZebraCachedStateNoRpc | FullSyncFromGenesis { .. } | UpdateCachedState => {
self.launches_lightwalletd()
}
}
}
@ -89,7 +91,10 @@ impl TestType {
pub fn launches_lightwalletd(&self) -> bool {
match self {
UpdateZebraCachedStateNoRpc | UpdateZebraCachedStateWithRpc => false,
LaunchWithEmptyState | FullSyncFromGenesis { .. } | UpdateCachedState => true,
FullSyncFromGenesis { .. } | UpdateCachedState => true,
LaunchWithEmptyState {
launches_lightwalletd,
} => *launches_lightwalletd,
}
}
@ -100,7 +105,7 @@ impl TestType {
// - FullSyncFromGenesis: use it if available, timeout if it is already populated
// - UpdateCachedState: skip the test if it is not available
match self {
LaunchWithEmptyState
LaunchWithEmptyState { .. }
| FullSyncFromGenesis { .. }
| UpdateZebraCachedStateNoRpc
| UpdateZebraCachedStateWithRpc => false,
@ -111,7 +116,7 @@ impl TestType {
/// Does this test allow a `lightwalletd` cached state, even if it is not required?
pub fn allow_lightwalletd_cached_state(&self) -> bool {
match self {
LaunchWithEmptyState => false,
LaunchWithEmptyState { .. } => false,
FullSyncFromGenesis {
allow_lightwalletd_cached_state,
} => *allow_lightwalletd_cached_state,
@ -122,7 +127,7 @@ impl TestType {
/// Can this test create a new `LIGHTWALLETD_DATA_DIR` cached state?
pub fn can_create_lightwalletd_cached_state(&self) -> bool {
match self {
LaunchWithEmptyState => false,
LaunchWithEmptyState { .. } => false,
FullSyncFromGenesis { .. } | UpdateCachedState => true,
UpdateZebraCachedStateNoRpc | UpdateZebraCachedStateWithRpc => false,
}
@ -232,7 +237,7 @@ impl TestType {
/// Returns the `zebrad` timeout for this test type.
pub fn zebrad_timeout(&self) -> Duration {
match self {
LaunchWithEmptyState => LIGHTWALLETD_DELAY,
LaunchWithEmptyState { .. } => LIGHTWALLETD_DELAY,
FullSyncFromGenesis { .. } => LIGHTWALLETD_FULL_SYNC_TIP_DELAY,
UpdateCachedState | UpdateZebraCachedStateNoRpc => LIGHTWALLETD_UPDATE_TIP_DELAY,
UpdateZebraCachedStateWithRpc => FINISH_PARTIAL_SYNC_TIMEOUT,
@ -249,7 +254,7 @@ impl TestType {
// We use the same timeouts for zebrad and lightwalletd,
// because the tests check zebrad and lightwalletd concurrently.
match self {
LaunchWithEmptyState => LIGHTWALLETD_DELAY,
LaunchWithEmptyState { .. } => LIGHTWALLETD_DELAY,
FullSyncFromGenesis { .. } => LIGHTWALLETD_FULL_SYNC_TIP_DELAY,
UpdateCachedState | UpdateZebraCachedStateNoRpc | UpdateZebraCachedStateWithRpc => {
LIGHTWALLETD_UPDATE_TIP_DELAY
@ -270,7 +275,7 @@ impl TestType {
// Fail if we need a cached Zebra state, but it's empty
zebrad_failure_messages.push("loaded Zebra state cache .*tip.*=.*None".to_string());
}
if *self == LaunchWithEmptyState {
if matches!(*self, LaunchWithEmptyState { .. }) {
// Fail if we need an empty Zebra state, but it has blocks
zebrad_failure_messages
.push(r"loaded Zebra state cache .*tip.*=.*Height\([1-9][0-9]*\)".to_string());
@ -311,7 +316,7 @@ impl TestType {
lightwalletd_failure_messages.push("Found [1-9][0-9]* blocks in cache".to_string());
}
let lightwalletd_ignore_messages = if *self == LaunchWithEmptyState {
let lightwalletd_ignore_messages = if matches!(*self, LaunchWithEmptyState { .. }) {
LIGHTWALLETD_EMPTY_ZEBRA_STATE_IGNORE_MESSAGES.iter()
} else {
NO_MATCHES_REGEX_ITER.iter()