-
Notifications
You must be signed in to change notification settings - Fork 533
/
streaming_stats.rs
1284 lines (1139 loc) · 47.7 KB
/
streaming_stats.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::OnceLock;
use prometheus::core::{AtomicF64, AtomicI64, AtomicU64, GenericCounterVec, GenericGaugeVec};
use prometheus::{
exponential_buckets, histogram_opts, register_gauge_vec_with_registry,
register_histogram_with_registry, register_int_counter_vec_with_registry,
register_int_counter_with_registry, register_int_gauge_vec_with_registry,
register_int_gauge_with_registry, Histogram, IntCounter, IntCounterVec, IntGauge, Registry,
};
use risingwave_common::config::MetricLevel;
use risingwave_common::metrics::{
LabelGuardedHistogramVec, LabelGuardedIntCounterVec, LabelGuardedIntGaugeVec,
RelabeledGuardedHistogramVec,
};
use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY;
use risingwave_common::{
register_guarded_histogram_vec_with_registry, register_guarded_int_counter_vec_with_registry,
register_guarded_int_gauge_vec_with_registry,
};
use risingwave_connector::sink::SinkMetrics;
use crate::common::log_store_impl::kv_log_store::{
REWIND_BACKOFF_FACTOR, REWIND_BASE_DELAY, REWIND_MAX_DELAY,
};
#[derive(Clone)]
pub struct StreamingMetrics {
pub level: MetricLevel,
// Executor metrics (disabled by default)
pub executor_row_count: GenericCounterVec<AtomicU64>,
// Streaming actor metrics from tokio (disabled by default)
pub actor_execution_time: GenericGaugeVec<AtomicF64>,
pub actor_scheduled_duration: GenericGaugeVec<AtomicF64>,
pub actor_scheduled_cnt: GenericGaugeVec<AtomicI64>,
pub actor_fast_poll_duration: GenericGaugeVec<AtomicF64>,
pub actor_fast_poll_cnt: GenericGaugeVec<AtomicI64>,
pub actor_slow_poll_duration: GenericGaugeVec<AtomicF64>,
pub actor_slow_poll_cnt: GenericGaugeVec<AtomicI64>,
pub actor_poll_duration: GenericGaugeVec<AtomicF64>,
pub actor_poll_cnt: GenericGaugeVec<AtomicI64>,
pub actor_idle_duration: GenericGaugeVec<AtomicF64>,
pub actor_idle_cnt: GenericGaugeVec<AtomicI64>,
// Streaming actor
pub actor_memory_usage: GenericGaugeVec<AtomicI64>,
pub actor_in_record_cnt: LabelGuardedIntCounterVec<3>,
pub actor_out_record_cnt: LabelGuardedIntCounterVec<2>,
// Source
pub source_output_row_count: GenericCounterVec<AtomicU64>,
pub source_split_change_count: GenericCounterVec<AtomicU64>,
pub source_backfill_row_count: LabelGuardedIntCounterVec<4>,
// Sink & materialized view
pub sink_input_row_count: LabelGuardedIntCounterVec<3>,
pub mview_input_row_count: IntCounterVec,
pub sink_chunk_buffer_size: LabelGuardedIntGaugeVec<3>,
// Exchange (see also `compute::ExchangeServiceMetrics`)
pub exchange_frag_recv_size: GenericCounterVec<AtomicU64>,
// Backpressure
pub actor_output_buffer_blocking_duration_ns: LabelGuardedIntCounterVec<3>,
pub actor_input_buffer_blocking_duration_ns: LabelGuardedIntCounterVec<3>,
// Streaming Join
pub join_lookup_miss_count: LabelGuardedIntCounterVec<5>,
pub join_lookup_total_count: LabelGuardedIntCounterVec<5>,
pub join_insert_cache_miss_count: LabelGuardedIntCounterVec<5>,
pub join_actor_input_waiting_duration_ns: LabelGuardedIntCounterVec<2>,
pub join_match_duration_ns: LabelGuardedIntCounterVec<3>,
pub join_barrier_align_duration: RelabeledGuardedHistogramVec<3>,
pub join_cached_entry_count: LabelGuardedIntGaugeVec<3>,
pub join_matched_join_keys: RelabeledGuardedHistogramVec<3>,
// Streaming Aggregation
pub agg_lookup_miss_count: GenericCounterVec<AtomicU64>,
pub agg_total_lookup_count: GenericCounterVec<AtomicU64>,
pub agg_cached_entry_count: GenericGaugeVec<AtomicI64>,
pub agg_chunk_lookup_miss_count: GenericCounterVec<AtomicU64>,
pub agg_chunk_total_lookup_count: GenericCounterVec<AtomicU64>,
pub agg_distinct_cache_miss_count: GenericCounterVec<AtomicU64>,
pub agg_distinct_total_cache_count: GenericCounterVec<AtomicU64>,
pub agg_distinct_cached_entry_count: GenericGaugeVec<AtomicI64>,
pub agg_dirty_groups_count: GenericGaugeVec<AtomicI64>,
pub agg_dirty_groups_heap_size: GenericGaugeVec<AtomicI64>,
// Streaming TopN
pub group_top_n_cache_miss_count: GenericCounterVec<AtomicU64>,
pub group_top_n_total_query_cache_count: GenericCounterVec<AtomicU64>,
pub group_top_n_cached_entry_count: GenericGaugeVec<AtomicI64>,
pub group_top_n_appendonly_cache_miss_count: GenericCounterVec<AtomicU64>,
pub group_top_n_appendonly_total_query_cache_count: GenericCounterVec<AtomicU64>,
pub group_top_n_appendonly_cached_entry_count: GenericGaugeVec<AtomicI64>,
// Lookup executor
pub lookup_cache_miss_count: GenericCounterVec<AtomicU64>,
pub lookup_total_query_cache_count: GenericCounterVec<AtomicU64>,
pub lookup_cached_entry_count: GenericGaugeVec<AtomicI64>,
// temporal join
pub temporal_join_cache_miss_count: GenericCounterVec<AtomicU64>,
pub temporal_join_total_query_cache_count: GenericCounterVec<AtomicU64>,
pub temporal_join_cached_entry_count: GenericGaugeVec<AtomicI64>,
// Backfill
pub backfill_snapshot_read_row_count: LabelGuardedIntCounterVec<2>,
pub backfill_upstream_output_row_count: LabelGuardedIntCounterVec<2>,
// CDC Backfill
pub cdc_backfill_snapshot_read_row_count: GenericCounterVec<AtomicU64>,
pub cdc_backfill_upstream_output_row_count: GenericCounterVec<AtomicU64>,
// Over Window
pub over_window_cached_entry_count: GenericGaugeVec<AtomicI64>,
pub over_window_cache_lookup_count: GenericCounterVec<AtomicU64>,
pub over_window_cache_miss_count: GenericCounterVec<AtomicU64>,
pub over_window_range_cache_entry_count: GenericGaugeVec<AtomicI64>,
pub over_window_range_cache_lookup_count: GenericCounterVec<AtomicU64>,
pub over_window_range_cache_left_miss_count: GenericCounterVec<AtomicU64>,
pub over_window_range_cache_right_miss_count: GenericCounterVec<AtomicU64>,
/// The duration from receipt of barrier to all actors collection.
/// And the max of all node `barrier_inflight_latency` is the latency for a barrier
/// to flow through the graph.
pub barrier_inflight_latency: Histogram,
/// The duration of sync to storage.
pub barrier_sync_latency: Histogram,
/// The progress made by the earliest in-flight barriers in the local barrier manager.
pub barrier_manager_progress: IntCounter,
// Sink related metrics
pub sink_commit_duration: LabelGuardedHistogramVec<3>,
pub connector_sink_rows_received: LabelGuardedIntCounterVec<2>,
pub log_store_first_write_epoch: LabelGuardedIntGaugeVec<3>,
pub log_store_latest_write_epoch: LabelGuardedIntGaugeVec<3>,
pub log_store_write_rows: LabelGuardedIntCounterVec<3>,
pub log_store_latest_read_epoch: LabelGuardedIntGaugeVec<3>,
pub log_store_read_rows: LabelGuardedIntCounterVec<3>,
pub log_store_reader_wait_new_future_duration_ns: LabelGuardedIntCounterVec<3>,
pub kv_log_store_storage_write_count: LabelGuardedIntCounterVec<3>,
pub kv_log_store_storage_write_size: LabelGuardedIntCounterVec<3>,
pub kv_log_store_rewind_count: LabelGuardedIntCounterVec<3>,
pub kv_log_store_rewind_delay: LabelGuardedHistogramVec<3>,
pub kv_log_store_storage_read_count: LabelGuardedIntCounterVec<4>,
pub kv_log_store_storage_read_size: LabelGuardedIntCounterVec<4>,
pub kv_log_store_buffer_unconsumed_item_count: LabelGuardedIntGaugeVec<3>,
pub kv_log_store_buffer_unconsumed_row_count: LabelGuardedIntGaugeVec<3>,
pub kv_log_store_buffer_unconsumed_epoch_count: LabelGuardedIntGaugeVec<3>,
pub kv_log_store_buffer_unconsumed_min_epoch: LabelGuardedIntGaugeVec<3>,
// Sink iceberg metrics
pub iceberg_write_qps: LabelGuardedIntCounterVec<2>,
pub iceberg_write_latency: LabelGuardedHistogramVec<2>,
pub iceberg_rolling_unflushed_data_file: LabelGuardedIntGaugeVec<2>,
pub iceberg_position_delete_cache_num: LabelGuardedIntGaugeVec<2>,
pub iceberg_partition_num: LabelGuardedIntGaugeVec<2>,
// Memory management
// FIXME(yuhao): use u64 here
pub lru_current_watermark_time_ms: IntGauge,
pub lru_physical_now_ms: IntGauge,
pub lru_runtime_loop_count: IntCounter,
pub lru_watermark_step: IntGauge,
pub lru_evicted_watermark_time_ms: LabelGuardedIntGaugeVec<3>,
pub jemalloc_allocated_bytes: IntGauge,
pub jemalloc_active_bytes: IntGauge,
pub jemalloc_resident_bytes: IntGauge,
pub jemalloc_metadata_bytes: IntGauge,
pub jvm_allocated_bytes: IntGauge,
pub jvm_active_bytes: IntGauge,
// Materialize
pub materialize_cache_hit_count: GenericCounterVec<AtomicU64>,
pub materialize_cache_total_count: GenericCounterVec<AtomicU64>,
// Memory
pub stream_memory_usage: LabelGuardedIntGaugeVec<3>,
}
pub static GLOBAL_STREAMING_METRICS: OnceLock<StreamingMetrics> = OnceLock::new();
pub fn global_streaming_metrics(metric_level: MetricLevel) -> StreamingMetrics {
GLOBAL_STREAMING_METRICS
.get_or_init(|| StreamingMetrics::new(&GLOBAL_METRICS_REGISTRY, metric_level))
.clone()
}
impl StreamingMetrics {
fn new(registry: &Registry, level: MetricLevel) -> Self {
let executor_row_count = register_int_counter_vec_with_registry!(
"stream_executor_row_count",
"Total number of rows that have been output from each executor",
&["actor_id", "fragment_id", "executor_identity"],
registry
)
.unwrap();
let source_output_row_count = register_int_counter_vec_with_registry!(
"stream_source_output_rows_counts",
"Total number of rows that have been output from source",
&["source_id", "source_name", "actor_id", "fragment_id"],
registry
)
.unwrap();
let source_split_change_count = register_int_counter_vec_with_registry!(
"stream_source_split_change_event_count",
"Total number of split change events that have been operated by source",
&["source_id", "source_name", "actor_id", "fragment_id"],
registry
)
.unwrap();
let source_backfill_row_count = register_guarded_int_counter_vec_with_registry!(
"stream_source_backfill_rows_counts",
"Total number of rows that have been backfilled for source",
&["source_id", "source_name", "actor_id", "fragment_id"],
registry
)
.unwrap();
let sink_input_row_count = register_guarded_int_counter_vec_with_registry!(
"stream_sink_input_row_count",
"Total number of rows streamed into sink executors",
&["sink_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let mview_input_row_count = register_int_counter_vec_with_registry!(
"stream_mview_input_row_count",
"Total number of rows streamed into materialize executors",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let sink_chunk_buffer_size = register_guarded_int_gauge_vec_with_registry!(
"stream_sink_chunk_buffer_size",
"Total size of chunks buffered in a barrier",
&["sink_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let actor_execution_time = register_gauge_vec_with_registry!(
"stream_actor_actor_execution_time",
"Total execution time (s) of an actor",
&["actor_id"],
registry
)
.unwrap();
let actor_output_buffer_blocking_duration_ns =
register_guarded_int_counter_vec_with_registry!(
"stream_actor_output_buffer_blocking_duration_ns",
"Total blocking duration (ns) of output buffer",
&["actor_id", "fragment_id", "downstream_fragment_id"],
registry
)
.unwrap();
let actor_input_buffer_blocking_duration_ns =
register_guarded_int_counter_vec_with_registry!(
"stream_actor_input_buffer_blocking_duration_ns",
"Total blocking duration (ns) of input buffer",
&["actor_id", "fragment_id", "upstream_fragment_id"],
registry
)
.unwrap();
let exchange_frag_recv_size = register_int_counter_vec_with_registry!(
"stream_exchange_frag_recv_size",
"Total size of messages that have been received from upstream Fragment",
&["up_fragment_id", "down_fragment_id"],
registry
)
.unwrap();
let actor_fast_poll_duration = register_gauge_vec_with_registry!(
"stream_actor_fast_poll_duration",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_fast_poll_cnt = register_int_gauge_vec_with_registry!(
"stream_actor_fast_poll_cnt",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_slow_poll_duration = register_gauge_vec_with_registry!(
"stream_actor_slow_poll_duration",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_slow_poll_cnt = register_int_gauge_vec_with_registry!(
"stream_actor_slow_poll_cnt",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_poll_duration = register_gauge_vec_with_registry!(
"stream_actor_poll_duration",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_poll_cnt = register_int_gauge_vec_with_registry!(
"stream_actor_poll_cnt",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_scheduled_duration = register_gauge_vec_with_registry!(
"stream_actor_scheduled_duration",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_scheduled_cnt = register_int_gauge_vec_with_registry!(
"stream_actor_scheduled_cnt",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_idle_duration = register_gauge_vec_with_registry!(
"stream_actor_idle_duration",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_idle_cnt = register_int_gauge_vec_with_registry!(
"stream_actor_idle_cnt",
"tokio's metrics",
&["actor_id"],
registry
)
.unwrap();
let actor_in_record_cnt = register_guarded_int_counter_vec_with_registry!(
"stream_actor_in_record_cnt",
"Total number of rows actor received",
&["actor_id", "fragment_id", "upstream_fragment_id"],
registry
)
.unwrap();
let actor_out_record_cnt = register_guarded_int_counter_vec_with_registry!(
"stream_actor_out_record_cnt",
"Total number of rows actor sent",
&["actor_id", "fragment_id"],
registry
)
.unwrap();
let actor_memory_usage = register_int_gauge_vec_with_registry!(
"actor_memory_usage",
"Memory usage (bytes)",
&["actor_id", "fragment_id"],
registry,
)
.unwrap();
let join_lookup_miss_count = register_guarded_int_counter_vec_with_registry!(
"stream_join_lookup_miss_count",
"Join executor lookup miss duration",
&[
"side",
"join_table_id",
"degree_table_id",
"actor_id",
"fragment_id"
],
registry
)
.unwrap();
let join_lookup_total_count = register_guarded_int_counter_vec_with_registry!(
"stream_join_lookup_total_count",
"Join executor lookup total operation",
&[
"side",
"join_table_id",
"degree_table_id",
"actor_id",
"fragment_id"
],
registry
)
.unwrap();
let join_insert_cache_miss_count = register_guarded_int_counter_vec_with_registry!(
"stream_join_insert_cache_miss_count",
"Join executor cache miss when insert operation",
&[
"side",
"join_table_id",
"degree_table_id",
"actor_id",
"fragment_id"
],
registry
)
.unwrap();
let join_actor_input_waiting_duration_ns = register_guarded_int_counter_vec_with_registry!(
"stream_join_actor_input_waiting_duration_ns",
"Total waiting duration (ns) of input buffer of join actor",
&["actor_id", "fragment_id"],
registry
)
.unwrap();
let join_match_duration_ns = register_guarded_int_counter_vec_with_registry!(
"stream_join_match_duration_ns",
"Matching duration for each side",
&["actor_id", "fragment_id", "side"],
registry
)
.unwrap();
let opts = histogram_opts!(
"stream_join_barrier_align_duration",
"Duration of join align barrier",
exponential_buckets(0.0001, 2.0, 21).unwrap() // max 104s
);
let join_barrier_align_duration = register_guarded_histogram_vec_with_registry!(
opts,
&["actor_id", "fragment_id", "wait_side"],
registry
)
.unwrap();
let join_barrier_align_duration = RelabeledGuardedHistogramVec::with_metric_level_relabel_n(
MetricLevel::Debug,
join_barrier_align_duration,
level,
1,
);
let join_cached_entry_count = register_guarded_int_gauge_vec_with_registry!(
"stream_join_cached_entry_count",
"Number of cached entries in streaming join operators",
&["actor_id", "fragment_id", "side"],
registry
)
.unwrap();
let join_matched_join_keys_opts = histogram_opts!(
"stream_join_matched_join_keys",
"The number of keys matched in the opposite side",
exponential_buckets(16.0, 2.0, 28).unwrap() // max 2^31
);
let join_matched_join_keys = register_guarded_histogram_vec_with_registry!(
join_matched_join_keys_opts,
&["actor_id", "fragment_id", "table_id"],
registry
)
.unwrap();
let join_matched_join_keys = RelabeledGuardedHistogramVec::with_metric_level_relabel_n(
MetricLevel::Debug,
join_matched_join_keys,
level,
1,
);
let agg_lookup_miss_count = register_int_counter_vec_with_registry!(
"stream_agg_lookup_miss_count",
"Aggregation executor lookup miss duration",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_total_lookup_count = register_int_counter_vec_with_registry!(
"stream_agg_lookup_total_count",
"Aggregation executor lookup total operation",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_distinct_cache_miss_count = register_int_counter_vec_with_registry!(
"stream_agg_distinct_cache_miss_count",
"Aggregation executor dinsinct miss duration",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_distinct_total_cache_count = register_int_counter_vec_with_registry!(
"stream_agg_distinct_total_cache_count",
"Aggregation executor distinct total operation",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_distinct_cached_entry_count = register_int_gauge_vec_with_registry!(
"stream_agg_distinct_cached_entry_count",
"Total entry counts in distinct aggregation executor cache",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_dirty_groups_count = register_int_gauge_vec_with_registry!(
"stream_agg_dirty_groups_count",
"Total dirty group counts in aggregation executor",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_dirty_groups_heap_size = register_int_gauge_vec_with_registry!(
"stream_agg_dirty_groups_heap_size",
"Total dirty group heap size in aggregation executor",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let group_top_n_cache_miss_count = register_int_counter_vec_with_registry!(
"stream_group_top_n_cache_miss_count",
"Group top n executor cache miss count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let group_top_n_total_query_cache_count = register_int_counter_vec_with_registry!(
"stream_group_top_n_total_query_cache_count",
"Group top n executor query cache total count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let group_top_n_cached_entry_count = register_int_gauge_vec_with_registry!(
"stream_group_top_n_cached_entry_count",
"Total entry counts in group top n executor cache",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let group_top_n_appendonly_cache_miss_count = register_int_counter_vec_with_registry!(
"stream_group_top_n_appendonly_cache_miss_count",
"Group top n appendonly executor cache miss count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let group_top_n_appendonly_total_query_cache_count =
register_int_counter_vec_with_registry!(
"stream_group_top_n_appendonly_total_query_cache_count",
"Group top n appendonly executor total cache count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let group_top_n_appendonly_cached_entry_count = register_int_gauge_vec_with_registry!(
"stream_group_top_n_appendonly_cached_entry_count",
"Total entry counts in group top n appendonly executor cache",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let lookup_cache_miss_count = register_int_counter_vec_with_registry!(
"stream_lookup_cache_miss_count",
"Lookup executor cache miss count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let lookup_total_query_cache_count = register_int_counter_vec_with_registry!(
"stream_lookup_total_query_cache_count",
"Lookup executor query cache total count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let lookup_cached_entry_count = register_int_gauge_vec_with_registry!(
"stream_lookup_cached_entry_count",
"Total entry counts in lookup executor cache",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let temporal_join_cache_miss_count = register_int_counter_vec_with_registry!(
"stream_temporal_join_cache_miss_count",
"Temporal join executor cache miss count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let temporal_join_total_query_cache_count = register_int_counter_vec_with_registry!(
"stream_temporal_join_total_query_cache_count",
"Temporal join executor query cache total count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let temporal_join_cached_entry_count = register_int_gauge_vec_with_registry!(
"stream_temporal_join_cached_entry_count",
"Total entry count in temporal join executor cache",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_cached_entry_count = register_int_gauge_vec_with_registry!(
"stream_agg_cached_entry_count",
"Number of cached keys in streaming aggregation operators",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_chunk_lookup_miss_count = register_int_counter_vec_with_registry!(
"stream_agg_chunk_lookup_miss_count",
"Aggregation executor chunk-level lookup miss duration",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let agg_chunk_total_lookup_count = register_int_counter_vec_with_registry!(
"stream_agg_chunk_lookup_total_count",
"Aggregation executor chunk-level lookup total operation",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let backfill_snapshot_read_row_count = register_guarded_int_counter_vec_with_registry!(
"stream_backfill_snapshot_read_row_count",
"Total number of rows that have been read from the backfill snapshot",
&["table_id", "actor_id"],
registry
)
.unwrap();
let backfill_upstream_output_row_count = register_guarded_int_counter_vec_with_registry!(
"stream_backfill_upstream_output_row_count",
"Total number of rows that have been output from the backfill upstream",
&["table_id", "actor_id"],
registry
)
.unwrap();
let cdc_backfill_snapshot_read_row_count = register_int_counter_vec_with_registry!(
"stream_cdc_backfill_snapshot_read_row_count",
"Total number of rows that have been read from the cdc_backfill snapshot",
&["table_id", "actor_id"],
registry
)
.unwrap();
let cdc_backfill_upstream_output_row_count = register_int_counter_vec_with_registry!(
"stream_cdc_backfill_upstream_output_row_count",
"Total number of rows that have been output from the cdc_backfill upstream",
&["table_id", "actor_id"],
registry
)
.unwrap();
let over_window_cached_entry_count = register_int_gauge_vec_with_registry!(
"stream_over_window_cached_entry_count",
"Total entry (partition) count in over window executor cache",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let over_window_cache_lookup_count = register_int_counter_vec_with_registry!(
"stream_over_window_cache_lookup_count",
"Over window executor cache lookup count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let over_window_cache_miss_count = register_int_counter_vec_with_registry!(
"stream_over_window_cache_miss_count",
"Over window executor cache miss count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let over_window_range_cache_entry_count = register_int_gauge_vec_with_registry!(
"stream_over_window_range_cache_entry_count",
"Over window partition range cache entry count",
&["table_id", "actor_id", "fragment_id"],
registry,
)
.unwrap();
let over_window_range_cache_lookup_count = register_int_counter_vec_with_registry!(
"stream_over_window_range_cache_lookup_count",
"Over window partition range cache lookup count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let over_window_range_cache_left_miss_count = register_int_counter_vec_with_registry!(
"stream_over_window_range_cache_left_miss_count",
"Over window partition range cache left miss count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let over_window_range_cache_right_miss_count = register_int_counter_vec_with_registry!(
"stream_over_window_range_cache_right_miss_count",
"Over window partition range cache right miss count",
&["table_id", "actor_id", "fragment_id"],
registry
)
.unwrap();
let opts = histogram_opts!(
"stream_barrier_inflight_duration_seconds",
"barrier_inflight_latency",
exponential_buckets(0.1, 1.5, 16).unwrap() // max 43s
);
let barrier_inflight_latency = register_histogram_with_registry!(opts, registry).unwrap();
let opts = histogram_opts!(
"stream_barrier_sync_storage_duration_seconds",
"barrier_sync_latency",
exponential_buckets(0.1, 1.5, 16).unwrap() // max 43
);
let barrier_sync_latency = register_histogram_with_registry!(opts, registry).unwrap();
let barrier_manager_progress = register_int_counter_with_registry!(
"stream_barrier_manager_progress",
"The number of actors that have processed the earliest in-flight barriers",
registry
)
.unwrap();
let sink_commit_duration = register_guarded_histogram_vec_with_registry!(
"sink_commit_duration",
"Duration of commit op in sink",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let connector_sink_rows_received = register_guarded_int_counter_vec_with_registry!(
"connector_sink_rows_received",
"Number of rows received by sink",
&["connector_type", "sink_id"],
registry
)
.unwrap();
let log_store_first_write_epoch = register_guarded_int_gauge_vec_with_registry!(
"log_store_first_write_epoch",
"The first write epoch of log store",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let log_store_latest_write_epoch = register_guarded_int_gauge_vec_with_registry!(
"log_store_latest_write_epoch",
"The latest write epoch of log store",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let log_store_write_rows = register_guarded_int_counter_vec_with_registry!(
"log_store_write_rows",
"The write rate of rows",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let log_store_latest_read_epoch = register_guarded_int_gauge_vec_with_registry!(
"log_store_latest_read_epoch",
"The latest read epoch of log store",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let log_store_read_rows = register_guarded_int_counter_vec_with_registry!(
"log_store_read_rows",
"The read rate of rows",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let log_store_reader_wait_new_future_duration_ns =
register_guarded_int_counter_vec_with_registry!(
"log_store_reader_wait_new_future_duration_ns",
"Accumulated duration of LogReader to wait for next call to create future",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_storage_write_count = register_guarded_int_counter_vec_with_registry!(
"kv_log_store_storage_write_count",
"Write row count throughput of kv log store",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_storage_write_size = register_guarded_int_counter_vec_with_registry!(
"kv_log_store_storage_write_size",
"Write size throughput of kv log store",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_storage_read_count = register_guarded_int_counter_vec_with_registry!(
"kv_log_store_storage_read_count",
"Write row count throughput of kv log store",
&["executor_id", "connector", "sink_id", "read_type"],
registry
)
.unwrap();
let kv_log_store_storage_read_size = register_guarded_int_counter_vec_with_registry!(
"kv_log_store_storage_read_size",
"Write size throughput of kv log store",
&["executor_id", "connector", "sink_id", "read_type"],
registry
)
.unwrap();
let kv_log_store_rewind_count = register_guarded_int_counter_vec_with_registry!(
"kv_log_store_rewind_count",
"Kv log store rewind rate",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_rewind_delay_opts = {
assert_eq!(2, REWIND_BACKOFF_FACTOR);
let bucket_count = (REWIND_MAX_DELAY.as_secs_f64().log2()
- REWIND_BASE_DELAY.as_secs_f64().log2())
.ceil() as usize;
let buckets = exponential_buckets(
REWIND_BASE_DELAY.as_secs_f64(),
REWIND_BACKOFF_FACTOR as _,
bucket_count,
)
.unwrap();
histogram_opts!(
"kv_log_store_rewind_delay",
"Kv log store rewind delay",
buckets,
)
};
let kv_log_store_rewind_delay = register_guarded_histogram_vec_with_registry!(
kv_log_store_rewind_delay_opts,
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_buffer_unconsumed_item_count =
register_guarded_int_gauge_vec_with_registry!(
"kv_log_store_buffer_unconsumed_item_count",
"Number of Unconsumed Item in buffer",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_buffer_unconsumed_row_count =
register_guarded_int_gauge_vec_with_registry!(
"kv_log_store_buffer_unconsumed_row_count",
"Number of Unconsumed Row in buffer",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_buffer_unconsumed_epoch_count =
register_guarded_int_gauge_vec_with_registry!(
"kv_log_store_buffer_unconsumed_epoch_count",
"Number of Unconsumed Epoch in buffer",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let kv_log_store_buffer_unconsumed_min_epoch =
register_guarded_int_gauge_vec_with_registry!(
"kv_log_store_buffer_unconsumed_min_epoch",
"Number of Unconsumed Epoch in buffer",
&["executor_id", "connector", "sink_id"],
registry
)
.unwrap();
let lru_current_watermark_time_ms = register_int_gauge_with_registry!(
"lru_current_watermark_time_ms",
"Current LRU manager watermark time(ms)",
registry
)
.unwrap();
let lru_physical_now_ms = register_int_gauge_with_registry!(
"lru_physical_now_ms",
"Current physical time in Risingwave(ms)",
registry
)
.unwrap();
let lru_runtime_loop_count = register_int_counter_with_registry!(
"lru_runtime_loop_count",
"The counts of the eviction loop in LRU manager per second",
registry
)
.unwrap();
let lru_watermark_step = register_int_gauge_with_registry!(
"lru_watermark_step",
"The steps increase in 1 loop",
registry
)
.unwrap();
let lru_evicted_watermark_time_ms = register_guarded_int_gauge_vec_with_registry!(
"lru_evicted_watermark_time_ms",
"The latest evicted watermark time by actors",
&["table_id", "actor_id", "desc"],
registry
)
.unwrap();
let jemalloc_allocated_bytes = register_int_gauge_with_registry!(
"jemalloc_allocated_bytes",
"The allocated memory jemalloc, got from jemalloc_ctl",
registry
)
.unwrap();