mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-07 05:22:56 +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")]
|
#[serde(with = "humantime_serde")]
|
||||||
pub heartbeat_interval: Duration,
|
pub heartbeat_interval: Duration,
|
||||||
|
|
||||||
|
#[serde(with = "humantime_serde")]
|
||||||
|
pub long_reconcile_threshold: Option<Duration>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl NeonStorageControllerConf {
|
impl NeonStorageControllerConf {
|
||||||
@@ -190,6 +193,7 @@ impl Default for NeonStorageControllerConf {
|
|||||||
split_threshold: None,
|
split_threshold: None,
|
||||||
max_secondary_lag_bytes: None,
|
max_secondary_lag_bytes: None,
|
||||||
heartbeat_interval: Self::DEFAULT_HEARTBEAT_INTERVAL,
|
heartbeat_interval: Self::DEFAULT_HEARTBEAT_INTERVAL,
|
||||||
|
long_reconcile_threshold: None,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -517,6 +517,13 @@ impl StorageController {
|
|||||||
args.push(format!("--max-secondary-lag-bytes={lag}"))
|
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!(
|
args.push(format!(
|
||||||
"--neon-local-repo-dir={}",
|
"--neon-local-repo-dir={}",
|
||||||
self.env.base_data_dir.display()
|
self.env.base_data_dir.display()
|
||||||
|
|||||||
@@ -11,8 +11,8 @@ use storage_controller::metrics::preinitialize_metrics;
|
|||||||
use storage_controller::persistence::Persistence;
|
use storage_controller::persistence::Persistence;
|
||||||
use storage_controller::service::chaos_injector::ChaosInjector;
|
use storage_controller::service::chaos_injector::ChaosInjector;
|
||||||
use storage_controller::service::{
|
use storage_controller::service::{
|
||||||
Config, Service, HEARTBEAT_INTERVAL_DEFAULT, MAX_OFFLINE_INTERVAL_DEFAULT,
|
Config, Service, HEARTBEAT_INTERVAL_DEFAULT, LONG_RECONCILE_THRESHOLD_DEFAULT,
|
||||||
MAX_WARMING_UP_INTERVAL_DEFAULT, RECONCILER_CONCURRENCY_DEFAULT,
|
MAX_OFFLINE_INTERVAL_DEFAULT, MAX_WARMING_UP_INTERVAL_DEFAULT, RECONCILER_CONCURRENCY_DEFAULT,
|
||||||
};
|
};
|
||||||
use tokio::signal::unix::SignalKind;
|
use tokio::signal::unix::SignalKind;
|
||||||
use tokio_util::sync::CancellationToken;
|
use tokio_util::sync::CancellationToken;
|
||||||
@@ -108,6 +108,9 @@ struct Cli {
|
|||||||
// Period with which to send heartbeats to registered nodes
|
// Period with which to send heartbeats to registered nodes
|
||||||
#[arg(long)]
|
#[arg(long)]
|
||||||
heartbeat_interval: Option<humantime::Duration>,
|
heartbeat_interval: Option<humantime::Duration>,
|
||||||
|
|
||||||
|
#[arg(long)]
|
||||||
|
long_reconcile_threshold: Option<humantime::Duration>,
|
||||||
}
|
}
|
||||||
|
|
||||||
enum StrictMode {
|
enum StrictMode {
|
||||||
@@ -293,6 +296,10 @@ async fn async_main() -> anyhow::Result<()> {
|
|||||||
.heartbeat_interval
|
.heartbeat_interval
|
||||||
.map(humantime::Duration::into)
|
.map(humantime::Duration::into)
|
||||||
.unwrap_or(HEARTBEAT_INTERVAL_DEFAULT),
|
.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,
|
address_for_peers: args.address_for_peers,
|
||||||
start_as_candidate: args.start_as_candidate,
|
start_as_candidate: args.start_as_candidate,
|
||||||
http_service_port: args.listen.port() as i32,
|
http_service_port: args.listen.port() as i32,
|
||||||
|
|||||||
@@ -87,6 +87,10 @@ pub(crate) struct StorageControllerMetricGroup {
|
|||||||
measured::HistogramVec<DatabaseQueryLatencyLabelGroupSet, 5>,
|
measured::HistogramVec<DatabaseQueryLatencyLabelGroupSet, 5>,
|
||||||
|
|
||||||
pub(crate) storage_controller_leadership_status: measured::GaugeVec<LeadershipStatusGroupSet>,
|
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 {
|
impl StorageControllerMetrics {
|
||||||
@@ -168,6 +172,17 @@ pub(crate) struct LeadershipStatusGroup {
|
|||||||
pub(crate) status: LeadershipStatus,
|
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)]
|
#[derive(FixedCardinalityLabel, Clone, Copy)]
|
||||||
pub(crate) enum ReconcileOutcome {
|
pub(crate) enum ReconcileOutcome {
|
||||||
#[label(rename = "ok")]
|
#[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?
|
/// How often to send heartbeats to registered nodes?
|
||||||
pub const HEARTBEAT_INTERVAL_DEFAULT: Duration = Duration::from_secs(5);
|
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)]
|
#[derive(Clone, strum_macros::Display)]
|
||||||
enum TenantOperations {
|
enum TenantOperations {
|
||||||
Create,
|
Create,
|
||||||
@@ -348,6 +351,8 @@ pub struct Config {
|
|||||||
pub start_as_candidate: bool,
|
pub start_as_candidate: bool,
|
||||||
|
|
||||||
pub http_service_port: i32,
|
pub http_service_port: i32,
|
||||||
|
|
||||||
|
pub long_reconcile_threshold: Duration,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl From<DatabaseError> for ApiError {
|
impl From<DatabaseError> for ApiError {
|
||||||
|
|||||||
@@ -5,7 +5,9 @@ use std::{
|
|||||||
};
|
};
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
metrics::{self, ReconcileCompleteLabelGroup, ReconcileOutcome},
|
metrics::{
|
||||||
|
self, ReconcileCompleteLabelGroup, ReconcileLongRunningLabelGroup, ReconcileOutcome,
|
||||||
|
},
|
||||||
persistence::TenantShardPersistence,
|
persistence::TenantShardPersistence,
|
||||||
reconciler::{ReconcileUnits, ReconcilerConfig},
|
reconciler::{ReconcileUnits, ReconcilerConfig},
|
||||||
scheduler::{
|
scheduler::{
|
||||||
@@ -14,6 +16,7 @@ use crate::{
|
|||||||
},
|
},
|
||||||
service::ReconcileResultRequest,
|
service::ReconcileResultRequest,
|
||||||
};
|
};
|
||||||
|
use futures::future::{self, Either};
|
||||||
use pageserver_api::controller_api::{
|
use pageserver_api::controller_api::{
|
||||||
AvailabilityZone, NodeSchedulingPolicy, PlacementPolicy, ShardSchedulingPolicy,
|
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)]
|
#[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()))]
|
#[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(
|
pub(crate) fn spawn_reconciler(
|
||||||
@@ -1122,7 +1166,7 @@ impl TenantShard {
|
|||||||
|
|
||||||
let reconciler_cancel = cancel.child_token();
|
let reconciler_cancel = cancel.child_token();
|
||||||
let reconciler_intent = TargetState::from_intent(pageservers, &self.intent);
|
let reconciler_intent = TargetState::from_intent(pageservers, &self.intent);
|
||||||
let mut reconciler = Reconciler {
|
let reconciler = Reconciler {
|
||||||
tenant_shard_id: self.tenant_shard_id,
|
tenant_shard_id: self.tenant_shard_id,
|
||||||
shard: self.shard,
|
shard: self.shard,
|
||||||
placement_policy: self.policy.clone(),
|
placement_policy: self.policy.clone(),
|
||||||
@@ -1142,6 +1186,7 @@ impl TenantShard {
|
|||||||
};
|
};
|
||||||
|
|
||||||
let reconcile_seq = self.sequence;
|
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);
|
tracing::info!(seq=%reconcile_seq, "Spawning Reconciler for sequence {}", self.sequence);
|
||||||
let must_notify = self.pending_compute_notification;
|
let must_notify = self.pending_compute_notification;
|
||||||
@@ -1178,41 +1223,55 @@ impl TenantShard {
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Attempt to make observed state match intent state
|
let (tenant_id_label, shard_number_label, sequence_label) = {
|
||||||
let result = reconciler.reconcile().await;
|
(
|
||||||
|
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
|
let label_group = ReconcileLongRunningLabelGroup {
|
||||||
// of whether the above reconcile() did any work
|
tenant_id: &tenant_id_label,
|
||||||
if result.is_ok() && must_notify {
|
shard_number: &shard_number_label,
|
||||||
// If this fails we will send the need to retry in [`ReconcileResult::pending_compute_notification`]
|
sequence: &sequence_label,
|
||||||
reconciler.compute_notify().await.ok();
|
};
|
||||||
|
|
||||||
|
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
|
result_tx
|
||||||
.send(ReconcileResultRequest::ReconcileResult(result))
|
.send(ReconcileResultRequest::ReconcileResult(result))
|
||||||
.ok();
|
.ok();
|
||||||
|
|||||||
@@ -2641,6 +2641,12 @@ def test_storage_controller_proxy_during_migration(
|
|||||||
"""
|
"""
|
||||||
neon_env_builder.num_pageservers = 2
|
neon_env_builder.num_pageservers = 2
|
||||||
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
|
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 = neon_env_builder.init_configs()
|
||||||
env.start()
|
env.start()
|
||||||
|
|
||||||
@@ -2648,6 +2654,12 @@ def test_storage_controller_proxy_during_migration(
|
|||||||
timeline_id = env.initial_timeline
|
timeline_id = env.initial_timeline
|
||||||
env.neon_cli.create_tenant(tenant_id, timeline_id)
|
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
|
# 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.
|
# to the new pageserver: this should result in requests routed to the new pageserver.
|
||||||
env.storage_controller.configure_failpoints((migration_failpoint.value, "pause"))
|
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)
|
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]
|
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:
|
try:
|
||||||
with concurrent.futures.ThreadPoolExecutor(max_workers=2) as executor:
|
with concurrent.futures.ThreadPoolExecutor(max_workers=2) as executor:
|
||||||
migrate_fut = executor.submit(
|
migrate_fut = executor.submit(
|
||||||
@@ -2685,9 +2715,14 @@ def test_storage_controller_proxy_during_migration(
|
|||||||
# We expect request to land on the origin
|
# We expect request to land on the origin
|
||||||
assert tenant_info["generation"] == 1
|
assert tenant_info["generation"] == 1
|
||||||
|
|
||||||
|
wait_until(10, 1, long_migration_metric_published)
|
||||||
|
|
||||||
# Eventually migration completes
|
# Eventually migration completes
|
||||||
env.storage_controller.configure_failpoints((migration_failpoint.value, "off"))
|
env.storage_controller.configure_failpoints((migration_failpoint.value, "off"))
|
||||||
migrate_fut.result()
|
migrate_fut.result()
|
||||||
|
|
||||||
|
assert_long_migration_metric_not_published()
|
||||||
|
|
||||||
except:
|
except:
|
||||||
# Always disable 'pause' failpoints, even on failure, to avoid hanging in shutdown
|
# Always disable 'pause' failpoints, even on failure, to avoid hanging in shutdown
|
||||||
env.storage_controller.configure_failpoints((migration_failpoint.value, "off"))
|
env.storage_controller.configure_failpoints((migration_failpoint.value, "off"))
|
||||||
|
|||||||
Reference in New Issue
Block a user