1use async_stream::stream;
13use mz_persist_types::stats::PartStatsMetrics;
14use std::collections::BTreeMap;
15use std::sync::{Arc, Mutex, Weak};
16use std::time::{Duration, Instant};
17use tokio::sync::{OnceCell, OwnedSemaphorePermit, Semaphore};
18
19use async_trait::async_trait;
20use bytes::Bytes;
21use futures_util::StreamExt;
22use mz_ore::bytes::SegmentedBytes;
23use mz_ore::cast::{CastFrom, CastLossy};
24use mz_ore::instrument;
25use mz_ore::metric;
26use mz_ore::metrics::{
27 ComputedGauge, ComputedIntGauge, ComputedUIntGauge, Counter, DeleteOnDropCounter,
28 DeleteOnDropGauge, IntCounter, MakeCollector, MetricVecExt, MetricsRegistry, UIntGauge,
29 UIntGaugeVec, raw,
30};
31use mz_ore::stats::histogram_seconds_buckets;
32use mz_persist::location::{
33 Blob, BlobMetadata, CaSResult, Consensus, ExternalError, ResultStream, SeqNo, VersionedData,
34};
35use mz_persist::metrics::{ColumnarMetrics, S3BlobMetrics};
36use mz_persist::retry::RetryStream;
37use mz_persist_types::Codec64;
38use mz_postgres_client::metrics::PostgresClientMetrics;
39use prometheus::core::{AtomicI64, AtomicU64, Collector, Desc, GenericGauge};
40use prometheus::proto::MetricFamily;
41use prometheus::{CounterVec, Gauge, GaugeVec, Histogram, HistogramVec, IntCounterVec};
42use timely::progress::Antichain;
43use tokio_metrics::TaskMonitor;
44use tracing::{Instrument, debug, info, info_span};
45
46use crate::fetch::{FETCH_SEMAPHORE_COST_ADJUSTMENT, FETCH_SEMAPHORE_PERMIT_ADJUSTMENT};
47use crate::internal::paths::BlobKey;
48use crate::{PersistConfig, ShardId};
49
50pub struct Metrics {
55 _vecs: MetricsVecs,
56 _uptime: ComputedGauge,
57
58 pub blob: BlobMetrics,
60 pub consensus: ConsensusMetrics,
62 pub cmds: CmdsMetrics,
64 pub retries: RetriesMetrics,
66 pub user: BatchWriteMetrics,
69 pub read: BatchPartReadMetrics,
71 pub compaction: CompactionMetrics,
73 pub gc: GcMetrics,
75 pub lease: LeaseMetrics,
77 pub codecs: CodecsMetrics,
79 pub state: StateMetrics,
81 pub shards: ShardsMetrics,
83 pub audit: UsageAuditMetrics,
85 pub locks: LocksMetrics,
87 pub watch: WatchMetrics,
89 pub pubsub_client: PubSubClientMetrics,
91 pub pushdown: PushdownMetrics,
93 pub consolidation: ConsolidationMetrics,
95 pub blob_cache_mem: BlobMemCache,
97 pub tasks: TasksMetrics,
99 pub columnar: ColumnarMetrics,
101 pub schema: SchemaMetrics,
103 pub inline: InlineMetrics,
105 pub(crate) semaphore: SemaphoreMetrics,
107
108 pub sink: SinkMetrics,
110
111 pub s3_blob: S3BlobMetrics,
113 pub postgres_consensus: PostgresClientMetrics,
115
116 #[allow(dead_code)]
117 pub(crate) registry: MetricsRegistry,
118}
119
120impl std::fmt::Debug for Metrics {
121 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
122 f.debug_struct("Metrics").finish_non_exhaustive()
123 }
124}
125
126impl Metrics {
127 pub fn new(cfg: &PersistConfig, registry: &MetricsRegistry) -> Self {
129 let vecs = MetricsVecs::new(registry);
130 let start = Instant::now();
131 let uptime = registry.register_computed_gauge(
132 metric!(
133 name: "mz_persist_metadata_seconds",
134 help: "server uptime, labels are build metadata",
135 const_labels: {
136 "version" => cfg.build_version,
137 "build_type" => if cfg!(release) { "release" } else { "debug" }
138 },
139 ),
140 move || start.elapsed().as_secs_f64(),
141 );
142 let s3_blob = S3BlobMetrics::new(registry);
143 let columnar = ColumnarMetrics::new(
144 registry,
145 &s3_blob.lgbytes,
146 Arc::clone(&cfg.configs),
147 cfg.is_cc_active,
148 );
149 Metrics {
150 blob: vecs.blob_metrics(),
151 consensus: vecs.consensus_metrics(),
152 cmds: vecs.cmds_metrics(registry),
153 retries: vecs.retries_metrics(),
154 codecs: vecs.codecs_metrics(),
155 user: BatchWriteMetrics::new(registry, "user"),
156 read: vecs.batch_part_read_metrics(),
157 compaction: CompactionMetrics::new(registry),
158 gc: GcMetrics::new(registry),
159 lease: LeaseMetrics::new(registry),
160 state: StateMetrics::new(registry),
161 shards: ShardsMetrics::new(registry),
162 audit: UsageAuditMetrics::new(registry),
163 locks: vecs.locks_metrics(),
164 watch: WatchMetrics::new(registry),
165 pubsub_client: PubSubClientMetrics::new(registry),
166 pushdown: PushdownMetrics::new(registry),
167 consolidation: ConsolidationMetrics::new(registry),
168 blob_cache_mem: BlobMemCache::new(registry),
169 tasks: TasksMetrics::new(registry),
170 columnar,
171 schema: SchemaMetrics::new(registry),
172 inline: InlineMetrics::new(registry),
173 semaphore: SemaphoreMetrics::new(cfg.clone(), registry.clone()),
174 sink: SinkMetrics::new(registry),
175 s3_blob,
176 postgres_consensus: PostgresClientMetrics::new(registry, "mz_persist"),
177 _vecs: vecs,
178 _uptime: uptime,
179 registry: registry.clone(),
180 }
181 }
182
183 pub fn write_amplification(&self) -> f64 {
188 let total_written = self.blob.set.bytes.get();
191 let user_written = self.user.goodbytes.get();
192 #[allow(clippy::as_conversions)]
193 {
194 total_written as f64 / user_written as f64
195 }
196 }
197}
198
199#[derive(Debug)]
200struct MetricsVecs {
201 cmd_started: IntCounterVec,
202 cmd_cas_mismatch: IntCounterVec,
203 cmd_succeeded: IntCounterVec,
204 cmd_failed: IntCounterVec,
205 cmd_seconds: CounterVec,
206
207 external_op_started: IntCounterVec,
208 external_op_succeeded: IntCounterVec,
209 external_op_failed: IntCounterVec,
210 external_op_bytes: IntCounterVec,
211 external_op_seconds: CounterVec,
212 external_consensus_truncated_count: IntCounter,
213 external_blob_delete_noop_count: IntCounter,
214 external_blob_sizes: Histogram,
215 external_rtt_latency: GaugeVec,
216 external_op_latency: HistogramVec,
217
218 retry_started: IntCounterVec,
219 retry_finished: IntCounterVec,
220 retry_retries: IntCounterVec,
221 retry_sleep_seconds: CounterVec,
222
223 encode_count: IntCounterVec,
224 encode_seconds: CounterVec,
225 decode_count: IntCounterVec,
226 decode_seconds: CounterVec,
227
228 read_part_bytes: IntCounterVec,
229 read_part_goodbytes: IntCounterVec,
230 read_part_count: IntCounterVec,
231 read_part_seconds: CounterVec,
232 read_ts_rewrite: IntCounterVec,
233
234 lock_acquire_count: IntCounterVec,
235 lock_blocking_acquire_count: IntCounterVec,
236 lock_blocking_seconds: CounterVec,
237
238 alerts_metrics: Arc<AlertsMetrics>,
240}
241
242impl MetricsVecs {
243 fn new(registry: &MetricsRegistry) -> Self {
244 MetricsVecs {
245 cmd_started: registry.register(metric!(
246 name: "mz_persist_cmd_started_count",
247 help: "count of commands started",
248 var_labels: ["cmd"],
249 )),
250 cmd_cas_mismatch: registry.register(metric!(
251 name: "mz_persist_cmd_cas_mismatch_count",
252 help: "count of command retries from CaS mismatch",
253 var_labels: ["cmd"],
254 )),
255 cmd_succeeded: registry.register(metric!(
256 name: "mz_persist_cmd_succeeded_count",
257 help: "count of commands succeeded",
258 var_labels: ["cmd"],
259 )),
260 cmd_failed: registry.register(metric!(
261 name: "mz_persist_cmd_failed_count",
262 help: "count of commands failed",
263 var_labels: ["cmd"],
264 )),
265 cmd_seconds: registry.register(metric!(
266 name: "mz_persist_cmd_seconds",
267 help: "time spent applying commands",
268 var_labels: ["cmd"],
269 )),
270
271 external_op_started: registry.register(metric!(
272 name: "mz_persist_external_started_count",
273 help: "count of external service calls started",
274 var_labels: ["op"],
275 )),
276 external_op_succeeded: registry.register(metric!(
277 name: "mz_persist_external_succeeded_count",
278 help: "count of external service calls succeeded",
279 var_labels: ["op"],
280 )),
281 external_op_failed: registry.register(metric!(
282 name: "mz_persist_external_failed_count",
283 help: "count of external service calls failed",
284 var_labels: ["op"],
285 )),
286 external_op_bytes: registry.register(metric!(
287 name: "mz_persist_external_bytes_count",
288 help: "total size represented by external service calls",
289 var_labels: ["op"],
290 )),
291 external_op_seconds: registry.register(metric!(
292 name: "mz_persist_external_seconds",
293 help: "time spent in external service calls",
294 var_labels: ["op"],
295 )),
296 external_consensus_truncated_count: registry.register(metric!(
297 name: "mz_persist_external_consensus_truncated_count",
298 help: "count of versions deleted by consensus truncate calls",
299 )),
300 external_blob_delete_noop_count: registry.register(metric!(
301 name: "mz_persist_external_blob_delete_noop_count",
302 help: "count of blob delete calls that deleted a non-existent key",
303 )),
304 external_blob_sizes: registry.register(metric!(
305 name: "mz_persist_external_blob_sizes",
306 help: "histogram of blob sizes at put time",
307 buckets: mz_ore::stats::HISTOGRAM_BYTE_BUCKETS.to_vec(),
308 )),
309 external_rtt_latency: registry.register(metric!(
310 name: "mz_persist_external_rtt_latency",
311 help: "roundtrip-time to external service as seen by this process",
312 var_labels: ["external"],
313 )),
314 external_op_latency: registry.register(metric!(
315 name: "mz_persist_external_op_latency",
316 help: "rountrip latency observed by individual performance-critical operations",
317 var_labels: ["op"],
318 buckets: histogram_seconds_buckets(0.000_500, 32.0),
321 )),
322
323 retry_started: registry.register(metric!(
324 name: "mz_persist_retry_started_count",
325 help: "count of retry loops started",
326 var_labels: ["op"],
327 )),
328 retry_finished: registry.register(metric!(
329 name: "mz_persist_retry_finished_count",
330 help: "count of retry loops finished",
331 var_labels: ["op"],
332 )),
333 retry_retries: registry.register(metric!(
334 name: "mz_persist_retry_retries_count",
335 help: "count of total attempts by retry loops",
336 var_labels: ["op"],
337 )),
338 retry_sleep_seconds: registry.register(metric!(
339 name: "mz_persist_retry_sleep_seconds",
340 help: "time spent in retry loop backoff",
341 var_labels: ["op"],
342 )),
343
344 encode_count: registry.register(metric!(
345 name: "mz_persist_encode_count",
346 help: "count of op encodes",
347 var_labels: ["op"],
348 )),
349 encode_seconds: registry.register(metric!(
350 name: "mz_persist_encode_seconds",
351 help: "time spent in op encodes",
352 var_labels: ["op"],
353 )),
354 decode_count: registry.register(metric!(
355 name: "mz_persist_decode_count",
356 help: "count of op decodes",
357 var_labels: ["op"],
358 )),
359 decode_seconds: registry.register(metric!(
360 name: "mz_persist_decode_seconds",
361 help: "time spent in op decodes",
362 var_labels: ["op"],
363 )),
364
365 read_part_bytes: registry.register(metric!(
366 name: "mz_persist_read_batch_part_bytes",
367 help: "total encoded size of batch parts read",
368 var_labels: ["op"],
369 )),
370 read_part_goodbytes: registry.register(metric!(
371 name: "mz_persist_read_batch_part_goodbytes",
372 help: "total logical size of batch parts read",
373 var_labels: ["op"],
374 )),
375 read_part_count: registry.register(metric!(
376 name: "mz_persist_read_batch_part_count",
377 help: "count of batch parts read",
378 var_labels: ["op"],
379 )),
380 read_part_seconds: registry.register(metric!(
381 name: "mz_persist_read_batch_part_seconds",
382 help: "time spent reading batch parts",
383 var_labels: ["op"],
384 )),
385 read_ts_rewrite: registry.register(metric!(
386 name: "mz_persist_read_ts_rewite",
387 help: "count of updates read with rewritten ts",
388 var_labels: ["op"],
389 )),
390
391 lock_acquire_count: registry.register(metric!(
392 name: "mz_persist_lock_acquire_count",
393 help: "count of locks acquired",
394 var_labels: ["op"],
395 )),
396 lock_blocking_acquire_count: registry.register(metric!(
397 name: "mz_persist_lock_blocking_acquire_count",
398 help: "count of locks acquired that required blocking",
399 var_labels: ["op"],
400 )),
401 lock_blocking_seconds: registry.register(metric!(
402 name: "mz_persist_lock_blocking_seconds",
403 help: "time spent blocked for a lock",
404 var_labels: ["op"],
405 )),
406
407 alerts_metrics: Arc::new(AlertsMetrics::new(registry)),
408 }
409 }
410
411 fn cmds_metrics(&self, registry: &MetricsRegistry) -> CmdsMetrics {
412 CmdsMetrics {
413 init_state: self.cmd_metrics("init_state"),
414 add_rollup: self.cmd_metrics("add_rollup"),
415 remove_rollups: self.cmd_metrics("remove_rollups"),
416 register: self.cmd_metrics("register"),
417 compare_and_append: self.cmd_metrics("compare_and_append"),
418 compare_and_append_noop: registry.register(metric!(
419 name: "mz_persist_cmd_compare_and_append_noop",
420 help: "count of compare_and_append retries that were discoverd to have already committed",
421 )),
422 compare_and_downgrade_since: self.cmd_metrics("compare_and_downgrade_since"),
423 downgrade_since: self.cmd_metrics("downgrade_since"),
424 heartbeat_reader: self.cmd_metrics("heartbeat_reader"),
425 expire_reader: self.cmd_metrics("expire_reader"),
426 expire_writer: self.cmd_metrics("expire_writer"),
427 merge_res: self.cmd_metrics("merge_res"),
428 become_tombstone: self.cmd_metrics("become_tombstone"),
429 compare_and_evolve_schema: self.cmd_metrics("compare_and_evolve_schema"),
430 spine_exert: self.cmd_metrics("spine_exert"),
431 fetch_upper_count: registry.register(metric!(
432 name: "mz_persist_cmd_fetch_upper_count",
433 help: "count of fetch_upper calls",
434 ))
435 }
436 }
437
438 fn cmd_metrics(&self, cmd: &str) -> CmdMetrics {
439 CmdMetrics {
440 name: cmd.to_owned(),
441 started: self.cmd_started.with_label_values(&[cmd]),
442 succeeded: self.cmd_succeeded.with_label_values(&[cmd]),
443 cas_mismatch: self.cmd_cas_mismatch.with_label_values(&[cmd]),
444 failed: self.cmd_failed.with_label_values(&[cmd]),
445 seconds: self.cmd_seconds.with_label_values(&[cmd]),
446 }
447 }
448
449 fn retries_metrics(&self) -> RetriesMetrics {
450 RetriesMetrics {
451 determinate: RetryDeterminate {
452 apply_unbatched_cmd_cas: self.retry_metrics("apply_unbatched_cmd::cas"),
453 },
454 external: RetryExternal {
455 batch_delete: Arc::new(self.retry_metrics("batch::delete")),
456 batch_set: self.retry_metrics("batch::set"),
457 blob_open: self.retry_metrics("blob::open"),
458 compaction_noop_delete: Arc::new(self.retry_metrics("compaction_noop::delete")),
459 consensus_open: self.retry_metrics("consensus::open"),
460 fetch_batch_get: self.retry_metrics("fetch_batch::get"),
461 fetch_state_scan: self.retry_metrics("fetch_state::scan"),
462 gc_truncate: self.retry_metrics("gc::truncate"),
463 maybe_init_cas: self.retry_metrics("maybe_init::cas"),
464 rollup_delete: self.retry_metrics("rollup::delete"),
465 rollup_get: self.retry_metrics("rollup::get"),
466 rollup_set: self.retry_metrics("rollup::set"),
467 hollow_run_get: self.retry_metrics("hollow_run::get"),
468 hollow_run_set: self.retry_metrics("hollow_run::set"),
469 storage_usage_shard_size: self.retry_metrics("storage_usage::shard_size"),
470 },
471 compare_and_append_idempotent: self.retry_metrics("compare_and_append_idempotent"),
472 fetch_latest_state: self.retry_metrics("fetch_latest_state"),
473 fetch_live_states: self.retry_metrics("fetch_live_states"),
474 idempotent_cmd: self.retry_metrics("idempotent_cmd"),
475 next_listen_batch: self.retry_metrics("next_listen_batch"),
476 snapshot: self.retry_metrics("snapshot"),
477 }
478 }
479
480 fn retry_metrics(&self, name: &str) -> RetryMetrics {
481 RetryMetrics {
482 name: name.to_owned(),
483 started: self.retry_started.with_label_values(&[name]),
484 finished: self.retry_finished.with_label_values(&[name]),
485 retries: self.retry_retries.with_label_values(&[name]),
486 sleep_seconds: self.retry_sleep_seconds.with_label_values(&[name]),
487 }
488 }
489
490 fn codecs_metrics(&self) -> CodecsMetrics {
491 CodecsMetrics {
492 state: self.codec_metrics("state"),
493 state_diff: self.codec_metrics("state_diff"),
494 batch: self.codec_metrics("batch"),
495 key: self.codec_metrics("key"),
496 val: self.codec_metrics("val"),
497 }
498 }
499
500 fn codec_metrics(&self, op: &str) -> CodecMetrics {
501 CodecMetrics {
502 encode_count: self.encode_count.with_label_values(&[op]),
503 encode_seconds: self.encode_seconds.with_label_values(&[op]),
504 decode_count: self.decode_count.with_label_values(&[op]),
505 decode_seconds: self.decode_seconds.with_label_values(&[op]),
506 }
507 }
508
509 fn blob_metrics(&self) -> BlobMetrics {
510 BlobMetrics {
511 set: self.external_op_metrics("blob_set", true),
512 get: self.external_op_metrics("blob_get", true),
513 list_keys: self.external_op_metrics("blob_list_keys", false),
514 delete: self.external_op_metrics("blob_delete", false),
515 restore: self.external_op_metrics("restore", false),
516 delete_noop: self.external_blob_delete_noop_count.clone(),
517 blob_sizes: self.external_blob_sizes.clone(),
518 rtt_latency: self.external_rtt_latency.with_label_values(&["blob"]),
519 }
520 }
521
522 fn consensus_metrics(&self) -> ConsensusMetrics {
523 ConsensusMetrics {
524 list_keys: self.external_op_metrics("consensus_list_keys", false),
525 head: self.external_op_metrics("consensus_head", false),
526 compare_and_set: self.external_op_metrics("consensus_cas", true),
527 scan: self.external_op_metrics("consensus_scan", false),
528 truncate: self.external_op_metrics("consensus_truncate", false),
529 truncated_count: self.external_consensus_truncated_count.clone(),
530 rtt_latency: self.external_rtt_latency.with_label_values(&["consensus"]),
531 }
532 }
533
534 fn external_op_metrics(&self, op: &str, latency_histogram: bool) -> ExternalOpMetrics {
535 ExternalOpMetrics {
536 started: self.external_op_started.with_label_values(&[op]),
537 succeeded: self.external_op_succeeded.with_label_values(&[op]),
538 failed: self.external_op_failed.with_label_values(&[op]),
539 bytes: self.external_op_bytes.with_label_values(&[op]),
540 seconds: self.external_op_seconds.with_label_values(&[op]),
541 seconds_histogram: if latency_histogram {
542 Some(self.external_op_latency.with_label_values(&[op]))
543 } else {
544 None
545 },
546 alerts_metrics: Arc::clone(&self.alerts_metrics),
547 }
548 }
549
550 fn batch_part_read_metrics(&self) -> BatchPartReadMetrics {
551 BatchPartReadMetrics {
552 listen: self.read_metrics("listen"),
553 snapshot: self.read_metrics("snapshot"),
554 batch_fetcher: self.read_metrics("batch_fetcher"),
555 compaction: self.read_metrics("compaction"),
556 unindexed: self.read_metrics("unindexed"),
557 }
558 }
559
560 fn read_metrics(&self, op: &str) -> ReadMetrics {
561 ReadMetrics {
562 part_bytes: self.read_part_bytes.with_label_values(&[op]),
563 part_goodbytes: self.read_part_goodbytes.with_label_values(&[op]),
564 part_count: self.read_part_count.with_label_values(&[op]),
565 seconds: self.read_part_seconds.with_label_values(&[op]),
566 ts_rewrite: self.read_ts_rewrite.with_label_values(&[op]),
567 }
568 }
569
570 fn locks_metrics(&self) -> LocksMetrics {
571 LocksMetrics {
572 applier_read_cacheable: self.lock_metrics("applier_read_cacheable"),
573 applier_read_noncacheable: self.lock_metrics("applier_read_noncacheable"),
574 applier_write: self.lock_metrics("applier_write"),
575 watch: self.lock_metrics("watch"),
576 }
577 }
578
579 fn lock_metrics(&self, op: &str) -> LockMetrics {
580 LockMetrics {
581 acquire_count: self.lock_acquire_count.with_label_values(&[op]),
582 blocking_acquire_count: self.lock_blocking_acquire_count.with_label_values(&[op]),
583 blocking_seconds: self.lock_blocking_seconds.with_label_values(&[op]),
584 }
585 }
586}
587
588#[derive(Debug)]
589pub struct CmdMetrics {
590 pub(crate) name: String,
591 pub(crate) started: IntCounter,
592 pub(crate) cas_mismatch: IntCounter,
593 pub(crate) succeeded: IntCounter,
594 pub(crate) failed: IntCounter,
595 pub(crate) seconds: Counter,
596}
597
598impl CmdMetrics {
599 pub async fn run_cmd<R, E, F, CmdFn>(
600 &self,
601 shard_metrics: &ShardMetrics,
602 cmd_fn: CmdFn,
603 ) -> Result<R, E>
604 where
605 F: std::future::Future<Output = Result<R, E>>,
606 CmdFn: FnOnce() -> F,
607 {
608 self.started.inc();
609 let start = Instant::now();
610 let res = cmd_fn().await;
611 self.seconds.inc_by(start.elapsed().as_secs_f64());
612 match res.as_ref() {
613 Ok(_) => {
614 self.succeeded.inc();
615 shard_metrics.cmd_succeeded.inc();
616 }
617 Err(_) => self.failed.inc(),
618 };
619 res
620 }
621}
622
623#[derive(Debug)]
624pub struct CmdsMetrics {
625 pub(crate) init_state: CmdMetrics,
626 pub(crate) add_rollup: CmdMetrics,
627 pub(crate) remove_rollups: CmdMetrics,
628 pub(crate) register: CmdMetrics,
629 pub(crate) compare_and_append: CmdMetrics,
630 pub(crate) compare_and_append_noop: IntCounter,
631 pub(crate) compare_and_downgrade_since: CmdMetrics,
632 pub(crate) downgrade_since: CmdMetrics,
633 pub(crate) heartbeat_reader: CmdMetrics,
634 pub(crate) expire_reader: CmdMetrics,
635 pub(crate) expire_writer: CmdMetrics,
636 pub(crate) merge_res: CmdMetrics,
637 pub(crate) become_tombstone: CmdMetrics,
638 pub(crate) compare_and_evolve_schema: CmdMetrics,
639 pub(crate) spine_exert: CmdMetrics,
640 pub(crate) fetch_upper_count: IntCounter,
641}
642
643#[derive(Debug)]
644pub struct RetryMetrics {
645 pub(crate) name: String,
646 pub(crate) started: IntCounter,
647 pub(crate) finished: IntCounter,
648 pub(crate) retries: IntCounter,
649 pub(crate) sleep_seconds: Counter,
650}
651
652impl RetryMetrics {
653 pub(crate) fn stream(&self, retry: RetryStream) -> MetricsRetryStream {
654 MetricsRetryStream::new(retry, self)
655 }
656}
657
658#[derive(Debug)]
659pub struct RetryDeterminate {
660 pub(crate) apply_unbatched_cmd_cas: RetryMetrics,
661}
662
663#[derive(Debug)]
664pub struct RetryExternal {
665 pub(crate) batch_delete: Arc<RetryMetrics>,
666 pub(crate) batch_set: RetryMetrics,
667 pub(crate) blob_open: RetryMetrics,
668 pub(crate) compaction_noop_delete: Arc<RetryMetrics>,
669 pub(crate) consensus_open: RetryMetrics,
670 pub(crate) fetch_batch_get: RetryMetrics,
671 pub(crate) fetch_state_scan: RetryMetrics,
672 pub(crate) gc_truncate: RetryMetrics,
673 pub(crate) maybe_init_cas: RetryMetrics,
674 pub(crate) rollup_delete: RetryMetrics,
675 pub(crate) rollup_get: RetryMetrics,
676 pub(crate) rollup_set: RetryMetrics,
677 pub(crate) hollow_run_get: RetryMetrics,
678 pub(crate) hollow_run_set: RetryMetrics,
679 pub(crate) storage_usage_shard_size: RetryMetrics,
680}
681
682#[derive(Debug)]
683pub struct RetriesMetrics {
684 pub(crate) determinate: RetryDeterminate,
685 pub(crate) external: RetryExternal,
686
687 pub(crate) compare_and_append_idempotent: RetryMetrics,
688 pub(crate) fetch_latest_state: RetryMetrics,
689 pub(crate) fetch_live_states: RetryMetrics,
690 pub(crate) idempotent_cmd: RetryMetrics,
691 pub(crate) next_listen_batch: RetryMetrics,
692 pub(crate) snapshot: RetryMetrics,
693}
694
695#[derive(Debug)]
696pub struct BatchPartReadMetrics {
697 pub(crate) listen: ReadMetrics,
698 pub(crate) snapshot: ReadMetrics,
699 pub(crate) batch_fetcher: ReadMetrics,
700 pub(crate) compaction: ReadMetrics,
701 pub(crate) unindexed: ReadMetrics,
702}
703
704#[derive(Debug, Clone)]
705pub struct ReadMetrics {
706 pub(crate) part_bytes: IntCounter,
707 pub(crate) part_goodbytes: IntCounter,
708 pub(crate) part_count: IntCounter,
709 pub(crate) seconds: Counter,
710 pub(crate) ts_rewrite: IntCounter,
711}
712
713#[derive(Debug, Clone)]
716pub struct BatchWriteMetrics {
717 pub(crate) bytes: IntCounter,
718 pub(crate) goodbytes: IntCounter,
719 pub(crate) seconds: Counter,
720 pub(crate) write_stalls: IntCounter,
721 pub(crate) key_lower_too_big: IntCounter,
722
723 pub(crate) unordered: IntCounter,
724 pub(crate) codec_order: IntCounter,
725 pub(crate) structured_order: IntCounter,
726 _order_counts: IntCounterVec,
727
728 pub(crate) step_stats: Counter,
729 pub(crate) step_part_writing: Counter,
730 pub(crate) step_inline: Counter,
731}
732
733impl BatchWriteMetrics {
734 fn new(registry: &MetricsRegistry, name: &str) -> Self {
735 let order_counts: IntCounterVec = registry.register(metric!(
736 name: format!("mz_persist_{}_write_batch_order", name),
737 help: "count of batches by the data ordering",
738 var_labels: ["order"],
739 ));
740 let unordered = order_counts.with_label_values(&["unordered"]);
741 let codec_order = order_counts.with_label_values(&["codec"]);
742 let structured_order = order_counts.with_label_values(&["structured"]);
743
744 BatchWriteMetrics {
745 bytes: registry.register(metric!(
746 name: format!("mz_persist_{}_bytes", name),
747 help: format!("total encoded size of {} batches written", name),
748 )),
749 goodbytes: registry.register(metric!(
750 name: format!("mz_persist_{}_goodbytes", name),
751 help: format!("total logical size of {} batches written", name),
752 )),
753 seconds: registry.register(metric!(
754 name: format!("mz_persist_{}_write_batch_part_seconds", name),
755 help: format!("time spent writing {} batches", name),
756 )),
757 write_stalls: registry.register(metric!(
758 name: format!("mz_persist_{}_write_stall_count", name),
759 help: format!(
760 "count of {} writes stalling to await max outstanding reqs",
761 name
762 ),
763 )),
764 key_lower_too_big: registry.register(metric!(
765 name: format!("mz_persist_{}_key_lower_too_big", name),
766 help: format!(
767 "count of {} writes that were unable to write a key lower, because the size threshold was too low",
768 name
769 ),
770 )),
771 unordered,
772 codec_order,
773 structured_order,
774 _order_counts: order_counts,
775 step_stats: registry.register(metric!(
776 name: format!("mz_persist_{}_step_stats", name),
777 help: format!("time spent computing {} update stats", name),
778 )),
779 step_part_writing: registry.register(metric!(
780 name: format!("mz_persist_{}_step_part_writing", name),
781 help: format!("blocking time spent writing parts for {} updates", name),
782 )),
783 step_inline: registry.register(metric!(
784 name: format!("mz_persist_{}_step_inline", name),
785 help: format!("time spent encoding {} inline batches", name)
786 )),
787 }
788 }
789}
790
791#[derive(Debug)]
792pub struct CompactionMetrics {
793 pub(crate) requested: IntCounter,
794 pub(crate) dropped: IntCounter,
795 pub(crate) disabled: IntCounter,
796 pub(crate) skipped: IntCounter,
797 pub(crate) started: IntCounter,
798 pub(crate) applied: IntCounter,
799 pub(crate) timed_out: IntCounter,
800 pub(crate) failed: IntCounter,
801 pub(crate) noop: IntCounter,
802 pub(crate) seconds: Counter,
803 pub(crate) concurrency_waits: IntCounter,
804 pub(crate) queued_seconds: Counter,
805 pub(crate) memory_violations: IntCounter,
806 pub(crate) runs_compacted: IntCounter,
807 pub(crate) chunks_compacted: IntCounter,
808 pub(crate) not_all_prefetched: IntCounter,
809 pub(crate) parts_prefetched: IntCounter,
810 pub(crate) parts_waited: IntCounter,
811 pub(crate) fast_path_eligible: IntCounter,
812 pub(crate) admin_count: IntCounter,
813
814 pub(crate) applied_exact_match: IntCounter,
815 pub(crate) applied_subset_match: IntCounter,
816 pub(crate) not_applied_too_many_updates: IntCounter,
817
818 pub(crate) batch: BatchWriteMetrics,
819 pub(crate) steps: CompactionStepTimings,
820 pub(crate) schema_selection: CompactionSchemaSelection,
821
822 pub(crate) _steps_vec: CounterVec,
823}
824
825impl CompactionMetrics {
826 fn new(registry: &MetricsRegistry) -> Self {
827 let step_timings: CounterVec = registry.register(metric!(
828 name: "mz_persist_compaction_step_seconds",
829 help: "time spent on individual steps of compaction",
830 var_labels: ["step"],
831 ));
832 let schema_selection: CounterVec = registry.register(metric!(
833 name: "mz_persist_compaction_schema_selection",
834 help: "count of compactions and how we did schema selection",
835 var_labels: ["selection"],
836 ));
837
838 CompactionMetrics {
839 requested: registry.register(metric!(
840 name: "mz_persist_compaction_requested",
841 help: "count of total compaction requests",
842 )),
843 dropped: registry.register(metric!(
844 name: "mz_persist_compaction_dropped",
845 help: "count of total compaction requests dropped due to a full queue",
846 )),
847 disabled: registry.register(metric!(
848 name: "mz_persist_compaction_disabled",
849 help: "count of total compaction requests dropped because compaction was disabled",
850 )),
851 skipped: registry.register(metric!(
852 name: "mz_persist_compaction_skipped",
853 help: "count of compactions skipped due to heuristics",
854 )),
855 started: registry.register(metric!(
856 name: "mz_persist_compaction_started",
857 help: "count of compactions started",
858 )),
859 failed: registry.register(metric!(
860 name: "mz_persist_compaction_failed",
861 help: "count of compactions failed",
862 )),
863 applied: registry.register(metric!(
864 name: "mz_persist_compaction_applied",
865 help: "count of compactions applied to state",
866 )),
867 timed_out: registry.register(metric!(
868 name: "mz_persist_compaction_timed_out",
869 help: "count of compactions that timed out",
870 )),
871 noop: registry.register(metric!(
872 name: "mz_persist_compaction_noop",
873 help: "count of compactions discarded (obsolete)",
874 )),
875 seconds: registry.register(metric!(
876 name: "mz_persist_compaction_seconds",
877 help: "time spent in compaction",
878 )),
879 concurrency_waits: registry.register(metric!(
880 name: "mz_persist_compaction_concurrency_waits",
881 help: "count of compaction requests that ever blocked due to concurrency limit",
882 )),
883 queued_seconds: registry.register(metric!(
884 name: "mz_persist_compaction_queued_seconds",
885 help: "time that compaction requests spent queued",
886 )),
887 memory_violations: registry.register(metric!(
888 name: "mz_persist_compaction_memory_violations",
889 help: "count of compaction memory requirement violations",
890 )),
891 runs_compacted: registry.register(metric!(
892 name: "mz_persist_compaction_runs_compacted",
893 help: "count of runs compacted",
894 )),
895 chunks_compacted: registry.register(metric!(
896 name: "mz_persist_compaction_chunks_compacted",
897 help: "count of run chunks compacted",
898 )),
899 not_all_prefetched: registry.register(metric!(
900 name: "mz_persist_compaction_not_all_prefetched",
901 help: "count of compactions where not all inputs were prefetched",
902 )),
903 parts_prefetched: registry.register(metric!(
904 name: "mz_persist_compaction_parts_prefetched",
905 help: "count of compaction parts completely prefetched by the time they're needed",
906 )),
907 parts_waited: registry.register(metric!(
908 name: "mz_persist_compaction_parts_waited",
909 help: "count of compaction parts that had to be waited on",
910 )),
911 fast_path_eligible: registry.register(metric!(
912 name: "mz_persist_compaction_fast_path_eligible",
913 help: "count of compaction requests that could have used the fast-path optimization",
914 )),
915 admin_count: registry.register(metric!(
916 name: "mz_persist_compaction_admin_count",
917 help: "count of compaction requests that were performed by admin tooling",
918 )),
919 applied_exact_match: registry.register(metric!(
920 name: "mz_persist_compaction_applied_exact_match",
921 help: "count of merge results that exactly replaced a SpineBatch",
922 )),
923 applied_subset_match: registry.register(metric!(
924 name: "mz_persist_compaction_applied_subset_match",
925 help: "count of merge results that replaced a subset of a SpineBatch",
926 )),
927 not_applied_too_many_updates: registry.register(metric!(
928 name: "mz_persist_compaction_not_applied_too_many_updates",
929 help: "count of merge results that did not apply due to too many updates",
930 )),
931 batch: BatchWriteMetrics::new(registry, "compaction"),
932 steps: CompactionStepTimings::new(step_timings.clone()),
933 schema_selection: CompactionSchemaSelection::new(schema_selection.clone()),
934 _steps_vec: step_timings,
935 }
936 }
937}
938
939#[derive(Debug)]
940pub struct CompactionStepTimings {
941 pub(crate) part_fetch_seconds: Counter,
942 pub(crate) heap_population_seconds: Counter,
943}
944
945impl CompactionStepTimings {
946 fn new(step_timings: CounterVec) -> CompactionStepTimings {
947 CompactionStepTimings {
948 part_fetch_seconds: step_timings.with_label_values(&["part_fetch"]),
949 heap_population_seconds: step_timings.with_label_values(&["heap_population"]),
950 }
951 }
952}
953
954#[derive(Debug)]
955pub struct CompactionSchemaSelection {
956 pub(crate) recent_schema: Counter,
957 pub(crate) no_schema: Counter,
958 pub(crate) disabled: Counter,
959}
960
961impl CompactionSchemaSelection {
962 fn new(schema_selection: CounterVec) -> CompactionSchemaSelection {
963 CompactionSchemaSelection {
964 recent_schema: schema_selection.with_label_values(&["recent"]),
965 no_schema: schema_selection.with_label_values(&["none"]),
966 disabled: schema_selection.with_label_values(&["disabled"]),
967 }
968 }
969}
970
971#[derive(Debug)]
972pub struct GcMetrics {
973 pub(crate) noop: IntCounter,
974 pub(crate) started: IntCounter,
975 pub(crate) finished: IntCounter,
976 pub(crate) merged: IntCounter,
977 pub(crate) seconds: Counter,
978 pub(crate) steps: GcStepTimings,
979}
980
981#[derive(Debug)]
982pub struct GcStepTimings {
983 pub(crate) find_removable_rollups: Counter,
984 pub(crate) fetch_seconds: Counter,
985 pub(crate) find_deletable_blobs_seconds: Counter,
986 pub(crate) delete_rollup_seconds: Counter,
987 pub(crate) delete_batch_part_seconds: Counter,
988 pub(crate) truncate_diff_seconds: Counter,
989 pub(crate) remove_rollups_from_state: Counter,
990 pub(crate) post_gc_calculations_seconds: Counter,
991}
992
993impl GcStepTimings {
994 fn new(step_timings: CounterVec) -> Self {
995 Self {
996 find_removable_rollups: step_timings.with_label_values(&["find_removable_rollups"]),
997 fetch_seconds: step_timings.with_label_values(&["fetch"]),
998 find_deletable_blobs_seconds: step_timings.with_label_values(&["find_deletable_blobs"]),
999 delete_rollup_seconds: step_timings.with_label_values(&["delete_rollup"]),
1000 delete_batch_part_seconds: step_timings.with_label_values(&["delete_batch_part"]),
1001 truncate_diff_seconds: step_timings.with_label_values(&["truncate_diff"]),
1002 remove_rollups_from_state: step_timings
1003 .with_label_values(&["remove_rollups_from_state"]),
1004 post_gc_calculations_seconds: step_timings.with_label_values(&["post_gc_calculations"]),
1005 }
1006 }
1007}
1008
1009impl GcMetrics {
1010 fn new(registry: &MetricsRegistry) -> Self {
1011 let step_timings: CounterVec = registry.register(metric!(
1012 name: "mz_persist_gc_step_seconds",
1013 help: "time spent on individual steps of gc",
1014 var_labels: ["step"],
1015 ));
1016 GcMetrics {
1017 noop: registry.register(metric!(
1018 name: "mz_persist_gc_noop",
1019 help: "count of garbage collections skipped because they were already done",
1020 )),
1021 started: registry.register(metric!(
1022 name: "mz_persist_gc_started",
1023 help: "count of garbage collections started",
1024 )),
1025 finished: registry.register(metric!(
1026 name: "mz_persist_gc_finished",
1027 help: "count of garbage collections finished",
1028 )),
1029 merged: registry.register(metric!(
1030 name: "mz_persist_gc_merged_reqs",
1031 help: "count of garbage collection requests merged",
1032 )),
1033 seconds: registry.register(metric!(
1034 name: "mz_persist_gc_seconds",
1035 help: "time spent in garbage collections",
1036 )),
1037 steps: GcStepTimings::new(step_timings),
1038 }
1039 }
1040}
1041
1042#[derive(Debug)]
1043pub struct LeaseMetrics {
1044 pub(crate) timeout_read: IntCounter,
1045 pub(crate) dropped_part: IntCounter,
1046}
1047
1048impl LeaseMetrics {
1049 fn new(registry: &MetricsRegistry) -> Self {
1050 LeaseMetrics {
1051 timeout_read: registry.register(metric!(
1052 name: "mz_persist_lease_timeout_read",
1053 help: "count of readers whose lease timed out",
1054 )),
1055 dropped_part: registry.register(metric!(
1056 name: "mz_persist_lease_dropped_part",
1057 help: "count of LeasedBatchParts that were dropped without being politely returned",
1058 )),
1059 }
1060 }
1061}
1062
1063struct IncOnDrop(IntCounter);
1064
1065impl Drop for IncOnDrop {
1066 fn drop(&mut self) {
1067 self.0.inc()
1068 }
1069}
1070
1071pub struct MetricsRetryStream {
1072 retry: RetryStream,
1073 pub(crate) retries: IntCounter,
1074 sleep_seconds: Counter,
1075 _finished: IncOnDrop,
1076}
1077
1078impl MetricsRetryStream {
1079 pub fn new(retry: RetryStream, metrics: &RetryMetrics) -> Self {
1080 metrics.started.inc();
1081 MetricsRetryStream {
1082 retry,
1083 retries: metrics.retries.clone(),
1084 sleep_seconds: metrics.sleep_seconds.clone(),
1085 _finished: IncOnDrop(metrics.finished.clone()),
1086 }
1087 }
1088
1089 pub fn attempt(&self) -> usize {
1091 self.retry.attempt()
1092 }
1093
1094 pub fn next_sleep(&self) -> Duration {
1096 self.retry.next_sleep()
1097 }
1098
1099 pub async fn sleep(self) -> Self {
1104 self.retries.inc();
1105 self.sleep_seconds
1106 .inc_by(self.retry.next_sleep().as_secs_f64());
1107 let retry = self.retry.sleep().await;
1108 MetricsRetryStream {
1109 retry,
1110 retries: self.retries,
1111 sleep_seconds: self.sleep_seconds,
1112 _finished: self._finished,
1113 }
1114 }
1115}
1116
1117#[derive(Debug)]
1118pub struct CodecsMetrics {
1119 pub(crate) state: CodecMetrics,
1120 pub(crate) state_diff: CodecMetrics,
1121 pub(crate) batch: CodecMetrics,
1122 pub(crate) key: CodecMetrics,
1123 pub(crate) val: CodecMetrics,
1124 }
1127
1128#[derive(Debug)]
1129pub struct CodecMetrics {
1130 pub(crate) encode_count: IntCounter,
1131 pub(crate) encode_seconds: Counter,
1132 pub(crate) decode_count: IntCounter,
1133 pub(crate) decode_seconds: Counter,
1134}
1135
1136impl CodecMetrics {
1137 pub(crate) fn encode<R, F: FnOnce() -> R>(&self, f: F) -> R {
1138 let now = Instant::now();
1139 let r = f();
1140 self.encode_count.inc();
1141 self.encode_seconds.inc_by(now.elapsed().as_secs_f64());
1142 r
1143 }
1144
1145 pub(crate) fn decode<R, F: FnOnce() -> R>(&self, f: F) -> R {
1146 let now = Instant::now();
1147 let r = f();
1148 self.decode_count.inc();
1149 self.decode_seconds.inc_by(now.elapsed().as_secs_f64());
1150 r
1151 }
1152}
1153
1154#[derive(Debug)]
1155pub struct StateMetrics {
1156 pub(crate) apply_spine_fast_path: IntCounter,
1157 pub(crate) apply_spine_slow_path: IntCounter,
1158 pub(crate) apply_spine_slow_path_lenient: IntCounter,
1159 pub(crate) apply_spine_slow_path_lenient_adjustment: IntCounter,
1160 pub(crate) apply_spine_slow_path_with_reconstruction: IntCounter,
1161 pub(crate) apply_spine_flattened: IntCounter,
1162 pub(crate) update_state_noop_path: IntCounter,
1163 pub(crate) update_state_empty_path: IntCounter,
1164 pub(crate) update_state_fast_path: IntCounter,
1165 pub(crate) update_state_slow_path: IntCounter,
1166 pub(crate) rollup_at_seqno_migration: IntCounter,
1167 pub(crate) fetch_recent_live_diffs_fast_path: IntCounter,
1168 pub(crate) fetch_recent_live_diffs_slow_path: IntCounter,
1169 pub(crate) writer_added: IntCounter,
1170 pub(crate) writer_removed: IntCounter,
1171 pub(crate) force_apply_hostname: IntCounter,
1172 pub(crate) rollup_write_success: IntCounter,
1173 pub(crate) rollup_write_noop_latest: IntCounter,
1174 pub(crate) rollup_write_noop_truncated: IntCounter,
1175}
1176
1177impl StateMetrics {
1178 pub(crate) fn new(registry: &MetricsRegistry) -> Self {
1179 let rollup_write_noop: IntCounterVec = registry.register(metric!(
1180 name: "mz_persist_state_rollup_write_noop",
1181 help: "count of no-op rollup writes",
1182 var_labels: ["reason"],
1183 ));
1184
1185 StateMetrics {
1186 apply_spine_fast_path: registry.register(metric!(
1187 name: "mz_persist_state_apply_spine_fast_path",
1188 help: "count of spine diff applications that hit the fast path",
1189 )),
1190 apply_spine_slow_path: registry.register(metric!(
1191 name: "mz_persist_state_apply_spine_slow_path",
1192 help: "count of spine diff applications that hit the slow path",
1193 )),
1194 apply_spine_slow_path_lenient: registry.register(metric!(
1195 name: "mz_persist_state_apply_spine_slow_path_lenient",
1196 help: "count of spine diff applications that hit the lenient compaction apply path",
1197 )),
1198 apply_spine_slow_path_lenient_adjustment: registry.register(metric!(
1199 name: "mz_persist_state_apply_spine_slow_path_lenient_adjustment",
1200 help: "count of adjustments made by the lenient compaction apply path",
1201 )),
1202 apply_spine_slow_path_with_reconstruction: registry.register(metric!(
1203 name: "mz_persist_state_apply_spine_slow_path_with_reconstruction",
1204 help: "count of spine diff applications that hit the slow path with extra spine reconstruction step",
1205 )),
1206 apply_spine_flattened: registry.register(metric!(
1207 name: "mz_persist_state_apply_spine_flattened",
1208 help: "count of spine diff applications that flatten the trace",
1209 )),
1210 update_state_noop_path: registry.register(metric!(
1211 name: "mz_persist_state_update_state_noop_path",
1212 help: "count of state update applications that no-oped due to shared state",
1213 )),
1214 update_state_empty_path: registry.register(metric!(
1215 name: "mz_persist_state_update_state_empty_path",
1216 help: "count of state update applications that found no new updates",
1217 )),
1218 update_state_fast_path: registry.register(metric!(
1219 name: "mz_persist_state_update_state_fast_path",
1220 help: "count of state update applications that hit the fast path",
1221 )),
1222 update_state_slow_path: registry.register(metric!(
1223 name: "mz_persist_state_update_state_slow_path",
1224 help: "count of state update applications that hit the slow path",
1225 )),
1226 rollup_at_seqno_migration: registry.register(metric!(
1227 name: "mz_persist_state_rollup_at_seqno_migration",
1228 help: "count of fetch_rollup_at_seqno calls that only worked because of the migration",
1229 )),
1230 fetch_recent_live_diffs_fast_path: registry.register(metric!(
1231 name: "mz_persist_state_fetch_recent_live_diffs_fast_path",
1232 help: "count of fetch_recent_live_diffs that hit the fast path",
1233 )),
1234 fetch_recent_live_diffs_slow_path: registry.register(metric!(
1235 name: "mz_persist_state_fetch_recent_live_diffs_slow_path",
1236 help: "count of fetch_recent_live_diffs that hit the slow path",
1237 )),
1238 writer_added: registry.register(metric!(
1239 name: "mz_persist_state_writer_added",
1240 help: "count of writers added to the state",
1241 )),
1242 writer_removed: registry.register(metric!(
1243 name: "mz_persist_state_writer_removed",
1244 help: "count of writers removed from the state",
1245 )),
1246 force_apply_hostname: registry.register(metric!(
1247 name: "mz_persist_state_force_applied_hostname",
1248 help: "count of when hostname diffs needed to be force applied",
1249 )),
1250 rollup_write_success: registry.register(metric!(
1251 name: "mz_persist_state_rollup_write_success",
1252 help: "count of rollups written successful (may not be linked in to state)",
1253 )),
1254 rollup_write_noop_latest: rollup_write_noop.with_label_values(&["latest"]),
1255 rollup_write_noop_truncated: rollup_write_noop.with_label_values(&["truncated"]),
1256 }
1257 }
1258}
1259
1260#[derive(Debug)]
1261pub struct ShardsMetrics {
1262 _count: ComputedIntGauge,
1266 since: mz_ore::metrics::IntGaugeVec,
1267 upper: mz_ore::metrics::IntGaugeVec,
1268 encoded_rollup_size: mz_ore::metrics::UIntGaugeVec,
1269 encoded_diff_size: mz_ore::metrics::IntCounterVec,
1270 hollow_batch_count: mz_ore::metrics::UIntGaugeVec,
1271 spine_batch_count: mz_ore::metrics::UIntGaugeVec,
1272 batch_part_count: mz_ore::metrics::UIntGaugeVec,
1273 batch_part_version_count: mz_ore::metrics::UIntGaugeVec,
1274 batch_part_version_bytes: mz_ore::metrics::UIntGaugeVec,
1275 update_count: mz_ore::metrics::UIntGaugeVec,
1276 rollup_count: mz_ore::metrics::UIntGaugeVec,
1277 largest_batch_size: mz_ore::metrics::UIntGaugeVec,
1278 seqnos_held: mz_ore::metrics::UIntGaugeVec,
1279 seqnos_since_last_rollup: mz_ore::metrics::UIntGaugeVec,
1280 gc_seqno_held_parts: mz_ore::metrics::UIntGaugeVec,
1281 gc_live_diffs: mz_ore::metrics::UIntGaugeVec,
1282 gc_finished: mz_ore::metrics::IntCounterVec,
1283 compaction_applied: mz_ore::metrics::IntCounterVec,
1284 cmd_succeeded: mz_ore::metrics::IntCounterVec,
1285 usage_current_state_batches_bytes: mz_ore::metrics::UIntGaugeVec,
1286 usage_current_state_rollups_bytes: mz_ore::metrics::UIntGaugeVec,
1287 usage_referenced_not_current_state_bytes: mz_ore::metrics::UIntGaugeVec,
1288 usage_not_leaked_not_referenced_bytes: mz_ore::metrics::UIntGaugeVec,
1289 usage_leaked_bytes: mz_ore::metrics::UIntGaugeVec,
1290 pubsub_push_diff_applied: mz_ore::metrics::IntCounterVec,
1291 pubsub_push_diff_not_applied_stale: mz_ore::metrics::IntCounterVec,
1292 pubsub_push_diff_not_applied_out_of_order: mz_ore::metrics::IntCounterVec,
1293 blob_gets: mz_ore::metrics::IntCounterVec,
1294 blob_sets: mz_ore::metrics::IntCounterVec,
1295 live_writers: mz_ore::metrics::UIntGaugeVec,
1296 unconsolidated_snapshot: mz_ore::metrics::IntCounterVec,
1297 backpressure_emitted_bytes: IntCounterVec,
1298 backpressure_last_backpressured_bytes: UIntGaugeVec,
1299 backpressure_retired_bytes: IntCounterVec,
1300 rewrite_part_count: UIntGaugeVec,
1301 inline_part_count: UIntGaugeVec,
1302 inline_part_bytes: UIntGaugeVec,
1303 compact_batches: UIntGaugeVec,
1304 compacting_batches: UIntGaugeVec,
1305 noncompact_batches: UIntGaugeVec,
1306 schema_registry_version_count: UIntGaugeVec,
1307 inline_backpressure_count: IntCounterVec,
1308 shards: Arc<Mutex<BTreeMap<ShardId, Weak<ShardMetrics>>>>,
1312}
1313
1314impl ShardsMetrics {
1315 fn new(registry: &MetricsRegistry) -> Self {
1316 let shards = Arc::new(Mutex::new(BTreeMap::new()));
1317 let shards_count = Arc::clone(&shards);
1318 ShardsMetrics {
1319 _count: registry.register_computed_gauge(
1320 metric!(
1321 name: "mz_persist_shard_count",
1322 help: "count of all active shards on this process",
1323 ),
1324 move || {
1325 let mut ret = 0;
1326 Self::compute(&shards_count, |_m| ret += 1);
1327 ret
1328 },
1329 ),
1330 since: registry.register(metric!(
1331 name: "mz_persist_shard_since",
1332 help: "since by shard",
1333 var_labels: ["shard", "name"],
1334 )),
1335 upper: registry.register(metric!(
1336 name: "mz_persist_shard_upper",
1337 help: "upper by shard",
1338 var_labels: ["shard", "name"],
1339 )),
1340 encoded_rollup_size: registry.register(metric!(
1341 name: "mz_persist_shard_rollup_size_bytes",
1342 help: "total encoded rollup size by shard",
1343 var_labels: ["shard", "name"],
1344 )),
1345 encoded_diff_size: registry.register(metric!(
1346 name: "mz_persist_shard_diff_size_bytes",
1347 help: "total encoded diff size by shard",
1348 var_labels: ["shard", "name"],
1349 )),
1350 hollow_batch_count: registry.register(metric!(
1351 name: "mz_persist_shard_hollow_batch_count",
1352 help: "count of hollow batches by shard",
1353 var_labels: ["shard", "name"],
1354 )),
1355 spine_batch_count: registry.register(metric!(
1356 name: "mz_persist_shard_spine_batch_count",
1357 help: "count of spine batches by shard",
1358 var_labels: ["shard", "name"],
1359 )),
1360 batch_part_count: registry.register(metric!(
1361 name: "mz_persist_shard_batch_part_count",
1362 help: "count of batch parts by shard",
1363 var_labels: ["shard", "name"],
1364 )),
1365 batch_part_version_count: registry.register(metric!(
1366 name: "mz_persist_shard_batch_part_version_count",
1367 help: "count of batch parts by shard and version",
1368 var_labels: ["shard", "name", "version"],
1369 )),
1370 batch_part_version_bytes: registry.register(metric!(
1371 name: "mz_persist_shard_batch_part_version_bytes",
1372 help: "total bytes in batch parts by shard and version",
1373 var_labels: ["shard", "name", "version"],
1374 )),
1375 update_count: registry.register(metric!(
1376 name: "mz_persist_shard_update_count",
1377 help: "count of updates by shard",
1378 var_labels: ["shard", "name"],
1379 )),
1380 rollup_count: registry.register(metric!(
1381 name: "mz_persist_shard_rollup_count",
1382 help: "count of rollups by shard",
1383 var_labels: ["shard", "name"],
1384 )),
1385 largest_batch_size: registry.register(metric!(
1386 name: "mz_persist_shard_largest_batch_size",
1387 help: "largest encoded batch size by shard",
1388 var_labels: ["shard", "name"],
1389 )),
1390 seqnos_held: registry.register(metric!(
1391 name: "mz_persist_shard_seqnos_held",
1392 help: "maximum count of gc-ineligible states by shard",
1393 var_labels: ["shard", "name"],
1394 )),
1395 seqnos_since_last_rollup: registry.register(metric!(
1396 name: "mz_persist_shard_seqnos_since_last_rollup",
1397 help: "count of seqnos since last rollup",
1398 var_labels: ["shard", "name"],
1399 )),
1400 gc_seqno_held_parts: registry.register(metric!(
1401 name: "mz_persist_shard_gc_seqno_held_parts",
1402 help: "count of parts referenced by some live state but not the current state (ie. parts kept only to satisfy seqno holds) at GC time",
1403 var_labels: ["shard", "name"],
1404 )),
1405 gc_live_diffs: registry.register(metric!(
1406 name: "mz_persist_shard_gc_live_diffs",
1407 help: "the number of diffs (or, alternatively, the number of seqnos) present in consensus state at GC time",
1408 var_labels: ["shard", "name"],
1409 )),
1410 gc_finished: registry.register(metric!(
1411 name: "mz_persist_shard_gc_finished",
1412 help: "count of garbage collections finished by shard",
1413 var_labels: ["shard", "name"],
1414 )),
1415 compaction_applied: registry.register(metric!(
1416 name: "mz_persist_shard_compaction_applied",
1417 help: "count of compactions applied to state by shard",
1418 var_labels: ["shard", "name"],
1419 )),
1420 cmd_succeeded: registry.register(metric!(
1421 name: "mz_persist_shard_cmd_succeeded",
1422 help: "count of commands succeeded by shard",
1423 var_labels: ["shard", "name"],
1424 )),
1425 usage_current_state_batches_bytes: registry.register(metric!(
1426 name: "mz_persist_shard_usage_current_state_batches_bytes",
1427 help: "data in batches/parts referenced by current version of state",
1428 var_labels: ["shard", "name"],
1429 )),
1430 usage_current_state_rollups_bytes: registry.register(metric!(
1431 name: "mz_persist_shard_usage_current_state_rollups_bytes",
1432 help: "data in rollups referenced by current version of state",
1433 var_labels: ["shard", "name"],
1434 )),
1435 usage_referenced_not_current_state_bytes: registry.register(metric!(
1436 name: "mz_persist_shard_usage_referenced_not_current_state_bytes",
1437 help: "data referenced only by a previous version of state",
1438 var_labels: ["shard", "name"],
1439 )),
1440 usage_not_leaked_not_referenced_bytes: registry.register(metric!(
1441 name: "mz_persist_shard_usage_not_leaked_not_referenced_bytes",
1442 help: "data written by an active writer but not referenced by any version of state",
1443 var_labels: ["shard", "name"],
1444 )),
1445 usage_leaked_bytes: registry.register(metric!(
1446 name: "mz_persist_shard_usage_leaked_bytes",
1447 help: "data reclaimable by a leaked blob detector",
1448 var_labels: ["shard", "name"],
1449 )),
1450 pubsub_push_diff_applied: registry.register(metric!(
1451 name: "mz_persist_shard_pubsub_diff_applied",
1452 help: "number of diffs received via pubsub that applied",
1453 var_labels: ["shard", "name"],
1454 )),
1455 pubsub_push_diff_not_applied_stale: registry.register(metric!(
1456 name: "mz_persist_shard_pubsub_diff_not_applied_stale",
1457 help: "number of diffs received via pubsub that did not apply due to staleness",
1458 var_labels: ["shard", "name"],
1459 )),
1460 pubsub_push_diff_not_applied_out_of_order: registry.register(metric!(
1461 name: "mz_persist_shard_pubsub_diff_not_applied_out_of_order",
1462 help: "number of diffs received via pubsub that did not apply due to out-of-order delivery",
1463 var_labels: ["shard", "name"],
1464 )),
1465 blob_gets: registry.register(metric!(
1466 name: "mz_persist_shard_blob_gets",
1467 help: "number of Blob::get calls for this shard",
1468 var_labels: ["shard", "name"],
1469 )),
1470 blob_sets: registry.register(metric!(
1471 name: "mz_persist_shard_blob_sets",
1472 help: "number of Blob::set calls for this shard",
1473 var_labels: ["shard", "name"],
1474 )),
1475 live_writers: registry.register(metric!(
1476 name: "mz_persist_shard_live_writers",
1477 help: "number of writers that have recently appended updates to this shard",
1478 var_labels: ["shard", "name"],
1479 )),
1480 unconsolidated_snapshot: registry.register(metric!(
1481 name: "mz_persist_shard_unconsolidated_snapshot",
1482 help: "in snapshot_and_read, the number of times consolidating the raw data wasn't enough to produce consolidated output",
1483 var_labels: ["shard", "name"],
1484 )),
1485 backpressure_emitted_bytes: registry.register(metric!(
1486 name: "mz_persist_backpressure_emitted_bytes",
1487 help: "A counter with the number of emitted bytes.",
1488 var_labels: ["shard", "name"],
1489 )),
1490 backpressure_last_backpressured_bytes: registry.register(metric!(
1491 name: "mz_persist_backpressure_last_backpressured_bytes",
1492 help: "The last count of bytes we are waiting to be retired in \
1493 the operator. This cannot be directly compared to \
1494 `retired_bytes`, but CAN indicate that backpressure is happening.",
1495 var_labels: ["shard", "name"],
1496 )),
1497 backpressure_retired_bytes: registry.register(metric!(
1498 name: "mz_persist_backpressure_retired_bytes",
1499 help:"A counter with the number of bytes retired by downstream processing.",
1500 var_labels: ["shard", "name"],
1501 )),
1502 rewrite_part_count: registry.register(metric!(
1503 name: "mz_persist_shard_rewrite_part_count",
1504 help: "count of batch parts with rewrites by shard",
1505 var_labels: ["shard", "name"],
1506 )),
1507 inline_part_count: registry.register(metric!(
1508 name: "mz_persist_shard_inline_part_count",
1509 help: "count of parts inline in shard metadata",
1510 var_labels: ["shard", "name"],
1511 )),
1512 inline_part_bytes: registry.register(metric!(
1513 name: "mz_persist_shard_inline_part_bytes",
1514 help: "total size of parts inline in shard metadata",
1515 var_labels: ["shard", "name"],
1516 )),
1517 compact_batches: registry.register(metric!(
1518 name: "mz_persist_shard_compact_batches",
1519 help: "number of fully compact batches in the shard",
1520 var_labels: ["shard", "name"],
1521 )),
1522 compacting_batches: registry.register(metric!(
1523 name: "mz_persist_shard_compacting_batches",
1524 help: "number of batches in the shard with compactions in progress",
1525 var_labels: ["shard", "name"],
1526 )),
1527 noncompact_batches: registry.register(metric!(
1528 name: "mz_persist_shard_noncompact_batches",
1529 help: "number of batches in the shard that aren't compact and have no ongoing compaction",
1530 var_labels: ["shard", "name"],
1531 )),
1532 schema_registry_version_count: registry.register(metric!(
1533 name: "mz_persist_shard_schema_registry_version_count",
1534 help: "count of versions in the schema registry",
1535 var_labels: ["shard", "name"],
1536 )),
1537 inline_backpressure_count: registry.register(metric!(
1538 name: "mz_persist_shard_inline_backpressure_count",
1539 help: "count of CaA attempts retried because of inline backpressure",
1540 var_labels: ["shard", "name"],
1541 )),
1542 shards,
1543 }
1544 }
1545
1546 pub fn shard(&self, shard_id: &ShardId, name: &str) -> Arc<ShardMetrics> {
1547 let mut shards = self.shards.lock().expect("mutex poisoned");
1548 if let Some(shard) = shards.get(shard_id) {
1549 if let Some(shard) = shard.upgrade() {
1550 return Arc::clone(&shard);
1551 } else {
1552 assert!(shards.remove(shard_id).is_some());
1553 }
1554 }
1555 let shard = Arc::new(ShardMetrics::new(shard_id, name, self));
1556 assert!(
1557 shards
1558 .insert(shard_id.clone(), Arc::downgrade(&shard))
1559 .is_none()
1560 );
1561 shard
1562 }
1563
1564 fn compute<F: FnMut(&ShardMetrics)>(
1565 shards: &Arc<Mutex<BTreeMap<ShardId, Weak<ShardMetrics>>>>,
1566 mut f: F,
1567 ) {
1568 let mut shards = shards.lock().expect("mutex poisoned");
1569 let mut deleted_shards = Vec::new();
1570 for (shard_id, metrics) in shards.iter() {
1571 if let Some(metrics) = metrics.upgrade() {
1572 f(&metrics);
1573 } else {
1574 deleted_shards.push(shard_id.clone());
1575 }
1576 }
1577 for deleted_shard_id in deleted_shards {
1578 assert!(shards.remove(&deleted_shard_id).is_some());
1579 }
1580 }
1581}
1582
1583#[derive(Debug)]
1584pub struct ShardMetrics {
1585 pub shard_id: ShardId,
1586 pub name: String,
1587 pub since: DeleteOnDropGauge<AtomicI64, Vec<String>>,
1588 pub upper: DeleteOnDropGauge<AtomicI64, Vec<String>>,
1589 pub largest_batch_size: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1590 pub latest_rollup_size: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1591 pub encoded_diff_size: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1592 pub hollow_batch_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1593 pub spine_batch_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1594 pub batch_part_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1595 batch_part_version_count: mz_ore::metrics::UIntGaugeVec,
1596 batch_part_version_bytes: mz_ore::metrics::UIntGaugeVec,
1597 batch_part_version_map: Mutex<BTreeMap<String, BatchPartVersionMetrics>>,
1598 pub update_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1599 pub rollup_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1600 pub seqnos_held: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1601 pub seqnos_since_last_rollup: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1602 pub gc_seqno_held_parts: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1603 pub gc_live_diffs: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1604 pub usage_current_state_batches_bytes: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1605 pub usage_current_state_rollups_bytes: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1606 pub usage_referenced_not_current_state_bytes: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1607 pub usage_not_leaked_not_referenced_bytes: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1608 pub usage_leaked_bytes: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1609 pub gc_finished: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1610 pub compaction_applied: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1611 pub cmd_succeeded: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1612 pub pubsub_push_diff_applied: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1613 pub pubsub_push_diff_not_applied_stale: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1614 pub pubsub_push_diff_not_applied_out_of_order: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1615 pub blob_gets: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1616 pub blob_sets: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1617 pub live_writers: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1618 pub unconsolidated_snapshot: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1619 pub backpressure_emitted_bytes: Arc<DeleteOnDropCounter<AtomicU64, Vec<String>>>,
1620 pub backpressure_last_backpressured_bytes: Arc<DeleteOnDropGauge<AtomicU64, Vec<String>>>,
1621 pub backpressure_retired_bytes: Arc<DeleteOnDropCounter<AtomicU64, Vec<String>>>,
1622 pub rewrite_part_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1623 pub inline_part_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1624 pub inline_part_bytes: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1625 pub compact_batches: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1626 pub compacting_batches: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1627 pub noncompact_batches: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1628 pub schema_registry_version_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1629 pub inline_backpressure_count: DeleteOnDropCounter<AtomicU64, Vec<String>>,
1630}
1631
1632impl ShardMetrics {
1633 pub fn new(shard_id: &ShardId, name: &str, shards_metrics: &ShardsMetrics) -> Self {
1634 let shard = shard_id.to_string();
1635 ShardMetrics {
1636 shard_id: *shard_id,
1637 name: name.to_string(),
1638 since: shards_metrics
1639 .since
1640 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1641 upper: shards_metrics
1642 .upper
1643 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1644 latest_rollup_size: shards_metrics
1645 .encoded_rollup_size
1646 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1647 encoded_diff_size: shards_metrics
1648 .encoded_diff_size
1649 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1650 hollow_batch_count: shards_metrics
1651 .hollow_batch_count
1652 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1653 spine_batch_count: shards_metrics
1654 .spine_batch_count
1655 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1656 batch_part_count: shards_metrics
1657 .batch_part_count
1658 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1659 batch_part_version_count: shards_metrics.batch_part_version_count.clone(),
1660 batch_part_version_bytes: shards_metrics.batch_part_version_bytes.clone(),
1661 batch_part_version_map: Mutex::new(BTreeMap::new()),
1662 update_count: shards_metrics
1663 .update_count
1664 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1665 rollup_count: shards_metrics
1666 .rollup_count
1667 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1668 largest_batch_size: shards_metrics
1669 .largest_batch_size
1670 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1671 seqnos_held: shards_metrics
1672 .seqnos_held
1673 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1674 seqnos_since_last_rollup: shards_metrics
1675 .seqnos_since_last_rollup
1676 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1677 gc_seqno_held_parts: shards_metrics
1678 .gc_seqno_held_parts
1679 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1680 gc_live_diffs: shards_metrics
1681 .gc_live_diffs
1682 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1683 gc_finished: shards_metrics
1684 .gc_finished
1685 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1686 compaction_applied: shards_metrics
1687 .compaction_applied
1688 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1689 cmd_succeeded: shards_metrics
1690 .cmd_succeeded
1691 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1692 usage_current_state_batches_bytes: shards_metrics
1693 .usage_current_state_batches_bytes
1694 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1695 usage_current_state_rollups_bytes: shards_metrics
1696 .usage_current_state_rollups_bytes
1697 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1698 usage_referenced_not_current_state_bytes: shards_metrics
1699 .usage_referenced_not_current_state_bytes
1700 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1701 usage_not_leaked_not_referenced_bytes: shards_metrics
1702 .usage_not_leaked_not_referenced_bytes
1703 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1704 usage_leaked_bytes: shards_metrics
1705 .usage_leaked_bytes
1706 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1707 pubsub_push_diff_applied: shards_metrics
1708 .pubsub_push_diff_applied
1709 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1710 pubsub_push_diff_not_applied_stale: shards_metrics
1711 .pubsub_push_diff_not_applied_stale
1712 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1713 pubsub_push_diff_not_applied_out_of_order: shards_metrics
1714 .pubsub_push_diff_not_applied_out_of_order
1715 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1716 blob_gets: shards_metrics
1717 .blob_gets
1718 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1719 blob_sets: shards_metrics
1720 .blob_sets
1721 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1722 live_writers: shards_metrics
1723 .live_writers
1724 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1725 unconsolidated_snapshot: shards_metrics
1726 .unconsolidated_snapshot
1727 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1728 backpressure_emitted_bytes: Arc::new(
1729 shards_metrics
1730 .backpressure_emitted_bytes
1731 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1732 ),
1733 backpressure_last_backpressured_bytes: Arc::new(
1734 shards_metrics
1735 .backpressure_last_backpressured_bytes
1736 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1737 ),
1738 backpressure_retired_bytes: Arc::new(
1739 shards_metrics
1740 .backpressure_retired_bytes
1741 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1742 ),
1743 rewrite_part_count: shards_metrics
1744 .rewrite_part_count
1745 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1746 inline_part_count: shards_metrics
1747 .inline_part_count
1748 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1749 inline_part_bytes: shards_metrics
1750 .inline_part_bytes
1751 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1752 compact_batches: shards_metrics
1753 .compact_batches
1754 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1755 compacting_batches: shards_metrics
1756 .compacting_batches
1757 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1758 noncompact_batches: shards_metrics
1759 .noncompact_batches
1760 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1761 schema_registry_version_count: shards_metrics
1762 .schema_registry_version_count
1763 .get_delete_on_drop_metric(vec![shard.clone(), name.to_string()]),
1764 inline_backpressure_count: shards_metrics
1765 .inline_backpressure_count
1766 .get_delete_on_drop_metric(vec![shard, name.to_string()]),
1767 }
1768 }
1769
1770 pub fn set_since<T: Codec64>(&self, since: &Antichain<T>) {
1771 self.since.set(encode_ts_metric(since))
1772 }
1773
1774 pub fn set_upper<T: Codec64>(&self, upper: &Antichain<T>) {
1775 self.upper.set(encode_ts_metric(upper))
1776 }
1777
1778 pub(crate) fn set_batch_part_versions<'a>(
1779 &self,
1780 batch_parts_by_version: impl Iterator<Item = (&'a str, usize)>,
1781 ) {
1782 let mut map = self
1783 .batch_part_version_map
1784 .lock()
1785 .expect("mutex should not be poisoned");
1786 for x in map.values() {
1793 x.batch_part_version_count.set(0);
1794 x.batch_part_version_bytes.set(0);
1795 }
1796
1797 for (key, bytes) in batch_parts_by_version {
1800 if !map.contains_key(key) {
1801 map.insert(
1802 key.to_owned(),
1803 BatchPartVersionMetrics {
1804 batch_part_version_count: self
1805 .batch_part_version_count
1806 .get_delete_on_drop_metric(vec![
1807 self.shard_id.to_string(),
1808 self.name.clone(),
1809 key.to_owned(),
1810 ]),
1811 batch_part_version_bytes: self
1812 .batch_part_version_bytes
1813 .get_delete_on_drop_metric(vec![
1814 self.shard_id.to_string(),
1815 self.name.clone(),
1816 key.to_owned(),
1817 ]),
1818 },
1819 );
1820 }
1821 let value = map.get(key).expect("inserted above");
1822 value.batch_part_version_count.inc();
1823 value.batch_part_version_bytes.add(u64::cast_from(bytes));
1824 }
1825 }
1826}
1827
1828#[derive(Debug)]
1829pub struct BatchPartVersionMetrics {
1830 pub batch_part_version_count: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1831 pub batch_part_version_bytes: DeleteOnDropGauge<AtomicU64, Vec<String>>,
1832}
1833
1834#[derive(Debug)]
1836pub struct UsageAuditMetrics {
1837 pub blob_batch_part_bytes: UIntGauge,
1839 pub blob_batch_part_count: UIntGauge,
1841 pub blob_rollup_bytes: UIntGauge,
1843 pub blob_rollup_count: UIntGauge,
1845 pub blob_bytes: UIntGauge,
1847 pub blob_count: UIntGauge,
1849 pub step_blob_metadata: Counter,
1851 pub step_state: Counter,
1853 pub step_math: Counter,
1855}
1856
1857impl UsageAuditMetrics {
1858 fn new(registry: &MetricsRegistry) -> Self {
1859 let step_timings: CounterVec = registry.register(metric!(
1860 name: "mz_persist_audit_step_seconds",
1861 help: "time spent on individual steps of audit",
1862 var_labels: ["step"],
1863 ));
1864 UsageAuditMetrics {
1865 blob_batch_part_bytes: registry.register(metric!(
1866 name: "mz_persist_audit_blob_batch_part_bytes",
1867 help: "total size of batch parts in blob",
1868 )),
1869 blob_batch_part_count: registry.register(metric!(
1870 name: "mz_persist_audit_blob_batch_part_count",
1871 help: "count of batch parts in blob",
1872 )),
1873 blob_rollup_bytes: registry.register(metric!(
1874 name: "mz_persist_audit_blob_rollup_bytes",
1875 help: "total size of state rollups stored in blob",
1876 )),
1877 blob_rollup_count: registry.register(metric!(
1878 name: "mz_persist_audit_blob_rollup_count",
1879 help: "count of all state rollups in blob",
1880 )),
1881 blob_bytes: registry.register(metric!(
1882 name: "mz_persist_audit_blob_bytes",
1883 help: "total size of blob",
1884 )),
1885 blob_count: registry.register(metric!(
1886 name: "mz_persist_audit_blob_count",
1887 help: "count of all blobs",
1888 )),
1889 step_blob_metadata: step_timings.with_label_values(&["blob_metadata"]),
1890 step_state: step_timings.with_label_values(&["state"]),
1891 step_math: step_timings.with_label_values(&["math"]),
1892 }
1893 }
1894}
1895
1896#[derive(Debug)]
1899pub enum UpdateDelta {
1900 Negative(u64),
1902 NonNegative(u64),
1904}
1905
1906impl UpdateDelta {
1907 pub fn new(new: usize, old: usize) -> Self {
1910 if new < old {
1911 UpdateDelta::Negative(CastFrom::cast_from(old - new))
1912 } else {
1913 UpdateDelta::NonNegative(CastFrom::cast_from(new - old))
1914 }
1915 }
1916}
1917
1918#[derive(Debug, Clone)]
1921pub struct SinkMetrics {
1922 correction_insertions_total: IntCounter,
1924 correction_deletions_total: IntCounter,
1926 correction_capacity_increases_total: IntCounter,
1928 correction_capacity_decreases_total: IntCounter,
1930 correction_max_per_sink_worker_len_updates: raw::UIntGaugeVec,
1932 correction_max_per_sink_worker_capacity_updates: raw::UIntGaugeVec,
1934}
1935
1936impl SinkMetrics {
1937 fn new(registry: &MetricsRegistry) -> Self {
1938 SinkMetrics {
1939 correction_insertions_total: registry.register(metric!(
1940 name: "mz_persist_sink_correction_insertions_total",
1941 help: "The cumulative insertions observed on the correction buffer across workers and persist sinks.",
1942 )),
1943 correction_deletions_total: registry.register(metric!(
1944 name: "mz_persist_sink_correction_deletions_total",
1945 help: "The cumulative deletions observed on the correction buffer across workers and persist sinks.",
1946 )),
1947 correction_capacity_increases_total: registry.register(metric!(
1948 name: "mz_persist_sink_correction_capacity_increases_total",
1949 help: "The cumulative capacity increases observed on the correction buffer across workers and persist sinks.",
1950 )),
1951 correction_capacity_decreases_total: registry.register(metric!(
1952 name: "mz_persist_sink_correction_capacity_decreases_total",
1953 help: "The cumulative capacity decreases observed on the correction buffer across workers and persist sinks.",
1954 )),
1955 correction_max_per_sink_worker_len_updates: registry.register(metric!(
1956 name: "mz_persist_sink_correction_max_per_sink_worker_len_updates",
1957 help: "The maximum length observed for the correction buffer of any single persist sink per worker.",
1958 var_labels: ["worker_id"],
1959 )),
1960 correction_max_per_sink_worker_capacity_updates: registry.register(metric!(
1961 name: "mz_persist_sink_correction_max_per_sink_worker_capacity_updates",
1962 help: "The maximum capacity observed for the correction buffer of any single persist sink per worker.",
1963 var_labels: ["worker_id"],
1964 )),
1965 }
1966 }
1967
1968 pub fn for_worker(&self, worker_id: usize) -> SinkWorkerMetrics {
1973 let worker = worker_id.to_string();
1974 let correction_max_per_sink_worker_len_updates = self
1975 .correction_max_per_sink_worker_len_updates
1976 .with_label_values(&[&worker]);
1977 let correction_max_per_sink_worker_capacity_updates = self
1978 .correction_max_per_sink_worker_capacity_updates
1979 .with_label_values(&[&worker]);
1980 SinkWorkerMetrics {
1981 correction_max_per_sink_worker_len_updates,
1982 correction_max_per_sink_worker_capacity_updates,
1983 }
1984 }
1985
1986 pub fn report_correction_update_deltas(
1992 &self,
1993 correction_len_delta: UpdateDelta,
1994 correction_cap_delta: UpdateDelta,
1995 ) {
1996 match correction_len_delta {
1998 UpdateDelta::NonNegative(delta) => {
1999 if delta > 0 {
2000 self.correction_insertions_total.inc_by(delta)
2001 }
2002 }
2003 UpdateDelta::Negative(delta) => self.correction_deletions_total.inc_by(delta),
2004 }
2005 match correction_cap_delta {
2007 UpdateDelta::NonNegative(delta) => {
2008 if delta > 0 {
2009 self.correction_capacity_increases_total.inc_by(delta)
2010 }
2011 }
2012 UpdateDelta::Negative(delta) => self.correction_capacity_decreases_total.inc_by(delta),
2013 }
2014 }
2015}
2016
2017#[derive(Clone, Debug)]
2019pub struct SinkWorkerMetrics {
2020 correction_max_per_sink_worker_len_updates: UIntGauge,
2021 correction_max_per_sink_worker_capacity_updates: UIntGauge,
2022}
2023
2024impl SinkWorkerMetrics {
2025 pub fn report_correction_update_totals(&self, correction_len: usize, correction_cap: usize) {
2030 let correction_len = CastFrom::cast_from(correction_len);
2032 if correction_len > self.correction_max_per_sink_worker_len_updates.get() {
2033 self.correction_max_per_sink_worker_len_updates
2034 .set(correction_len);
2035 }
2036 let correction_cap = CastFrom::cast_from(correction_cap);
2037 if correction_cap > self.correction_max_per_sink_worker_capacity_updates.get() {
2038 self.correction_max_per_sink_worker_capacity_updates
2039 .set(correction_cap);
2040 }
2041 }
2042}
2043
2044#[derive(Debug)]
2046pub struct AlertsMetrics {
2047 pub(crate) blob_failures: IntCounter,
2048 pub(crate) consensus_failures: IntCounter,
2049}
2050
2051impl AlertsMetrics {
2052 fn new(registry: &MetricsRegistry) -> Self {
2053 AlertsMetrics {
2054 blob_failures: registry.register(metric!(
2055 name: "mz_persist_blob_failures",
2056 help: "count of all blob operation failures",
2057 const_labels: {"honeycomb" => "import"},
2058 )),
2059 consensus_failures: registry.register(metric!(
2060 name: "mz_persist_consensus_failures",
2061 help: "count of determinate consensus operation failures",
2062 const_labels: {"honeycomb" => "import"},
2063 )),
2064 }
2065 }
2066}
2067
2068#[derive(Debug)]
2070pub struct PubSubServerMetrics {
2071 pub(crate) active_connections: UIntGauge,
2072 pub(crate) broadcasted_diff_count: IntCounter,
2073 pub(crate) broadcasted_diff_bytes: IntCounter,
2074 pub(crate) broadcasted_diff_dropped_channel_full: IntCounter,
2075
2076 pub(crate) push_seconds: Counter,
2077 pub(crate) subscribe_seconds: Counter,
2078 pub(crate) unsubscribe_seconds: Counter,
2079 pub(crate) connection_cleanup_seconds: Counter,
2080
2081 pub(crate) push_call_count: IntCounter,
2082 pub(crate) subscribe_call_count: IntCounter,
2083 pub(crate) unsubscribe_call_count: IntCounter,
2084}
2085
2086impl PubSubServerMetrics {
2087 pub(crate) fn new(registry: &MetricsRegistry) -> Self {
2088 let op_timings: CounterVec = registry.register(metric!(
2089 name: "mz_persist_pubsub_server_operation_seconds",
2090 help: "time spent in pubsub server performing each operation",
2091 var_labels: ["op"],
2092 ));
2093 let call_count: IntCounterVec = registry.register(metric!(
2094 name: "mz_persist_pubsub_server_call_count",
2095 help: "count of each pubsub server message received",
2096 var_labels: ["call"],
2097 ));
2098
2099 Self {
2100 active_connections: registry.register(metric!(
2101 name: "mz_persist_pubsub_server_active_connections",
2102 help: "number of active connections to server",
2103 )),
2104 broadcasted_diff_count: registry.register(metric!(
2105 name: "mz_persist_pubsub_server_broadcasted_diff_count",
2106 help: "count of total broadcast diff messages sent",
2107 )),
2108 broadcasted_diff_bytes: registry.register(metric!(
2109 name: "mz_persist_pubsub_server_broadcasted_diff_bytes",
2110 help: "count of total broadcast diff bytes sent",
2111 )),
2112 broadcasted_diff_dropped_channel_full: registry.register(metric!(
2113 name: "mz_persist_pubsub_server_broadcasted_diff_dropped_channel_full",
2114 help: "count of diffs dropped due to full connection channel",
2115 )),
2116
2117 push_seconds: op_timings.with_label_values(&["push"]),
2118 subscribe_seconds: op_timings.with_label_values(&["subscribe"]),
2119 unsubscribe_seconds: op_timings.with_label_values(&["unsubscribe"]),
2120 connection_cleanup_seconds: op_timings.with_label_values(&["cleanup"]),
2121
2122 push_call_count: call_count.with_label_values(&["push"]),
2123 subscribe_call_count: call_count.with_label_values(&["subscribe"]),
2124 unsubscribe_call_count: call_count.with_label_values(&["unsubscribe"]),
2125 }
2126 }
2127}
2128
2129#[derive(Debug)]
2131pub struct PubSubClientMetrics {
2132 pub sender: PubSubClientSenderMetrics,
2133 pub receiver: PubSubClientReceiverMetrics,
2134 pub grpc_connection: PubSubGrpcClientConnectionMetrics,
2135}
2136
2137impl PubSubClientMetrics {
2138 fn new(registry: &MetricsRegistry) -> Self {
2139 PubSubClientMetrics {
2140 sender: PubSubClientSenderMetrics::new(registry),
2141 receiver: PubSubClientReceiverMetrics::new(registry),
2142 grpc_connection: PubSubGrpcClientConnectionMetrics::new(registry),
2143 }
2144 }
2145}
2146
2147#[derive(Debug)]
2148pub struct PubSubGrpcClientConnectionMetrics {
2149 pub(crate) connected: UIntGauge,
2150 pub(crate) connection_established_count: IntCounter,
2151 pub(crate) connect_call_attempt_count: IntCounter,
2152 pub(crate) broadcast_recv_lagged_count: IntCounter,
2153 pub(crate) grpc_error_count: IntCounter,
2154}
2155
2156impl PubSubGrpcClientConnectionMetrics {
2157 fn new(registry: &MetricsRegistry) -> Self {
2158 Self {
2159 connected: registry.register(metric!(
2160 name: "mz_persist_pubsub_client_grpc_connected",
2161 help: "whether the grpc client is currently connected",
2162 )),
2163 connection_established_count: registry.register(metric!(
2164 name: "mz_persist_pubsub_client_grpc_connection_established_count",
2165 help: "count of grpc connection establishments to pubsub server",
2166 )),
2167 connect_call_attempt_count: registry.register(metric!(
2168 name: "mz_persist_pubsub_client_grpc_connect_call_attempt_count",
2169 help: "count of connection call attempts (including retries) to pubsub server",
2170 )),
2171 broadcast_recv_lagged_count: registry.register(metric!(
2172 name: "mz_persist_pubsub_client_grpc_broadcast_recv_lagged_count",
2173 help: "times a message was missed by broadcast receiver due to lag",
2174 )),
2175 grpc_error_count: registry.register(metric!(
2176 name: "mz_persist_pubsub_client_grpc_error_count",
2177 help: "count of grpc errors received",
2178 )),
2179 }
2180 }
2181}
2182
2183#[derive(Clone, Debug)]
2184pub struct PubSubClientReceiverMetrics {
2185 pub(crate) push_received: IntCounter,
2186 pub(crate) unknown_message_received: IntCounter,
2187 pub(crate) approx_diff_latency_seconds: Histogram,
2188
2189 pub(crate) state_pushed_diff_fast_path: IntCounter,
2190 pub(crate) state_pushed_diff_slow_path_succeeded: IntCounter,
2191 pub(crate) state_pushed_diff_slow_path_failed: IntCounter,
2192}
2193
2194impl PubSubClientReceiverMetrics {
2195 fn new(registry: &MetricsRegistry) -> Self {
2196 let call_received: IntCounterVec = registry.register(metric!(
2197 name: "mz_persist_pubsub_client_call_received",
2198 help: "times a pubsub client call was received",
2199 var_labels: ["call"],
2200 ));
2201
2202 Self {
2203 push_received: call_received.with_label_values(&["push"]),
2204 unknown_message_received: call_received.with_label_values(&["unknown"]),
2205 approx_diff_latency_seconds: registry.register(metric!(
2206 name: "mz_persist_pubsub_client_approx_diff_apply_latency_seconds",
2207 help: "histogram of (approximate) latency between sending a diff and applying it",
2208 buckets: prometheus::exponential_buckets(0.001, 2.0, 13).expect("buckets"),
2209 )),
2210
2211 state_pushed_diff_fast_path: registry.register(metric!(
2212 name: "mz_persist_pubsub_client_receiver_state_push_diff_fast_path",
2213 help: "count fast-path state push_diff calls",
2214 )),
2215 state_pushed_diff_slow_path_succeeded: registry.register(metric!(
2216 name: "mz_persist_pubsub_client_receiver_state_push_diff_slow_path_succeeded",
2217 help: "count of successful slow-path state push_diff calls",
2218 )),
2219 state_pushed_diff_slow_path_failed: registry.register(metric!(
2220 name: "mz_persist_pubsub_client_receiver_state_push_diff_slow_path_failed",
2221 help: "count of unsuccessful slow-path state push_diff calls",
2222 )),
2223 }
2224 }
2225}
2226
2227#[derive(Debug)]
2228pub struct PubSubClientSenderMetrics {
2229 pub push: PubSubClientCallMetrics,
2230 pub subscribe: PubSubClientCallMetrics,
2231 pub unsubscribe: PubSubClientCallMetrics,
2232}
2233
2234#[derive(Debug)]
2235pub struct PubSubClientCallMetrics {
2236 pub(crate) succeeded: IntCounter,
2237 pub(crate) bytes_sent: IntCounter,
2238 pub(crate) failed: IntCounter,
2239}
2240
2241impl PubSubClientSenderMetrics {
2242 fn new(registry: &MetricsRegistry) -> Self {
2243 let call_bytes_sent: IntCounterVec = registry.register(metric!(
2244 name: "mz_persist_pubsub_client_call_bytes_sent",
2245 help: "number of bytes sent for a given pubsub client call",
2246 var_labels: ["call"],
2247 ));
2248 let call_succeeded: IntCounterVec = registry.register(metric!(
2249 name: "mz_persist_pubsub_client_call_succeeded",
2250 help: "times a pubsub client call succeeded",
2251 var_labels: ["call"],
2252 ));
2253 let call_failed: IntCounterVec = registry.register(metric!(
2254 name: "mz_persist_pubsub_client_call_failed",
2255 help: "times a pubsub client call failed",
2256 var_labels: ["call"],
2257 ));
2258
2259 Self {
2260 push: PubSubClientCallMetrics {
2261 succeeded: call_succeeded.with_label_values(&["push"]),
2262 failed: call_failed.with_label_values(&["push"]),
2263 bytes_sent: call_bytes_sent.with_label_values(&["push"]),
2264 },
2265 subscribe: PubSubClientCallMetrics {
2266 succeeded: call_succeeded.with_label_values(&["subscribe"]),
2267 failed: call_failed.with_label_values(&["subscribe"]),
2268 bytes_sent: call_bytes_sent.with_label_values(&["subscribe"]),
2269 },
2270 unsubscribe: PubSubClientCallMetrics {
2271 succeeded: call_succeeded.with_label_values(&["unsubscribe"]),
2272 failed: call_failed.with_label_values(&["unsubscribe"]),
2273 bytes_sent: call_bytes_sent.with_label_values(&["unsubscribe"]),
2274 },
2275 }
2276 }
2277}
2278
2279#[derive(Debug)]
2280pub struct LocksMetrics {
2281 pub(crate) applier_read_cacheable: LockMetrics,
2282 pub(crate) applier_read_noncacheable: LockMetrics,
2283 pub(crate) applier_write: LockMetrics,
2284 pub(crate) watch: LockMetrics,
2285}
2286
2287#[derive(Debug, Clone)]
2288pub struct LockMetrics {
2289 pub(crate) acquire_count: IntCounter,
2290 pub(crate) blocking_acquire_count: IntCounter,
2291 pub(crate) blocking_seconds: Counter,
2292}
2293
2294#[derive(Debug)]
2295pub struct WatchMetrics {
2296 pub(crate) listen_woken_via_watch: IntCounter,
2297 pub(crate) listen_woken_via_sleep: IntCounter,
2298 pub(crate) listen_resolved_via_watch: IntCounter,
2299 pub(crate) listen_resolved_via_sleep: IntCounter,
2300 pub(crate) snapshot_woken_via_watch: IntCounter,
2301 pub(crate) snapshot_woken_via_sleep: IntCounter,
2302 pub(crate) notify_sent: IntCounter,
2303 pub(crate) notify_noop: IntCounter,
2304 pub(crate) notify_recv: IntCounter,
2305 pub(crate) notify_lagged: IntCounter,
2306 pub(crate) notify_wait_started: IntCounter,
2307 pub(crate) notify_wait_finished: IntCounter,
2308}
2309
2310impl WatchMetrics {
2311 fn new(registry: &MetricsRegistry) -> Self {
2312 WatchMetrics {
2313 listen_woken_via_watch: registry.register(metric!(
2314 name: "mz_persist_listen_woken_via_watch",
2315 help: "count of listen next batches wakes via watch notify",
2316 )),
2317 listen_woken_via_sleep: registry.register(metric!(
2318 name: "mz_persist_listen_woken_via_sleep",
2319 help: "count of listen next batches wakes via sleep",
2320 )),
2321 listen_resolved_via_watch: registry.register(metric!(
2322 name: "mz_persist_listen_resolved_via_watch",
2323 help: "count of listen next batches resolved via watch notify",
2324 )),
2325 listen_resolved_via_sleep: registry.register(metric!(
2326 name: "mz_persist_listen_resolved_via_sleep",
2327 help: "count of listen next batches resolved via sleep",
2328 )),
2329 snapshot_woken_via_watch: registry.register(metric!(
2330 name: "mz_persist_snapshot_woken_via_watch",
2331 help: "count of snapshot wakes via watch notify",
2332 )),
2333 snapshot_woken_via_sleep: registry.register(metric!(
2334 name: "mz_persist_snapshot_woken_via_sleep",
2335 help: "count of snapshot wakes via sleep",
2336 )),
2337 notify_sent: registry.register(metric!(
2338 name: "mz_persist_watch_notify_sent",
2339 help: "count of watch notifications sent to a non-empty broadcast channel",
2340 )),
2341 notify_noop: registry.register(metric!(
2342 name: "mz_persist_watch_notify_noop",
2343 help: "count of watch notifications sent to an broadcast channel",
2344 )),
2345 notify_recv: registry.register(metric!(
2346 name: "mz_persist_watch_notify_recv",
2347 help: "count of watch notifications received from the broadcast channel",
2348 )),
2349 notify_lagged: registry.register(metric!(
2350 name: "mz_persist_watch_notify_lagged",
2351 help: "count of lagged events in the watch notification broadcast channel",
2352 )),
2353 notify_wait_started: registry.register(metric!(
2354 name: "mz_persist_watch_notify_wait_started",
2355 help: "count of watch wait calls started",
2356 )),
2357 notify_wait_finished: registry.register(metric!(
2358 name: "mz_persist_watch_notify_wait_finished",
2359 help: "count of watch wait calls resolved",
2360 )),
2361 }
2362 }
2363}
2364
2365#[derive(Debug)]
2366pub struct PushdownMetrics {
2367 pub(crate) parts_filtered_count: IntCounter,
2368 pub(crate) parts_filtered_bytes: IntCounter,
2369 pub(crate) parts_fetched_count: IntCounter,
2370 pub(crate) parts_fetched_bytes: IntCounter,
2371 pub(crate) parts_audited_count: IntCounter,
2372 pub(crate) parts_audited_bytes: IntCounter,
2373 pub(crate) parts_inline_count: IntCounter,
2374 pub(crate) parts_inline_bytes: IntCounter,
2375 pub(crate) parts_faked_count: IntCounter,
2376 pub(crate) parts_faked_bytes: IntCounter,
2377 pub(crate) parts_stats_trimmed_count: IntCounter,
2378 pub(crate) parts_stats_trimmed_bytes: IntCounter,
2379 pub(crate) parts_projection_trimmed_bytes: IntCounter,
2380 pub part_stats: PartStatsMetrics,
2381}
2382
2383impl PushdownMetrics {
2384 fn new(registry: &MetricsRegistry) -> Self {
2385 PushdownMetrics {
2386 parts_filtered_count: registry.register(metric!(
2387 name: "mz_persist_pushdown_parts_filtered_count",
2388 help: "count of parts filtered by pushdown",
2389 )),
2390 parts_filtered_bytes: registry.register(metric!(
2391 name: "mz_persist_pushdown_parts_filtered_bytes",
2392 help: "total size of parts filtered by pushdown in bytes",
2393 )),
2394 parts_fetched_count: registry.register(metric!(
2395 name: "mz_persist_pushdown_parts_fetched_count",
2396 help: "count of parts not filtered by pushdown",
2397 )),
2398 parts_fetched_bytes: registry.register(metric!(
2399 name: "mz_persist_pushdown_parts_fetched_bytes",
2400 help: "total size of parts not filtered by pushdown in bytes",
2401 )),
2402 parts_audited_count: registry.register(metric!(
2403 name: "mz_persist_pushdown_parts_audited_count",
2404 help: "count of parts fetched only for pushdown audit",
2405 )),
2406 parts_audited_bytes: registry.register(metric!(
2407 name: "mz_persist_pushdown_parts_audited_bytes",
2408 help: "total size of parts fetched only for pushdown audit",
2409 )),
2410 parts_inline_count: registry.register(metric!(
2411 name: "mz_persist_pushdown_parts_inline_count",
2412 help: "count of parts not fetched because they were inline",
2413 )),
2414 parts_inline_bytes: registry.register(metric!(
2415 name: "mz_persist_pushdown_parts_inline_bytes",
2416 help: "total size of parts not fetched because they were inline",
2417 )),
2418 parts_faked_count: registry.register(metric!(
2419 name: "mz_persist_pushdown_parts_faked_count",
2420 help: "count of parts faked because of aggressive projection pushdown",
2421 )),
2422 parts_faked_bytes: registry.register(metric!(
2423 name: "mz_persist_pushdown_parts_faked_bytes",
2424 help: "total size of parts replaced with fakes by aggressive projection pushdown",
2425 )),
2426 parts_stats_trimmed_count: registry.register(metric!(
2427 name: "mz_persist_pushdown_parts_stats_trimmed_count",
2428 help: "count of trimmed part stats",
2429 )),
2430 parts_stats_trimmed_bytes: registry.register(metric!(
2431 name: "mz_persist_pushdown_parts_stats_trimmed_bytes",
2432 help: "total bytes trimmed from part stats",
2433 )),
2434 parts_projection_trimmed_bytes: registry.register(metric!(
2435 name: "mz_persist_pushdown_parts_projection_trimmed_bytes",
2436 help: "total bytes trimmed from columnar data because of projection pushdown",
2437 )),
2438 part_stats: PartStatsMetrics::new(registry),
2439 }
2440 }
2441}
2442
2443#[derive(Debug)]
2444pub struct ConsolidationMetrics {
2445 pub(crate) parts_fetched: IntCounter,
2446 pub(crate) parts_skipped: IntCounter,
2447 pub(crate) parts_wasted: IntCounter,
2448 pub(crate) wrong_sort: IntCounter,
2449}
2450
2451impl ConsolidationMetrics {
2452 fn new(registry: &MetricsRegistry) -> Self {
2453 ConsolidationMetrics {
2454 parts_fetched: registry.register(metric!(
2455 name: "mz_persist_consolidation_parts_fetched_count",
2456 help: "count of parts that were fetched and used during consolidation",
2457 )),
2458 parts_skipped: registry.register(metric!(
2459 name: "mz_persist_consolidation_parts_skipped_count",
2460 help: "count of parts that were never needed during consolidation",
2461 )),
2462 parts_wasted: registry.register(metric!(
2463 name: "mz_persist_consolidation_parts_wasted_count",
2464 help: "count of parts that were fetched but not needed during consolidation",
2465 )),
2466 wrong_sort: registry.register(metric!(
2467 name: "mz_persist_consolidation_wrong_sort_count",
2468 help: "count of runs that were sorted using the wrong ordering for the current consolidation",
2469 )),
2470 }
2471 }
2472}
2473
2474#[derive(Debug)]
2475pub struct BlobMemCache {
2476 pub(crate) size_blobs: UIntGauge,
2477 pub(crate) size_bytes: UIntGauge,
2478 pub(crate) hits_blobs: IntCounter,
2479 pub(crate) hits_bytes: IntCounter,
2480 pub(crate) evictions: IntCounter,
2481}
2482
2483impl BlobMemCache {
2484 fn new(registry: &MetricsRegistry) -> Self {
2485 BlobMemCache {
2486 size_blobs: registry.register(metric!(
2487 name: "mz_persist_blob_cache_size_blobs",
2488 help: "count of blobs in the cache",
2489 const_labels: {"cache" => "mem"},
2490 )),
2491 size_bytes: registry.register(metric!(
2492 name: "mz_persist_blob_cache_size_bytes",
2493 help: "total size of blobs in the cache",
2494 const_labels: {"cache" => "mem"},
2495 )),
2496 hits_blobs: registry.register(metric!(
2497 name: "mz_persist_blob_cache_hits_blobs",
2498 help: "count of blobs served via cache instead of s3",
2499 const_labels: {"cache" => "mem"},
2500 )),
2501 hits_bytes: registry.register(metric!(
2502 name: "mz_persist_blob_cache_hits_bytes",
2503 help: "total size of blobs served via cache instead of s3",
2504 const_labels: {"cache" => "mem"},
2505 )),
2506 evictions: registry.register(metric!(
2507 name: "mz_persist_blob_cache_evictions",
2508 help: "count of capacity-based cache evictions",
2509 const_labels: {"cache" => "mem"},
2510 )),
2511 }
2512 }
2513}
2514
2515#[derive(Debug)]
2516pub struct SemaphoreMetrics {
2517 cfg: PersistConfig,
2518 registry: MetricsRegistry,
2519 fetch: OnceCell<MetricsSemaphore>,
2520}
2521
2522impl SemaphoreMetrics {
2523 fn new(cfg: PersistConfig, registry: MetricsRegistry) -> Self {
2524 SemaphoreMetrics {
2525 cfg,
2526 registry,
2527 fetch: OnceCell::new(),
2528 }
2529 }
2530
2531 async fn fetch(&self) -> &MetricsSemaphore {
2535 if let Some(x) = self.fetch.get() {
2536 return x;
2538 }
2539 let cfg = self.cfg.clone();
2540 let registry = self.registry.clone();
2541 let init = async move {
2542 let total_permits = match cfg.announce_memory_limit {
2543 Some(mem) if cfg.is_cc_active => {
2546 info!("fetch semaphore awaiting first dyncfg values");
2549 let () = cfg.configs_synced_once().await;
2550 let total_permits = usize::cast_lossy(
2551 f64::cast_lossy(mem) * FETCH_SEMAPHORE_PERMIT_ADJUSTMENT.get(&cfg),
2552 );
2553 info!("fetch_semaphore got first dyncfg values");
2554 total_permits
2555 }
2556 Some(_) | None => Semaphore::MAX_PERMITS,
2557 };
2558 MetricsSemaphore::new(®istry, "fetch", total_permits)
2559 };
2560 self.fetch.get_or_init(|| init).await
2561 }
2562
2563 pub(crate) async fn acquire_fetch_permits(&self, encoded_size_bytes: usize) -> MetricsPermits {
2564 let requested_permits = f64::cast_lossy(encoded_size_bytes);
2567 let requested_permits = requested_permits * FETCH_SEMAPHORE_COST_ADJUSTMENT.get(&self.cfg);
2568 let requested_permits = usize::cast_lossy(requested_permits);
2569 self.fetch().await.acquire_permits(requested_permits).await
2570 }
2571}
2572
2573#[derive(Debug)]
2574pub struct MetricsSemaphore {
2575 name: &'static str,
2576 semaphore: Arc<Semaphore>,
2577 total_permits: usize,
2578 acquire_count: IntCounter,
2579 blocking_count: IntCounter,
2580 blocking_seconds: Counter,
2581 acquired_permits: IntCounter,
2582 released_permits: IntCounter,
2583 _available_permits: ComputedUIntGauge,
2584}
2585
2586impl MetricsSemaphore {
2587 pub fn new(registry: &MetricsRegistry, name: &'static str, total_permits: usize) -> Self {
2588 let total_permits = std::cmp::min(total_permits, Semaphore::MAX_PERMITS);
2589 let semaphore = Arc::new(Semaphore::new(total_permits));
2592 MetricsSemaphore {
2593 name,
2594 total_permits,
2595 acquire_count: registry.register(metric!(
2596 name: "mz_persist_semaphore_acquire_count",
2597 help: "count of acquire calls (not acquired permits count)",
2598 const_labels: {"name" => name},
2599 )),
2600 blocking_count: registry.register(metric!(
2601 name: "mz_persist_semaphore_blocking_count",
2602 help: "count of acquire calls that had to block",
2603 const_labels: {"name" => name},
2604 )),
2605 blocking_seconds: registry.register(metric!(
2606 name: "mz_persist_semaphore_blocking_seconds",
2607 help: "total time spent blocking on permit acquisition",
2608 const_labels: {"name" => name},
2609 )),
2610 acquired_permits: registry.register(metric!(
2611 name: "mz_persist_semaphore_acquired_permits",
2612 help: "total sum of acquired permits",
2613 const_labels: {"name" => name},
2614 )),
2615 released_permits: registry.register(metric!(
2616 name: "mz_persist_semaphore_released_permits",
2617 help: "total sum of released permits",
2618 const_labels: {"name" => name},
2619 )),
2620 _available_permits: registry.register_computed_gauge(
2621 metric!(
2622 name: "mz_persist_semaphore_available_permits",
2623 help: "currently available permits according to the semaphore",
2624 ),
2625 {
2626 let semaphore = Arc::clone(&semaphore);
2627 move || u64::cast_from(semaphore.available_permits())
2628 },
2629 ),
2630 semaphore,
2631 }
2632 }
2633
2634 pub async fn acquire_permits(&self, requested_permits: usize) -> MetricsPermits {
2635 let total_permits = u32::try_from(self.total_permits).unwrap_or(u32::MAX);
2638 let requested_permits = u32::try_from(requested_permits).unwrap_or(u32::MAX);
2639 let requested_permits = std::cmp::min(requested_permits, total_permits);
2640 let wrap = |_permit| {
2641 self.acquired_permits.inc_by(u64::from(requested_permits));
2642 MetricsPermits {
2643 _permit,
2644 released_metric: self.released_permits.clone(),
2645 count: requested_permits,
2646 }
2647 };
2648
2649 self.acquire_count.inc();
2651 match Arc::clone(&self.semaphore).try_acquire_many_owned(requested_permits) {
2652 Ok(x) => return wrap(x),
2653 Err(_) => {}
2654 };
2655
2656 self.blocking_count.inc();
2658 let start = Instant::now();
2659 let ret = Arc::clone(&self.semaphore)
2660 .acquire_many_owned(requested_permits)
2661 .instrument(info_span!("acquire_permits"))
2662 .await;
2663 let elapsed = start.elapsed();
2664 self.blocking_seconds.inc_by(elapsed.as_secs_f64());
2665 debug!(
2666 "acquisition of {} {} permits blocked for {:?}",
2667 self.name, requested_permits, elapsed
2668 );
2669 wrap(ret.expect("semaphore is never closed"))
2670 }
2671}
2672
2673#[derive(Debug)]
2674pub struct MetricsPermits {
2675 _permit: OwnedSemaphorePermit,
2676 released_metric: IntCounter,
2677 count: u32,
2678}
2679
2680impl Drop for MetricsPermits {
2681 fn drop(&mut self) {
2682 self.released_metric.inc_by(u64::from(self.count))
2683 }
2684}
2685
2686#[derive(Debug)]
2687pub struct ExternalOpMetrics {
2688 started: IntCounter,
2689 succeeded: IntCounter,
2690 failed: IntCounter,
2691 bytes: IntCounter,
2692 seconds: Counter,
2693 seconds_histogram: Option<Histogram>,
2694 alerts_metrics: Arc<AlertsMetrics>,
2695}
2696
2697impl ExternalOpMetrics {
2698 async fn run_op<R, F, OpFn, ErrFn>(
2699 &self,
2700 op_fn: OpFn,
2701 on_err_fn: ErrFn,
2702 ) -> Result<R, ExternalError>
2703 where
2704 F: std::future::Future<Output = Result<R, ExternalError>>,
2705 OpFn: FnOnce() -> F,
2706 ErrFn: FnOnce(&AlertsMetrics, &ExternalError),
2707 {
2708 self.started.inc();
2709 let start = Instant::now();
2710 let res = op_fn().await;
2711 let elapsed_seconds = start.elapsed().as_secs_f64();
2712 self.seconds.inc_by(elapsed_seconds);
2713 if let Some(h) = &self.seconds_histogram {
2714 h.observe(elapsed_seconds);
2715 }
2716 match res.as_ref() {
2717 Ok(_) => self.succeeded.inc(),
2718 Err(err) => {
2719 self.failed.inc();
2720 on_err_fn(&self.alerts_metrics, err);
2721 }
2722 };
2723 res
2724 }
2725
2726 fn run_stream<'a, R: 'a, S, OpFn, ErrFn>(
2727 &'a self,
2728 op_fn: OpFn,
2729 mut on_err_fn: ErrFn,
2730 ) -> impl futures::Stream<Item = Result<R, ExternalError>> + 'a
2731 where
2732 S: futures::Stream<Item = Result<R, ExternalError>> + Unpin + 'a,
2733 OpFn: FnOnce() -> S,
2734 ErrFn: FnMut(&AlertsMetrics, &ExternalError) + 'a,
2735 {
2736 self.started.inc();
2737 let start = Instant::now();
2738 let mut stream = op_fn();
2739 stream! {
2740 let mut succeeded = true;
2741 while let Some(res) = stream.next().await {
2742 if let Err(err) = res.as_ref() {
2743 on_err_fn(&self.alerts_metrics, err);
2744 succeeded = false;
2745 }
2746 yield res;
2747 }
2748 if succeeded {
2749 self.succeeded.inc()
2750 } else {
2751 self.failed.inc()
2752 }
2753 let elapsed_seconds = start.elapsed().as_secs_f64();
2754 self.seconds.inc_by(elapsed_seconds);
2755 if let Some(h) = &self.seconds_histogram {
2756 h.observe(elapsed_seconds);
2757 }
2758 }
2759 }
2760}
2761
2762#[derive(Debug)]
2763pub struct BlobMetrics {
2764 set: ExternalOpMetrics,
2765 get: ExternalOpMetrics,
2766 list_keys: ExternalOpMetrics,
2767 delete: ExternalOpMetrics,
2768 restore: ExternalOpMetrics,
2769 delete_noop: IntCounter,
2770 blob_sizes: Histogram,
2771 pub rtt_latency: Gauge,
2772}
2773
2774#[derive(Debug)]
2775pub struct MetricsBlob {
2776 blob: Arc<dyn Blob>,
2777 metrics: Arc<Metrics>,
2778}
2779
2780impl MetricsBlob {
2781 pub fn new(blob: Arc<dyn Blob>, metrics: Arc<Metrics>) -> Self {
2782 MetricsBlob { blob, metrics }
2783 }
2784
2785 fn on_err(alerts_metrics: &AlertsMetrics, _err: &ExternalError) {
2786 alerts_metrics.blob_failures.inc()
2787 }
2788}
2789
2790#[async_trait]
2791impl Blob for MetricsBlob {
2792 #[instrument(name = "blob::get", fields(shard=blob_key_shard_id(key)))]
2793 async fn get(&self, key: &str) -> Result<Option<SegmentedBytes>, ExternalError> {
2794 let res = self
2795 .metrics
2796 .blob
2797 .get
2798 .run_op(|| self.blob.get(key), Self::on_err)
2799 .await;
2800 if let Ok(Some(value)) = res.as_ref() {
2801 self.metrics
2802 .blob
2803 .get
2804 .bytes
2805 .inc_by(u64::cast_from(value.len()));
2806 }
2807 res
2808 }
2809
2810 #[instrument(name = "blob::list_keys_and_metadata", fields(shard=blob_key_shard_id(key_prefix)))]
2811 async fn list_keys_and_metadata(
2812 &self,
2813 key_prefix: &str,
2814 f: &mut (dyn FnMut(BlobMetadata) + Send + Sync),
2815 ) -> Result<(), ExternalError> {
2816 let mut byte_total = 0;
2817 let mut instrumented = |blob_metadata: BlobMetadata| {
2818 byte_total += blob_metadata.key.len();
2821 f(blob_metadata)
2822 };
2823
2824 let res = self
2825 .metrics
2826 .blob
2827 .list_keys
2828 .run_op(
2829 || {
2830 self.blob
2831 .list_keys_and_metadata(key_prefix, &mut instrumented)
2832 },
2833 Self::on_err,
2834 )
2835 .await;
2836
2837 self.metrics
2838 .blob
2839 .list_keys
2840 .bytes
2841 .inc_by(u64::cast_from(byte_total));
2842
2843 res
2844 }
2845
2846 #[instrument(name = "blob::set", fields(shard=blob_key_shard_id(key),size_bytes=value.len()))]
2847 async fn set(&self, key: &str, value: Bytes) -> Result<(), ExternalError> {
2848 let bytes = value.len();
2849 let res = self
2850 .metrics
2851 .blob
2852 .set
2853 .run_op(|| self.blob.set(key, value), Self::on_err)
2854 .await;
2855 if res.is_ok() {
2856 self.metrics.blob.set.bytes.inc_by(u64::cast_from(bytes));
2857 self.metrics.blob.blob_sizes.observe(f64::cast_lossy(bytes));
2858 }
2859 res
2860 }
2861
2862 #[instrument(name = "blob::delete", fields(shard=blob_key_shard_id(key)))]
2863 async fn delete(&self, key: &str) -> Result<Option<usize>, ExternalError> {
2864 let bytes = self
2865 .metrics
2866 .blob
2867 .delete
2868 .run_op(|| self.blob.delete(key), Self::on_err)
2869 .await?;
2870 if let Some(bytes) = bytes {
2871 self.metrics.blob.delete.bytes.inc_by(u64::cast_from(bytes));
2872 } else {
2873 self.metrics.blob.delete_noop.inc();
2874 }
2875 Ok(bytes)
2876 }
2877
2878 async fn restore(&self, key: &str) -> Result<(), ExternalError> {
2879 self.metrics
2880 .blob
2881 .restore
2882 .run_op(|| self.blob.restore(key), Self::on_err)
2883 .await
2884 }
2885}
2886
2887#[derive(Debug)]
2888pub struct ConsensusMetrics {
2889 list_keys: ExternalOpMetrics,
2890 head: ExternalOpMetrics,
2891 compare_and_set: ExternalOpMetrics,
2892 scan: ExternalOpMetrics,
2893 truncate: ExternalOpMetrics,
2894 truncated_count: IntCounter,
2895 pub rtt_latency: Gauge,
2896}
2897
2898#[derive(Debug)]
2899pub struct MetricsConsensus {
2900 consensus: Arc<dyn Consensus>,
2901 metrics: Arc<Metrics>,
2902}
2903
2904impl MetricsConsensus {
2905 pub fn new(consensus: Arc<dyn Consensus>, metrics: Arc<Metrics>) -> Self {
2906 MetricsConsensus { consensus, metrics }
2907 }
2908
2909 fn on_err(alerts_metrics: &AlertsMetrics, err: &ExternalError) {
2910 if let ExternalError::Indeterminate(_) = err {
2914 alerts_metrics.consensus_failures.inc()
2915 }
2916 }
2917}
2918
2919#[async_trait]
2920impl Consensus for MetricsConsensus {
2921 fn list_keys(&self) -> ResultStream<String> {
2922 Box::pin(
2923 self.metrics
2924 .consensus
2925 .list_keys
2926 .run_stream(|| self.consensus.list_keys(), Self::on_err),
2927 )
2928 }
2929
2930 #[instrument(name = "consensus::head", fields(shard=key))]
2931 async fn head(&self, key: &str) -> Result<Option<VersionedData>, ExternalError> {
2932 let res = self
2933 .metrics
2934 .consensus
2935 .head
2936 .run_op(|| self.consensus.head(key), Self::on_err)
2937 .await;
2938 if let Ok(Some(data)) = res.as_ref() {
2939 self.metrics
2940 .consensus
2941 .head
2942 .bytes
2943 .inc_by(u64::cast_from(data.data.len()));
2944 }
2945 res
2946 }
2947
2948 #[instrument(name = "consensus::compare_and_set", fields(shard=key,size_bytes=new.data.len()))]
2949 async fn compare_and_set(
2950 &self,
2951 key: &str,
2952 expected: Option<SeqNo>,
2953 new: VersionedData,
2954 ) -> Result<CaSResult, ExternalError> {
2955 let bytes = new.data.len();
2956 let res = self
2957 .metrics
2958 .consensus
2959 .compare_and_set
2960 .run_op(
2961 || self.consensus.compare_and_set(key, expected, new),
2962 Self::on_err,
2963 )
2964 .await;
2965 match res.as_ref() {
2966 Ok(CaSResult::Committed) => self
2967 .metrics
2968 .consensus
2969 .compare_and_set
2970 .bytes
2971 .inc_by(u64::cast_from(bytes)),
2972 Ok(CaSResult::ExpectationMismatch) | Err(_) => {}
2973 }
2974 res
2975 }
2976
2977 #[instrument(name = "consensus::scan", fields(shard=key))]
2978 async fn scan(
2979 &self,
2980 key: &str,
2981 from: SeqNo,
2982 limit: usize,
2983 ) -> Result<Vec<VersionedData>, ExternalError> {
2984 let res = self
2985 .metrics
2986 .consensus
2987 .scan
2988 .run_op(|| self.consensus.scan(key, from, limit), Self::on_err)
2989 .await;
2990 if let Ok(dataz) = res.as_ref() {
2991 let bytes: usize = dataz.iter().map(|x| x.data.len()).sum();
2992 self.metrics
2993 .consensus
2994 .scan
2995 .bytes
2996 .inc_by(u64::cast_from(bytes));
2997 }
2998 res
2999 }
3000
3001 #[instrument(name = "consensus::truncate", fields(shard=key))]
3002 async fn truncate(&self, key: &str, seqno: SeqNo) -> Result<usize, ExternalError> {
3003 let deleted = self
3004 .metrics
3005 .consensus
3006 .truncate
3007 .run_op(|| self.consensus.truncate(key, seqno), Self::on_err)
3008 .await?;
3009 self.metrics
3010 .consensus
3011 .truncated_count
3012 .inc_by(u64::cast_from(deleted));
3013 Ok(deleted)
3014 }
3015}
3016
3017#[derive(Debug, Clone)]
3020pub struct TaskMetrics {
3021 f64_gauges: Vec<(Gauge, fn(&tokio_metrics::TaskMetrics) -> f64)>,
3022 u64_gauges: Vec<(
3023 GenericGauge<AtomicU64>,
3024 fn(&tokio_metrics::TaskMetrics) -> u64,
3025 )>,
3026 monitor: TaskMonitor,
3027}
3028
3029impl TaskMetrics {
3030 pub fn new(name: &str) -> Self {
3031 let monitor = TaskMonitor::new();
3032 Self {
3033 f64_gauges: vec![
3034 (
3035 Gauge::make_collector(metric!(
3036 name: "mz_persist_task_total_idle_duration",
3037 help: "Seconds of time spent idling, ie. waiting for a task to be woken up.",
3038 const_labels: {"name" => name}
3039 )),
3040 |m| m.total_idle_duration.as_secs_f64(),
3041 ),
3042 (
3043 Gauge::make_collector(metric!(
3044 name: "mz_persist_task_total_scheduled_duration",
3045 help: "Seconds of time spent scheduled, ie. ready to poll but not yet polled.",
3046 const_labels: {"name" => name}
3047 )),
3048 |m| m.total_scheduled_duration.as_secs_f64(),
3049 ),
3050 ],
3051 u64_gauges: vec![
3052 (
3053 MakeCollector::make_collector(metric!(
3054 name: "mz_persist_task_total_scheduled_count",
3055 help: "The total number of task schedules. Useful for computing the average scheduled time.",
3056 const_labels: {"name" => name}
3057 )),
3058 |m| m.total_scheduled_count,
3059 ),
3060 (
3061 MakeCollector::make_collector(metric!(
3062 name: "mz_persist_task_total_idled_count",
3063 help: "The total number of task idles. Useful for computing the average idle time.",
3064 const_labels: {"name" => name}
3065 ,
3066 )),
3067 |m| m.total_idled_count,
3068 ),
3069 ],
3070 monitor,
3071 }
3072 }
3073
3074 pub fn instrument_task<F>(&self, task: F) -> tokio_metrics::Instrumented<F> {
3077 TaskMonitor::instrument(&self.monitor, task)
3078 }
3079}
3080
3081impl Collector for TaskMetrics {
3082 fn desc(&self) -> Vec<&Desc> {
3083 let mut descs = Vec::with_capacity(self.f64_gauges.len() + self.u64_gauges.len());
3084 for (g, _) in &self.f64_gauges {
3085 descs.extend(g.desc());
3086 }
3087 for (g, _) in &self.u64_gauges {
3088 descs.extend(g.desc());
3089 }
3090 descs
3091 }
3092
3093 fn collect(&self) -> Vec<MetricFamily> {
3094 let mut families = Vec::with_capacity(self.f64_gauges.len() + self.u64_gauges.len());
3095 let metrics = self.monitor.cumulative();
3096 for (g, metrics_fn) in &self.f64_gauges {
3097 g.set(metrics_fn(&metrics));
3098 families.extend(g.collect());
3099 }
3100 for (g, metrics_fn) in &self.u64_gauges {
3101 g.set(metrics_fn(&metrics));
3102 families.extend(g.collect());
3103 }
3104 families
3105 }
3106}
3107
3108#[derive(Debug)]
3109pub struct TasksMetrics {
3110 pub heartbeat_read: TaskMetrics,
3111}
3112
3113impl TasksMetrics {
3114 fn new(registry: &MetricsRegistry) -> Self {
3115 let heartbeat_read = TaskMetrics::new("heartbeat_read");
3116 registry.register_collector(heartbeat_read.clone());
3117 TasksMetrics { heartbeat_read }
3118 }
3119}
3120
3121#[derive(Debug)]
3122pub struct SchemaMetrics {
3123 pub(crate) cache_fetch_state_count: IntCounter,
3124 pub(crate) cache_schema: SchemaCacheMetrics,
3125 pub(crate) cache_migration: SchemaCacheMetrics,
3126 pub(crate) migration_count_same: IntCounter,
3127 pub(crate) migration_count_codec: IntCounter,
3128 pub(crate) migration_count_either: IntCounter,
3129 pub(crate) migration_len_legacy_codec: IntCounter,
3130 pub(crate) migration_len_either_codec: IntCounter,
3131 pub(crate) migration_len_either_arrow: IntCounter,
3132 pub(crate) migration_new_count: IntCounter,
3133 pub(crate) migration_new_seconds: Counter,
3134 pub(crate) migration_migrate_seconds: Counter,
3135}
3136
3137impl SchemaMetrics {
3138 fn new(registry: &MetricsRegistry) -> Self {
3139 let cached: IntCounterVec = registry.register(metric!(
3140 name: "mz_persist_schema_cache_cached_count",
3141 help: "count of schema cache entries served from cache",
3142 var_labels: ["op"],
3143 ));
3144 let computed: IntCounterVec = registry.register(metric!(
3145 name: "mz_persist_schema_cache_computed_count",
3146 help: "count of schema cache entries computed",
3147 var_labels: ["op"],
3148 ));
3149 let unavailable: IntCounterVec = registry.register(metric!(
3150 name: "mz_persist_schema_cache_unavailable_count",
3151 help: "count of schema cache entries unavailable at current state",
3152 var_labels: ["op"],
3153 ));
3154 let added: IntCounterVec = registry.register(metric!(
3155 name: "mz_persist_schema_cache_added_count",
3156 help: "count of schema cache entries added",
3157 var_labels: ["op"],
3158 ));
3159 let dropped: IntCounterVec = registry.register(metric!(
3160 name: "mz_persist_schema_cache_dropped_count",
3161 help: "count of schema cache entries dropped",
3162 var_labels: ["op"],
3163 ));
3164 let cache = |name| SchemaCacheMetrics {
3165 cached_count: cached.with_label_values(&[name]),
3166 computed_count: computed.with_label_values(&[name]),
3167 unavailable_count: unavailable.with_label_values(&[name]),
3168 added_count: added.with_label_values(&[name]),
3169 dropped_count: dropped.with_label_values(&[name]),
3170 };
3171 let migration_count: IntCounterVec = registry.register(metric!(
3172 name: "mz_persist_schema_migration_count",
3173 help: "count of fetch part migrations",
3174 var_labels: ["op"],
3175 ));
3176 let migration_len: IntCounterVec = registry.register(metric!(
3177 name: "mz_persist_schema_migration_len",
3178 help: "count of migrated update records",
3179 var_labels: ["op"],
3180 ));
3181 SchemaMetrics {
3182 cache_fetch_state_count: registry.register(metric!(
3183 name: "mz_persist_schema_cache_fetch_state_count",
3184 help: "count of state fetches by the schema cache",
3185 )),
3186 cache_schema: cache("schema"),
3187 cache_migration: cache("migration"),
3188 migration_count_same: migration_count.with_label_values(&["same"]),
3189 migration_count_codec: migration_count.with_label_values(&["codec"]),
3190 migration_count_either: migration_count.with_label_values(&["either"]),
3191 migration_len_legacy_codec: migration_len.with_label_values(&["legacy_codec"]),
3192 migration_len_either_codec: migration_len.with_label_values(&["either_codec"]),
3193 migration_len_either_arrow: migration_len.with_label_values(&["either_arrow"]),
3194 migration_new_count: registry.register(metric!(
3195 name: "mz_persist_schema_migration_new_count",
3196 help: "count of migrations constructed",
3197 )),
3198 migration_new_seconds: registry.register(metric!(
3199 name: "mz_persist_schema_migration_new_seconds",
3200 help: "seconds spent constructing migration logic",
3201 )),
3202 migration_migrate_seconds: registry.register(metric!(
3203 name: "mz_persist_schema_migration_migrate_seconds",
3204 help: "seconds spent applying migration logic",
3205 )),
3206 }
3207 }
3208}
3209
3210#[derive(Debug, Clone)]
3211pub struct SchemaCacheMetrics {
3212 pub(crate) cached_count: IntCounter,
3213 pub(crate) computed_count: IntCounter,
3214 pub(crate) unavailable_count: IntCounter,
3215 pub(crate) added_count: IntCounter,
3216 pub(crate) dropped_count: IntCounter,
3217}
3218
3219#[derive(Debug)]
3220pub struct InlineMetrics {
3221 pub(crate) part_commit_count: IntCounter,
3222 pub(crate) part_commit_bytes: IntCounter,
3223 pub(crate) backpressure: BatchWriteMetrics,
3224}
3225
3226impl InlineMetrics {
3227 fn new(registry: &MetricsRegistry) -> Self {
3228 InlineMetrics {
3229 part_commit_count: registry.register(metric!(
3230 name: "mz_persist_inline_part_commit_count",
3231 help: "count of inline parts committed to state",
3232 )),
3233 part_commit_bytes: registry.register(metric!(
3234 name: "mz_persist_inline_part_commit_bytes",
3235 help: "total size of of inline parts committed to state",
3236 )),
3237 backpressure: BatchWriteMetrics::new(registry, "inline_backpressure"),
3238 }
3239 }
3240}
3241
3242fn blob_key_shard_id(key: &str) -> Option<String> {
3243 let (shard_id, _) = BlobKey::parse_ids(key).ok()?;
3244 Some(shard_id.to_string())
3245}
3246
3247pub fn encode_ts_metric<T: Codec64>(ts: &Antichain<T>) -> i64 {
3249 match ts.elements().first() {
3259 Some(ts) => i64::from_le_bytes(Codec64::encode(ts)),
3260 None => i64::MAX,
3261 }
3262}