mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-04 12:02:55 +00:00
storcon: add metric for long running reconciles (#9207)
## Problem
We don't have an alert for long running reconciles. Stuck reconciles are
problematic
as we've seen in a recent incident.
## Summary of changes
Add a new metric `storage_controller_reconcile_long_running_total` with
labels: `{tenant_id, shard_number, seq}`.
The metric is removed after the long running reconcile finishes. These
events should be rare, so we won't break
the bank on cardinality.
Related https://github.com/neondatabase/neon/issues/9150
This commit is contained in:
@@ -168,6 +168,9 @@ pub struct NeonStorageControllerConf {
|
||||
|
||||
#[serde(with = "humantime_serde")]
|
||||
pub heartbeat_interval: Duration,
|
||||
|
||||
#[serde(with = "humantime_serde")]
|
||||
pub long_reconcile_threshold: Option<Duration>,
|
||||
}
|
||||
|
||||
impl NeonStorageControllerConf {
|
||||
@@ -190,6 +193,7 @@ impl Default for NeonStorageControllerConf {
|
||||
split_threshold: None,
|
||||
max_secondary_lag_bytes: None,
|
||||
heartbeat_interval: Self::DEFAULT_HEARTBEAT_INTERVAL,
|
||||
long_reconcile_threshold: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -517,6 +517,13 @@ impl StorageController {
|
||||
args.push(format!("--max-secondary-lag-bytes={lag}"))
|
||||
}
|
||||
|
||||
if let Some(threshold) = self.config.long_reconcile_threshold {
|
||||
args.push(format!(
|
||||
"--long-reconcile-threshold={}",
|
||||
humantime::Duration::from(threshold)
|
||||
))
|
||||
}
|
||||
|
||||
args.push(format!(
|
||||
"--neon-local-repo-dir={}",
|
||||
self.env.base_data_dir.display()
|
||||
|
||||
@@ -11,8 +11,8 @@ use storage_controller::metrics::preinitialize_metrics;
|
||||
use storage_controller::persistence::Persistence;
|
||||
use storage_controller::service::chaos_injector::ChaosInjector;
|
||||
use storage_controller::service::{
|
||||
Config, Service, HEARTBEAT_INTERVAL_DEFAULT, MAX_OFFLINE_INTERVAL_DEFAULT,
|
||||
MAX_WARMING_UP_INTERVAL_DEFAULT, RECONCILER_CONCURRENCY_DEFAULT,
|
||||
Config, Service, HEARTBEAT_INTERVAL_DEFAULT, LONG_RECONCILE_THRESHOLD_DEFAULT,
|
||||
MAX_OFFLINE_INTERVAL_DEFAULT, MAX_WARMING_UP_INTERVAL_DEFAULT, RECONCILER_CONCURRENCY_DEFAULT,
|
||||
};
|
||||
use tokio::signal::unix::SignalKind;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
@@ -108,6 +108,9 @@ struct Cli {
|
||||
// Period with which to send heartbeats to registered nodes
|
||||
#[arg(long)]
|
||||
heartbeat_interval: Option<humantime::Duration>,
|
||||
|
||||
#[arg(long)]
|
||||
long_reconcile_threshold: Option<humantime::Duration>,
|
||||
}
|
||||
|
||||
enum StrictMode {
|
||||
@@ -293,6 +296,10 @@ async fn async_main() -> anyhow::Result<()> {
|
||||
.heartbeat_interval
|
||||
.map(humantime::Duration::into)
|
||||
.unwrap_or(HEARTBEAT_INTERVAL_DEFAULT),
|
||||
long_reconcile_threshold: args
|
||||
.long_reconcile_threshold
|
||||
.map(humantime::Duration::into)
|
||||
.unwrap_or(LONG_RECONCILE_THRESHOLD_DEFAULT),
|
||||
address_for_peers: args.address_for_peers,
|
||||
start_as_candidate: args.start_as_candidate,
|
||||
http_service_port: args.listen.port() as i32,
|
||||
|
||||
@@ -87,6 +87,10 @@ pub(crate) struct StorageControllerMetricGroup {
|
||||
measured::HistogramVec<DatabaseQueryLatencyLabelGroupSet, 5>,
|
||||
|
||||
pub(crate) storage_controller_leadership_status: measured::GaugeVec<LeadershipStatusGroupSet>,
|
||||
|
||||
/// HTTP request status counters for handled requests
|
||||
pub(crate) storage_controller_reconcile_long_running:
|
||||
measured::CounterVec<ReconcileLongRunningLabelGroupSet>,
|
||||
}
|
||||
|
||||
impl StorageControllerMetrics {
|
||||
@@ -168,6 +172,17 @@ pub(crate) struct LeadershipStatusGroup {
|
||||
pub(crate) status: LeadershipStatus,
|
||||
}
|
||||
|
||||
#[derive(measured::LabelGroup, Clone)]
|
||||
#[label(set = ReconcileLongRunningLabelGroupSet)]
|
||||
pub(crate) struct ReconcileLongRunningLabelGroup<'a> {
|
||||
#[label(dynamic_with = lasso::ThreadedRodeo, default)]
|
||||
pub(crate) tenant_id: &'a str,
|
||||
#[label(dynamic_with = lasso::ThreadedRodeo, default)]
|
||||
pub(crate) shard_number: &'a str,
|
||||
#[label(dynamic_with = lasso::ThreadedRodeo, default)]
|
||||
pub(crate) sequence: &'a str,
|
||||
}
|
||||
|
||||
#[derive(FixedCardinalityLabel, Clone, Copy)]
|
||||
pub(crate) enum ReconcileOutcome {
|
||||
#[label(rename = "ok")]
|
||||
|
||||
@@ -128,6 +128,9 @@ pub const MAX_WARMING_UP_INTERVAL_DEFAULT: Duration = Duration::from_secs(300);
|
||||
/// How often to send heartbeats to registered nodes?
|
||||
pub const HEARTBEAT_INTERVAL_DEFAULT: Duration = Duration::from_secs(5);
|
||||
|
||||
/// How long is too long for a reconciliation?
|
||||
pub const LONG_RECONCILE_THRESHOLD_DEFAULT: Duration = Duration::from_secs(120);
|
||||
|
||||
#[derive(Clone, strum_macros::Display)]
|
||||
enum TenantOperations {
|
||||
Create,
|
||||
@@ -348,6 +351,8 @@ pub struct Config {
|
||||
pub start_as_candidate: bool,
|
||||
|
||||
pub http_service_port: i32,
|
||||
|
||||
pub long_reconcile_threshold: Duration,
|
||||
}
|
||||
|
||||
impl From<DatabaseError> for ApiError {
|
||||
|
||||
@@ -5,7 +5,9 @@ use std::{
|
||||
};
|
||||
|
||||
use crate::{
|
||||
metrics::{self, ReconcileCompleteLabelGroup, ReconcileOutcome},
|
||||
metrics::{
|
||||
self, ReconcileCompleteLabelGroup, ReconcileLongRunningLabelGroup, ReconcileOutcome,
|
||||
},
|
||||
persistence::TenantShardPersistence,
|
||||
reconciler::{ReconcileUnits, ReconcilerConfig},
|
||||
scheduler::{
|
||||
@@ -14,6 +16,7 @@ use crate::{
|
||||
},
|
||||
service::ReconcileResultRequest,
|
||||
};
|
||||
use futures::future::{self, Either};
|
||||
use pageserver_api::controller_api::{
|
||||
AvailabilityZone, NodeSchedulingPolicy, PlacementPolicy, ShardSchedulingPolicy,
|
||||
};
|
||||
@@ -1083,6 +1086,47 @@ impl TenantShard {
|
||||
}
|
||||
}
|
||||
|
||||
async fn reconcile(
|
||||
sequence: Sequence,
|
||||
mut reconciler: Reconciler,
|
||||
must_notify: bool,
|
||||
) -> ReconcileResult {
|
||||
// Attempt to make observed state match intent state
|
||||
let result = reconciler.reconcile().await;
|
||||
|
||||
// If we know we had a pending compute notification from some previous action, send a notification irrespective
|
||||
// of whether the above reconcile() did any work
|
||||
if result.is_ok() && must_notify {
|
||||
// If this fails we will send the need to retry in [`ReconcileResult::pending_compute_notification`]
|
||||
reconciler.compute_notify().await.ok();
|
||||
}
|
||||
|
||||
// Update result counter
|
||||
let outcome_label = match &result {
|
||||
Ok(_) => ReconcileOutcome::Success,
|
||||
Err(ReconcileError::Cancel) => ReconcileOutcome::Cancel,
|
||||
Err(_) => ReconcileOutcome::Error,
|
||||
};
|
||||
|
||||
metrics::METRICS_REGISTRY
|
||||
.metrics_group
|
||||
.storage_controller_reconcile_complete
|
||||
.inc(ReconcileCompleteLabelGroup {
|
||||
status: outcome_label,
|
||||
});
|
||||
|
||||
// Constructing result implicitly drops Reconciler, freeing any ReconcileUnits before the Service might
|
||||
// try and schedule more work in response to our result.
|
||||
ReconcileResult {
|
||||
sequence,
|
||||
result,
|
||||
tenant_shard_id: reconciler.tenant_shard_id,
|
||||
generation: reconciler.generation,
|
||||
observed: reconciler.observed,
|
||||
pending_compute_notification: reconciler.compute_notify_failure,
|
||||
}
|
||||
}
|
||||
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
#[instrument(skip_all, fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug()))]
|
||||
pub(crate) fn spawn_reconciler(
|
||||
@@ -1122,7 +1166,7 @@ impl TenantShard {
|
||||
|
||||
let reconciler_cancel = cancel.child_token();
|
||||
let reconciler_intent = TargetState::from_intent(pageservers, &self.intent);
|
||||
let mut reconciler = Reconciler {
|
||||
let reconciler = Reconciler {
|
||||
tenant_shard_id: self.tenant_shard_id,
|
||||
shard: self.shard,
|
||||
placement_policy: self.policy.clone(),
|
||||
@@ -1142,6 +1186,7 @@ impl TenantShard {
|
||||
};
|
||||
|
||||
let reconcile_seq = self.sequence;
|
||||
let long_reconcile_threshold = service_config.long_reconcile_threshold;
|
||||
|
||||
tracing::info!(seq=%reconcile_seq, "Spawning Reconciler for sequence {}", self.sequence);
|
||||
let must_notify = self.pending_compute_notification;
|
||||
@@ -1178,41 +1223,55 @@ impl TenantShard {
|
||||
return;
|
||||
}
|
||||
|
||||
// Attempt to make observed state match intent state
|
||||
let result = reconciler.reconcile().await;
|
||||
let (tenant_id_label, shard_number_label, sequence_label) = {
|
||||
(
|
||||
reconciler.tenant_shard_id.tenant_id.to_string(),
|
||||
reconciler.tenant_shard_id.shard_number.0.to_string(),
|
||||
reconcile_seq.to_string(),
|
||||
)
|
||||
};
|
||||
|
||||
// If we know we had a pending compute notification from some previous action, send a notification irrespective
|
||||
// of whether the above reconcile() did any work
|
||||
if result.is_ok() && must_notify {
|
||||
// If this fails we will send the need to retry in [`ReconcileResult::pending_compute_notification`]
|
||||
reconciler.compute_notify().await.ok();
|
||||
let label_group = ReconcileLongRunningLabelGroup {
|
||||
tenant_id: &tenant_id_label,
|
||||
shard_number: &shard_number_label,
|
||||
sequence: &sequence_label,
|
||||
};
|
||||
|
||||
let reconcile_fut = Self::reconcile(reconcile_seq, reconciler, must_notify);
|
||||
let long_reconcile_fut = {
|
||||
let label_group = label_group.clone();
|
||||
async move {
|
||||
tokio::time::sleep(long_reconcile_threshold).await;
|
||||
|
||||
tracing::warn!("Reconcile passed the long running threshold of {long_reconcile_threshold:?}");
|
||||
|
||||
metrics::METRICS_REGISTRY
|
||||
.metrics_group
|
||||
.storage_controller_reconcile_long_running
|
||||
.inc(label_group);
|
||||
}
|
||||
};
|
||||
|
||||
let reconcile_fut = std::pin::pin!(reconcile_fut);
|
||||
let long_reconcile_fut = std::pin::pin!(long_reconcile_fut);
|
||||
|
||||
let (was_long, result) =
|
||||
match future::select(reconcile_fut, long_reconcile_fut).await {
|
||||
Either::Left((reconcile_result, _)) => (false, reconcile_result),
|
||||
Either::Right((_, reconcile_fut)) => (true, reconcile_fut.await),
|
||||
};
|
||||
|
||||
if was_long {
|
||||
let id = metrics::METRICS_REGISTRY
|
||||
.metrics_group
|
||||
.storage_controller_reconcile_long_running
|
||||
.with_labels(label_group);
|
||||
metrics::METRICS_REGISTRY
|
||||
.metrics_group
|
||||
.storage_controller_reconcile_long_running
|
||||
.remove_metric(id);
|
||||
}
|
||||
|
||||
// Update result counter
|
||||
let outcome_label = match &result {
|
||||
Ok(_) => ReconcileOutcome::Success,
|
||||
Err(ReconcileError::Cancel) => ReconcileOutcome::Cancel,
|
||||
Err(_) => ReconcileOutcome::Error,
|
||||
};
|
||||
|
||||
metrics::METRICS_REGISTRY
|
||||
.metrics_group
|
||||
.storage_controller_reconcile_complete
|
||||
.inc(ReconcileCompleteLabelGroup {
|
||||
status: outcome_label,
|
||||
});
|
||||
|
||||
// Constructing result implicitly drops Reconciler, freeing any ReconcileUnits before the Service might
|
||||
// try and schedule more work in response to our result.
|
||||
let result = ReconcileResult {
|
||||
sequence: reconcile_seq,
|
||||
result,
|
||||
tenant_shard_id: reconciler.tenant_shard_id,
|
||||
generation: reconciler.generation,
|
||||
observed: reconciler.observed,
|
||||
pending_compute_notification: reconciler.compute_notify_failure,
|
||||
};
|
||||
|
||||
result_tx
|
||||
.send(ReconcileResultRequest::ReconcileResult(result))
|
||||
.ok();
|
||||
|
||||
@@ -2641,6 +2641,12 @@ def test_storage_controller_proxy_during_migration(
|
||||
"""
|
||||
neon_env_builder.num_pageservers = 2
|
||||
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
|
||||
|
||||
neon_env_builder.storage_controller_config = {
|
||||
# Publish long reconcile metric early
|
||||
"long_reconcile_threshold": "5s",
|
||||
}
|
||||
|
||||
env = neon_env_builder.init_configs()
|
||||
env.start()
|
||||
|
||||
@@ -2648,6 +2654,12 @@ def test_storage_controller_proxy_during_migration(
|
||||
timeline_id = env.initial_timeline
|
||||
env.neon_cli.create_tenant(tenant_id, timeline_id)
|
||||
|
||||
# The test stalls a reconcile on purpose to check if the long running
|
||||
# reconcile alert fires.
|
||||
env.storage_controller.allowed_errors.extend(
|
||||
[".*Reconcile passed the long running threshold.*"]
|
||||
)
|
||||
|
||||
# Activate a failpoint that will cause live migration to get stuck _after_ the generation has been issued
|
||||
# to the new pageserver: this should result in requests routed to the new pageserver.
|
||||
env.storage_controller.configure_failpoints((migration_failpoint.value, "pause"))
|
||||
@@ -2655,6 +2667,24 @@ def test_storage_controller_proxy_during_migration(
|
||||
origin_pageserver = env.get_tenant_pageserver(tenant_id)
|
||||
dest_ps_id = [p.id for p in env.pageservers if p.id != origin_pageserver.id][0]
|
||||
|
||||
def long_migration_metric_published():
|
||||
assert (
|
||||
env.storage_controller.get_metric_value(
|
||||
"storage_controller_reconcile_long_running_total",
|
||||
filter={"tenant_id": str(tenant_id), "shard_number": "0"},
|
||||
)
|
||||
== 1
|
||||
)
|
||||
|
||||
def assert_long_migration_metric_not_published():
|
||||
assert (
|
||||
env.storage_controller.get_metric_value(
|
||||
"storage_controller_reconcile_long_running_total",
|
||||
filter={"tenant_id": str(tenant_id), "shard_number": "0"},
|
||||
)
|
||||
is None
|
||||
)
|
||||
|
||||
try:
|
||||
with concurrent.futures.ThreadPoolExecutor(max_workers=2) as executor:
|
||||
migrate_fut = executor.submit(
|
||||
@@ -2685,9 +2715,14 @@ def test_storage_controller_proxy_during_migration(
|
||||
# We expect request to land on the origin
|
||||
assert tenant_info["generation"] == 1
|
||||
|
||||
wait_until(10, 1, long_migration_metric_published)
|
||||
|
||||
# Eventually migration completes
|
||||
env.storage_controller.configure_failpoints((migration_failpoint.value, "off"))
|
||||
migrate_fut.result()
|
||||
|
||||
assert_long_migration_metric_not_published()
|
||||
|
||||
except:
|
||||
# Always disable 'pause' failpoints, even on failure, to avoid hanging in shutdown
|
||||
env.storage_controller.configure_failpoints((migration_failpoint.value, "off"))
|
||||
|
||||
Reference in New Issue
Block a user