feat(storage): database/transaction/cursor metrics (#5149)

This commit is contained in:
Alexey Shekhirin
2023-10-27 18:47:41 +01:00
committed by GitHub
parent 7019e023f5
commit a9fa281816
12 changed files with 954 additions and 181 deletions

View File

@ -25,6 +25,7 @@ use clap::{value_parser, Parser};
use eyre::Context; use eyre::Context;
use fdlimit::raise_fd_limit; use fdlimit::raise_fd_limit;
use futures::{future::Either, pin_mut, stream, stream_select, StreamExt}; use futures::{future::Either, pin_mut, stream, stream_select, StreamExt};
use metrics_exporter_prometheus::PrometheusHandle;
use reth_auto_seal_consensus::{AutoSealBuilder, AutoSealConsensus, MiningMode}; use reth_auto_seal_consensus::{AutoSealBuilder, AutoSealConsensus, MiningMode};
use reth_beacon_consensus::{ use reth_beacon_consensus::{
hooks::{EngineHooks, PruneHook}, hooks::{EngineHooks, PruneHook},
@ -72,7 +73,6 @@ use reth_stages::{
IndexAccountHistoryStage, IndexStorageHistoryStage, MerkleStage, SenderRecoveryStage, IndexAccountHistoryStage, IndexStorageHistoryStage, MerkleStage, SenderRecoveryStage,
StorageHashingStage, TotalDifficultyStage, TransactionLookupStage, StorageHashingStage, TotalDifficultyStage, TransactionLookupStage,
}, },
MetricEventsSender, MetricsListener,
}; };
use reth_tasks::TaskExecutor; use reth_tasks::TaskExecutor;
use reth_transaction_pool::{ use reth_transaction_pool::{
@ -247,12 +247,14 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
// always store reth.toml in the data dir, not the chain specific data dir // always store reth.toml in the data dir, not the chain specific data dir
info!(target: "reth::cli", path = ?config_path, "Configuration loaded"); info!(target: "reth::cli", path = ?config_path, "Configuration loaded");
let prometheus_handle = self.install_prometheus_recorder()?;
let db_path = data_dir.db_path(); let db_path = data_dir.db_path();
info!(target: "reth::cli", path = ?db_path, "Opening database"); info!(target: "reth::cli", path = ?db_path, "Opening database");
let db = Arc::new(init_db(&db_path, self.db.log_level)?); let db = Arc::new(init_db(&db_path, self.db.log_level)?.with_metrics());
info!(target: "reth::cli", "Database opened"); info!(target: "reth::cli", "Database opened");
self.start_metrics_endpoint(Arc::clone(&db)).await?; self.start_metrics_endpoint(prometheus_handle, Arc::clone(&db)).await?;
debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis"); debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis");
@ -269,10 +271,10 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
self.init_trusted_nodes(&mut config); self.init_trusted_nodes(&mut config);
debug!(target: "reth::cli", "Spawning metrics listener task"); debug!(target: "reth::cli", "Spawning stages metrics listener task");
let (metrics_tx, metrics_rx) = unbounded_channel(); let (sync_metrics_tx, sync_metrics_rx) = unbounded_channel();
let metrics_listener = MetricsListener::new(metrics_rx); let sync_metrics_listener = reth_stages::MetricsListener::new(sync_metrics_rx);
ctx.task_executor.spawn_critical("metrics listener task", metrics_listener); ctx.task_executor.spawn_critical("stages metrics listener task", sync_metrics_listener);
let prune_config = let prune_config =
self.pruning.prune_config(Arc::clone(&self.chain))?.or(config.prune.clone()); self.pruning.prune_config(Arc::clone(&self.chain))?.or(config.prune.clone());
@ -289,7 +291,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
BlockchainTreeConfig::default(), BlockchainTreeConfig::default(),
prune_config.clone().map(|config| config.segments), prune_config.clone().map(|config| config.segments),
)? )?
.with_sync_metrics_tx(metrics_tx.clone()); .with_sync_metrics_tx(sync_metrics_tx.clone());
let canon_state_notification_sender = tree.canon_state_notification_sender(); let canon_state_notification_sender = tree.canon_state_notification_sender();
let blockchain_tree = ShareableBlockchainTree::new(tree); let blockchain_tree = ShareableBlockchainTree::new(tree);
debug!(target: "reth::cli", "configured blockchain tree"); debug!(target: "reth::cli", "configured blockchain tree");
@ -409,7 +411,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
Arc::clone(&consensus), Arc::clone(&consensus),
db.clone(), db.clone(),
&ctx.task_executor, &ctx.task_executor,
metrics_tx, sync_metrics_tx,
prune_config.clone(), prune_config.clone(),
max_block, max_block,
) )
@ -429,7 +431,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
Arc::clone(&consensus), Arc::clone(&consensus),
db.clone(), db.clone(),
&ctx.task_executor, &ctx.task_executor,
metrics_tx, sync_metrics_tx,
prune_config.clone(), prune_config.clone(),
max_block, max_block,
) )
@ -565,7 +567,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
consensus: Arc<dyn Consensus>, consensus: Arc<dyn Consensus>,
db: DB, db: DB,
task_executor: &TaskExecutor, task_executor: &TaskExecutor,
metrics_tx: MetricEventsSender, metrics_tx: reth_stages::MetricEventsSender,
prune_config: Option<PruneConfig>, prune_config: Option<PruneConfig>,
max_block: Option<BlockNumber>, max_block: Option<BlockNumber>,
) -> eyre::Result<Pipeline<DB>> ) -> eyre::Result<Pipeline<DB>>
@ -628,11 +630,24 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
} }
} }
async fn start_metrics_endpoint(&self, db: Arc<DatabaseEnv>) -> eyre::Result<()> { fn install_prometheus_recorder(&self) -> eyre::Result<PrometheusHandle> {
prometheus_exporter::install_recorder()
}
async fn start_metrics_endpoint(
&self,
prometheus_handle: PrometheusHandle,
db: Arc<DatabaseEnv>,
) -> eyre::Result<()> {
if let Some(listen_addr) = self.metrics { if let Some(listen_addr) = self.metrics {
info!(target: "reth::cli", addr = %listen_addr, "Starting metrics endpoint"); info!(target: "reth::cli", addr = %listen_addr, "Starting metrics endpoint");
prometheus_exporter::initialize(listen_addr, db, metrics_process::Collector::default()) prometheus_exporter::serve(
.await?; listen_addr,
prometheus_handle,
db,
metrics_process::Collector::default(),
)
.await?;
} }
Ok(()) Ok(())
@ -790,7 +805,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
consensus: Arc<dyn Consensus>, consensus: Arc<dyn Consensus>,
max_block: Option<u64>, max_block: Option<u64>,
continuous: bool, continuous: bool,
metrics_tx: MetricEventsSender, metrics_tx: reth_stages::MetricEventsSender,
prune_config: Option<PruneConfig>, prune_config: Option<PruneConfig>,
) -> eyre::Result<Pipeline<DB>> ) -> eyre::Result<Pipeline<DB>>
where where

View File

@ -15,30 +15,36 @@ use tracing::error;
pub(crate) trait Hook: Fn() + Send + Sync {} pub(crate) trait Hook: Fn() + Send + Sync {}
impl<T: Fn() + Send + Sync> Hook for T {} impl<T: Fn() + Send + Sync> Hook for T {}
/// Installs Prometheus as the metrics recorder and serves it over HTTP with hooks. /// Installs Prometheus as the metrics recorder.
/// pub(crate) fn install_recorder() -> eyre::Result<PrometheusHandle> {
/// 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 initialize_with_hooks<F: Hook + 'static>(
listen_addr: SocketAddr,
hooks: impl IntoIterator<Item = F>,
) -> eyre::Result<()> {
let recorder = PrometheusBuilder::new().build_recorder(); let recorder = PrometheusBuilder::new().build_recorder();
let handle = recorder.handle(); let handle = recorder.handle();
let hooks: Vec<_> = hooks.into_iter().collect();
// Start endpoint
start_endpoint(listen_addr, handle, Arc::new(move || hooks.iter().for_each(|hook| hook())))
.await
.wrap_err("Could not start Prometheus endpoint")?;
// Build metrics stack // Build metrics stack
Stack::new(recorder) Stack::new(recorder)
.push(PrefixLayer::new("reth")) .push(PrefixLayer::new("reth"))
.install() .install()
.wrap_err("Couldn't set metrics recorder.")?; .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>,
) -> 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())))
.await
.wrap_err("Could not start Prometheus endpoint")?;
Ok(()) Ok(())
} }
@ -67,10 +73,10 @@ async fn start_endpoint<F: Hook + 'static>(
Ok(()) Ok(())
} }
/// Installs Prometheus as the metrics recorder and serves it over HTTP with database and process /// Serves Prometheus metrics over HTTP with database and process metrics.
/// metrics. pub(crate) async fn serve(
pub(crate) async fn initialize(
listen_addr: SocketAddr, listen_addr: SocketAddr,
handle: PrometheusHandle,
db: Arc<DatabaseEnv>, db: Arc<DatabaseEnv>,
process: metrics_process::Collector, process: metrics_process::Collector,
) -> eyre::Result<()> { ) -> eyre::Result<()> {
@ -119,7 +125,7 @@ pub(crate) async fn initialize(
Box::new(move || cloned_process.collect()), Box::new(move || cloned_process.collect()),
Box::new(collect_memory_stats), Box::new(collect_memory_stats),
]; ];
initialize_with_hooks(listen_addr, hooks).await?; serve_with_hooks(listen_addr, handle, hooks).await?;
// We describe the metrics after the recorder is installed, otherwise this information is not // We describe the metrics after the recorder is installed, otherwise this information is not
// registered // registered

View File

@ -131,8 +131,9 @@ impl Command {
if let Some(listen_addr) = self.metrics { if let Some(listen_addr) = self.metrics {
info!(target: "reth::cli", "Starting metrics endpoint at {}", listen_addr); info!(target: "reth::cli", "Starting metrics endpoint at {}", listen_addr);
prometheus_exporter::initialize( prometheus_exporter::serve(
listen_addr, listen_addr,
prometheus_exporter::install_recorder()?,
Arc::clone(&db), Arc::clone(&db),
metrics_process::Collector::default(), metrics_process::Collector::default(),
) )

View File

@ -63,7 +63,7 @@ impl DbTx for TxMock {
Ok(true) Ok(true)
} }
fn drop(self) {} fn abort(self) {}
fn cursor_read<T: Table>(&self) -> Result<<Self as DbTxGAT<'_>>::Cursor<T>, DatabaseError> { fn cursor_read<T: Table>(&self) -> Result<<Self as DbTxGAT<'_>>::Cursor<T>, DatabaseError> {
Ok(CursorMock { _cursor: 0 }) Ok(CursorMock { _cursor: 0 })

View File

@ -39,8 +39,8 @@ pub trait DbTx: for<'a> DbTxGAT<'a> {
/// Commit for read only transaction will consume and free transaction and allows /// Commit for read only transaction will consume and free transaction and allows
/// freeing of memory pages /// freeing of memory pages
fn commit(self) -> Result<bool, DatabaseError>; fn commit(self) -> Result<bool, DatabaseError>;
/// Drops transaction /// Aborts transaction
fn drop(self); fn abort(self);
/// Iterate over read only values in table. /// Iterate over read only values in table.
fn cursor_read<T: Table>(&self) -> Result<<Self as DbTxGAT<'_>>::Cursor<T>, DatabaseError>; fn cursor_read<T: Table>(&self) -> Result<<Self as DbTxGAT<'_>>::Cursor<T>, DatabaseError>;
/// Iterate over read only values in dup sorted table. /// Iterate over read only values in dup sorted table.

View File

@ -1,7 +1,7 @@
//! Cursor wrapper for libmdbx-sys. //! Cursor wrapper for libmdbx-sys.
use reth_interfaces::db::DatabaseWriteOperation; use reth_interfaces::db::DatabaseWriteOperation;
use std::{borrow::Cow, collections::Bound, ops::RangeBounds}; use std::{borrow::Cow, collections::Bound, marker::PhantomData, ops::RangeBounds};
use crate::{ use crate::{
common::{PairResult, ValueOnlyResult}, common::{PairResult, ValueOnlyResult},
@ -9,6 +9,7 @@ use crate::{
DbCursorRO, DbCursorRW, DbDupCursorRO, DbDupCursorRW, DupWalker, RangeWalker, DbCursorRO, DbCursorRW, DbDupCursorRO, DbDupCursorRW, DupWalker, RangeWalker,
ReverseWalker, Walker, ReverseWalker, Walker,
}, },
metrics::{Operation, OperationMetrics},
table::{Compress, DupSort, Encode, Table}, table::{Compress, DupSort, Encode, Table},
tables::utils::*, tables::utils::*,
DatabaseError, DatabaseError,
@ -24,13 +25,38 @@ pub type CursorRW<'tx, T> = Cursor<'tx, RW, T>;
#[derive(Debug)] #[derive(Debug)]
pub struct Cursor<'tx, K: TransactionKind, T: Table> { pub struct Cursor<'tx, K: TransactionKind, T: Table> {
/// Inner `libmdbx` cursor. /// Inner `libmdbx` cursor.
pub inner: reth_libmdbx::Cursor<'tx, K>, pub(crate) inner: reth_libmdbx::Cursor<'tx, K>,
/// Table name as is inside the database.
pub table: &'static str,
/// Phantom data to enforce encoding/decoding.
pub _dbi: std::marker::PhantomData<T>,
/// Cache buffer that receives compressed values. /// Cache buffer that receives compressed values.
pub buf: Vec<u8>, buf: Vec<u8>,
/// Whether to record metrics or not.
with_metrics: bool,
/// Phantom data to enforce encoding/decoding.
_dbi: PhantomData<T>,
}
impl<'tx, K: TransactionKind, T: Table> Cursor<'tx, K, T> {
pub(crate) fn new_with_metrics(
inner: reth_libmdbx::Cursor<'tx, K>,
with_metrics: bool,
) -> Self {
Self { inner, buf: Vec::new(), with_metrics, _dbi: PhantomData }
}
/// If `self.with_metrics == true`, record a metric with the provided operation and value size.
///
/// Otherwise, just execute the closure.
fn execute_with_operation_metric<R>(
&mut self,
operation: Operation,
value_size: Option<usize>,
f: impl FnOnce(&mut Self) -> R,
) -> R {
if self.with_metrics {
OperationMetrics::record(T::NAME, operation, value_size, || f(self))
} else {
f(self)
}
}
} }
/// Takes `(key, value)` from the database and decodes it appropriately. /// Takes `(key, value)` from the database and decodes it appropriately.
@ -43,14 +69,14 @@ macro_rules! decode {
/// Some types don't support compression (eg. B256), and we don't want to be copying them to the /// Some types don't support compression (eg. B256), and we don't want to be copying them to the
/// allocated buffer when we can just use their reference. /// allocated buffer when we can just use their reference.
macro_rules! compress_or_ref { macro_rules! compress_to_buf_or_ref {
($self:expr, $value:expr) => { ($self:expr, $value:expr) => {
if let Some(value) = $value.uncompressable_ref() { if let Some(value) = $value.uncompressable_ref() {
value Some(value)
} else { } else {
$self.buf.truncate(0); $self.buf.truncate(0);
$value.compress_to_buf(&mut $self.buf); $value.compress_to_buf(&mut $self.buf);
$self.buf.as_ref() None
} }
}; };
} }
@ -229,61 +255,92 @@ impl<T: Table> DbCursorRW<T> for Cursor<'_, RW, T> {
/// found, before calling `upsert`. /// found, before calling `upsert`.
fn upsert(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> { fn upsert(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> {
let key = key.encode(); let key = key.encode();
// Default `WriteFlags` is UPSERT let value = compress_to_buf_or_ref!(self, value);
self.inner.put(key.as_ref(), compress_or_ref!(self, value), WriteFlags::UPSERT).map_err( self.execute_with_operation_metric(
|e| DatabaseError::Write { Operation::CursorUpsert,
code: e.into(), Some(value.unwrap_or(&self.buf).len()),
operation: DatabaseWriteOperation::CursorUpsert, |this| {
table_name: T::NAME, this.inner
key: Box::from(key.as_ref()), .put(key.as_ref(), value.unwrap_or(&this.buf), WriteFlags::UPSERT)
.map_err(|e| DatabaseError::Write {
code: e.into(),
operation: DatabaseWriteOperation::CursorUpsert,
table_name: T::NAME,
key: Box::from(key.as_ref()),
})
}, },
) )
} }
fn insert(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> { fn insert(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> {
let key = key.encode(); let key = key.encode();
self.inner let value = compress_to_buf_or_ref!(self, value);
.put(key.as_ref(), compress_or_ref!(self, value), WriteFlags::NO_OVERWRITE) self.execute_with_operation_metric(
.map_err(|e| DatabaseError::Write { Operation::CursorInsert,
code: e.into(), Some(value.unwrap_or(&self.buf).len()),
operation: DatabaseWriteOperation::CursorInsert, |this| {
table_name: T::NAME, this.inner
key: Box::from(key.as_ref()), .put(key.as_ref(), value.unwrap_or(&this.buf), WriteFlags::NO_OVERWRITE)
}) .map_err(|e| DatabaseError::Write {
code: e.into(),
operation: DatabaseWriteOperation::CursorInsert,
table_name: T::NAME,
key: Box::from(key.as_ref()),
})
},
)
} }
/// Appends the data to the end of the table. Consequently, the append operation /// Appends the data to the end of the table. Consequently, the append operation
/// will fail if the inserted key is less than the last table key /// will fail if the inserted key is less than the last table key
fn append(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> { fn append(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> {
let key = key.encode(); let key = key.encode();
self.inner.put(key.as_ref(), compress_or_ref!(self, value), WriteFlags::APPEND).map_err( let value = compress_to_buf_or_ref!(self, value);
|e| DatabaseError::Write { self.execute_with_operation_metric(
code: e.into(), Operation::CursorAppend,
operation: DatabaseWriteOperation::CursorAppend, Some(value.unwrap_or(&self.buf).len()),
table_name: T::NAME, |this| {
key: Box::from(key.as_ref()), this.inner
.put(key.as_ref(), value.unwrap_or(&this.buf), WriteFlags::APPEND)
.map_err(|e| DatabaseError::Write {
code: e.into(),
operation: DatabaseWriteOperation::CursorAppend,
table_name: T::NAME,
key: Box::from(key.as_ref()),
})
}, },
) )
} }
fn delete_current(&mut self) -> Result<(), DatabaseError> { fn delete_current(&mut self) -> Result<(), DatabaseError> {
self.inner.del(WriteFlags::CURRENT).map_err(|e| DatabaseError::Delete(e.into())) self.execute_with_operation_metric(Operation::CursorDeleteCurrent, None, |this| {
this.inner.del(WriteFlags::CURRENT).map_err(|e| DatabaseError::Delete(e.into()))
})
} }
} }
impl<T: DupSort> DbDupCursorRW<T> for Cursor<'_, RW, T> { impl<T: DupSort> DbDupCursorRW<T> for Cursor<'_, RW, T> {
fn delete_current_duplicates(&mut self) -> Result<(), DatabaseError> { fn delete_current_duplicates(&mut self) -> Result<(), DatabaseError> {
self.inner.del(WriteFlags::NO_DUP_DATA).map_err(|e| DatabaseError::Delete(e.into())) self.execute_with_operation_metric(Operation::CursorDeleteCurrentDuplicates, None, |this| {
this.inner.del(WriteFlags::NO_DUP_DATA).map_err(|e| DatabaseError::Delete(e.into()))
})
} }
fn append_dup(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> { fn append_dup(&mut self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> {
let key = key.encode(); let key = key.encode();
self.inner.put(key.as_ref(), compress_or_ref!(self, value), WriteFlags::APPEND_DUP).map_err( let value = compress_to_buf_or_ref!(self, value);
|e| DatabaseError::Write { self.execute_with_operation_metric(
code: e.into(), Operation::CursorAppendDup,
operation: DatabaseWriteOperation::CursorAppendDup, Some(value.unwrap_or(&self.buf).len()),
table_name: T::NAME, |this| {
key: Box::from(key.as_ref()), this.inner
.put(key.as_ref(), value.unwrap_or(&this.buf), WriteFlags::APPEND_DUP)
.map_err(|e| DatabaseError::Write {
code: e.into(),
operation: DatabaseWriteOperation::CursorAppendDup,
table_name: T::NAME,
key: Box::from(key.as_ref()),
})
}, },
) )
} }

View File

@ -37,6 +37,8 @@ pub enum EnvKind {
pub struct Env<E: EnvironmentKind> { pub struct Env<E: EnvironmentKind> {
/// Libmdbx-sys environment. /// Libmdbx-sys environment.
pub inner: Environment<E>, pub inner: Environment<E>,
/// Whether to record metrics or not.
with_metrics: bool,
} }
impl<'a, E: EnvironmentKind> DatabaseGAT<'a> for Env<E> { impl<'a, E: EnvironmentKind> DatabaseGAT<'a> for Env<E> {
@ -46,11 +48,17 @@ impl<'a, E: EnvironmentKind> DatabaseGAT<'a> for Env<E> {
impl<E: EnvironmentKind> Database for Env<E> { impl<E: EnvironmentKind> Database for Env<E> {
fn tx(&self) -> Result<<Self as DatabaseGAT<'_>>::TX, DatabaseError> { fn tx(&self) -> Result<<Self as DatabaseGAT<'_>>::TX, DatabaseError> {
Ok(Tx::new(self.inner.begin_ro_txn().map_err(|e| DatabaseError::InitTx(e.into()))?)) Ok(Tx::new_with_metrics(
self.inner.begin_ro_txn().map_err(|e| DatabaseError::InitTx(e.into()))?,
self.with_metrics,
))
} }
fn tx_mut(&self) -> Result<<Self as DatabaseGAT<'_>>::TXMut, DatabaseError> { fn tx_mut(&self) -> Result<<Self as DatabaseGAT<'_>>::TXMut, DatabaseError> {
Ok(Tx::new(self.inner.begin_rw_txn().map_err(|e| DatabaseError::InitTx(e.into()))?)) Ok(Tx::new_with_metrics(
self.inner.begin_rw_txn().map_err(|e| DatabaseError::InitTx(e.into()))?,
self.with_metrics,
))
} }
} }
@ -116,11 +124,20 @@ impl<E: EnvironmentKind> Env<E> {
} }
} }
let env = Env { inner: inner_env.open(path).map_err(|e| DatabaseError::Open(e.into()))? }; let env = Env {
inner: inner_env.open(path).map_err(|e| DatabaseError::Open(e.into()))?,
with_metrics: false,
};
Ok(env) Ok(env)
} }
/// Enables metrics on the database.
pub fn with_metrics(mut self) -> Self {
self.with_metrics = true;
self
}
/// Creates all the defined tables, if necessary. /// Creates all the defined tables, if necessary.
pub fn create_tables(&self) -> Result<(), DatabaseError> { pub fn create_tables(&self) -> Result<(), DatabaseError> {
let tx = self.inner.begin_rw_txn().map_err(|e| DatabaseError::InitTx(e.into()))?; let tx = self.inner.begin_rw_txn().map_err(|e| DatabaseError::InitTx(e.into()))?;

View File

@ -2,6 +2,9 @@
use super::cursor::Cursor; use super::cursor::Cursor;
use crate::{ use crate::{
metrics::{
Operation, OperationMetrics, TransactionMetrics, TransactionMode, TransactionOutcome,
},
table::{Compress, DupSort, Encode, Table, TableImporter}, table::{Compress, DupSort, Encode, Table, TableImporter},
tables::{utils::decode_one, Tables, NUM_TABLES}, tables::{utils::decode_one, Tables, NUM_TABLES},
transaction::{DbTx, DbTxGAT, DbTxMut, DbTxMutGAT}, transaction::{DbTx, DbTxGAT, DbTxMut, DbTxMutGAT},
@ -10,7 +13,6 @@ use crate::{
use parking_lot::RwLock; use parking_lot::RwLock;
use reth_interfaces::db::DatabaseWriteOperation; use reth_interfaces::db::DatabaseWriteOperation;
use reth_libmdbx::{ffi::DBI, EnvironmentKind, Transaction, TransactionKind, WriteFlags, RW}; use reth_libmdbx::{ffi::DBI, EnvironmentKind, Transaction, TransactionKind, WriteFlags, RW};
use reth_metrics::metrics::histogram;
use std::{marker::PhantomData, str::FromStr, sync::Arc, time::Instant}; use std::{marker::PhantomData, str::FromStr, sync::Arc, time::Instant};
/// Wrapper for the libmdbx transaction. /// Wrapper for the libmdbx transaction.
@ -18,8 +20,13 @@ use std::{marker::PhantomData, str::FromStr, sync::Arc, time::Instant};
pub struct Tx<'a, K: TransactionKind, E: EnvironmentKind> { pub struct Tx<'a, K: TransactionKind, E: EnvironmentKind> {
/// Libmdbx-sys transaction. /// Libmdbx-sys transaction.
pub inner: Transaction<'a, K, E>, pub inner: Transaction<'a, K, E>,
/// Database table handle cache /// Database table handle cache.
pub db_handles: Arc<RwLock<[Option<DBI>; NUM_TABLES]>>, pub(crate) db_handles: Arc<RwLock<[Option<DBI>; NUM_TABLES]>>,
/// Handler for metrics with its own [Drop] implementation for cases when the transaction isn't
/// closed by [Tx::commit] or [Tx::abort], but we still need to report it in the metrics.
///
/// If [Some], then metrics are reported.
metrics_handler: Option<MetricsHandler<K>>,
} }
impl<'env, K: TransactionKind, E: EnvironmentKind> Tx<'env, K, E> { impl<'env, K: TransactionKind, E: EnvironmentKind> Tx<'env, K, E> {
@ -28,12 +35,30 @@ impl<'env, K: TransactionKind, E: EnvironmentKind> Tx<'env, K, E> {
where where
'a: 'env, 'a: 'env,
{ {
Self { inner, db_handles: Default::default() } Self { inner, db_handles: Default::default(), metrics_handler: None }
}
/// Creates new `Tx` object with a `RO` or `RW` transaction and optionally enables metrics.
pub fn new_with_metrics<'a>(inner: Transaction<'a, K, E>, with_metrics: bool) -> Self
where
'a: 'env,
{
let metrics_handler = with_metrics.then(|| {
let handler = MetricsHandler::<K> {
txn_id: inner.id(),
start: Instant::now(),
close_recorded: false,
_marker: PhantomData,
};
TransactionMetrics::record_open(handler.transaction_mode());
handler
});
Self { inner, db_handles: Default::default(), metrics_handler }
} }
/// Gets this transaction ID. /// Gets this transaction ID.
pub fn id(&self) -> u64 { pub fn id(&self) -> u64 {
self.inner.id() self.metrics_handler.as_ref().map_or_else(|| self.inner.id(), |handler| handler.txn_id)
} }
/// Gets a table database handle if it exists, otherwise creates it. /// Gets a table database handle if it exists, otherwise creates it.
@ -57,15 +82,94 @@ impl<'env, K: TransactionKind, E: EnvironmentKind> Tx<'env, K, E> {
/// Create db Cursor /// Create db Cursor
pub fn new_cursor<T: Table>(&self) -> Result<Cursor<'env, K, T>, DatabaseError> { pub fn new_cursor<T: Table>(&self) -> Result<Cursor<'env, K, T>, DatabaseError> {
Ok(Cursor { let inner = self
inner: self .inner
.inner .cursor_with_dbi(self.get_dbi::<T>()?)
.cursor_with_dbi(self.get_dbi::<T>()?) .map_err(|e| DatabaseError::InitCursor(e.into()))?;
.map_err(|e| DatabaseError::InitCursor(e.into()))?,
table: T::NAME, Ok(Cursor::new_with_metrics(inner, self.metrics_handler.is_some()))
_dbi: PhantomData, }
buf: vec![],
}) /// If `self.metrics_handler == Some(_)`, measure the time it takes to execute the closure and
/// record a metric with the provided transaction outcome.
///
/// Otherwise, just execute the closure.
fn execute_with_close_transaction_metric<R>(
mut self,
outcome: TransactionOutcome,
f: impl FnOnce(Self) -> R,
) -> R {
if let Some(mut metrics_handler) = self.metrics_handler.take() {
metrics_handler.close_recorded = true;
let start = Instant::now();
let result = f(self);
let close_duration = start.elapsed();
let open_duration = metrics_handler.start.elapsed();
TransactionMetrics::record_close(
metrics_handler.transaction_mode(),
outcome,
open_duration,
Some(close_duration),
);
result
} else {
f(self)
}
}
/// If `self.metrics_handler == Some(_)`, measure the time it takes to execute the closure and
/// record a metric with the provided operation.
///
/// Otherwise, just execute the closure.
fn execute_with_operation_metric<T: Table, R>(
&self,
operation: Operation,
value_size: Option<usize>,
f: impl FnOnce(&Transaction<'_, K, E>) -> R,
) -> R {
if self.metrics_handler.is_some() {
OperationMetrics::record(T::NAME, operation, value_size, || f(&self.inner))
} else {
f(&self.inner)
}
}
}
#[derive(Debug)]
struct MetricsHandler<K: TransactionKind> {
/// Cached internal transaction ID provided by libmdbx.
txn_id: u64,
/// The time when transaction has started.
start: Instant,
/// If true, the metric about transaction closing has already been recorded and we don't need
/// to do anything on [Drop::drop].
close_recorded: bool,
_marker: PhantomData<K>,
}
impl<K: TransactionKind> MetricsHandler<K> {
const fn transaction_mode(&self) -> TransactionMode {
if K::IS_READ_ONLY {
TransactionMode::ReadOnly
} else {
TransactionMode::ReadWrite
}
}
}
impl<K: TransactionKind> Drop for MetricsHandler<K> {
fn drop(&mut self) {
if !self.close_recorded {
TransactionMetrics::record_close(
self.transaction_mode(),
TransactionOutcome::Drop,
self.start.elapsed(),
None,
);
}
} }
} }
@ -83,22 +187,24 @@ impl<E: EnvironmentKind> TableImporter for Tx<'_, RW, E> {}
impl<K: TransactionKind, E: EnvironmentKind> DbTx for Tx<'_, K, E> { impl<K: TransactionKind, E: EnvironmentKind> DbTx for Tx<'_, K, E> {
fn get<T: Table>(&self, key: T::Key) -> Result<Option<<T as Table>::Value>, DatabaseError> { fn get<T: Table>(&self, key: T::Key) -> Result<Option<<T as Table>::Value>, DatabaseError> {
self.inner self.execute_with_operation_metric::<T, _>(Operation::Get, None, |tx| {
.get(self.get_dbi::<T>()?, key.encode().as_ref()) tx.get(self.get_dbi::<T>()?, key.encode().as_ref())
.map_err(|e| DatabaseError::Read(e.into()))? .map_err(|e| DatabaseError::Read(e.into()))?
.map(decode_one::<T>) .map(decode_one::<T>)
.transpose() .transpose()
})
} }
fn commit(self) -> Result<bool, DatabaseError> { fn commit(self) -> Result<bool, DatabaseError> {
let start = Instant::now(); self.execute_with_close_transaction_metric(TransactionOutcome::Commit, |this| {
let result = self.inner.commit().map_err(|e| DatabaseError::Commit(e.into())); this.inner.commit().map_err(|e| DatabaseError::Commit(e.into()))
histogram!("tx.commit", start.elapsed()); })
result
} }
fn drop(self) { fn abort(self) {
drop(self.inner) self.execute_with_close_transaction_metric(TransactionOutcome::Abort, |this| {
drop(this.inner)
})
} }
// Iterate over read only values in database. // Iterate over read only values in database.
@ -126,14 +232,21 @@ impl<K: TransactionKind, E: EnvironmentKind> DbTx for Tx<'_, K, E> {
impl<E: EnvironmentKind> DbTxMut for Tx<'_, RW, E> { impl<E: EnvironmentKind> DbTxMut for Tx<'_, RW, E> {
fn put<T: Table>(&self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> { fn put<T: Table>(&self, key: T::Key, value: T::Value) -> Result<(), DatabaseError> {
let key = key.encode(); let key = key.encode();
self.inner let value = value.compress();
.put(self.get_dbi::<T>()?, key.as_ref(), &value.compress(), WriteFlags::UPSERT) self.execute_with_operation_metric::<T, _>(
.map_err(|e| DatabaseError::Write { Operation::Put,
code: e.into(), Some(value.as_ref().len()),
operation: DatabaseWriteOperation::Put, |tx| {
table_name: T::NAME, tx.put(self.get_dbi::<T>()?, key.as_ref(), value, WriteFlags::UPSERT).map_err(|e| {
key: Box::from(key.as_ref()), DatabaseError::Write {
}) code: e.into(),
operation: DatabaseWriteOperation::Put,
table_name: T::NAME,
key: Box::from(key.as_ref()),
}
})
},
)
} }
fn delete<T: Table>( fn delete<T: Table>(
@ -148,9 +261,10 @@ impl<E: EnvironmentKind> DbTxMut for Tx<'_, RW, E> {
data = Some(value.as_ref()); data = Some(value.as_ref());
}; };
self.inner self.execute_with_operation_metric::<T, _>(Operation::Delete, None, |tx| {
.del(self.get_dbi::<T>()?, key.encode(), data) tx.del(self.get_dbi::<T>()?, key.encode(), data)
.map_err(|e| DatabaseError::Delete(e.into())) .map_err(|e| DatabaseError::Delete(e.into()))
})
} }
fn clear<T: Table>(&self) -> Result<(), DatabaseError> { fn clear<T: Table>(&self) -> Result<(), DatabaseError> {

View File

@ -68,6 +68,7 @@
pub mod abstraction; pub mod abstraction;
mod implementation; mod implementation;
mod metrics;
pub mod snapshot; pub mod snapshot;
pub mod tables; pub mod tables;
mod utils; mod utils;

View File

@ -0,0 +1,168 @@
use metrics::{Gauge, Histogram};
use reth_metrics::{metrics::Counter, Metrics};
use std::time::{Duration, Instant};
const LARGE_VALUE_THRESHOLD_BYTES: usize = 4096;
#[derive(Debug, Clone, Copy, Eq, PartialEq, Hash)]
#[allow(missing_docs)]
pub(crate) enum TransactionMode {
ReadOnly,
ReadWrite,
}
impl TransactionMode {
pub(crate) const fn as_str(&self) -> &'static str {
match self {
TransactionMode::ReadOnly => "read-only",
TransactionMode::ReadWrite => "read-write",
}
}
}
#[derive(Debug, Clone, Copy, Eq, PartialEq, Hash)]
#[allow(missing_docs)]
pub(crate) enum TransactionOutcome {
Commit,
Abort,
Drop,
}
impl TransactionOutcome {
pub(crate) const fn as_str(&self) -> &'static str {
match self {
TransactionOutcome::Commit => "commit",
TransactionOutcome::Abort => "abort",
TransactionOutcome::Drop => "drop",
}
}
}
#[derive(Debug, Clone, Copy, Eq, PartialEq, Hash)]
#[allow(missing_docs)]
pub(crate) enum Operation {
Get,
Put,
Delete,
CursorUpsert,
CursorInsert,
CursorAppend,
CursorAppendDup,
CursorDeleteCurrent,
CursorDeleteCurrentDuplicates,
}
impl Operation {
pub(crate) const fn as_str(&self) -> &'static str {
match self {
Operation::Get => "get",
Operation::Put => "put",
Operation::Delete => "delete",
Operation::CursorUpsert => "cursor-upsert",
Operation::CursorInsert => "cursor-insert",
Operation::CursorAppend => "cursor-append",
Operation::CursorAppendDup => "cursor-append-dup",
Operation::CursorDeleteCurrent => "cursor-delete-current",
Operation::CursorDeleteCurrentDuplicates => "cursor-delete-current-duplicates",
}
}
}
enum Labels {
Table,
TransactionMode,
TransactionOutcome,
Operation,
}
impl Labels {
pub(crate) fn as_str(&self) -> &'static str {
match self {
Labels::Table => "table",
Labels::TransactionMode => "mode",
Labels::TransactionOutcome => "outcome",
Labels::Operation => "operation",
}
}
}
#[derive(Metrics, Clone)]
#[metrics(scope = "database.transaction")]
pub(crate) struct TransactionMetrics {
/// Total number of currently open database transactions
open_total: Gauge,
/// The time a database transaction has been open
open_duration_seconds: Histogram,
/// The time it took to close a database transaction
close_duration_seconds: Histogram,
}
impl TransactionMetrics {
/// Record transaction opening.
pub(crate) fn record_open(mode: TransactionMode) {
let metrics = Self::new_with_labels(&[(Labels::TransactionMode.as_str(), mode.as_str())]);
metrics.open_total.increment(1.0);
}
/// Record transaction closing with the duration it was open and the duration it took to close
/// it.
pub(crate) fn record_close(
mode: TransactionMode,
outcome: TransactionOutcome,
open_duration: Duration,
close_duration: Option<Duration>,
) {
let metrics = Self::new_with_labels(&[(Labels::TransactionMode.as_str(), mode.as_str())]);
metrics.open_total.decrement(1.0);
let metrics = Self::new_with_labels(&[
(Labels::TransactionMode.as_str(), mode.as_str()),
(Labels::TransactionOutcome.as_str(), outcome.as_str()),
]);
metrics.open_duration_seconds.record(open_duration);
if let Some(close_duration) = close_duration {
metrics.close_duration_seconds.record(close_duration)
}
}
}
#[derive(Metrics, Clone)]
#[metrics(scope = "database.operation")]
pub(crate) struct OperationMetrics {
/// Total number of database operations made
calls_total: Counter,
/// The time it took to execute a database operation (put/upsert/insert/append/append_dup) with
/// value larger than [LARGE_VALUE_THRESHOLD_BYTES] bytes.
large_value_duration_seconds: Histogram,
}
impl OperationMetrics {
/// Record operation metric.
///
/// The duration it took to execute the closure is recorded only if the provided `value_size` is
/// larger than [LARGE_VALUE_THRESHOLD_BYTES].
pub(crate) fn record<T>(
table: &'static str,
operation: Operation,
value_size: Option<usize>,
f: impl FnOnce() -> T,
) -> T {
let metrics = Self::new_with_labels(&[
(Labels::Table.as_str(), table),
(Labels::Operation.as_str(), operation.as_str()),
]);
metrics.calls_total.increment(1);
// Record duration only for large values to prevent the performance hit of clock syscall
// on small operations
if value_size.map_or(false, |size| size > LARGE_VALUE_THRESHOLD_BYTES) {
let start = Instant::now();
let result = f();
metrics.large_value_duration_seconds.record(start.elapsed());
result
} else {
f()
}
}
}

View File

@ -23,12 +23,15 @@ mod private {
impl Sealed for RW {} impl Sealed for RW {}
} }
pub trait TransactionKind: private::Sealed + Debug + 'static { pub trait TransactionKind: private::Sealed + Send + Sync + Debug + 'static {
#[doc(hidden)] #[doc(hidden)]
const ONLY_CLEAN: bool; const ONLY_CLEAN: bool;
#[doc(hidden)] #[doc(hidden)]
const OPEN_FLAGS: MDBX_txn_flags_t; const OPEN_FLAGS: MDBX_txn_flags_t;
#[doc(hidden)]
const IS_READ_ONLY: bool;
} }
#[derive(Debug)] #[derive(Debug)]
@ -42,10 +45,12 @@ pub struct RW;
impl TransactionKind for RO { impl TransactionKind for RO {
const ONLY_CLEAN: bool = true; const ONLY_CLEAN: bool = true;
const OPEN_FLAGS: MDBX_txn_flags_t = MDBX_TXN_RDONLY; const OPEN_FLAGS: MDBX_txn_flags_t = MDBX_TXN_RDONLY;
const IS_READ_ONLY: bool = true;
} }
impl TransactionKind for RW { impl TransactionKind for RW {
const ONLY_CLEAN: bool = false; const ONLY_CLEAN: bool = false;
const OPEN_FLAGS: MDBX_txn_flags_t = MDBX_TXN_READWRITE; const OPEN_FLAGS: MDBX_txn_flags_t = MDBX_TXN_READWRITE;
const IS_READ_ONLY: bool = false;
} }
/// An MDBX transaction. /// An MDBX transaction.

View File

@ -497,7 +497,6 @@
} }
}, },
"mappings": [], "mappings": [],
"min": 0,
"thresholds": { "thresholds": {
"mode": "absolute", "mode": "absolute",
"steps": [ "steps": [
@ -523,7 +522,7 @@
"calcs": [], "calcs": [],
"displayMode": "list", "displayMode": "list",
"placement": "bottom", "placement": "bottom",
"showLegend": true "showLegend": false
}, },
"tooltip": { "tooltip": {
"mode": "single", "mode": "single",
@ -539,7 +538,7 @@
}, },
"editorMode": "code", "editorMode": "code",
"exemplar": false, "exemplar": false,
"expr": "rate(reth_tx_commit_sum{instance=~\"$instance\"}[$__rate_interval]) / rate(reth_tx_commit_count{instance=~\"$instance\"}[$__rate_interval])", "expr": "avg(rate(reth_database_transaction_close_duration_seconds_sum{instance=~\"$instance\", outcome=\"commit\"}[$__rate_interval]) / rate(reth_database_transaction_close_duration_seconds_count{instance=~\"$instance\", outcome=\"commit\"}[$__rate_interval]) >= 0)",
"format": "time_series", "format": "time_series",
"instant": false, "instant": false,
"legendFormat": "Commit time", "legendFormat": "Commit time",
@ -628,7 +627,7 @@
}, },
"editorMode": "code", "editorMode": "code",
"exemplar": false, "exemplar": false,
"expr": "sum(increase(reth_tx_commit{instance=~\"$instance\"}[$__interval])) by (quantile)", "expr": "avg(avg_over_time(reth_database_transaction_close_duration_seconds{instance=~\"$instance\", outcome=\"commit\"}[$__interval])) by (quantile)",
"format": "time_series", "format": "time_series",
"instant": false, "instant": false,
"legendFormat": "{{quantile}}", "legendFormat": "{{quantile}}",
@ -639,6 +638,394 @@
"title": "Commit time heatmap", "title": "Commit time heatmap",
"type": "heatmap" "type": "heatmap"
}, },
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "The average time a database transaction was open.",
"fieldConfig": {
"defaults": {
"color": {
"mode": "palette-classic",
"seriesBy": "last"
},
"custom": {
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "points",
"fillOpacity": 0,
"gradientMode": "none",
"hideFrom": {
"legend": false,
"tooltip": false,
"viz": false
},
"insertNulls": false,
"lineInterpolation": "linear",
"lineWidth": 1,
"pointSize": 5,
"scaleDistribution": {
"type": "linear"
},
"showPoints": "auto",
"spanNulls": false,
"stacking": {
"group": "A",
"mode": "none"
},
"thresholdsStyle": {
"mode": "off"
}
},
"mappings": [],
"thresholds": {
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
}
]
},
"unit": "s"
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 12,
"x": 0,
"y": 26
},
"id": 117,
"options": {
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom",
"showLegend": true
},
"tooltip": {
"mode": "single",
"sort": "none"
}
},
"pluginVersion": "9.3.6",
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"exemplar": false,
"expr": "sum(rate(reth_database_transaction_open_duration_seconds_sum{instance=~\"$instance\", outcome!=\"\"}[$__rate_interval]) / rate(reth_database_transaction_open_duration_seconds_count{instance=~\"$instance\", outcome!=\"\"}[$__rate_interval])) by (outcome, mode)",
"format": "time_series",
"instant": false,
"legendFormat": "{{mode}}, {{outcome}}",
"range": true,
"refId": "A"
}
],
"title": "Average transaction open time",
"type": "timeseries"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "The maximum time the database transaction was open.",
"fieldConfig": {
"defaults": {
"color": {
"mode": "palette-classic"
},
"custom": {
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "points",
"fillOpacity": 0,
"gradientMode": "none",
"hideFrom": {
"legend": false,
"tooltip": false,
"viz": false
},
"insertNulls": false,
"lineInterpolation": "linear",
"lineWidth": 1,
"pointSize": 5,
"scaleDistribution": {
"type": "linear"
},
"showPoints": "auto",
"spanNulls": false,
"stacking": {
"group": "A",
"mode": "none"
},
"thresholdsStyle": {
"mode": "off"
}
},
"mappings": [],
"thresholds": {
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
}
]
},
"unit": "s"
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 12,
"x": 12,
"y": 26
},
"id": 116,
"options": {
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom",
"showLegend": true
},
"tooltip": {
"mode": "single",
"sort": "none"
}
},
"pluginVersion": "10.1.0",
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"exemplar": false,
"expr": "max(max_over_time(reth_database_transaction_open_duration_seconds{instance=~\"$instance\", outcome!=\"\", quantile=\"1\"}[$__interval])) by (outcome, mode)",
"format": "time_series",
"instant": false,
"legendFormat": "{{mode}}, {{outcome}}",
"range": true,
"refId": "A"
}
],
"title": "Max transaction open time",
"type": "timeseries"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "",
"fieldConfig": {
"defaults": {
"color": {
"mode": "palette-classic"
},
"custom": {
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
"fillOpacity": 0,
"gradientMode": "none",
"hideFrom": {
"legend": false,
"tooltip": false,
"viz": false
},
"insertNulls": false,
"lineInterpolation": "linear",
"lineWidth": 1,
"pointSize": 5,
"scaleDistribution": {
"type": "linear"
},
"showPoints": "auto",
"spanNulls": false,
"stacking": {
"group": "A",
"mode": "none"
},
"thresholdsStyle": {
"mode": "off"
}
},
"mappings": [],
"thresholds": {
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
},
{
"color": "red",
"value": 80
}
]
},
"unit": "none"
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 12,
"x": 0,
"y": 34
},
"id": 119,
"options": {
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom",
"showLegend": true
},
"tooltip": {
"mode": "single",
"sort": "none"
}
},
"pluginVersion": "10.1.0",
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"exemplar": false,
"expr": "sum(reth_database_transaction_open_total{instance=~\"$instance\"}) by (mode)",
"format": "time_series",
"instant": false,
"legendFormat": "{{mode}}",
"range": true,
"refId": "A"
}
],
"title": "Number of open transactions",
"type": "timeseries"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "The maximum time the database transaction operation which inserts a large value took.",
"fieldConfig": {
"defaults": {
"color": {
"mode": "palette-classic"
},
"custom": {
"axisCenteredZero": false,
"axisColorMode": "text",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "points",
"fillOpacity": 0,
"gradientMode": "none",
"hideFrom": {
"legend": false,
"tooltip": false,
"viz": false
},
"insertNulls": false,
"lineInterpolation": "linear",
"lineWidth": 1,
"pointSize": 5,
"scaleDistribution": {
"type": "linear"
},
"showPoints": "auto",
"spanNulls": false,
"stacking": {
"group": "A",
"mode": "none"
},
"thresholdsStyle": {
"mode": "off"
}
},
"mappings": [],
"thresholds": {
"mode": "absolute",
"steps": [
{
"color": "green",
"value": null
},
{
"color": "red",
"value": 80
}
]
},
"unit": "s"
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 12,
"x": 12,
"y": 34
},
"id": 118,
"options": {
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom",
"showLegend": true
},
"tooltip": {
"mode": "single",
"sort": "none"
}
},
"pluginVersion": "10.1.0",
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"exemplar": false,
"expr": "max(max_over_time(reth_database_operation_large_value_duration_seconds{instance=~\"$instance\", quantile=\"1\"}[$__interval]) > 0) by (table)",
"format": "time_series",
"instant": false,
"legendFormat": "{{table}}",
"range": true,
"refId": "A"
}
],
"title": "Max insertion operation time",
"type": "timeseries"
},
{ {
"datasource": { "datasource": {
"type": "prometheus", "type": "prometheus",
@ -666,7 +1053,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 26 "y": 42
}, },
"id": 48, "id": 48,
"options": { "options": {
@ -776,7 +1163,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 26 "y": 42
}, },
"id": 52, "id": 52,
"options": { "options": {
@ -834,7 +1221,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 34 "y": 50
}, },
"id": 50, "id": 50,
"options": { "options": {
@ -1002,7 +1389,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 34 "y": 50
}, },
"id": 58, "id": 58,
"options": { "options": {
@ -1101,7 +1488,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 42 "y": 58
}, },
"id": 113, "id": 113,
"options": { "options": {
@ -1138,7 +1525,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 50 "y": 66
}, },
"id": 46, "id": 46,
"panels": [], "panels": [],
@ -1207,7 +1594,7 @@
"h": 8, "h": 8,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 51 "y": 67
}, },
"id": 56, "id": 56,
"options": { "options": {
@ -1280,7 +1667,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 59 "y": 75
}, },
"id": 6, "id": 6,
"panels": [], "panels": [],
@ -1352,7 +1739,7 @@
"h": 8, "h": 8,
"w": 8, "w": 8,
"x": 0, "x": 0,
"y": 60 "y": 76
}, },
"id": 18, "id": 18,
"options": { "options": {
@ -1446,7 +1833,7 @@
"h": 8, "h": 8,
"w": 8, "w": 8,
"x": 8, "x": 8,
"y": 60 "y": 76
}, },
"id": 16, "id": 16,
"options": { "options": {
@ -1566,7 +1953,7 @@
"h": 8, "h": 8,
"w": 8, "w": 8,
"x": 16, "x": 16,
"y": 60 "y": 76
}, },
"id": 8, "id": 8,
"options": { "options": {
@ -1649,7 +2036,7 @@
"h": 8, "h": 8,
"w": 8, "w": 8,
"x": 0, "x": 0,
"y": 68 "y": 84
}, },
"id": 54, "id": 54,
"options": { "options": {
@ -1870,7 +2257,7 @@
"h": 8, "h": 8,
"w": 14, "w": 14,
"x": 8, "x": 8,
"y": 68 "y": 84
}, },
"id": 103, "id": 103,
"options": { "options": {
@ -1907,7 +2294,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 76 "y": 92
}, },
"id": 24, "id": 24,
"panels": [], "panels": [],
@ -2003,7 +2390,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 77 "y": 93
}, },
"id": 26, "id": 26,
"options": { "options": {
@ -2135,7 +2522,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 77 "y": 93
}, },
"id": 33, "id": 33,
"options": { "options": {
@ -2253,7 +2640,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 85 "y": 101
}, },
"id": 36, "id": 36,
"options": { "options": {
@ -2302,7 +2689,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 93 "y": 109
}, },
"id": 32, "id": 32,
"panels": [], "panels": [],
@ -2358,7 +2745,8 @@
"mode": "absolute", "mode": "absolute",
"steps": [ "steps": [
{ {
"color": "green" "color": "green",
"value": null
}, },
{ {
"color": "red", "color": "red",
@ -2407,7 +2795,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 94 "y": 110
}, },
"id": 30, "id": 30,
"options": { "options": {
@ -2558,7 +2946,8 @@
"mode": "absolute", "mode": "absolute",
"steps": [ "steps": [
{ {
"color": "green" "color": "green",
"value": null
} }
] ]
}, },
@ -2570,7 +2959,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 94 "y": 110
}, },
"id": 28, "id": 28,
"options": { "options": {
@ -2687,7 +3076,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 102 "y": 118
}, },
"id": 35, "id": 35,
"options": { "options": {
@ -2810,7 +3199,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 102 "y": 118
}, },
"id": 73, "id": 73,
"options": { "options": {
@ -2860,7 +3249,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 110 "y": 126
}, },
"id": 89, "id": 89,
"panels": [], "panels": [],
@ -2932,7 +3321,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 111 "y": 127
}, },
"id": 91, "id": 91,
"options": { "options": {
@ -3049,7 +3438,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 111 "y": 127
}, },
"id": 92, "id": 92,
"options": { "options": {
@ -3184,7 +3573,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 119 "y": 135
}, },
"id": 102, "id": 102,
"options": { "options": {
@ -3303,7 +3692,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 119 "y": 135
}, },
"id": 94, "id": 94,
"options": { "options": {
@ -3397,7 +3786,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 127 "y": 143
}, },
"id": 104, "id": 104,
"options": { "options": {
@ -3521,7 +3910,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 127 "y": 143
}, },
"id": 93, "id": 93,
"options": { "options": {
@ -3664,7 +4053,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 135 "y": 151
}, },
"id": 95, "id": 95,
"options": { "options": {
@ -3783,7 +4172,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 135 "y": 151
}, },
"id": 115, "id": 115,
"options": { "options": {
@ -3876,7 +4265,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 143 "y": 159
}, },
"id": 79, "id": 79,
"panels": [], "panels": [],
@ -3947,7 +4336,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 144 "y": 160
}, },
"id": 74, "id": 74,
"options": { "options": {
@ -4041,7 +4430,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 144 "y": 160
}, },
"id": 80, "id": 80,
"options": { "options": {
@ -4135,7 +4524,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 152 "y": 168
}, },
"id": 81, "id": 81,
"options": { "options": {
@ -4229,7 +4618,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 152 "y": 168
}, },
"id": 114, "id": 114,
"options": { "options": {
@ -4267,7 +4656,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 160 "y": 176
}, },
"id": 87, "id": 87,
"panels": [], "panels": [],
@ -4338,7 +4727,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 161 "y": 177
}, },
"id": 83, "id": 83,
"options": { "options": {
@ -4431,7 +4820,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 161 "y": 177
}, },
"id": 84, "id": 84,
"options": { "options": {
@ -4536,7 +4925,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 169 "y": 185
}, },
"id": 85, "id": 85,
"options": { "options": {
@ -4573,7 +4962,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 177 "y": 193
}, },
"id": 68, "id": 68,
"panels": [], "panels": [],
@ -4644,7 +5033,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 178 "y": 194
}, },
"id": 60, "id": 60,
"options": { "options": {
@ -4737,7 +5126,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 178 "y": 194
}, },
"id": 62, "id": 62,
"options": { "options": {
@ -4830,7 +5219,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 186 "y": 202
}, },
"id": 64, "id": 64,
"options": { "options": {
@ -4867,7 +5256,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 194 "y": 210
}, },
"id": 97, "id": 97,
"panels": [], "panels": [],
@ -4936,7 +5325,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 195 "y": 211
}, },
"id": 98, "id": 98,
"options": { "options": {
@ -5096,7 +5485,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 195 "y": 211
}, },
"id": 101, "id": 101,
"options": { "options": {
@ -5191,7 +5580,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 203 "y": 219
}, },
"id": 99, "id": 99,
"options": { "options": {
@ -5286,7 +5675,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 203 "y": 219
}, },
"id": 100, "id": 100,
"options": { "options": {
@ -5324,7 +5713,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 211 "y": 227
}, },
"id": 105, "id": 105,
"panels": [], "panels": [],
@ -5394,7 +5783,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 212 "y": 228
}, },
"id": 106, "id": 106,
"options": { "options": {
@ -5489,7 +5878,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 212 "y": 228
}, },
"id": 107, "id": 107,
"options": { "options": {
@ -5527,7 +5916,7 @@
"h": 1, "h": 1,
"w": 24, "w": 24,
"x": 0, "x": 0,
"y": 220 "y": 236
}, },
"id": 108, "id": 108,
"panels": [], "panels": [],
@ -5550,7 +5939,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 221 "y": 237
}, },
"hiddenSeries": false, "hiddenSeries": false,
"id": 109, "id": 109,
@ -5638,7 +6027,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 221 "y": 237
}, },
"hiddenSeries": false, "hiddenSeries": false,
"id": 110, "id": 110,
@ -5735,7 +6124,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 0, "x": 0,
"y": 229 "y": 245
}, },
"id": 111, "id": 111,
"maxDataPoints": 25, "maxDataPoints": 25,
@ -5824,7 +6213,7 @@
"h": 8, "h": 8,
"w": 12, "w": 12,
"x": 12, "x": 12,
"y": 229 "y": 245
}, },
"id": 112, "id": 112,
"maxDataPoints": 25, "maxDataPoints": 25,
@ -5928,6 +6317,6 @@
"timezone": "", "timezone": "",
"title": "reth", "title": "reth",
"uid": "2k8BXz24x", "uid": "2k8BXz24x",
"version": 10, "version": 11,
"weekStart": "" "weekStart": ""
} }