mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-15 09:22:55 +00:00
many metrics removed and still compiles and can be started
This commit is contained in:
@@ -20,7 +20,6 @@ use pageserver::config::{PageServerConf, PageserverIdentity, ignored_fields};
|
||||
use pageserver::controller_upcall_client::StorageControllerUpcallClient;
|
||||
use pageserver::deletion_queue::DeletionQueue;
|
||||
use pageserver::disk_usage_eviction_task::{self, launch_disk_usage_global_eviction_task};
|
||||
use pageserver::metrics::{STARTUP_DURATION, STARTUP_IS_LOADING};
|
||||
use pageserver::task_mgr::{
|
||||
BACKGROUND_RUNTIME, COMPUTE_REQUEST_RUNTIME, MGMT_REQUEST_RUNTIME, WALRECEIVER_RUNTIME,
|
||||
};
|
||||
@@ -321,10 +320,9 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn startup_checkpoint(started_at: Instant, phase: &str, human_phase: &str) {
|
||||
fn startup_checkpoint(started_at: Instant, _phase: &str, human_phase: &str) {
|
||||
let elapsed = started_at.elapsed();
|
||||
let secs = elapsed.as_secs_f64();
|
||||
STARTUP_DURATION.with_label_values(&[phase]).set(secs);
|
||||
|
||||
info!(
|
||||
elapsed_ms = elapsed.as_millis(),
|
||||
@@ -502,7 +500,6 @@ fn start_pageserver(
|
||||
// Up to this point no significant I/O has been done: this should have been fast. Record
|
||||
// duration prior to starting I/O intensive phase of startup.
|
||||
startup_checkpoint(started_startup_at, "initial", "Starting loading tenants");
|
||||
STARTUP_IS_LOADING.set(1);
|
||||
|
||||
// Startup staging or optimizing:
|
||||
//
|
||||
@@ -578,7 +575,6 @@ fn start_pageserver(
|
||||
"initial_tenant_load",
|
||||
"Initial load completed",
|
||||
);
|
||||
STARTUP_IS_LOADING.set(0);
|
||||
});
|
||||
|
||||
let WaitForPhaseResult {
|
||||
|
||||
@@ -519,36 +519,6 @@ static DIRECTORY_ENTRIES_COUNT: Lazy<UIntGaugeVec> = Lazy::new(|| {
|
||||
.expect("failed to define a metric")
|
||||
});
|
||||
|
||||
pub(crate) static TENANT_STATE_METRIC: Lazy<UIntGaugeVec> = Lazy::new(|| {
|
||||
register_uint_gauge_vec!(
|
||||
"pageserver_tenant_states_count",
|
||||
"Count of tenants per state",
|
||||
&["state"]
|
||||
)
|
||||
.expect("Failed to register pageserver_tenant_states_count metric")
|
||||
});
|
||||
|
||||
/// A set of broken tenants.
|
||||
///
|
||||
/// These are expected to be so rare that a set is fine. Set as in a new timeseries per each broken
|
||||
/// tenant.
|
||||
pub(crate) static BROKEN_TENANTS_SET: Lazy<UIntGaugeVec> = Lazy::new(|| {
|
||||
register_uint_gauge_vec!(
|
||||
"pageserver_broken_tenants_count",
|
||||
"Set of broken tenants",
|
||||
&["tenant_id", "shard_id"]
|
||||
)
|
||||
.expect("Failed to register pageserver_tenant_states_count metric")
|
||||
});
|
||||
|
||||
pub(crate) static TENANT_SYNTHETIC_SIZE_METRIC: Lazy<UIntGaugeVec> = Lazy::new(|| {
|
||||
register_uint_gauge_vec!(
|
||||
"pageserver_tenant_synthetic_cached_size_bytes",
|
||||
"Synthetic size of each tenant in bytes",
|
||||
&["tenant_id"]
|
||||
)
|
||||
.expect("Failed to register pageserver_tenant_synthetic_cached_size_bytes metric")
|
||||
});
|
||||
|
||||
pub(crate) static EVICTION_ITERATION_DURATION: Lazy<HistogramVec> = Lazy::new(|| {
|
||||
register_histogram_vec!(
|
||||
@@ -1438,14 +1408,6 @@ pub(crate) static WALRECEIVER_ACTIVE_MANAGERS: Lazy<IntGauge> = Lazy::new(|| {
|
||||
.expect("failed to define a metric")
|
||||
});
|
||||
|
||||
pub(crate) static WALRECEIVER_SWITCHES: Lazy<IntCounterVec> = Lazy::new(|| {
|
||||
register_int_counter_vec!(
|
||||
"pageserver_walreceiver_switches_total",
|
||||
"Number of walreceiver manager change_connection calls",
|
||||
&["reason"]
|
||||
)
|
||||
.expect("failed to define a metric")
|
||||
});
|
||||
|
||||
pub(crate) static WALRECEIVER_BROKER_UPDATES: Lazy<IntCounter> = Lazy::new(|| {
|
||||
register_int_counter!(
|
||||
@@ -1778,10 +1740,7 @@ pub(crate) struct TimelineMetrics {
|
||||
pub imitate_logical_size_histo: StorageTimeMetrics,
|
||||
pub garbage_collect_histo: StorageTimeMetrics,
|
||||
pub find_gc_cutoffs_histo: StorageTimeMetrics,
|
||||
pub pitr_history_size: UIntGauge,
|
||||
pub archival_size: UIntGauge,
|
||||
pub standby_horizon_gauge: IntGauge,
|
||||
pub visible_physical_size_gauge: UIntGauge,
|
||||
/// copy of LayeredTimeline.current_logical_size
|
||||
pub current_logical_size_gauge: UIntGauge,
|
||||
pub aux_file_size_gauge: IntGauge,
|
||||
@@ -1844,20 +1803,11 @@ impl TimelineMetrics {
|
||||
&timeline_id,
|
||||
);
|
||||
|
||||
let pitr_history_size = PITR_HISTORY_SIZE
|
||||
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
|
||||
.unwrap();
|
||||
|
||||
let archival_size = TIMELINE_ARCHIVE_SIZE
|
||||
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
|
||||
.unwrap();
|
||||
|
||||
|
||||
let standby_horizon_gauge = STANDBY_HORIZON
|
||||
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
|
||||
.unwrap();
|
||||
let visible_physical_size_gauge = VISIBLE_PHYSICAL_SIZE
|
||||
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
|
||||
.unwrap();
|
||||
// TODO: we shouldn't expose this metric
|
||||
let current_logical_size_gauge = CURRENT_LOGICAL_SIZE
|
||||
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
|
||||
@@ -1902,10 +1852,7 @@ impl TimelineMetrics {
|
||||
imitate_logical_size_histo,
|
||||
garbage_collect_histo,
|
||||
find_gc_cutoffs_histo,
|
||||
pitr_history_size,
|
||||
archival_size,
|
||||
standby_horizon_gauge,
|
||||
visible_physical_size_gauge,
|
||||
current_logical_size_gauge,
|
||||
aux_file_size_gauge,
|
||||
directory_entries_count_gauge,
|
||||
@@ -2066,11 +2013,6 @@ impl TimelineMetrics {
|
||||
}
|
||||
|
||||
pub(crate) fn remove_tenant_metrics(tenant_shard_id: &TenantShardId) {
|
||||
// Only shard zero deals in synthetic sizes
|
||||
if tenant_shard_id.is_shard_zero() {
|
||||
let tid = tenant_shard_id.tenant_id.to_string();
|
||||
let _ = TENANT_SYNTHETIC_SIZE_METRIC.remove_label_values(&[&tid]);
|
||||
}
|
||||
|
||||
tenant_throttling::remove_tenant_metrics(tenant_shard_id);
|
||||
|
||||
|
||||
@@ -84,9 +84,9 @@ use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::deletion_queue::{DeletionQueueClient, DeletionQueueError};
|
||||
use crate::l0_flush::L0FlushGlobalState;
|
||||
use crate::metrics::{
|
||||
BROKEN_TENANTS_SET, CIRCUIT_BREAKERS_BROKEN, CIRCUIT_BREAKERS_UNBROKEN, CONCURRENT_INITDBS,
|
||||
INITDB_RUN_TIME, INITDB_SEMAPHORE_ACQUISITION_TIME, TENANT, TENANT_STATE_METRIC,
|
||||
TENANT_SYNTHETIC_SIZE_METRIC, remove_tenant_metrics,
|
||||
CIRCUIT_BREAKERS_BROKEN, CIRCUIT_BREAKERS_UNBROKEN, CONCURRENT_INITDBS,
|
||||
INITDB_RUN_TIME, INITDB_SEMAPHORE_ACQUISITION_TIME, TENANT,
|
||||
remove_tenant_metrics,
|
||||
};
|
||||
use crate::task_mgr::TaskKind;
|
||||
use crate::tenant::config::LocationMode;
|
||||
@@ -4223,55 +4223,15 @@ impl Tenant {
|
||||
let (state, mut rx) = watch::channel(state);
|
||||
|
||||
tokio::spawn(async move {
|
||||
// reflect tenant state in metrics:
|
||||
// - global per tenant state: TENANT_STATE_METRIC
|
||||
// - "set" of broken tenants: BROKEN_TENANTS_SET
|
||||
//
|
||||
// set of broken tenants should not have zero counts so that it remains accessible for
|
||||
// alerting.
|
||||
|
||||
let tid = tenant_shard_id.to_string();
|
||||
let shard_id = tenant_shard_id.shard_slug().to_string();
|
||||
let set_key = &[tid.as_str(), shard_id.as_str()][..];
|
||||
|
||||
fn inspect_state(state: &TenantState) -> ([&'static str; 1], bool) {
|
||||
([state.into()], matches!(state, TenantState::Broken { .. }))
|
||||
}
|
||||
|
||||
let mut tuple = inspect_state(&rx.borrow_and_update());
|
||||
|
||||
let is_broken = tuple.1;
|
||||
let mut counted_broken = if is_broken {
|
||||
// add the id to the set right away, there should not be any updates on the channel
|
||||
// after before tenant is removed, if ever
|
||||
BROKEN_TENANTS_SET.with_label_values(set_key).set(1);
|
||||
true
|
||||
} else {
|
||||
false
|
||||
};
|
||||
|
||||
|
||||
loop {
|
||||
let labels = &tuple.0;
|
||||
let current = TENANT_STATE_METRIC.with_label_values(labels);
|
||||
current.inc();
|
||||
|
||||
|
||||
if rx.changed().await.is_err() {
|
||||
// tenant has been dropped
|
||||
current.dec();
|
||||
drop(BROKEN_TENANTS_SET.remove_label_values(set_key));
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
current.dec();
|
||||
tuple = inspect_state(&rx.borrow_and_update());
|
||||
|
||||
let is_broken = tuple.1;
|
||||
if is_broken && !counted_broken {
|
||||
counted_broken = true;
|
||||
// insert the tenant_id (back) into the set while avoiding needless counter
|
||||
// access
|
||||
BROKEN_TENANTS_SET.with_label_values(set_key).set(1);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
@@ -4655,22 +4615,6 @@ impl Tenant {
|
||||
}
|
||||
}
|
||||
|
||||
// Update metrics that depend on GC state
|
||||
timeline
|
||||
.metrics
|
||||
.archival_size
|
||||
.set(if target.within_ancestor_pitr {
|
||||
timeline.metrics.current_logical_size_gauge.get()
|
||||
} else {
|
||||
0
|
||||
});
|
||||
timeline.metrics.pitr_history_size.set(
|
||||
timeline
|
||||
.get_last_record_lsn()
|
||||
.checked_sub(target.cutoffs.time)
|
||||
.unwrap_or(Lsn(0))
|
||||
.0,
|
||||
);
|
||||
|
||||
// Apply the cutoffs we found to the Timeline's GcInfo. Why might we _not_ have cutoffs for a timeline?
|
||||
// - this timeline was created while we were finding cutoffs
|
||||
@@ -5420,10 +5364,6 @@ impl Tenant {
|
||||
// Only shard zero should be calculating synthetic sizes
|
||||
debug_assert!(self.shard_identity.is_shard_zero());
|
||||
|
||||
TENANT_SYNTHETIC_SIZE_METRIC
|
||||
.get_metric_with_label_values(&[&self.tenant_shard_id.tenant_id.to_string()])
|
||||
.unwrap()
|
||||
.set(size);
|
||||
}
|
||||
|
||||
pub fn cached_synthetic_size(&self) -> u64 {
|
||||
@@ -5500,16 +5440,7 @@ impl Tenant {
|
||||
/// than they report here, due to layer eviction. Tenants with many active branches may
|
||||
/// actually use more than they report here.
|
||||
pub(crate) fn local_storage_wanted(&self) -> u64 {
|
||||
let timelines = self.timelines.lock().unwrap();
|
||||
|
||||
// Heuristic: we use the max() of the timelines' visible sizes, rather than the sum. This
|
||||
// reflects the observation that on tenants with multiple large branches, typically only one
|
||||
// of them is used actively enough to occupy space on disk.
|
||||
timelines
|
||||
.values()
|
||||
.map(|t| t.metrics.visible_physical_size_gauge.get())
|
||||
.max()
|
||||
.unwrap_or(0)
|
||||
1000
|
||||
}
|
||||
|
||||
/// Builds a new tenant manifest, and uploads it if it differs from the last-known tenant
|
||||
|
||||
@@ -191,7 +191,7 @@ impl SecondaryDetailTimeline {
|
||||
tenant_shard_id: &TenantShardId,
|
||||
timeline_id: &TimelineId,
|
||||
touched: &HeatMapLayer,
|
||||
resident_metric: &UIntGauge,
|
||||
_resident_metric: &UIntGauge,
|
||||
local_path: F,
|
||||
) where
|
||||
F: FnOnce() -> Utf8PathBuf,
|
||||
@@ -211,7 +211,6 @@ impl SecondaryDetailTimeline {
|
||||
touched.access_time,
|
||||
local_path(),
|
||||
));
|
||||
resident_metric.add(touched.metadata.file_size);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -526,12 +526,6 @@ impl Layer {
|
||||
}
|
||||
}
|
||||
|
||||
// Update the timeline's visible bytes count
|
||||
if let Some(tl) = self.0.timeline.upgrade() {
|
||||
tl.metrics
|
||||
.visible_physical_size_gauge
|
||||
.add(self.0.desc.file_size)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -540,23 +534,10 @@ impl Layer {
|
||||
use LayerVisibilityHint::*;
|
||||
match (old_visibility, visibility) {
|
||||
(Visible, Covered) => {
|
||||
// Subtract this layer's contribution to the visible size metric
|
||||
if let Some(tl) = self.0.timeline.upgrade() {
|
||||
debug_assert!(
|
||||
tl.metrics.visible_physical_size_gauge.get() >= self.0.desc.file_size
|
||||
);
|
||||
tl.metrics
|
||||
.visible_physical_size_gauge
|
||||
.sub(self.0.desc.file_size)
|
||||
}
|
||||
|
||||
}
|
||||
(Covered, Visible) => {
|
||||
// Add this layer's contribution to the visible size metric
|
||||
if let Some(tl) = self.0.timeline.upgrade() {
|
||||
tl.metrics
|
||||
.visible_physical_size_gauge
|
||||
.add(self.0.desc.file_size)
|
||||
}
|
||||
|
||||
}
|
||||
(Covered, Covered) | (Visible, Visible) => {
|
||||
// no change
|
||||
@@ -759,13 +740,6 @@ impl Drop for LayerInner {
|
||||
timeline.metrics.dec_layer(&self.desc);
|
||||
|
||||
if matches!(self.access_stats.visibility(), LayerVisibilityHint::Visible) {
|
||||
debug_assert!(
|
||||
timeline.metrics.visible_physical_size_gauge.get() >= self.desc.file_size
|
||||
);
|
||||
timeline
|
||||
.metrics
|
||||
.visible_physical_size_gauge
|
||||
.sub(self.desc.file_size);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -868,12 +842,6 @@ impl LayerInner {
|
||||
// This object acts as a RAII guard on these metrics: increment on construction
|
||||
timeline.metrics.inc_layer(&desc);
|
||||
|
||||
// New layers are visible by default. This metric is later updated on drop or in set_visibility
|
||||
timeline
|
||||
.metrics
|
||||
.visible_physical_size_gauge
|
||||
.add(desc.file_size);
|
||||
|
||||
LayerInner {
|
||||
conf,
|
||||
path: local_path,
|
||||
|
||||
@@ -41,7 +41,7 @@ use super::{TaskEvent, TaskHandle, TaskStateUpdate, WalReceiverConf};
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::metrics::{
|
||||
WALRECEIVER_ACTIVE_MANAGERS, WALRECEIVER_BROKER_UPDATES, WALRECEIVER_CANDIDATES_ADDED,
|
||||
WALRECEIVER_CANDIDATES_REMOVED, WALRECEIVER_SWITCHES,
|
||||
WALRECEIVER_CANDIDATES_REMOVED,
|
||||
};
|
||||
use crate::task_mgr::TaskKind;
|
||||
use crate::tenant::{Timeline, debug_assert_current_span_has_tenant_and_timeline_id};
|
||||
@@ -526,9 +526,6 @@ impl ConnectionManagerState {
|
||||
|
||||
/// Shuts down the current connection (if any) and immediately starts another one with the given connection string.
|
||||
async fn change_connection(&mut self, new_sk: NewWalConnectionCandidate, ctx: &RequestContext) {
|
||||
WALRECEIVER_SWITCHES
|
||||
.with_label_values(&[new_sk.reason.name()])
|
||||
.inc();
|
||||
|
||||
self.drop_old_connection(true).await;
|
||||
|
||||
@@ -1078,6 +1075,7 @@ struct NewWalConnectionCandidate {
|
||||
safekeeper_id: NodeId,
|
||||
wal_source_connconf: PgConnectionConfig,
|
||||
availability_zone: Option<String>,
|
||||
#[allow(dead_code)]
|
||||
reason: ReconnectReason,
|
||||
}
|
||||
|
||||
@@ -1106,18 +1104,6 @@ enum ReconnectReason {
|
||||
},
|
||||
}
|
||||
|
||||
impl ReconnectReason {
|
||||
fn name(&self) -> &str {
|
||||
match self {
|
||||
ReconnectReason::NoExistingConnection => "NoExistingConnection",
|
||||
ReconnectReason::LaggingWal { .. } => "LaggingWal",
|
||||
ReconnectReason::SwitchAvailabilityZone => "SwitchAvailabilityZone",
|
||||
ReconnectReason::NoWalTimeout { .. } => "NoWalTimeout",
|
||||
ReconnectReason::NoKeepAlives { .. } => "NoKeepAlives",
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use pageserver_api::config::defaults::DEFAULT_WAL_RECEIVER_PROTOCOL;
|
||||
|
||||
Reference in New Issue
Block a user