mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: move metrics to its own crate (#9691)
Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
44
Cargo.lock
generated
44
Cargo.lock
generated
@ -6249,6 +6249,7 @@ dependencies = [
|
||||
"reth-node-core",
|
||||
"reth-node-ethereum",
|
||||
"reth-node-events",
|
||||
"reth-node-metrics",
|
||||
"reth-node-optimism",
|
||||
"reth-optimism-cli",
|
||||
"reth-optimism-primitives",
|
||||
@ -6542,7 +6543,6 @@ dependencies = [
|
||||
"futures",
|
||||
"human_bytes",
|
||||
"itertools 0.13.0",
|
||||
"metrics-process",
|
||||
"proptest",
|
||||
"proptest-arbitrary-interop",
|
||||
"ratatui",
|
||||
@ -6565,6 +6565,7 @@ dependencies = [
|
||||
"reth-node-builder",
|
||||
"reth-node-core",
|
||||
"reth-node-events",
|
||||
"reth-node-metrics",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
"reth-prune",
|
||||
@ -7674,6 +7675,7 @@ dependencies = [
|
||||
"reth-node-api",
|
||||
"reth-node-core",
|
||||
"reth-node-events",
|
||||
"reth-node-metrics",
|
||||
"reth-payload-builder",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
@ -7708,15 +7710,7 @@ dependencies = [
|
||||
"dirs-next",
|
||||
"eyre",
|
||||
"futures",
|
||||
"http 1.1.0",
|
||||
"humantime",
|
||||
"jsonrpsee",
|
||||
"metrics",
|
||||
"metrics-exporter-prometheus",
|
||||
"metrics-process",
|
||||
"metrics-util",
|
||||
"once_cell",
|
||||
"procfs",
|
||||
"proptest",
|
||||
"rand 0.8.5",
|
||||
"reth-chainspec",
|
||||
@ -7728,7 +7722,6 @@ dependencies = [
|
||||
"reth-discv4",
|
||||
"reth-discv5",
|
||||
"reth-fs-util",
|
||||
"reth-metrics",
|
||||
"reth-net-nat",
|
||||
"reth-network",
|
||||
"reth-network-p2p",
|
||||
@ -7744,15 +7737,12 @@ dependencies = [
|
||||
"reth-rpc-types-compat",
|
||||
"reth-stages-types",
|
||||
"reth-storage-errors",
|
||||
"reth-tasks",
|
||||
"reth-tracing",
|
||||
"reth-transaction-pool",
|
||||
"secp256k1",
|
||||
"serde_json",
|
||||
"shellexpand",
|
||||
"tikv-jemalloc-ctl",
|
||||
"tokio",
|
||||
"tower",
|
||||
"tracing",
|
||||
"vergen",
|
||||
]
|
||||
@ -7821,6 +7811,34 @@ dependencies = [
|
||||
"tracing",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "reth-node-metrics"
|
||||
version = "1.0.3"
|
||||
dependencies = [
|
||||
"eyre",
|
||||
"http 1.1.0",
|
||||
"jsonrpsee",
|
||||
"metrics",
|
||||
"metrics-exporter-prometheus",
|
||||
"metrics-process",
|
||||
"metrics-util",
|
||||
"once_cell",
|
||||
"procfs",
|
||||
"reqwest",
|
||||
"reth-chainspec",
|
||||
"reth-db",
|
||||
"reth-db-api",
|
||||
"reth-metrics",
|
||||
"reth-provider",
|
||||
"reth-tasks",
|
||||
"socket2 0.4.10",
|
||||
"tikv-jemalloc-ctl",
|
||||
"tokio",
|
||||
"tower",
|
||||
"tracing",
|
||||
"vergen",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "reth-node-optimism"
|
||||
version = "1.0.3"
|
||||
|
||||
@ -66,6 +66,7 @@ members = [
|
||||
"crates/node/api/",
|
||||
"crates/node/builder/",
|
||||
"crates/node/events/",
|
||||
"crates/node/metrics",
|
||||
"crates/optimism/cli",
|
||||
"crates/optimism/consensus",
|
||||
"crates/optimism/evm/",
|
||||
@ -334,6 +335,7 @@ reth-node-builder = { path = "crates/node/builder" }
|
||||
reth-node-core = { path = "crates/node/core" }
|
||||
reth-node-ethereum = { path = "crates/ethereum/node" }
|
||||
reth-node-events = { path = "crates/node/events" }
|
||||
reth-node-metrics = { path = "crates/node/metrics" }
|
||||
reth-node-optimism = { path = "crates/optimism/node" }
|
||||
reth-optimism-cli = { path = "crates/optimism/cli" }
|
||||
reth-optimism-consensus = { path = "crates/optimism/consensus" }
|
||||
|
||||
@ -83,7 +83,7 @@ default = ["jemalloc"]
|
||||
|
||||
asm-keccak = ["reth-primitives/asm-keccak"]
|
||||
|
||||
jemalloc = ["dep:tikv-jemallocator", "reth-node-core/jemalloc"]
|
||||
jemalloc = ["dep:tikv-jemallocator"]
|
||||
jemalloc-prof = ["jemalloc", "tikv-jemallocator?/profiling"]
|
||||
|
||||
min-error-logs = ["tracing/release_max_level_error"]
|
||||
|
||||
@ -65,6 +65,7 @@ reth-db-common.workspace = true
|
||||
reth-node-ethereum.workspace = true
|
||||
reth-node-builder.workspace = true
|
||||
reth-node-events.workspace = true
|
||||
reth-node-metrics.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
reth-optimism-primitives.workspace = true
|
||||
reth-engine-util.workspace = true
|
||||
@ -124,7 +125,7 @@ dev = ["reth-cli-commands/dev"]
|
||||
|
||||
asm-keccak = ["reth-primitives/asm-keccak"]
|
||||
|
||||
jemalloc = ["dep:tikv-jemallocator", "reth-node-core/jemalloc"]
|
||||
jemalloc = ["dep:tikv-jemallocator", "reth-node-metrics/jemalloc"]
|
||||
jemalloc-prof = ["jemalloc", "tikv-jemallocator?/profiling"]
|
||||
|
||||
min-error-logs = ["tracing/release_max_level_error"]
|
||||
|
||||
@ -59,9 +59,9 @@ pub mod core {
|
||||
pub use reth_node_core::*;
|
||||
}
|
||||
|
||||
/// Re-exported from `reth_node_core`.
|
||||
/// Re-exported from `reth_node_metrics`.
|
||||
pub mod prometheus_exporter {
|
||||
pub use reth_node_core::prometheus_exporter::*;
|
||||
pub use reth_node_metrics::recorder::*;
|
||||
}
|
||||
|
||||
/// Re-export of the `reth_node_core` types specifically in the `args` module.
|
||||
|
||||
@ -28,6 +28,7 @@ reth-network-p2p.workspace = true
|
||||
reth-node-builder.workspace = true
|
||||
reth-node-core.workspace = true
|
||||
reth-node-events.workspace = true
|
||||
reth-node-metrics.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-prune.workspace = true
|
||||
@ -63,9 +64,6 @@ ratatui = { version = "0.27", default-features = false, features = [
|
||||
"crossterm",
|
||||
] }
|
||||
|
||||
# metrics
|
||||
metrics-process.workspace = true
|
||||
|
||||
# reth test-vectors
|
||||
proptest = { workspace = true, optional = true }
|
||||
arbitrary = { workspace = true, optional = true }
|
||||
|
||||
@ -15,6 +15,7 @@ use reth_node_core::{
|
||||
node_config::NodeConfig,
|
||||
version,
|
||||
};
|
||||
use reth_node_metrics::recorder::install_prometheus_recorder;
|
||||
use std::{ffi::OsString, fmt, future::Future, net::SocketAddr, path::PathBuf, sync::Arc};
|
||||
|
||||
/// Start the node
|
||||
@ -173,7 +174,7 @@ impl<Ext: clap::Args + fmt::Debug> NodeCommand<Ext> {
|
||||
|
||||
// Register the prometheus recorder before creating the database,
|
||||
// because database init needs it to register metrics.
|
||||
let _ = node_config.install_prometheus_recorder()?;
|
||||
let _ = install_prometheus_recorder();
|
||||
|
||||
let data_dir = node_config.datadir();
|
||||
let db_path = data_dir.db();
|
||||
|
||||
@ -13,7 +13,15 @@ use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_node_core::{
|
||||
args::{NetworkArgs, StageEnum},
|
||||
prometheus_exporter,
|
||||
version::{
|
||||
BUILD_PROFILE_NAME, CARGO_PKG_VERSION, VERGEN_BUILD_TIMESTAMP, VERGEN_CARGO_FEATURES,
|
||||
VERGEN_CARGO_TARGET_TRIPLE, VERGEN_GIT_SHA,
|
||||
},
|
||||
};
|
||||
use reth_node_metrics::{
|
||||
hooks::Hooks,
|
||||
server::{MetricServer, MetricServerConfig},
|
||||
version::VersionInfo,
|
||||
};
|
||||
use reth_provider::{
|
||||
ChainSpecProvider, StageCheckpointReader, StageCheckpointWriter, StaticFileProviderFactory,
|
||||
@ -99,15 +107,24 @@ impl Command {
|
||||
|
||||
if let Some(listen_addr) = self.metrics {
|
||||
info!(target: "reth::cli", "Starting metrics endpoint at {}", listen_addr);
|
||||
prometheus_exporter::serve(
|
||||
let config = MetricServerConfig::new(
|
||||
listen_addr,
|
||||
prometheus_exporter::install_recorder()?,
|
||||
VersionInfo {
|
||||
version: CARGO_PKG_VERSION,
|
||||
build_timestamp: VERGEN_BUILD_TIMESTAMP,
|
||||
cargo_features: VERGEN_CARGO_FEATURES,
|
||||
git_sha: VERGEN_GIT_SHA,
|
||||
target_triple: VERGEN_CARGO_TARGET_TRIPLE,
|
||||
build_profile: BUILD_PROFILE_NAME,
|
||||
},
|
||||
ctx.task_executor,
|
||||
Hooks::new(
|
||||
provider_factory.db_ref().clone(),
|
||||
provider_factory.static_file_provider(),
|
||||
metrics_process::Collector::default(),
|
||||
ctx.task_executor,
|
||||
)
|
||||
.await?;
|
||||
),
|
||||
);
|
||||
|
||||
MetricServer::new(config).serve().await?;
|
||||
}
|
||||
|
||||
let batch_size = self.batch_size.unwrap_or(self.to.saturating_sub(self.from) + 1);
|
||||
|
||||
@ -29,6 +29,7 @@ reth-rpc-builder.workspace = true
|
||||
reth-rpc-layer.workspace = true
|
||||
reth-node-api.workspace = true
|
||||
reth-node-core.workspace = true
|
||||
reth-node-metrics.workspace = true
|
||||
reth-network.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-payload-builder.workspace = true
|
||||
|
||||
@ -25,6 +25,15 @@ use reth_node_api::FullNodeTypes;
|
||||
use reth_node_core::{
|
||||
dirs::{ChainPath, DataDirPath},
|
||||
node_config::NodeConfig,
|
||||
version::{
|
||||
BUILD_PROFILE_NAME, CARGO_PKG_VERSION, VERGEN_BUILD_TIMESTAMP, VERGEN_CARGO_FEATURES,
|
||||
VERGEN_CARGO_TARGET_TRIPLE, VERGEN_GIT_SHA,
|
||||
},
|
||||
};
|
||||
use reth_node_metrics::{
|
||||
hooks::Hooks,
|
||||
server::{MetricServer, MetricServerConfig},
|
||||
version::VersionInfo,
|
||||
};
|
||||
use reth_primitives::{BlockNumber, Head, B256};
|
||||
use reth_provider::{
|
||||
@ -483,15 +492,27 @@ where
|
||||
|
||||
/// Starts the prometheus endpoint.
|
||||
pub async fn start_prometheus_endpoint(&self) -> eyre::Result<()> {
|
||||
let prometheus_handle = self.node_config().install_prometheus_recorder()?;
|
||||
self.node_config()
|
||||
.start_metrics_endpoint(
|
||||
prometheus_handle,
|
||||
self.database().clone(),
|
||||
self.static_file_provider(),
|
||||
let listen_addr = self.node_config().metrics;
|
||||
if let Some(addr) = listen_addr {
|
||||
info!(target: "reth::cli", "Starting metrics endpoint at {}", addr);
|
||||
let config = MetricServerConfig::new(
|
||||
addr,
|
||||
VersionInfo {
|
||||
version: CARGO_PKG_VERSION,
|
||||
build_timestamp: VERGEN_BUILD_TIMESTAMP,
|
||||
cargo_features: VERGEN_CARGO_FEATURES,
|
||||
git_sha: VERGEN_GIT_SHA,
|
||||
target_triple: VERGEN_CARGO_TARGET_TRIPLE,
|
||||
build_profile: BUILD_PROFILE_NAME,
|
||||
},
|
||||
self.task_executor().clone(),
|
||||
)
|
||||
.await
|
||||
Hooks::new(self.database().clone(), self.static_file_provider()),
|
||||
);
|
||||
|
||||
MetricServer::new(config).serve().await?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Convenience function to [`Self::init_genesis`]
|
||||
|
||||
@ -6,8 +6,6 @@ mod exex;
|
||||
pub use common::LaunchContext;
|
||||
pub use exex::ExExLauncher;
|
||||
|
||||
use std::{future::Future, sync::Arc};
|
||||
|
||||
use futures::{future::Either, stream, stream_select, StreamExt};
|
||||
use reth_beacon_consensus::{
|
||||
hooks::{EngineHooks, PruneHook, StaticFileHook},
|
||||
@ -33,6 +31,7 @@ use reth_rpc_types::engine::ClientVersionV1;
|
||||
use reth_tasks::TaskExecutor;
|
||||
use reth_tracing::tracing::{debug, info};
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
use std::{future::Future, sync::Arc};
|
||||
use tokio::sync::{mpsc::unbounded_channel, oneshot};
|
||||
use tokio_stream::wrappers::UnboundedReceiverStream;
|
||||
|
||||
|
||||
@ -35,7 +35,6 @@ reth-discv4.workspace = true
|
||||
reth-discv5.workspace = true
|
||||
reth-net-nat.workspace = true
|
||||
reth-network-peers.workspace = true
|
||||
reth-tasks.workspace = true
|
||||
reth-consensus-common.workspace = true
|
||||
reth-prune-types.workspace = true
|
||||
reth-stages-types.workspace = true
|
||||
@ -44,15 +43,6 @@ reth-stages-types.workspace = true
|
||||
alloy-genesis.workspace = true
|
||||
alloy-rpc-types-engine.workspace = true
|
||||
|
||||
# async
|
||||
tokio.workspace = true
|
||||
|
||||
# metrics
|
||||
reth-metrics.workspace = true
|
||||
metrics.workspace = true
|
||||
metrics-exporter-prometheus.workspace = true
|
||||
metrics-process.workspace = true
|
||||
metrics-util.workspace = true
|
||||
|
||||
# misc
|
||||
eyre.workspace = true
|
||||
@ -61,17 +51,13 @@ humantime.workspace = true
|
||||
const_format.workspace = true
|
||||
rand.workspace = true
|
||||
derive_more.workspace = true
|
||||
once_cell.workspace = true
|
||||
|
||||
# io
|
||||
dirs-next = "2.0.0"
|
||||
shellexpand.workspace = true
|
||||
serde_json.workspace = true
|
||||
|
||||
# http/rpc
|
||||
http.workspace = true
|
||||
jsonrpsee.workspace = true
|
||||
tower.workspace = true
|
||||
|
||||
|
||||
# tracing
|
||||
tracing.workspace = true
|
||||
@ -86,15 +72,11 @@ secp256k1 = { workspace = true, features = [
|
||||
# async
|
||||
futures.workspace = true
|
||||
|
||||
[target.'cfg(unix)'.dependencies]
|
||||
tikv-jemalloc-ctl = { version = "0.5.0", optional = true }
|
||||
|
||||
[target.'cfg(target_os = "linux")'.dependencies]
|
||||
procfs = "0.16.0"
|
||||
|
||||
[dev-dependencies]
|
||||
# test vectors generation
|
||||
proptest.workspace = true
|
||||
tokio.workspace = true
|
||||
|
||||
[features]
|
||||
optimism = [
|
||||
@ -105,7 +87,7 @@ optimism = [
|
||||
"reth-rpc-eth-types/optimism",
|
||||
]
|
||||
|
||||
jemalloc = ["dep:tikv-jemalloc-ctl"]
|
||||
|
||||
|
||||
[build-dependencies]
|
||||
vergen = { version = "8.0.0", features = ["build", "cargo", "git", "gitcl"] }
|
||||
|
||||
@ -12,14 +12,9 @@ pub mod args;
|
||||
pub mod cli;
|
||||
pub mod dirs;
|
||||
pub mod exit;
|
||||
pub mod metrics;
|
||||
pub mod node_config;
|
||||
pub mod utils;
|
||||
pub mod version;
|
||||
|
||||
// Re-export for backwards compatibility.
|
||||
pub use metrics::prometheus_exporter;
|
||||
|
||||
/// Re-exported from `reth_primitives`.
|
||||
pub mod primitives {
|
||||
pub use reth_primitives::*;
|
||||
|
||||
@ -1,4 +0,0 @@
|
||||
//! Metrics utilities for the node.
|
||||
|
||||
pub mod prometheus_exporter;
|
||||
pub mod version_metrics;
|
||||
@ -1,319 +0,0 @@
|
||||
//! Prometheus exporter
|
||||
|
||||
use crate::metrics::version_metrics::VersionInfo;
|
||||
use eyre::WrapErr;
|
||||
use http::{header::CONTENT_TYPE, HeaderValue, Response};
|
||||
use metrics::describe_gauge;
|
||||
use metrics_exporter_prometheus::{PrometheusBuilder, PrometheusHandle};
|
||||
use metrics_util::layers::{PrefixLayer, Stack};
|
||||
use reth_db_api::database_metrics::DatabaseMetrics;
|
||||
use reth_metrics::metrics::Unit;
|
||||
use reth_provider::providers::StaticFileProvider;
|
||||
use reth_tasks::TaskExecutor;
|
||||
use std::{convert::Infallible, net::SocketAddr, sync::Arc};
|
||||
|
||||
pub(crate) trait Hook: Fn() + Send + Sync {}
|
||||
impl<T: Fn() + Send + Sync> Hook for T {}
|
||||
|
||||
/// Installs Prometheus as the metrics recorder.
|
||||
pub fn install_recorder() -> eyre::Result<PrometheusHandle> {
|
||||
let recorder = PrometheusBuilder::new().build_recorder();
|
||||
let handle = recorder.handle();
|
||||
|
||||
// Build metrics stack
|
||||
Stack::new(recorder)
|
||||
.push(PrefixLayer::new("reth"))
|
||||
.install()
|
||||
.wrap_err("Couldn't set metrics recorder.")?;
|
||||
|
||||
Ok(handle)
|
||||
}
|
||||
|
||||
/// Serves Prometheus metrics over HTTP with hooks.
|
||||
///
|
||||
/// The hooks are called every time the metrics are requested at the given endpoint, and can be used
|
||||
/// to record values for pull-style metrics, i.e. metrics that are not automatically updated.
|
||||
pub(crate) async fn serve_with_hooks<F: Hook + 'static>(
|
||||
listen_addr: SocketAddr,
|
||||
handle: PrometheusHandle,
|
||||
hooks: impl IntoIterator<Item = F>,
|
||||
task_executor: TaskExecutor,
|
||||
) -> eyre::Result<()> {
|
||||
let hooks: Vec<_> = hooks.into_iter().collect();
|
||||
|
||||
// Start endpoint
|
||||
start_endpoint(
|
||||
listen_addr,
|
||||
handle,
|
||||
Arc::new(move || hooks.iter().for_each(|hook| hook())),
|
||||
task_executor,
|
||||
)
|
||||
.await
|
||||
.wrap_err("Could not start Prometheus endpoint")?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Starts an endpoint at the given address to serve Prometheus metrics.
|
||||
async fn start_endpoint<F: Hook + 'static>(
|
||||
listen_addr: SocketAddr,
|
||||
handle: PrometheusHandle,
|
||||
hook: Arc<F>,
|
||||
task_executor: TaskExecutor,
|
||||
) -> eyre::Result<()> {
|
||||
let listener =
|
||||
tokio::net::TcpListener::bind(listen_addr).await.wrap_err("Could not bind to address")?;
|
||||
|
||||
task_executor.spawn_with_graceful_shutdown_signal(|mut signal| async move {
|
||||
loop {
|
||||
let io = tokio::select! {
|
||||
_ = &mut signal => break,
|
||||
io = listener.accept() => {
|
||||
match io {
|
||||
Ok((stream, _remote_addr)) => stream,
|
||||
Err(err) => {
|
||||
tracing::error!(%err, "failed to accept connection");
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
let handle = handle.clone();
|
||||
let hook = hook.clone();
|
||||
let service = tower::service_fn(move |_| {
|
||||
(hook)();
|
||||
let metrics = handle.render();
|
||||
let mut response = Response::new(metrics);
|
||||
response.headers_mut().insert(CONTENT_TYPE, HeaderValue::from_static("text/plain"));
|
||||
async move { Ok::<_, Infallible>(response) }
|
||||
});
|
||||
|
||||
let mut shutdown = signal.clone().ignore_guard();
|
||||
tokio::task::spawn(async move {
|
||||
if let Err(error) =
|
||||
jsonrpsee::server::serve_with_graceful_shutdown(io, service, &mut shutdown)
|
||||
.await
|
||||
{
|
||||
tracing::debug!(%error, "failed to serve request")
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Serves Prometheus metrics over HTTP with database and process metrics.
|
||||
pub async fn serve<Metrics>(
|
||||
listen_addr: SocketAddr,
|
||||
handle: PrometheusHandle,
|
||||
db: Metrics,
|
||||
static_file_provider: StaticFileProvider,
|
||||
process: metrics_process::Collector,
|
||||
task_executor: TaskExecutor,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
Metrics: DatabaseMetrics + 'static + Send + Sync,
|
||||
{
|
||||
let db_metrics_hook = move || db.report_metrics();
|
||||
let static_file_metrics_hook = move || {
|
||||
let _ = static_file_provider.report_metrics().map_err(
|
||||
|error| tracing::error!(%error, "Failed to report static file provider metrics"),
|
||||
);
|
||||
};
|
||||
|
||||
// Clone `process` to move it into the hook and use the original `process` for describe below.
|
||||
let cloned_process = process.clone();
|
||||
let hooks: Vec<Box<dyn Hook<Output = ()>>> = vec![
|
||||
Box::new(db_metrics_hook),
|
||||
Box::new(static_file_metrics_hook),
|
||||
Box::new(move || cloned_process.collect()),
|
||||
Box::new(collect_memory_stats),
|
||||
Box::new(collect_io_stats),
|
||||
];
|
||||
serve_with_hooks(listen_addr, handle, hooks, task_executor).await?;
|
||||
|
||||
// We describe the metrics after the recorder is installed, otherwise this information is not
|
||||
// registered
|
||||
describe_gauge!("db.table_size", Unit::Bytes, "The size of a database table (in bytes)");
|
||||
describe_gauge!("db.table_pages", "The number of database pages for a table");
|
||||
describe_gauge!("db.table_entries", "The number of entries for a table");
|
||||
describe_gauge!("db.freelist", "The number of pages on the freelist");
|
||||
describe_gauge!("db.page_size", Unit::Bytes, "The size of a database page (in bytes)");
|
||||
describe_gauge!(
|
||||
"db.timed_out_not_aborted_transactions",
|
||||
"Number of timed out transactions that were not aborted by the user yet"
|
||||
);
|
||||
|
||||
describe_gauge!("static_files.segment_size", Unit::Bytes, "The size of a static file segment");
|
||||
describe_gauge!("static_files.segment_files", "The number of files for a static file segment");
|
||||
describe_gauge!(
|
||||
"static_files.segment_entries",
|
||||
"The number of entries for a static file segment"
|
||||
);
|
||||
|
||||
process.describe();
|
||||
describe_memory_stats();
|
||||
describe_io_stats();
|
||||
VersionInfo::default().register_version_metrics();
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[cfg(all(feature = "jemalloc", unix))]
|
||||
fn collect_memory_stats() {
|
||||
use metrics::gauge;
|
||||
use tikv_jemalloc_ctl::{epoch, stats};
|
||||
use tracing::error;
|
||||
|
||||
if epoch::advance().map_err(|error| error!(%error, "Failed to advance jemalloc epoch")).is_err()
|
||||
{
|
||||
return
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::active::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.active"))
|
||||
{
|
||||
gauge!("jemalloc.active").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::allocated::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.allocated"))
|
||||
{
|
||||
gauge!("jemalloc.allocated").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::mapped::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.mapped"))
|
||||
{
|
||||
gauge!("jemalloc.mapped").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::metadata::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.metadata"))
|
||||
{
|
||||
gauge!("jemalloc.metadata").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::resident::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.resident"))
|
||||
{
|
||||
gauge!("jemalloc.resident").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::retained::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.retained"))
|
||||
{
|
||||
gauge!("jemalloc.retained").set(value as f64);
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(all(feature = "jemalloc", unix))]
|
||||
fn describe_memory_stats() {
|
||||
describe_gauge!(
|
||||
"jemalloc.active",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in active pages allocated by the application"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.allocated",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes allocated by the application"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.mapped",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in active extents mapped by the allocator"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.metadata",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes dedicated to jemalloc metadata"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.resident",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in physically resident data pages mapped by the allocator"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.retained",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in virtual memory mappings that were retained rather than \
|
||||
being returned to the operating system via e.g. munmap(2)"
|
||||
);
|
||||
}
|
||||
|
||||
#[cfg(not(all(feature = "jemalloc", unix)))]
|
||||
fn collect_memory_stats() {}
|
||||
|
||||
#[cfg(not(all(feature = "jemalloc", unix)))]
|
||||
fn describe_memory_stats() {}
|
||||
|
||||
#[cfg(target_os = "linux")]
|
||||
fn collect_io_stats() {
|
||||
use metrics::counter;
|
||||
use tracing::error;
|
||||
|
||||
let Ok(process) = procfs::process::Process::myself()
|
||||
.map_err(|error| error!(%error, "Failed to get currently running process"))
|
||||
else {
|
||||
return
|
||||
};
|
||||
|
||||
let Ok(io) = process.io().map_err(
|
||||
|error| error!(%error, "Failed to get IO stats for the currently running process"),
|
||||
) else {
|
||||
return
|
||||
};
|
||||
|
||||
counter!("io.rchar").absolute(io.rchar);
|
||||
counter!("io.wchar").absolute(io.wchar);
|
||||
counter!("io.syscr").absolute(io.syscr);
|
||||
counter!("io.syscw").absolute(io.syscw);
|
||||
counter!("io.read_bytes").absolute(io.read_bytes);
|
||||
counter!("io.write_bytes").absolute(io.write_bytes);
|
||||
counter!("io.cancelled_write_bytes").absolute(io.cancelled_write_bytes);
|
||||
}
|
||||
|
||||
#[cfg(target_os = "linux")]
|
||||
fn describe_io_stats() {
|
||||
use metrics::describe_counter;
|
||||
|
||||
describe_counter!("io.rchar", "Characters read");
|
||||
describe_counter!("io.wchar", "Characters written");
|
||||
describe_counter!("io.syscr", "Read syscalls");
|
||||
describe_counter!("io.syscw", "Write syscalls");
|
||||
describe_counter!("io.read_bytes", Unit::Bytes, "Bytes read");
|
||||
describe_counter!("io.write_bytes", Unit::Bytes, "Bytes written");
|
||||
describe_counter!("io.cancelled_write_bytes", Unit::Bytes, "Cancelled write bytes");
|
||||
}
|
||||
|
||||
#[cfg(not(target_os = "linux"))]
|
||||
const fn collect_io_stats() {}
|
||||
|
||||
#[cfg(not(target_os = "linux"))]
|
||||
const fn describe_io_stats() {}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::node_config::PROMETHEUS_RECORDER_HANDLE;
|
||||
|
||||
// Dependencies using different version of the `metrics` crate (to be exact, 0.21 vs 0.22)
|
||||
// may not be able to communicate with each other through the global recorder.
|
||||
//
|
||||
// This test ensures that `metrics-process` dependency plays well with the current
|
||||
// `metrics-exporter-prometheus` dependency version.
|
||||
#[test]
|
||||
fn process_metrics() {
|
||||
// initialize the lazy handle
|
||||
let _ = &*PROMETHEUS_RECORDER_HANDLE;
|
||||
|
||||
let process = metrics_process::Collector::default();
|
||||
process.describe();
|
||||
process.collect();
|
||||
|
||||
let metrics = PROMETHEUS_RECORDER_HANDLE.render();
|
||||
assert!(metrics.contains("process_cpu_seconds_total"), "{metrics:?}");
|
||||
}
|
||||
}
|
||||
@ -6,39 +6,27 @@ use crate::{
|
||||
PruningArgs, RpcServerArgs, TxPoolArgs,
|
||||
},
|
||||
dirs::{ChainPath, DataDirPath},
|
||||
metrics::prometheus_exporter,
|
||||
utils::get_single_header,
|
||||
};
|
||||
use metrics_exporter_prometheus::PrometheusHandle;
|
||||
use once_cell::sync::Lazy;
|
||||
use reth_chainspec::{ChainSpec, MAINNET};
|
||||
use reth_config::config::PruneConfig;
|
||||
use reth_db_api::{database::Database, database_metrics::DatabaseMetrics};
|
||||
use reth_db_api::database::Database;
|
||||
use reth_network_p2p::headers::client::HeadersClient;
|
||||
|
||||
use reth_primitives::{
|
||||
revm_primitives::EnvKzgSettings, BlockHashOrNumber, BlockNumber, Head, SealedHeader, B256,
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::StaticFileProvider, BlockHashReader, HeaderProvider, ProviderFactory,
|
||||
StageCheckpointReader,
|
||||
};
|
||||
use reth_provider::{BlockHashReader, HeaderProvider, ProviderFactory, StageCheckpointReader};
|
||||
use reth_stages_types::StageId;
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use reth_tasks::TaskExecutor;
|
||||
use std::{net::SocketAddr, path::PathBuf, sync::Arc};
|
||||
use tracing::*;
|
||||
|
||||
/// The default prometheus recorder handle. We use a global static to ensure that it is only
|
||||
/// installed once.
|
||||
pub static PROMETHEUS_RECORDER_HANDLE: Lazy<PrometheusHandle> =
|
||||
Lazy::new(|| prometheus_exporter::install_recorder().unwrap());
|
||||
|
||||
/// This includes all necessary configuration to launch the node.
|
||||
/// The individual configuration options can be overwritten before launching the node.
|
||||
///
|
||||
/// # Example
|
||||
/// ```rust
|
||||
/// # use reth_tasks::{TaskManager, TaskSpawner};
|
||||
/// # use reth_node_core::{
|
||||
/// # node_config::NodeConfig,
|
||||
/// # args::RpcServerArgs,
|
||||
@ -47,10 +35,6 @@ pub static PROMETHEUS_RECORDER_HANDLE: Lazy<PrometheusHandle> =
|
||||
/// # use tokio::runtime::Handle;
|
||||
///
|
||||
/// async fn t() {
|
||||
/// let handle = Handle::current();
|
||||
/// let manager = TaskManager::new(handle);
|
||||
/// let executor = manager.executor();
|
||||
///
|
||||
/// // create the builder
|
||||
/// let builder = NodeConfig::default();
|
||||
///
|
||||
@ -66,7 +50,6 @@ pub static PROMETHEUS_RECORDER_HANDLE: Lazy<PrometheusHandle> =
|
||||
///
|
||||
/// # Example
|
||||
/// ```rust
|
||||
/// # use reth_tasks::{TaskManager, TaskSpawner};
|
||||
/// # use reth_node_core::{
|
||||
/// # node_config::NodeConfig,
|
||||
/// # args::RpcServerArgs,
|
||||
@ -75,10 +58,6 @@ pub static PROMETHEUS_RECORDER_HANDLE: Lazy<PrometheusHandle> =
|
||||
/// # use tokio::runtime::Handle;
|
||||
///
|
||||
/// async fn t() {
|
||||
/// let handle = Handle::current();
|
||||
/// let manager = TaskManager::new(handle);
|
||||
/// let executor = manager.executor();
|
||||
///
|
||||
/// // create the builder with a test database, using the `test` method
|
||||
/// let builder = NodeConfig::test();
|
||||
///
|
||||
@ -284,38 +263,6 @@ impl NodeConfig {
|
||||
Ok(EnvKzgSettings::Default)
|
||||
}
|
||||
|
||||
/// Installs the prometheus recorder.
|
||||
pub fn install_prometheus_recorder(&self) -> eyre::Result<PrometheusHandle> {
|
||||
Ok(PROMETHEUS_RECORDER_HANDLE.clone())
|
||||
}
|
||||
|
||||
/// Serves the prometheus endpoint over HTTP with the given database and prometheus handle.
|
||||
pub async fn start_metrics_endpoint<Metrics>(
|
||||
&self,
|
||||
prometheus_handle: PrometheusHandle,
|
||||
db: Metrics,
|
||||
static_file_provider: StaticFileProvider,
|
||||
task_executor: TaskExecutor,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
Metrics: DatabaseMetrics + 'static + Send + Sync,
|
||||
{
|
||||
if let Some(listen_addr) = self.metrics {
|
||||
info!(target: "reth::cli", addr = %listen_addr, "Starting metrics endpoint");
|
||||
prometheus_exporter::serve(
|
||||
listen_addr,
|
||||
prometheus_handle,
|
||||
db,
|
||||
static_file_provider,
|
||||
metrics_process::Collector::default(),
|
||||
task_executor,
|
||||
)
|
||||
.await?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Fetches the head block from the database.
|
||||
///
|
||||
/// If the database is empty, returns the genesis block.
|
||||
|
||||
@ -20,6 +20,12 @@ pub const VERGEN_GIT_SHA: &str = const_format::str_index!(VERGEN_GIT_SHA_LONG, .
|
||||
/// The build timestamp.
|
||||
pub const VERGEN_BUILD_TIMESTAMP: &str = env!("VERGEN_BUILD_TIMESTAMP");
|
||||
|
||||
/// The target triple.
|
||||
pub const VERGEN_CARGO_TARGET_TRIPLE: &str = env!("VERGEN_CARGO_TARGET_TRIPLE");
|
||||
|
||||
/// The build features.
|
||||
pub const VERGEN_CARGO_FEATURES: &str = env!("VERGEN_CARGO_FEATURES");
|
||||
|
||||
/// The short version information for reth.
|
||||
///
|
||||
/// - The latest version from Cargo.toml
|
||||
@ -73,7 +79,8 @@ pub const LONG_VERSION: &str = const_format::concatcp!(
|
||||
BUILD_PROFILE_NAME
|
||||
);
|
||||
|
||||
pub(crate) const BUILD_PROFILE_NAME: &str = {
|
||||
/// The build profile name.
|
||||
pub const BUILD_PROFILE_NAME: &str = {
|
||||
// Derived from https://stackoverflow.com/questions/73595435/how-to-get-profile-from-cargo-toml-in-build-rs-or-at-runtime
|
||||
// We split on the path separator of the *host* machine, which may be different from
|
||||
// `std::path::MAIN_SEPARATOR_STR`.
|
||||
|
||||
52
crates/node/metrics/Cargo.toml
Normal file
52
crates/node/metrics/Cargo.toml
Normal file
@ -0,0 +1,52 @@
|
||||
[package]
|
||||
name = "reth-node-metrics"
|
||||
version.workspace = true
|
||||
edition.workspace = true
|
||||
rust-version.workspace = true
|
||||
license.workspace = true
|
||||
homepage.workspace = true
|
||||
repository.workspace = true
|
||||
|
||||
[dependencies]
|
||||
reth-db-api.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-metrics.workspace = true
|
||||
reth-tasks.workspace = true
|
||||
|
||||
metrics.workspace = true
|
||||
metrics-exporter-prometheus.workspace = true
|
||||
metrics-process.workspace = true
|
||||
metrics-util.workspace = true
|
||||
|
||||
tokio.workspace = true
|
||||
|
||||
once_cell.workspace = true
|
||||
|
||||
jsonrpsee = { workspace = true, features = ["server"] }
|
||||
http.workspace = true
|
||||
tower.workspace = true
|
||||
|
||||
tracing.workspace = true
|
||||
eyre.workspace = true
|
||||
|
||||
[target.'cfg(unix)'.dependencies]
|
||||
tikv-jemalloc-ctl = { version = "0.5.0", optional = true }
|
||||
|
||||
[target.'cfg(target_os = "linux")'.dependencies]
|
||||
procfs = "0.16.0"
|
||||
|
||||
[dev-dependencies]
|
||||
reth-db = { workspace = true, features = ["test-utils"] }
|
||||
reqwest.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
socket2 = { version = "0.4", default-features = false }
|
||||
|
||||
[lints]
|
||||
workspace = true
|
||||
|
||||
[features]
|
||||
jemalloc = ["dep:tikv-jemalloc-ctl"]
|
||||
|
||||
|
||||
[build-dependencies]
|
||||
vergen = { version = "8.0.0", features = ["build", "cargo", "git", "gitcl"] }
|
||||
126
crates/node/metrics/src/hooks.rs
Normal file
126
crates/node/metrics/src/hooks.rs
Normal file
@ -0,0 +1,126 @@
|
||||
use metrics_process::Collector;
|
||||
use reth_db_api::database_metrics::DatabaseMetrics;
|
||||
use reth_provider::providers::StaticFileProvider;
|
||||
use std::{fmt, sync::Arc};
|
||||
pub(crate) trait Hook: Fn() + Send + Sync {}
|
||||
impl<T: Fn() + Send + Sync> Hook for T {}
|
||||
|
||||
impl fmt::Debug for Hooks {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
let hooks_len = self.inner.len();
|
||||
f.debug_struct("Hooks")
|
||||
.field("inner", &format!("Arc<Vec<Box<dyn Hook>>>, len: {}", hooks_len))
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
/// Helper type for managing hooks
|
||||
#[derive(Clone)]
|
||||
pub struct Hooks {
|
||||
inner: Arc<Vec<Box<dyn Hook<Output = ()>>>>,
|
||||
}
|
||||
|
||||
impl Hooks {
|
||||
/// Create a new set of hooks
|
||||
pub fn new<Metrics: DatabaseMetrics + 'static + Send + Sync>(
|
||||
db: Metrics,
|
||||
static_file_provider: StaticFileProvider,
|
||||
) -> Self {
|
||||
let hooks: Vec<Box<dyn Hook<Output = ()>>> = vec![
|
||||
Box::new(move || db.report_metrics()),
|
||||
Box::new(move || {
|
||||
let _ = static_file_provider.report_metrics().map_err(
|
||||
|error| tracing::error!(%error, "Failed to report static file provider metrics"),
|
||||
);
|
||||
}),
|
||||
Box::new(move || Collector::default().collect()),
|
||||
Box::new(collect_memory_stats),
|
||||
Box::new(collect_io_stats),
|
||||
];
|
||||
Self { inner: Arc::new(hooks) }
|
||||
}
|
||||
|
||||
pub(crate) fn iter(&self) -> impl Iterator<Item = &Box<dyn Hook<Output = ()>>> {
|
||||
self.inner.iter()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(all(feature = "jemalloc", unix))]
|
||||
fn collect_memory_stats() {
|
||||
use metrics::gauge;
|
||||
use tikv_jemalloc_ctl::{epoch, stats};
|
||||
use tracing::error;
|
||||
|
||||
if epoch::advance().map_err(|error| error!(%error, "Failed to advance jemalloc epoch")).is_err()
|
||||
{
|
||||
return
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::active::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.active"))
|
||||
{
|
||||
gauge!("jemalloc.active").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::allocated::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.allocated"))
|
||||
{
|
||||
gauge!("jemalloc.allocated").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::mapped::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.mapped"))
|
||||
{
|
||||
gauge!("jemalloc.mapped").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::metadata::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.metadata"))
|
||||
{
|
||||
gauge!("jemalloc.metadata").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::resident::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.resident"))
|
||||
{
|
||||
gauge!("jemalloc.resident").set(value as f64);
|
||||
}
|
||||
|
||||
if let Ok(value) = stats::retained::read()
|
||||
.map_err(|error| error!(%error, "Failed to read jemalloc.stats.retained"))
|
||||
{
|
||||
gauge!("jemalloc.retained").set(value as f64);
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(not(all(feature = "jemalloc", unix)))]
|
||||
const fn collect_memory_stats() {}
|
||||
|
||||
#[cfg(target_os = "linux")]
|
||||
fn collect_io_stats() {
|
||||
use metrics::counter;
|
||||
use tracing::error;
|
||||
|
||||
let Ok(process) = procfs::process::Process::myself()
|
||||
.map_err(|error| error!(%error, "Failed to get currently running process"))
|
||||
else {
|
||||
return
|
||||
};
|
||||
|
||||
let Ok(io) = process.io().map_err(
|
||||
|error| error!(%error, "Failed to get IO stats for the currently running process"),
|
||||
) else {
|
||||
return
|
||||
};
|
||||
|
||||
counter!("io.rchar").absolute(io.rchar);
|
||||
counter!("io.wchar").absolute(io.wchar);
|
||||
counter!("io.syscr").absolute(io.syscr);
|
||||
counter!("io.syscw").absolute(io.syscw);
|
||||
counter!("io.read_bytes").absolute(io.read_bytes);
|
||||
counter!("io.write_bytes").absolute(io.write_bytes);
|
||||
counter!("io.cancelled_write_bytes").absolute(io.cancelled_write_bytes);
|
||||
}
|
||||
|
||||
#[cfg(not(target_os = "linux"))]
|
||||
const fn collect_io_stats() {}
|
||||
18
crates/node/metrics/src/lib.rs
Normal file
18
crates/node/metrics/src/lib.rs
Normal file
@ -0,0 +1,18 @@
|
||||
//! Metrics utilities for the node.
|
||||
#![doc(
|
||||
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
|
||||
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
|
||||
issue_tracker_base_url = "https://github.com/paradigmxyz/reth/issues/"
|
||||
)]
|
||||
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
|
||||
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
||||
|
||||
/// The metrics hooks for prometheus.
|
||||
pub mod hooks;
|
||||
pub mod recorder;
|
||||
/// The metric server serving the metrics.
|
||||
pub mod server;
|
||||
pub mod version;
|
||||
|
||||
pub use metrics_exporter_prometheus::*;
|
||||
pub use metrics_process::*;
|
||||
58
crates/node/metrics/src/recorder.rs
Normal file
58
crates/node/metrics/src/recorder.rs
Normal file
@ -0,0 +1,58 @@
|
||||
//! Prometheus recorder
|
||||
|
||||
use eyre::WrapErr;
|
||||
use metrics_exporter_prometheus::{PrometheusBuilder, PrometheusHandle};
|
||||
use metrics_util::layers::{PrefixLayer, Stack};
|
||||
use once_cell::sync::Lazy;
|
||||
|
||||
/// Installs the Prometheus recorder as the global recorder.
|
||||
pub fn install_prometheus_recorder() -> &'static PrometheusHandle {
|
||||
&PROMETHEUS_RECORDER_HANDLE
|
||||
}
|
||||
|
||||
/// The default Prometheus recorder handle. We use a global static to ensure that it is only
|
||||
/// installed once.
|
||||
static PROMETHEUS_RECORDER_HANDLE: Lazy<PrometheusHandle> =
|
||||
Lazy::new(|| PrometheusRecorder::install().unwrap());
|
||||
|
||||
/// Prometheus recorder installer
|
||||
#[derive(Debug)]
|
||||
pub struct PrometheusRecorder;
|
||||
|
||||
impl PrometheusRecorder {
|
||||
/// Installs Prometheus as the metrics recorder.
|
||||
pub fn install() -> eyre::Result<PrometheusHandle> {
|
||||
let recorder = PrometheusBuilder::new().build_recorder();
|
||||
let handle = recorder.handle();
|
||||
|
||||
// Build metrics stack
|
||||
Stack::new(recorder)
|
||||
.push(PrefixLayer::new("reth"))
|
||||
.install()
|
||||
.wrap_err("Couldn't set metrics recorder.")?;
|
||||
|
||||
Ok(handle)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
// Dependencies using different version of the `metrics` crate (to be exact, 0.21 vs 0.22)
|
||||
// may not be able to communicate with each other through the global recorder.
|
||||
//
|
||||
// This test ensures that `metrics-process` dependency plays well with the current
|
||||
// `metrics-exporter-prometheus` dependency version.
|
||||
#[test]
|
||||
fn process_metrics() {
|
||||
// initialize the lazy handle
|
||||
let _ = &*PROMETHEUS_RECORDER_HANDLE;
|
||||
|
||||
let process = metrics_process::Collector::default();
|
||||
process.describe();
|
||||
process.collect();
|
||||
|
||||
let metrics = PROMETHEUS_RECORDER_HANDLE.render();
|
||||
assert!(metrics.contains("process_cpu_seconds_total"), "{metrics:?}");
|
||||
}
|
||||
}
|
||||
270
crates/node/metrics/src/server.rs
Normal file
270
crates/node/metrics/src/server.rs
Normal file
@ -0,0 +1,270 @@
|
||||
use crate::{
|
||||
hooks::{Hook, Hooks},
|
||||
recorder::install_prometheus_recorder,
|
||||
version::VersionInfo,
|
||||
};
|
||||
use eyre::WrapErr;
|
||||
use http::{header::CONTENT_TYPE, HeaderValue, Response};
|
||||
use metrics::describe_gauge;
|
||||
use metrics_process::Collector;
|
||||
use reth_metrics::metrics::Unit;
|
||||
use reth_tasks::TaskExecutor;
|
||||
use std::{convert::Infallible, net::SocketAddr, sync::Arc};
|
||||
use tracing::info;
|
||||
|
||||
/// Configuration for the [`MetricServer`]
|
||||
#[derive(Debug)]
|
||||
pub struct MetricServerConfig {
|
||||
listen_addr: SocketAddr,
|
||||
version_info: VersionInfo,
|
||||
task_executor: TaskExecutor,
|
||||
hooks: Hooks,
|
||||
}
|
||||
|
||||
impl MetricServerConfig {
|
||||
/// Create a new [`MetricServerConfig`] with the given configuration
|
||||
pub const fn new(
|
||||
listen_addr: SocketAddr,
|
||||
version_info: VersionInfo,
|
||||
task_executor: TaskExecutor,
|
||||
hooks: Hooks,
|
||||
) -> Self {
|
||||
Self { listen_addr, hooks, task_executor, version_info }
|
||||
}
|
||||
}
|
||||
|
||||
/// [`MetricServer`] responsible for serving the metrics endpoint
|
||||
#[derive(Debug)]
|
||||
pub struct MetricServer {
|
||||
config: MetricServerConfig,
|
||||
}
|
||||
|
||||
impl MetricServer {
|
||||
/// Create a new [`MetricServer`] with the given configuration
|
||||
pub const fn new(config: MetricServerConfig) -> Self {
|
||||
Self { config }
|
||||
}
|
||||
|
||||
/// Spawns the metrics server
|
||||
pub async fn serve(&self) -> eyre::Result<()> {
|
||||
let MetricServerConfig { listen_addr, hooks, task_executor, version_info } = &self.config;
|
||||
|
||||
info!(target: "reth::cli", addr = %listen_addr, "Starting metrics endpoint");
|
||||
|
||||
let hooks = hooks.clone();
|
||||
self.start_endpoint(
|
||||
*listen_addr,
|
||||
Arc::new(move || hooks.iter().for_each(|hook| hook())),
|
||||
task_executor.clone(),
|
||||
)
|
||||
.await
|
||||
.wrap_err("Could not start Prometheus endpoint")?;
|
||||
|
||||
// Describe metrics after recorder installation
|
||||
describe_db_metrics();
|
||||
describe_static_file_metrics();
|
||||
Collector::default().describe();
|
||||
describe_memory_stats();
|
||||
describe_io_stats();
|
||||
|
||||
version_info.register_version_metrics();
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn start_endpoint<F: Hook + 'static>(
|
||||
&self,
|
||||
listen_addr: SocketAddr,
|
||||
hook: Arc<F>,
|
||||
task_executor: TaskExecutor,
|
||||
) -> eyre::Result<()> {
|
||||
let listener = tokio::net::TcpListener::bind(listen_addr)
|
||||
.await
|
||||
.wrap_err("Could not bind to address")?;
|
||||
|
||||
task_executor.spawn_with_graceful_shutdown_signal(|mut signal| async move {
|
||||
loop {
|
||||
let io = tokio::select! {
|
||||
_ = &mut signal => break,
|
||||
io = listener.accept() => {
|
||||
match io {
|
||||
Ok((stream, _remote_addr)) => stream,
|
||||
Err(err) => {
|
||||
tracing::error!(%err, "failed to accept connection");
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
let handle = install_prometheus_recorder();
|
||||
let hook = hook.clone();
|
||||
let service = tower::service_fn(move |_| {
|
||||
(hook)();
|
||||
let metrics = handle.render();
|
||||
let mut response = Response::new(metrics);
|
||||
response
|
||||
.headers_mut()
|
||||
.insert(CONTENT_TYPE, HeaderValue::from_static("text/plain"));
|
||||
async move { Ok::<_, Infallible>(response) }
|
||||
});
|
||||
|
||||
let mut shutdown = signal.clone().ignore_guard();
|
||||
tokio::task::spawn(async move {
|
||||
if let Err(error) =
|
||||
jsonrpsee::server::serve_with_graceful_shutdown(io, service, &mut shutdown)
|
||||
.await
|
||||
{
|
||||
tracing::debug!(%error, "failed to serve request")
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn describe_db_metrics() {
|
||||
describe_gauge!("db.table_size", Unit::Bytes, "The size of a database table (in bytes)");
|
||||
describe_gauge!("db.table_pages", "The number of database pages for a table");
|
||||
describe_gauge!("db.table_entries", "The number of entries for a table");
|
||||
describe_gauge!("db.freelist", "The number of pages on the freelist");
|
||||
describe_gauge!("db.page_size", Unit::Bytes, "The size of a database page (in bytes)");
|
||||
describe_gauge!(
|
||||
"db.timed_out_not_aborted_transactions",
|
||||
"Number of timed out transactions that were not aborted by the user yet"
|
||||
);
|
||||
}
|
||||
|
||||
fn describe_static_file_metrics() {
|
||||
describe_gauge!("static_files.segment_size", Unit::Bytes, "The size of a static file segment");
|
||||
describe_gauge!("static_files.segment_files", "The number of files for a static file segment");
|
||||
describe_gauge!(
|
||||
"static_files.segment_entries",
|
||||
"The number of entries for a static file segment"
|
||||
);
|
||||
}
|
||||
|
||||
#[cfg(all(feature = "jemalloc", unix))]
|
||||
fn describe_memory_stats() {
|
||||
describe_gauge!(
|
||||
"jemalloc.active",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in active pages allocated by the application"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.allocated",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes allocated by the application"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.mapped",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in active extents mapped by the allocator"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.metadata",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes dedicated to jemalloc metadata"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.resident",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in physically resident data pages mapped by the allocator"
|
||||
);
|
||||
describe_gauge!(
|
||||
"jemalloc.retained",
|
||||
Unit::Bytes,
|
||||
"Total number of bytes in virtual memory mappings that were retained rather than \
|
||||
being returned to the operating system via e.g. munmap(2)"
|
||||
);
|
||||
}
|
||||
|
||||
#[cfg(not(all(feature = "jemalloc", unix)))]
|
||||
const fn describe_memory_stats() {}
|
||||
|
||||
#[cfg(target_os = "linux")]
|
||||
fn describe_io_stats() {
|
||||
use metrics::describe_counter;
|
||||
|
||||
describe_counter!("io.rchar", "Characters read");
|
||||
describe_counter!("io.wchar", "Characters written");
|
||||
describe_counter!("io.syscr", "Read syscalls");
|
||||
describe_counter!("io.syscw", "Write syscalls");
|
||||
describe_counter!("io.read_bytes", Unit::Bytes, "Bytes read");
|
||||
describe_counter!("io.write_bytes", Unit::Bytes, "Bytes written");
|
||||
describe_counter!("io.cancelled_write_bytes", Unit::Bytes, "Cancelled write bytes");
|
||||
}
|
||||
|
||||
#[cfg(not(target_os = "linux"))]
|
||||
const fn describe_io_stats() {}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use reqwest::Client;
|
||||
use reth_chainspec::MAINNET;
|
||||
use reth_db::{
|
||||
test_utils::{create_test_rw_db, create_test_static_files_dir, TempDatabase},
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::StaticFileProvider, ProviderFactory, StaticFileProviderFactory,
|
||||
};
|
||||
use reth_tasks::TaskManager;
|
||||
use socket2::{Domain, Socket, Type};
|
||||
use std::net::{SocketAddr, TcpListener};
|
||||
|
||||
fn create_test_db() -> ProviderFactory<Arc<TempDatabase<DatabaseEnv>>> {
|
||||
let (_, static_dir_path) = create_test_static_files_dir();
|
||||
ProviderFactory::new(
|
||||
create_test_rw_db(),
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
)
|
||||
}
|
||||
|
||||
fn get_random_available_addr() -> SocketAddr {
|
||||
let addr = &"127.0.0.1:0".parse::<SocketAddr>().unwrap().into();
|
||||
let socket = Socket::new(Domain::IPV4, Type::STREAM, None).unwrap();
|
||||
socket.set_reuse_address(true).unwrap();
|
||||
socket.bind(addr).unwrap();
|
||||
socket.listen(1).unwrap();
|
||||
let listener = TcpListener::from(socket);
|
||||
listener.local_addr().unwrap()
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_metrics_endpoint() {
|
||||
let version_info = VersionInfo {
|
||||
version: "test",
|
||||
build_timestamp: "test",
|
||||
cargo_features: "test",
|
||||
git_sha: "test",
|
||||
target_triple: "test",
|
||||
build_profile: "test",
|
||||
};
|
||||
|
||||
let tasks = TaskManager::current();
|
||||
let executor = tasks.executor();
|
||||
|
||||
let factory = create_test_db();
|
||||
let hooks = Hooks::new(factory.db_ref().clone(), factory.static_file_provider());
|
||||
|
||||
let listen_addr = get_random_available_addr();
|
||||
let config = MetricServerConfig::new(listen_addr, version_info, executor, hooks);
|
||||
|
||||
MetricServer::new(config).serve().await.unwrap();
|
||||
|
||||
// Send request to the metrics endpoint
|
||||
let url = format!("http://{}", listen_addr);
|
||||
let response = Client::new().get(&url).send().await.unwrap();
|
||||
assert!(response.status().is_success());
|
||||
|
||||
// Check the response body
|
||||
let body = response.text().await.unwrap();
|
||||
assert!(body.contains("reth_db_table_size"));
|
||||
assert!(body.contains("reth_jemalloc_metadata"));
|
||||
}
|
||||
}
|
||||
@ -1,6 +1,4 @@
|
||||
//! This exposes reth's version information over prometheus.
|
||||
|
||||
use crate::version::{BUILD_PROFILE_NAME, VERGEN_GIT_SHA};
|
||||
use metrics::gauge;
|
||||
|
||||
/// Contains version information for the application.
|
||||
@ -20,19 +18,6 @@ pub struct VersionInfo {
|
||||
pub build_profile: &'static str,
|
||||
}
|
||||
|
||||
impl Default for VersionInfo {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
version: env!("CARGO_PKG_VERSION"),
|
||||
build_timestamp: env!("VERGEN_BUILD_TIMESTAMP"),
|
||||
cargo_features: env!("VERGEN_CARGO_FEATURES"),
|
||||
git_sha: VERGEN_GIT_SHA,
|
||||
target_triple: env!("VERGEN_CARGO_TARGET_TRIPLE"),
|
||||
build_profile: BUILD_PROFILE_NAME,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl VersionInfo {
|
||||
/// This exposes reth's version information over prometheus.
|
||||
pub fn register_version_metrics(&self) {
|
||||
@ -45,7 +30,6 @@ impl VersionInfo {
|
||||
("build_profile", self.build_profile),
|
||||
];
|
||||
|
||||
let gauge = gauge!("info", &labels);
|
||||
gauge.set(1)
|
||||
let _gauge = gauge!("info", &labels);
|
||||
}
|
||||
}
|
||||
75
crates/node/metrics/src/version_metrics.rs
Normal file
75
crates/node/metrics/src/version_metrics.rs
Normal file
@ -0,0 +1,75 @@
|
||||
//! This exposes reth's version information over prometheus.
|
||||
use metrics::gauge;
|
||||
|
||||
/// The build timestamp.
|
||||
pub const VERGEN_BUILD_TIMESTAMP: &str = env!("VERGEN_BUILD_TIMESTAMP");
|
||||
/// The cargo features enabled for the build.
|
||||
pub const VERGEN_CARGO_FEATURES: &str = env!("VERGEN_CARGO_FEATURES");
|
||||
/// The target triple for the build.
|
||||
pub const VERGEN_CARGO_TARGET_TRIPLE: &str = env!("VERGEN_CARGO_TARGET_TRIPLE");
|
||||
/// The full SHA of the latest commit.
|
||||
pub const VERGEN_GIT_SHA_LONG: &str = env!("VERGEN_GIT_SHA");
|
||||
/// The 8 character short SHA of the latest commit.
|
||||
pub const VERGEN_GIT_SHA: &str = const_format::str_index!(VERGEN_GIT_SHA_LONG, ..8);
|
||||
|
||||
/// The build profile name.
|
||||
pub const BUILD_PROFILE_NAME: &str = {
|
||||
// Derived from https://stackoverflow.com/questions/73595435/how-to-get-profile-from-cargo-toml-in-build-rs-or-at-runtime
|
||||
// We split on the path separator of the *host* machine, which may be different from
|
||||
// `std::path::MAIN_SEPARATOR_STR`.
|
||||
const OUT_DIR: &str = env!("OUT_DIR");
|
||||
let unix_parts = const_format::str_split!(OUT_DIR, '/');
|
||||
if unix_parts.len() >= 4 {
|
||||
unix_parts[unix_parts.len() - 4]
|
||||
} else {
|
||||
let win_parts = const_format::str_split!(OUT_DIR, '\\');
|
||||
win_parts[win_parts.len() - 4]
|
||||
}
|
||||
};
|
||||
|
||||
/// Contains version information for the application.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct VersionInfo {
|
||||
/// The version of the application.
|
||||
pub version: &'static str,
|
||||
/// The build timestamp of the application.
|
||||
pub build_timestamp: &'static str,
|
||||
/// The cargo features enabled for the build.
|
||||
pub cargo_features: &'static str,
|
||||
/// The Git SHA of the build.
|
||||
pub git_sha: &'static str,
|
||||
/// The target triple for the build.
|
||||
pub target_triple: &'static str,
|
||||
/// The build profile (e.g., debug or release).
|
||||
pub build_profile: &'static str,
|
||||
}
|
||||
|
||||
impl Default for VersionInfo {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
version: env!("CARGO_PKG_VERSION"),
|
||||
build_timestamp: VERGEN_BUILD_TIMESTAMP,
|
||||
cargo_features: VERGEN_CARGO_FEATURES,
|
||||
git_sha: VERGEN_GIT_SHA,
|
||||
target_triple: VERGEN_CARGO_TARGET_TRIPLE,
|
||||
build_profile: BUILD_PROFILE_NAME,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl VersionInfo {
|
||||
/// This exposes reth's version information over prometheus.
|
||||
pub fn register_version_metrics(&self) {
|
||||
let labels: [(&str, &str); 6] = [
|
||||
("version", self.version),
|
||||
("build_timestamp", self.build_timestamp),
|
||||
("cargo_features", self.cargo_features),
|
||||
("git_sha", self.git_sha),
|
||||
("target_triple", self.target_triple),
|
||||
("build_profile", self.build_profile),
|
||||
];
|
||||
|
||||
let gauge = gauge!("info", &labels);
|
||||
gauge.set(1)
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user