feat(net): Cache a list of useful peers on disk (#6739)

* Rewrite some state cache docs to clarify

* Add a zebra_network::Config.cache_dir for peer address caches

* Add new config test files and fix config test failure message

* Create some zebra-chain and zebra-network convenience functions

* Add methods for reading and writing the peer address cache

* Add cached disk peers to the initial peers list

* Add metrics and logging for loading and storing the peer cache

* Replace log of useless redacted peer IP addresses

* Limit the peer cache minimum and maximum size, don't write empty caches

* Add a cacheable_peers() method to the address book

* Add a peer disk cache updater task to the peer set tasks

* Document that the peer cache is shared by multiple instances unless configured otherwise

* Disable peer cache read/write in disconnected tests

* Make initial peer cache updater sleep shorter for tests

* Add unit tests for reading and writing the peer cache

* Update the task list in the start command docs

* Modify the existing persistent acceptance test to check for peer caches

* Update the peer cache directory when writing test configs

* Add a CacheDir type so the default config can be enabled, but tests can disable it

* Update tests to use the CacheDir config type

* Rename some CacheDir internals

* Add config file test cases for each kind of CacheDir config

* Panic if the config contains invalid socket addresses, rather than continuing

* Add a network directory to state cache directory contents tests

* Add new network.cache_dir config to the config parsing tests
This commit is contained in:
teor 2023-06-06 18:28:14 +10:00 committed by GitHub
parent f197dfb75e
commit 04e96c2526
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 868 additions and 48 deletions

View File

@ -5847,6 +5847,7 @@ dependencies = [
"byteorder",
"bytes",
"chrono",
"dirs",
"futures",
"hex",
"howudoin",
@ -5863,6 +5864,7 @@ dependencies = [
"regex",
"serde",
"static_assertions",
"tempfile",
"thiserror",
"tokio",
"tokio-stream",

View File

@ -119,6 +119,11 @@ impl Network {
Network::Testnet => "test".to_string(),
}
}
/// Return the lowercase network name.
pub fn lowercase_name(&self) -> String {
self.to_string().to_ascii_lowercase()
}
}
impl FromStr for Network {

View File

@ -27,6 +27,7 @@ bitflags = "2.2.1"
byteorder = "1.4.3"
bytes = "1.4.0"
chrono = { version = "0.4.26", default-features = false, features = ["clock", "std"] }
dirs = "5.0.1"
hex = "0.4.3"
humantime-serde = "1.1.1"
indexmap = { version = "1.9.3", features = ["serde"] }
@ -37,10 +38,11 @@ rand = { version = "0.8.5", package = "rand" }
rayon = "1.7.0"
regex = "1.8.4"
serde = { version = "1.0.163", features = ["serde_derive"] }
tempfile = "3.5.0"
thiserror = "1.0.40"
futures = "0.3.28"
tokio = { version = "1.28.2", features = ["net", "time", "tracing", "macros", "rt-multi-thread"] }
tokio = { version = "1.28.2", features = ["fs", "net", "time", "tracing", "macros", "rt-multi-thread"] }
tokio-stream = { version = "0.1.14", features = ["sync", "time"] }
tokio-util = { version = "0.7.8", features = ["codec"] }
tower = { version = "0.4.13", features = ["retry", "discover", "load", "load-shed", "timeout", "util", "buffer"] }

View File

@ -240,7 +240,8 @@ impl AddressBook {
self.local_listener
}
/// Get the contents of `self` in random order with sanitized timestamps.
/// Get the active addresses in `self` in random order with sanitized timestamps,
/// including our local listener address.
pub fn sanitized(&self, now: chrono::DateTime<Utc>) -> Vec<MetaAddr> {
use rand::seq::SliceRandom;
let _guard = self.span.enter();
@ -254,10 +255,12 @@ impl AddressBook {
peers.insert(local_listener.addr, local_listener);
// Then sanitize and shuffle
let mut peers = peers
let mut peers: Vec<MetaAddr> = peers
.descending_values()
.filter_map(|meta_addr| meta_addr.sanitize(self.network))
// Security: remove peers that:
// # Security
//
// Remove peers that:
// - last responded more than three hours ago, or
// - haven't responded yet but were reported last seen more than three hours ago
//
@ -265,11 +268,36 @@ impl AddressBook {
// nodes impacts the network health, because connection attempts end up being wasted on
// peers that are less likely to respond.
.filter(|addr| addr.is_active_for_gossip(now))
.collect::<Vec<_>>();
.collect();
peers.shuffle(&mut rand::thread_rng());
peers
}
/// Get the active addresses in `self`, in preferred caching order,
/// excluding our local listener address.
pub fn cacheable(&self, now: chrono::DateTime<Utc>) -> Vec<MetaAddr> {
let _guard = self.span.enter();
let peers = self.by_addr.clone();
// Get peers in preferred order, then keep the recently active ones
peers
.descending_values()
// # Security
//
// Remove peers that:
// - last responded more than three hours ago, or
// - haven't responded yet but were reported last seen more than three hours ago
//
// This prevents Zebra from caching nodes that are likely unreachable,
// which improves startup time and reliability.
.filter(|addr| addr.is_active_for_gossip(now))
.cloned()
.collect()
}
/// Look up `addr` in the address book, and return its [`MetaAddr`].
///
/// Converts `addr` to a canonical address before looking it up.

View File

@ -2,6 +2,8 @@
use std::{
collections::HashSet,
ffi::OsString,
io::{self, ErrorKind},
net::{IpAddr, SocketAddr},
string::String,
time::Duration,
@ -9,21 +11,27 @@ use std::{
use indexmap::IndexSet;
use serde::{de, Deserialize, Deserializer};
use tempfile::NamedTempFile;
use tokio::{fs, io::AsyncWriteExt};
use zebra_chain::parameters::Network;
use crate::{
constants::{
DEFAULT_CRAWL_NEW_PEER_INTERVAL, DNS_LOOKUP_TIMEOUT, INBOUND_PEER_LIMIT_MULTIPLIER,
OUTBOUND_PEER_LIMIT_MULTIPLIER,
MAX_PEER_DISK_CACHE_SIZE, OUTBOUND_PEER_LIMIT_MULTIPLIER,
},
protocol::external::{canonical_peer_addr, canonical_socket_addr},
BoxError, PeerSocketAddr,
};
mod cache_dir;
#[cfg(test)]
mod tests;
pub use cache_dir::CacheDir;
/// The number of times Zebra will retry each initial peer's DNS resolution,
/// before checking if any other initial peers have returned addresses.
///
@ -71,9 +79,64 @@ pub struct Config {
/// testnet.
pub initial_testnet_peers: IndexSet<String>,
/// An optional root directory for storing cached peer address data.
///
/// # Configuration
///
/// Set to:
/// - `true` to read and write peer addresses to disk using the default cache path,
/// - `false` to disable reading and writing peer addresses to disk,
/// - `'/custom/cache/directory'` to read and write peer addresses to a custom directory.
///
/// By default, all Zebra instances run by the same user will share a single peer cache.
/// If you use a custom cache path, you might also want to change `state.cache_dir`.
///
/// # Functionality
///
/// The peer cache is a list of the addresses of some recently useful peers.
///
/// For privacy reasons, the cache does *not* include any other information about peers,
/// such as when they were connected to the node.
///
/// Deleting or modifying the peer cache can impact your node's:
/// - reliability: if DNS or the Zcash DNS seeders are unavailable or broken
/// - security: if DNS is compromised with malicious peers
///
/// If you delete it, Zebra will replace it with a fresh set of peers from the DNS seeders.
///
/// # Defaults
///
/// The default directory is platform dependent, based on
/// [`dirs::cache_dir()`](https://docs.rs/dirs/3.0.1/dirs/fn.cache_dir.html):
///
/// |Platform | Value | Example |
/// | ------- | ----------------------------------------------- | ------------------------------------ |
/// | Linux | `$XDG_CACHE_HOME/zebra` or `$HOME/.cache/zebra` | `/home/alice/.cache/zebra` |
/// | macOS | `$HOME/Library/Caches/zebra` | `/Users/Alice/Library/Caches/zebra` |
/// | Windows | `{FOLDERID_LocalAppData}\zebra` | `C:\Users\Alice\AppData\Local\zebra` |
/// | Other | `std::env::current_dir()/cache/zebra` | `/cache/zebra` |
///
/// # Security
///
/// If you are running Zebra with elevated permissions ("root"), create the
/// directory for this file before running Zebra, and make sure the Zebra user
/// account has exclusive access to that directory, and other users can't modify
/// its parent directories.
///
/// # Implementation Details
///
/// Each network has a separate peer list, which is updated regularly from the current
/// address book. These lists are stored in `network/mainnet.peers` and
/// `network/testnet.peers` files, underneath the `cache_dir` path.
///
/// Previous peer lists are automatically loaded at startup, and used to populate the
/// initial peer set and address book.
pub cache_dir: CacheDir,
/// The initial target size for the peer set.
///
/// Also used to limit the number of inbound and outbound connections made by Zebra.
/// Also used to limit the number of inbound and outbound connections made by Zebra,
/// and the size of the cached peer list.
///
/// If you have a slow network connection, and Zebra is having trouble
/// syncing, try reducing the peer set size. You can also reduce the peer
@ -144,9 +207,24 @@ impl Config {
}
}
/// Resolve initial seed peer IP addresses, based on the configured network.
/// Resolve initial seed peer IP addresses, based on the configured network,
/// and load cached peers from disk, if available.
///
/// # Panics
///
/// If a configured address is an invalid [`SocketAddr`] or DNS name.
pub async fn initial_peers(&self) -> HashSet<PeerSocketAddr> {
Config::resolve_peers(&self.initial_peer_hostnames().iter().cloned().collect()).await
// TODO: do DNS and disk in parallel if startup speed becomes important
let dns_peers =
Config::resolve_peers(&self.initial_peer_hostnames().iter().cloned().collect()).await;
// Ignore disk errors because the cache is optional and the method already logs them.
let disk_peers = self.load_peer_cache().await.unwrap_or_default();
dns_peers
.into_iter()
.chain(disk_peers.into_iter())
.collect()
}
/// Concurrently resolves `peers` into zero or more IP addresses, with a
@ -161,6 +239,7 @@ impl Config {
warn!(
"no initial peers in the network config. \
Hint: you must configure at least one peer IP or DNS seeder to run Zebra, \
give it some previously cached peer IP addresses on disk, \
or make sure Zebra's listener port gets inbound connections."
);
return HashSet::new();
@ -196,6 +275,10 @@ impl Config {
/// `max_retries` times.
///
/// If DNS continues to fail, returns an empty list of addresses.
///
/// # Panics
///
/// If a configured address is an invalid [`SocketAddr`] or DNS name.
async fn resolve_host(host: &str, max_retries: usize) -> HashSet<PeerSocketAddr> {
for retries in 0..=max_retries {
if let Ok(addresses) = Config::resolve_host_once(host).await {
@ -225,6 +308,10 @@ impl Config {
///
/// If `host` is a DNS name, performs DNS resolution with a timeout of a few seconds.
/// If DNS resolution fails or times out, returns an error.
///
/// # Panics
///
/// If a configured address is an invalid [`SocketAddr`] or DNS name.
async fn resolve_host_once(host: &str) -> Result<HashSet<PeerSocketAddr>, BoxError> {
let fut = tokio::net::lookup_host(host);
let fut = tokio::time::timeout(DNS_LOOKUP_TIMEOUT, fut);
@ -260,6 +347,13 @@ impl Config {
Ok(ip_addrs.into_iter().collect())
}
Ok(Err(e)) if e.kind() == ErrorKind::InvalidInput => {
// TODO: add testnet/mainnet ports, like we do with the listener address
panic!(
"Invalid peer IP address in Zebra config: addresses must have ports:\n\
resolving {host:?} returned {e:?}"
);
}
Ok(Err(e)) => {
tracing::info!(?host, ?e, "DNS error resolving peer IP addresses");
Err(e.into())
@ -270,6 +364,190 @@ impl Config {
}
}
}
/// Returns the addresses in the peer list cache file, if available.
pub async fn load_peer_cache(&self) -> io::Result<HashSet<PeerSocketAddr>> {
let Some(peer_cache_file) = self.cache_dir.peer_cache_file_path(self.network) else {
return Ok(HashSet::new());
};
let peer_list = match fs::read_to_string(&peer_cache_file).await {
Ok(peer_list) => peer_list,
Err(peer_list_error) => {
// We expect that the cache will be missing for new Zebra installs
if peer_list_error.kind() == ErrorKind::NotFound {
return Ok(HashSet::new());
} else {
info!(
?peer_list_error,
"could not load cached peer list, using default seed peers"
);
return Err(peer_list_error);
}
}
};
// Skip and log addresses that don't parse, and automatically deduplicate using the HashSet.
// (These issues shouldn't happen unless users modify the file.)
let peer_list: HashSet<PeerSocketAddr> = peer_list
.lines()
.filter_map(|peer| {
peer.parse()
.map_err(|peer_parse_error| {
info!(
?peer_parse_error,
"invalid peer address in cached peer list, skipping"
);
peer_parse_error
})
.ok()
})
.collect();
// This log is needed for user debugging, but it's annoying during tests.
#[cfg(not(test))]
info!(
cached_ip_count = ?peer_list.len(),
?peer_cache_file,
"loaded cached peer IP addresses"
);
#[cfg(test)]
debug!(
cached_ip_count = ?peer_list.len(),
?peer_cache_file,
"loaded cached peer IP addresses"
);
for ip in &peer_list {
// Count each initial peer, recording the cache file and loaded IP address.
//
// If an IP is returned by DNS seeders and the cache,
// each duplicate adds 1 to the initial peer count.
// (But we only make one initial connection attempt to each IP.)
metrics::counter!(
"zcash.net.peers.initial",
1,
"cache" => peer_cache_file.display().to_string(),
"remote_ip" => ip.to_string()
);
}
Ok(peer_list)
}
/// Atomically writes a new `peer_list` to the peer list cache file, if configured.
/// If the list is empty, keeps the previous cache file.
///
/// Also creates the peer cache directory, if it doesn't already exist.
///
/// Atomic writes avoid corrupting the cache if Zebra panics or crashes, or if multiple Zebra
/// instances try to read and write the same cache file.
pub async fn update_peer_cache(&self, peer_list: HashSet<PeerSocketAddr>) -> io::Result<()> {
let Some(peer_cache_file) = self.cache_dir.peer_cache_file_path(self.network) else {
return Ok(());
};
if peer_list.is_empty() {
info!(
?peer_cache_file,
"cacheable peer list was empty, keeping previous cache"
);
return Ok(());
}
// Turn IP addresses into strings
let mut peer_list: Vec<String> = peer_list
.iter()
.take(MAX_PEER_DISK_CACHE_SIZE)
.map(|redacted_peer| redacted_peer.remove_socket_addr_privacy().to_string())
.collect();
// # Privacy
//
// Sort to destroy any peer order, which could leak peer connection times.
// (Currently the HashSet argument does this as well.)
peer_list.sort();
// Make a newline-separated list
let peer_data = peer_list.join("\n");
// Write to a temporary file, so the cache is not corrupted if Zebra shuts down or crashes
// at the same time.
//
// # Concurrency
//
// We want to use async code to avoid blocking the tokio executor on filesystem operations,
// but `tempfile` is implemented using non-asyc methods. So we wrap its filesystem
// operations in `tokio::spawn_blocking()`.
//
// TODO: split this out into an atomic_write_to_tmp_file() method if we need to re-use it
// Create the peer cache directory if needed
let peer_cache_dir = peer_cache_file
.parent()
.expect("cache path always has a network directory")
.to_owned();
tokio::fs::create_dir_all(&peer_cache_dir).await?;
// Give the temporary file a similar name to the permanent cache file,
// but hide it in directory listings.
let mut tmp_peer_cache_prefix: OsString = ".tmp.".into();
tmp_peer_cache_prefix.push(
peer_cache_file
.file_name()
.expect("cache file always has a file name"),
);
// Create the temporary file.
// Do blocking filesystem operations on a dedicated thread.
let tmp_peer_cache_file = tokio::task::spawn_blocking(move || {
// Put the temporary file in the same directory as the permanent file,
// so atomic filesystem operations are possible.
tempfile::Builder::new()
.prefix(&tmp_peer_cache_prefix)
.tempfile_in(peer_cache_dir)
})
.await
.expect("unexpected panic creating temporary peer cache file")?;
// Write the list to the file asynchronously, by extracting the inner file, using it,
// then combining it back into a type that will correctly drop the file on error.
let (tmp_peer_cache_file, tmp_peer_cache_path) = tmp_peer_cache_file.into_parts();
let mut tmp_peer_cache_file = tokio::fs::File::from_std(tmp_peer_cache_file);
tmp_peer_cache_file.write_all(peer_data.as_bytes()).await?;
let tmp_peer_cache_file =
NamedTempFile::from_parts(tmp_peer_cache_file, tmp_peer_cache_path);
// Atomically replace the current cache with the temporary cache.
// Do blocking filesystem operations on a dedicated thread.
tokio::task::spawn_blocking(move || {
let result = tmp_peer_cache_file.persist(&peer_cache_file);
// Drops the temp file if needed
match result {
Ok(_temp_file) => {
info!(
cached_ip_count = ?peer_list.len(),
?peer_cache_file,
"updated cached peer IP addresses"
);
for ip in &peer_list {
metrics::counter!(
"zcash.net.peers.cache",
1,
"cache" => peer_cache_file.display().to_string(),
"remote_ip" => ip.to_string()
);
}
Ok(())
}
Err(error) => Err(error.error),
}
})
.await
.expect("unexpected panic making temporary peer cache file permanent")
}
}
impl Default for Config {
@ -300,6 +578,7 @@ impl Default for Config {
network: Network::Mainnet,
initial_mainnet_peers: mainnet_peers,
initial_testnet_peers: testnet_peers,
cache_dir: CacheDir::default(),
crawl_new_peer_interval: DEFAULT_CRAWL_NEW_PEER_INTERVAL,
// # Security
@ -326,6 +605,7 @@ impl<'de> Deserialize<'de> for Config {
network: Network,
initial_mainnet_peers: IndexSet<String>,
initial_testnet_peers: IndexSet<String>,
cache_dir: CacheDir,
peerset_initial_target_size: usize,
#[serde(alias = "new_peer_interval", with = "humantime_serde")]
crawl_new_peer_interval: Duration,
@ -339,6 +619,7 @@ impl<'de> Deserialize<'de> for Config {
network: config.network,
initial_mainnet_peers: config.initial_mainnet_peers,
initial_testnet_peers: config.initial_testnet_peers,
cache_dir: config.cache_dir,
peerset_initial_target_size: config.peerset_initial_target_size,
crawl_new_peer_interval: config.crawl_new_peer_interval,
}
@ -362,6 +643,7 @@ impl<'de> Deserialize<'de> for Config {
network: config.network,
initial_mainnet_peers: config.initial_mainnet_peers,
initial_testnet_peers: config.initial_testnet_peers,
cache_dir: config.cache_dir,
peerset_initial_target_size: config.peerset_initial_target_size,
crawl_new_peer_interval: config.crawl_new_peer_interval,
})

View File

@ -0,0 +1,74 @@
//! Cache directory configuration for zebra-network.
use std::path::{Path, PathBuf};
use zebra_chain::parameters::Network;
/// A cache directory config field.
///
/// This cache directory configuration field is optional.
/// It defaults to being enabled with the default config path,
/// but also allows a custom path to be set.
#[derive(Clone, Debug, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
#[serde(untagged)]
pub enum CacheDir {
/// Whether the cache directory is enabled with the default path (`true`),
/// or disabled (`false`).
IsEnabled(bool),
/// Enable the cache directory and use a custom path.
CustomPath(PathBuf),
}
impl CacheDir {
/// Returns a `CacheDir` enabled with the default path.
pub fn default_path() -> Self {
Self::IsEnabled(true)
}
/// Returns a disabled `CacheDir`.
pub fn disabled() -> Self {
Self::IsEnabled(false)
}
/// Returns a custom `CacheDir` enabled with `path`.
pub fn custom_path(path: impl AsRef<Path>) -> Self {
Self::CustomPath(path.as_ref().to_owned())
}
/// Returns `true` if this `CacheDir` is enabled with the default or a custom path.
pub fn is_enabled(&self) -> bool {
match self {
CacheDir::IsEnabled(is_enabled) => *is_enabled,
CacheDir::CustomPath(_) => true,
}
}
/// Returns the peer cache file path for `network`, if enabled.
pub fn peer_cache_file_path(&self, network: Network) -> Option<PathBuf> {
Some(
self.cache_dir()?
.join("network")
.join(format!("{}.peers", network.lowercase_name())),
)
}
/// Returns the `zebra-network` base cache directory, if enabled.
pub fn cache_dir(&self) -> Option<PathBuf> {
match self {
Self::IsEnabled(is_enabled) => is_enabled.then(|| {
dirs::cache_dir()
.unwrap_or_else(|| std::env::current_dir().unwrap().join("cache"))
.join("zebra")
}),
Self::CustomPath(cache_dir) => Some(cache_dir.to_owned()),
}
}
}
impl Default for CacheDir {
fn default() -> Self {
Self::default_path()
}
}

View File

@ -140,6 +140,22 @@ pub const INVENTORY_ROTATION_INTERVAL: Duration = Duration::from_secs(53);
/// don't synchronise with other crawls.
pub const DEFAULT_CRAWL_NEW_PEER_INTERVAL: Duration = Duration::from_secs(61);
/// The peer address disk cache update interval.
///
/// This should be longer than [`DEFAULT_CRAWL_NEW_PEER_INTERVAL`],
/// but shorter than [`MAX_PEER_ACTIVE_FOR_GOSSIP`].
///
/// We use a short interval so Zebra instances which are restarted frequently
/// still have useful caches.
pub const PEER_DISK_CACHE_UPDATE_INTERVAL: Duration = Duration::from_secs(5 * 60);
/// The maximum number of addresses in the peer disk cache.
///
/// This is chosen to be less than the number of active peers,
/// and approximately the same as the number of seed peers returned by DNS.
/// It is a tradeoff between fingerprinting attacks, DNS pollution risk, and cache pollution risk.
pub const MAX_PEER_DISK_CACHE_SIZE: usize = 75;
/// The maximum duration since a peer was last seen to consider it reachable.
///
/// This is used to prevent Zebra from gossiping addresses that are likely unreachable. Peers that

View File

@ -103,6 +103,7 @@
//!
//! Peer Inventory Service:
//! * tracks gossiped `inv` advertisements for each peer
//! * updated before each `PeerSet` request is processed
//! * tracks missing inventory for each peer
//! * used by the `PeerSet` to route block and transaction requests
//! to peers that have the requested data
@ -112,10 +113,15 @@
//! [`AddressBook`] Service:
//! * maintains a list of peer addresses and associated connection attempt metadata
//! * address book metadata is used to prioritise peer connection attempts
//! * updated by an independent thread based on peer connection status changes
//! * caches peer addresses to disk regularly using an independent task
//!
//! Initial Seed Peer Task:
//! * initiates new outbound peer connections to seed peers, resolving them via DNS if required
//! * adds seed peer addresses to the [`AddressBook`]
//! On startup:
//! * loads seed peers from the config, resolving them via DNS if required
//! * loads cached peer addresses from disk
//! * initiates new outbound peer connections to seed and cached peers
//! * adds seed and cached peer addresses to the [`AddressBook`]
//!
//! Peer Crawler Task:
//! * discovers new peer addresses by sending `Addr` requests to connected peers
@ -151,6 +157,7 @@ pub mod constants;
mod isolated;
mod meta_addr;
mod peer;
mod peer_cache_updater;
mod peer_set;
mod policies;
mod protocol;
@ -174,7 +181,7 @@ pub use crate::{
pub use crate::{
address_book::AddressBook,
address_book_peers::AddressBookPeers,
config::Config,
config::{CacheDir, Config},
isolated::{connect_isolated, connect_isolated_tcp_direct},
meta_addr::{PeerAddrState, PeerSocketAddr},
peer::{Client, ConnectedAddr, ConnectionInfo, HandshakeError, PeerError, SharedPeerError},

View File

@ -64,3 +64,11 @@ impl DerefMut for PeerSocketAddr {
&mut self.0
}
}
impl PeerSocketAddr {
/// Return the underlying [`SocketAddr`], which allows sensitive peer address information to
/// be printed and logged.
pub fn remove_socket_addr_privacy(&self) -> SocketAddr {
**self
}
}

View File

@ -0,0 +1,63 @@
//! An async task that regularly updates the peer cache on disk from the current address book.
use std::{
io,
sync::{Arc, Mutex},
};
use chrono::Utc;
use tokio::time::sleep;
use crate::{
constants::{DNS_LOOKUP_TIMEOUT, PEER_DISK_CACHE_UPDATE_INTERVAL},
meta_addr::MetaAddr,
AddressBook, BoxError, Config,
};
/// An ongoing task that regularly caches the current `address_book` to disk, based on `config`.
pub async fn peer_cache_updater(
config: Config,
address_book: Arc<Mutex<AddressBook>>,
) -> Result<(), BoxError> {
// Wait until we've queried DNS and (hopefully) sent peers to the address book.
// Ideally we'd wait for at least one peer crawl, but that makes tests very slow.
//
// TODO: turn the initial sleep time into a parameter of this function,
// and allow it to be set in tests
sleep(DNS_LOOKUP_TIMEOUT * 2).await;
loop {
// Ignore errors because updating the cache is optional.
// Errors are already logged by the functions we're calling.
let _ = update_peer_cache_once(&config, &address_book).await;
sleep(PEER_DISK_CACHE_UPDATE_INTERVAL).await;
}
}
/// Caches peers from the current `address_book` to disk, based on `config`.
pub async fn update_peer_cache_once(
config: &Config,
address_book: &Arc<Mutex<AddressBook>>,
) -> io::Result<()> {
let peer_list = cacheable_peers(address_book)
.iter()
.map(|meta_addr| meta_addr.addr)
.collect();
config.update_peer_cache(peer_list).await
}
/// Returns a list of cacheable peers, blocking for as short a time as possible.
fn cacheable_peers(address_book: &Arc<Mutex<AddressBook>>) -> Vec<MetaAddr> {
// TODO: use spawn_blocking() here, if needed to handle address book mutex load
let now = Utc::now();
// # Concurrency
//
// We return from this function immediately to make sure the address book is unlocked.
address_book
.lock()
.expect("unexpected panic in previous thread while accessing the address book")
.cacheable(now)
}

View File

@ -38,6 +38,7 @@ use crate::{
self, address_is_valid_for_inbound_listeners, HandshakeRequest, MinimumPeerVersion,
OutboundConnectorRequest, PeerPreference,
},
peer_cache_updater::peer_cache_updater,
peer_set::{set::MorePeers, ActiveConnectionCounter, CandidateSet, ConnectionTracker, PeerSet},
AddressBook, BoxError, Config, PeerSocketAddr, Request, Response,
};
@ -186,7 +187,7 @@ where
);
let listen_guard = tokio::spawn(listen_fut.in_current_span());
// 2. Initial peers, specified in the config.
// 2. Initial peers, specified in the config and cached on disk.
let initial_peers_fut = add_initial_peers(
config.clone(),
outbound_connector.clone(),
@ -224,8 +225,9 @@ where
let _ = demand_tx.try_send(MorePeers);
}
// Start the peer crawler
let crawl_fut = crawl_and_dial(
config,
config.clone(),
demand_tx,
demand_rx,
candidates,
@ -235,15 +237,24 @@ where
);
let crawl_guard = tokio::spawn(crawl_fut.in_current_span());
// Start the peer disk cache updater
let peer_cache_updater_fut = peer_cache_updater(config, address_book.clone());
let peer_cache_updater_guard = tokio::spawn(peer_cache_updater_fut.in_current_span());
handle_tx
.send(vec![listen_guard, crawl_guard, address_book_updater_guard])
.send(vec![
listen_guard,
crawl_guard,
address_book_updater_guard,
peer_cache_updater_guard,
])
.unwrap();
(peer_set, address_book)
}
/// Use the provided `outbound_connector` to connect to the configured initial peers,
/// then send the resulting peer connections over `peerset_tx`.
/// Use the provided `outbound_connector` to connect to the configured DNS seeder and
/// disk cache initial peers, then send the resulting peer connections over `peerset_tx`.
///
/// Also sends every initial peer address to the `address_book_updater`.
#[instrument(skip(config, outbound_connector, peerset_tx, address_book_updater))]
@ -273,9 +284,12 @@ where
"Outbound Connections",
);
// TODO: update when we add Tor peers or other kinds of addresses.
let ipv4_peer_count = initial_peers.iter().filter(|ip| ip.is_ipv4()).count();
let ipv6_peer_count = initial_peers.iter().filter(|ip| ip.is_ipv6()).count();
info!(
initial_peer_count = ?initial_peers.len(),
?initial_peers,
?ipv4_peer_count,
?ipv6_peer_count,
"connecting to initial peer set"
);
@ -385,7 +399,7 @@ where
?handshake_success_total,
?handshake_error_total,
?outbound_connections,
"finished connecting to initial seed peers"
"finished connecting to initial seed and disk cache peers"
);
Ok(active_outbound_connections)
@ -423,10 +437,10 @@ async fn limit_initial_peers(
.entry(preference)
.or_default()
.push(peer_addr),
Err(error) => warn!(
Err(error) => info!(
?peer_addr,
?error,
"invalid initial peer from DNS seeder or configured IP address",
"invalid initial peer from DNS seeder, configured IP address, or disk cache",
),
}
}

View File

@ -31,6 +31,7 @@ use zebra_test::net::random_known_port;
use crate::{
address_book_updater::AddressBookUpdater,
config::CacheDir,
constants, init,
meta_addr::{MetaAddr, PeerAddrState},
peer::{self, ClientTestHarness, HandshakeRequest, OutboundConnectorRequest},
@ -53,6 +54,11 @@ use Network::*;
/// Using a very short time can make the crawler not run at all.
const CRAWLER_TEST_DURATION: Duration = Duration::from_secs(10);
/// The amount of time to run the peer cache updater task, before testing what it has done.
///
/// Using a very short time can make the peer cache updater not run at all.
const PEER_CACHE_UPDATER_TEST_DURATION: Duration = Duration::from_secs(25);
/// The amount of time to run the listener, before testing what it has done.
///
/// Using a very short time can make the listener not run at all.
@ -288,6 +294,89 @@ async fn peer_limit_two_testnet() {
// Any number of address book peers is valid here, because some peers might have failed.
}
/// Test zebra-network writes a peer cache file, and can read it back manually.
#[tokio::test]
async fn written_peer_cache_can_be_read_manually() {
let _init_guard = zebra_test::init();
if zebra_test::net::zebra_skip_network_tests() {
return;
}
let nil_inbound_service = service_fn(|_| async { Ok(Response::Nil) });
// The default config should have an active peer cache
let config = Config::default();
let address_book =
init_with_peer_limit(25, nil_inbound_service, Mainnet, None, config.clone()).await;
// Let the peer cache updater run for a while.
tokio::time::sleep(PEER_CACHE_UPDATER_TEST_DURATION).await;
let approximate_peer_count = address_book
.lock()
.expect("previous thread panicked while holding address book lock")
.len();
if approximate_peer_count > 0 {
let cached_peers = config
.load_peer_cache()
.await
.expect("unexpected error reading peer cache");
assert!(
!cached_peers.is_empty(),
"unexpected empty peer cache from manual load: {:?}",
config.cache_dir.peer_cache_file_path(config.network)
);
}
}
/// Test zebra-network writes a peer cache file, and reads it back automatically.
#[tokio::test]
async fn written_peer_cache_is_automatically_read_on_startup() {
let _init_guard = zebra_test::init();
if zebra_test::net::zebra_skip_network_tests() {
return;
}
let nil_inbound_service = service_fn(|_| async { Ok(Response::Nil) });
// The default config should have an active peer cache
let mut config = Config::default();
let address_book =
init_with_peer_limit(25, nil_inbound_service, Mainnet, None, config.clone()).await;
// Let the peer cache updater run for a while.
tokio::time::sleep(PEER_CACHE_UPDATER_TEST_DURATION).await;
let approximate_peer_count = address_book
.lock()
.expect("previous thread panicked while holding address book lock")
.len();
if approximate_peer_count > 0 {
// Make sure our only peers are coming from the disk cache
config.initial_mainnet_peers = Default::default();
let address_book =
init_with_peer_limit(25, nil_inbound_service, Mainnet, None, config.clone()).await;
// Let the peer cache reader run and fill the address book.
tokio::time::sleep(CRAWLER_TEST_DURATION).await;
// We should have loaded at least one peer from the cache
let approximate_cached_peer_count = address_book
.lock()
.expect("previous thread panicked while holding address book lock")
.len();
assert!(
approximate_cached_peer_count > 0,
"unexpected empty address book using cache from previous instance: {:?}",
config.cache_dir.peer_cache_file_path(config.network)
);
}
}
/// Test the crawler with an outbound peer limit of zero peers, and a connector that panics.
#[tokio::test]
async fn crawler_peer_limit_zero_connect_panic() {
@ -1126,6 +1215,7 @@ async fn self_connections_should_fail() {
initial_mainnet_peers: IndexSet::new(),
initial_testnet_peers: IndexSet::new(),
cache_dir: CacheDir::disabled(),
..Config::default()
};
@ -1371,6 +1461,7 @@ async fn local_listener_port_with(listen_addr: SocketAddr, network: Network) {
// Stop Zebra making outbound connections
initial_mainnet_peers: IndexSet::new(),
initial_testnet_peers: IndexSet::new(),
cache_dir: CacheDir::disabled(),
..Config::default()
};
@ -1706,6 +1797,8 @@ where
let config = Config {
initial_mainnet_peers: peers,
// We want exactly the above list of peers, without any cached peers.
cache_dir: CacheDir::disabled(),
network: Network::Mainnet,
listen_addr: unused_v4,

View File

@ -15,22 +15,19 @@ use zebra_chain::parameters::Network;
#[derive(Clone, Debug, Deserialize, Serialize)]
#[serde(deny_unknown_fields, default)]
pub struct Config {
/// The root directory for storing cached data.
/// The root directory for storing cached block data.
///
/// Cached data includes any state that can be replicated from the network
/// (e.g., the chain state, the blocks, the UTXO set, etc.). It does *not*
/// include private data that cannot be replicated from the network, such as
/// wallet data. That data is not handled by `zebra-state`.
/// If you change this directory, you might also want to change `network.cache_dir`.
///
/// Each state format version and network has a separate state.
/// These states are stored in `state/vN/mainnet` and `state/vN/testnet` subdirectories,
/// underneath the `cache_dir` path, where `N` is the state format version.
/// This cache stores permanent blockchain state that can be replicated from
/// the network, including the best chain, blocks, the UTXO set, and other indexes.
/// Any state that can be rolled back is only stored in memory.
///
/// When Zebra's state format changes, it creates a new state subdirectory for that version,
/// and re-syncs from genesis.
/// The `zebra-state` cache does *not* include any private data, such as wallet data.
///
/// It is ok to delete the entire cached state directory.
/// If you do, Zebra will re-sync from genesis next time it is launched.
/// You can delete the entire cached state directory, but it will impact your node's
/// readiness and network usage. If you do, Zebra will re-sync from genesis the next
/// time it is launched.
///
/// The default directory is platform dependent, based on
/// [`dirs::cache_dir()`](https://docs.rs/dirs/3.0.1/dirs/fn.cache_dir.html):
@ -48,6 +45,18 @@ pub struct Config {
/// directory for this file before running Zebra, and make sure the Zebra user
/// account has exclusive access to that directory, and other users can't modify
/// its parent directories.
///
/// # Implementation Details
///
/// Each state format version and network has a separate state.
/// These states are stored in `state/vN/mainnet` and `state/vN/testnet` subdirectories,
/// underneath the `cache_dir` path, where `N` is the state format version.
///
/// When Zebra's state format changes, it creates a new state subdirectory for that version,
/// and re-syncs from genesis.
///
/// Old state versions are automatically deleted at startup. You can also manually delete old
/// state versions.
pub cache_dir: PathBuf,
/// Whether to use an ephemeral database.
@ -100,10 +109,7 @@ fn gen_temp_path(prefix: &str) -> PathBuf {
impl Config {
/// Returns the path for the finalized state database
pub fn db_path(&self, network: Network) -> PathBuf {
let net_dir = match network {
Network::Mainnet => "mainnet",
Network::Testnet => "testnet",
};
let net_dir = network.lowercase_name();
if self.ephemeral {
gen_temp_path(&format!(

View File

@ -2,7 +2,7 @@
//!
//! ## Application Structure
//!
//! A zebra node consists of the following services and tasks:
//! A zebra node consists of the following major services and tasks:
//!
//! Peers:
//! * Peer Connection Pool Service
@ -12,6 +12,9 @@
//! * maintains a list of peer addresses, and connection priority metadata
//! * discovers new peer addresses from existing peer connections
//! * initiates new outbound peer connections in response to demand from tasks within this node
//! * Peer Cache Service
//! * Reads previous peer cache on startup, and adds it to the configured DNS seed peers
//! * Periodically updates the peer cache on disk from the latest address book state
//!
//! Blocks & Mempool Transactions:
//! * Consensus Service

View File

@ -20,7 +20,7 @@ use zebra_chain::{
};
use zebra_consensus::{error::TransactionError, router::RouterError, transaction};
use zebra_network::{
canonical_peer_addr, connect_isolated_tcp_direct_with_inbound, types::InventoryHash,
canonical_peer_addr, connect_isolated_tcp_direct_with_inbound, types::InventoryHash, CacheDir,
Config as NetworkConfig, InventoryResponse, PeerError, Request, Response, SharedPeerError,
};
use zebra_node_services::mempool;
@ -647,6 +647,7 @@ async fn setup(
// Stop Zebra making outbound connections
initial_mainnet_peers: IndexSet::new(),
initial_testnet_peers: IndexSet::new(),
cache_dir: CacheDir::disabled(),
..NetworkConfig::default()
};

View File

@ -165,7 +165,10 @@ use common::{
config::{
config_file_full_path, configs_dir, default_test_config, persistent_test_config, testdir,
},
launch::{spawn_zebrad_for_rpc, ZebradTestDirExt, BETWEEN_NODES_DELAY, LAUNCH_DELAY},
launch::{
spawn_zebrad_for_rpc, ZebradTestDirExt, BETWEEN_NODES_DELAY, EXTENDED_LAUNCH_DELAY,
LAUNCH_DELAY,
},
lightwalletd::{can_spawn_lightwalletd_for_rpc, spawn_lightwalletd_for_rpc},
sync::{
create_cached_database_height, sync_until, MempoolBehavior, LARGE_CHECKPOINT_TEST_HEIGHT,
@ -371,6 +374,7 @@ async fn db_init_outside_future_executor() -> Result<()> {
Ok(())
}
/// Check that the block state and peer list caches are written to disk.
#[test]
fn persistent_mode() -> Result<()> {
let _init_guard = zebra_test::init();
@ -381,7 +385,7 @@ fn persistent_mode() -> Result<()> {
let mut child = testdir.spawn_child(args!["-v", "start"])?;
// Run the program and kill it after a few seconds
std::thread::sleep(LAUNCH_DELAY);
std::thread::sleep(EXTENDED_LAUNCH_DELAY);
child.kill(false)?;
let output = child.wait_with_output()?;
@ -395,6 +399,13 @@ fn persistent_mode() -> Result<()> {
"state directory empty despite persistent state config"
);
let cache_dir = testdir.path().join("network");
assert_with_context!(
cache_dir.read_dir()?.count() > 0,
&output,
"network directory empty despite persistent network config"
);
Ok(())
}
@ -424,6 +435,9 @@ fn misconfigured_ephemeral_missing_directory() -> Result<()> {
)
}
/// Check that the state directory created on disk matches the state config.
///
/// TODO: do a similar test for `network.cache_dir`
#[tracing::instrument]
fn ephemeral(cache_dir_config: EphemeralConfig, cache_dir_check: EphemeralCheck) -> Result<()> {
use std::io::ErrorKind;
@ -449,7 +463,7 @@ fn ephemeral(cache_dir_config: EphemeralConfig, cache_dir_check: EphemeralCheck)
.with_config(&mut config)?
.spawn_child(args!["start"])?;
// Run the program and kill it after a few seconds
std::thread::sleep(LAUNCH_DELAY);
std::thread::sleep(EXTENDED_LAUNCH_DELAY);
child.kill(false)?;
let output = child.wait_with_output()?;
@ -472,7 +486,7 @@ fn ephemeral(cache_dir_config: EphemeralConfig, cache_dir_check: EphemeralCheck)
ignored_cache_dir.read_dir().unwrap().collect::<Vec<_>>()
);
["state", "zebrad.toml"].iter()
["state", "network", "zebrad.toml"].iter()
}
// we didn't create the state directory, so it should not exist
@ -490,7 +504,7 @@ fn ephemeral(cache_dir_config: EphemeralConfig, cache_dir_check: EphemeralCheck)
ignored_cache_dir.read_dir().unwrap().collect::<Vec<_>>()
);
["zebrad.toml"].iter()
["network", "zebrad.toml"].iter()
}
};
@ -754,7 +768,7 @@ fn last_config_is_stored() -> Result<()> {
Or run: \n\
cargo build {}--bin zebrad && \n\
zebrad generate | \n\
sed \"s/cache_dir = '.*'/cache_dir = 'cache_dir'/\" > \n\
sed 's/cache_dir = \".*\"/cache_dir = \"cache_dir\"/' > \n\
zebrad/tests/common/configs/{}<next-release-tag>.toml",
if cfg!(feature = "getblocktemplate-rpcs") {
GET_BLOCK_TEMPLATE_CONFIG_PREFIX

View File

@ -0,0 +1,74 @@
# Default configuration for zebrad.
#
# This file can be used as a skeleton for custom configs.
#
# Unspecified fields use default values. Optional fields are Some(field) if the
# field is present and None if it is absent.
#
# This file is generated as an example using zebrad's current defaults.
# You should set only the config options you want to keep, and delete the rest.
# Only a subset of fields are present in the skeleton, since optional values
# whose default is None are omitted.
#
# The config format (including a complete list of sections and fields) is
# documented here:
# https://doc.zebra.zfnd.org/zebrad/config/struct.ZebradConfig.html
#
# zebrad attempts to load configs in the following order:
#
# 1. The -c flag on the command line, e.g., `zebrad -c myconfig.toml start`;
# 2. The file `zebrad.toml` in the users's preference directory (platform-dependent);
# 3. The default config.
[consensus]
checkpoint_sync = true
debug_skip_parameter_preload = false
[mempool]
eviction_memory_time = "1h"
tx_cost_limit = 80000000
[metrics]
[mining]
debug_like_zcashd = true
[network]
cache_dir = true
crawl_new_peer_interval = "1m 1s"
initial_mainnet_peers = [
"dnsseed.z.cash:8233",
"dnsseed.str4d.xyz:8233",
"mainnet.seeder.zfnd.org:8233",
"mainnet.is.yolo.money:8233",
]
initial_testnet_peers = [
"dnsseed.testnet.z.cash:18233",
"testnet.seeder.zfnd.org:18233",
"testnet.is.yolo.money:18233",
]
listen_addr = "0.0.0.0:8233"
network = "Mainnet"
peerset_initial_target_size = 25
[rpc]
debug_force_finished_sync = false
parallel_cpu_threads = 0
[state]
cache_dir = "cache_dir"
delete_old_database = true
ephemeral = false
[sync]
checkpoint_verify_concurrency_limit = 1000
download_concurrency_limit = 50
full_verify_concurrency_limit = 20
parallel_cpu_threads = 0
[tracing]
buffer_limit = 128000
force_use_color = false
use_color = true
use_journald = false

View File

@ -0,0 +1,16 @@
# Custom network.cache_dir config parsing test
[network]
# Enable the peer address cache with a custom path
cache_dir = "/tmp"
# Use a custom seed peer config
# https://en.wikipedia.org/wiki/IPv6_address#Documentation
initial_mainnet_peers = [
"192.0.2.0:8233",
"2001:db8::0:8233",
]
initial_testnet_peers = [
"192.0.2.1:18233",
"2001:db8::1:18233",
]

View File

@ -0,0 +1,9 @@
# Custom network.cache_dir config parsing test
[network]
# Disable the peer address cache
cache_dir = false
# Disable seed peers as well, to create an isolated node
initial_mainnet_peers = []
initial_testnet_peers = []

View File

@ -0,0 +1,18 @@
# Custom network.cache_dir config parsing test
[network]
# Enable the peer address cache with the default path
cache_dir = true
# Use the default seed peer config
initial_mainnet_peers = [
"dnsseed.z.cash:8233",
"dnsseed.str4d.xyz:8233",
"mainnet.seeder.zfnd.org:8233",
"mainnet.is.yolo.money:8233",
]
initial_testnet_peers = [
"dnsseed.testnet.z.cash:18233",
"testnet.seeder.zfnd.org:18233",
"testnet.is.yolo.money:18233",
]

View File

@ -0,0 +1,71 @@
# Default configuration for zebrad.
#
# This file can be used as a skeleton for custom configs.
#
# Unspecified fields use default values. Optional fields are Some(field) if the
# field is present and None if it is absent.
#
# This file is generated as an example using zebrad's current defaults.
# You should set only the config options you want to keep, and delete the rest.
# Only a subset of fields are present in the skeleton, since optional values
# whose default is None are omitted.
#
# The config format (including a complete list of sections and fields) is
# documented here:
# https://doc.zebra.zfnd.org/zebrad/config/struct.ZebradConfig.html
#
# zebrad attempts to load configs in the following order:
#
# 1. The -c flag on the command line, e.g., `zebrad -c myconfig.toml start`;
# 2. The file `zebrad.toml` in the users's preference directory (platform-dependent);
# 3. The default config.
[consensus]
checkpoint_sync = true
debug_skip_parameter_preload = false
[mempool]
eviction_memory_time = "1h"
tx_cost_limit = 80000000
[metrics]
[network]
cache_dir = true
crawl_new_peer_interval = "1m 1s"
initial_mainnet_peers = [
"dnsseed.z.cash:8233",
"dnsseed.str4d.xyz:8233",
"mainnet.seeder.zfnd.org:8233",
"mainnet.is.yolo.money:8233",
]
initial_testnet_peers = [
"dnsseed.testnet.z.cash:18233",
"testnet.seeder.zfnd.org:18233",
"testnet.is.yolo.money:18233",
]
listen_addr = "0.0.0.0:8233"
network = "Mainnet"
peerset_initial_target_size = 25
[rpc]
debug_force_finished_sync = false
parallel_cpu_threads = 1
[state]
cache_dir = "cache_dir"
delete_old_database = true
ephemeral = false
[sync]
checkpoint_verify_concurrency_limit = 1000
download_concurrency_limit = 50
full_verify_concurrency_limit = 20
parallel_cpu_threads = 0
[tracing]
buffer_limit = 128000
force_use_color = false
use_color = true
use_journald = false

View File

@ -17,6 +17,7 @@ use indexmap::IndexSet;
use tempfile::TempDir;
use zebra_chain::parameters::Network;
use zebra_network::CacheDir;
use zebra_test::{
args,
command::{Arguments, TestDirExt},
@ -36,6 +37,10 @@ use crate::common::{
/// metrics or tracing test failures in Windows CI.
pub const LAUNCH_DELAY: Duration = Duration::from_secs(15);
/// After we launch `zebrad`, wait this long in extended tests.
/// See [`LAUNCH_DELAY`] for details.
pub const EXTENDED_LAUNCH_DELAY: Duration = Duration::from_secs(25);
/// After we launch `lightwalletd`, wait this long for the command to start up,
/// take the actions expected by the quick tests, and log the expected logs.
///
@ -167,9 +172,16 @@ where
}
fn cache_config_update_helper(self, config: &mut ZebradConfig) -> Result<Self> {
let dir = self.as_ref();
let cache_dir = PathBuf::from(dir);
// If the peer cache has already been disabled, don't re-enable it
if config.network.cache_dir.is_enabled() {
config.network.cache_dir = CacheDir::custom_path(&cache_dir);
}
// Only replace the state cache directory if it's going to be used
if !config.state.ephemeral {
let dir = self.as_ref();
let cache_dir = PathBuf::from(dir);
config.state.cache_dir = cache_dir;
}
@ -232,6 +244,8 @@ pub fn spawn_zebrad_for_rpc<S: AsRef<str> + std::fmt::Debug>(
if !use_internet_connection {
config.network.initial_mainnet_peers = IndexSet::new();
config.network.initial_testnet_peers = IndexSet::new();
// Avoid re-using cached peers from disk when we're supposed to be a disconnected instance
config.network.cache_dir = CacheDir::disabled();
config.mempool.debug_enable_at_height = Some(0);
}