Merge pull request #7302 from influxdata/savage/add-hot-partition-persistence-metric
feat(persist): Add hot partition persistence counter metricpull/24376/head
commit
f029a42ac1
|
@ -320,8 +320,11 @@ where
|
||||||
// runs, such as if the configuration of the ingester was changed to persist
|
// runs, such as if the configuration of the ingester was changed to persist
|
||||||
// smaller partitions in-between executions because it was OOMing during WAL
|
// smaller partitions in-between executions because it was OOMing during WAL
|
||||||
// replay (and the configuration was changed to mitigate it).
|
// replay (and the configuration was changed to mitigate it).
|
||||||
let hot_partition_persister =
|
let hot_partition_persister = HotPartitionPersister::new(
|
||||||
HotPartitionPersister::new(Arc::clone(&persist_handle), persist_hot_partition_cost);
|
Arc::clone(&persist_handle),
|
||||||
|
persist_hot_partition_cost,
|
||||||
|
&metrics,
|
||||||
|
);
|
||||||
|
|
||||||
let buffer = Arc::new(BufferTree::new(
|
let buffer = Arc::new(BufferTree::new(
|
||||||
namespace_name_provider,
|
namespace_name_provider,
|
||||||
|
|
|
@ -13,16 +13,31 @@ use super::queue::PersistQueue;
|
||||||
pub(crate) struct HotPartitionPersister<P> {
|
pub(crate) struct HotPartitionPersister<P> {
|
||||||
persist_handle: P,
|
persist_handle: P,
|
||||||
max_estimated_persist_cost: usize,
|
max_estimated_persist_cost: usize,
|
||||||
|
|
||||||
|
/// A metric tracking the number of partitions persisted as "hot partitions".
|
||||||
|
persist_count: metric::U64Counter,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<P> HotPartitionPersister<P>
|
impl<P> HotPartitionPersister<P>
|
||||||
where
|
where
|
||||||
P: PersistQueue + Clone + Sync + 'static,
|
P: PersistQueue + Clone + Sync + 'static,
|
||||||
{
|
{
|
||||||
pub fn new(persist_handle: P, max_estimated_persist_cost: usize) -> Self {
|
pub fn new(
|
||||||
|
persist_handle: P,
|
||||||
|
max_estimated_persist_cost: usize,
|
||||||
|
metrics: &metric::Registry,
|
||||||
|
) -> Self {
|
||||||
|
let persist_count = metrics
|
||||||
|
.register_metric::<metric::U64Counter>(
|
||||||
|
"ingester_persist_hot_partition_enqueue_count",
|
||||||
|
"number of times persistence of a partition has been triggered \
|
||||||
|
because the persist cost exceeded the pre-configured limit",
|
||||||
|
)
|
||||||
|
.recorder(&[]);
|
||||||
Self {
|
Self {
|
||||||
persist_handle,
|
persist_handle,
|
||||||
max_estimated_persist_cost,
|
max_estimated_persist_cost,
|
||||||
|
persist_count,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -49,6 +64,8 @@ where
|
||||||
// There is no need to await on the completion handle.
|
// There is no need to await on the completion handle.
|
||||||
persist_handle.enqueue(partition, data).await;
|
persist_handle.enqueue(partition, data).await;
|
||||||
});
|
});
|
||||||
|
// Update any exported metrics.
|
||||||
|
self.persist_count.inc(1);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue