Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat(sink): add sink throughput( mb/s) metrics #19587

Merged
merged 5 commits into from
Dec 10, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion docker/dashboards/risingwave-dev-dashboard.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion docker/dashboards/risingwave-user-dashboard.json

Large diffs are not rendered by default.

20 changes: 20 additions & 0 deletions grafana/risingwave-dev-dashboard.dashboard.py
Original file line number Diff line number Diff line change
Expand Up @@ -970,6 +970,26 @@ def section_streaming(outer_panels):
),
],
),
panels.timeseries_bytesps(
"Sink Throughput(MB/s)",
"The figure shows the number of bytes written each sink per second. For sinks with 'sink_decouple = true', please refer to the 'Sink Metrics' section",
[
panels.target(
f"(sum(rate({metric('stream_sink_input_size')}[$__rate_interval])) by (sink_id) * on(sink_id) group_left(sink_name) group({metric('sink_info')}) by (sink_id, sink_name)) / (1000*1000)",
"sink {{sink_id}} {{sink_name}}",
),
],
),
panels.timeseries_bytesps(
"Sink Throughput(MB/s) per Partition *",
"The number of bytes streamed into each sink per second. For sinks with 'sink_decouple = true', please refer to the 'Sink Metrics' section",
[
panels.target(
f"(sum(rate({metric('stream_sink_input_size')}[$__rate_interval])) by (sink_id, actor_id) * on(actor_id) group_left(sink_name) {metric('sink_info')}) / (1000*1000)",
"sink {{sink_id}} {{sink_name}} - actor {{actor_id}}",
),
],
),
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
panels.timeseries_rowsps(
"Materialized View Throughput(rows/s)",
"The figure shows the number of rows written into each materialized view per second.",
Expand Down
2 changes: 1 addition & 1 deletion grafana/risingwave-dev-dashboard.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion grafana/risingwave-user-dashboard.json

Large diffs are not rendered by default.

6 changes: 3 additions & 3 deletions risedev.yml
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,9 @@ profile:
# database: metadata

# If you want to enable metrics or tracing, uncomment the following lines.
# - use: prometheus # metrics
# - use: tempo # tracing
# - use: grafana # visualization
- use: prometheus # metrics
- use: tempo # tracing
- use: grafana # visualization
xxhZs marked this conversation as resolved.
Show resolved Hide resolved

- use: meta-node
# meta-backend: postgres
Expand Down
12 changes: 12 additions & 0 deletions src/stream/src/executor/monitor/streaming_stats.rs
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ pub struct StreamingMetrics {

// Sink
sink_input_row_count: LabelGuardedIntCounterVec<3>,
sink_input_size: LabelGuardedIntCounterVec<3>,
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
sink_chunk_buffer_size: LabelGuardedIntGaugeVec<3>,

// Exchange (see also `compute::ExchangeServiceMetrics`)
Expand Down Expand Up @@ -244,6 +245,14 @@ impl StreamingMetrics {
)
.unwrap();

let sink_input_size = register_guarded_int_counter_vec_with_registry!(
"stream_sink_input_size",
"Total size of chunks streamed into sink executors",
&["sink_id", "actor_id", "fragment_id"],
registry
)
.unwrap();

let materialize_input_row_count = register_guarded_int_counter_vec_with_registry!(
"stream_mview_input_row_count",
"Total number of rows streamed into materialize executors",
Expand Down Expand Up @@ -1056,6 +1065,7 @@ impl StreamingMetrics {
source_split_change_count,
source_backfill_row_count,
sink_input_row_count,
sink_input_size,
sink_chunk_buffer_size,
exchange_frag_recv_size,
merge_barrier_align_duration,
Expand Down Expand Up @@ -1229,6 +1239,7 @@ impl StreamingMetrics {
sink_input_row_count: self
.sink_input_row_count
.with_guarded_label_values(label_list),
sink_input_size: self.sink_input_size.with_guarded_label_values(label_list),
sink_chunk_buffer_size: self
.sink_chunk_buffer_size
.with_guarded_label_values(label_list),
Expand Down Expand Up @@ -1520,6 +1531,7 @@ pub struct ActorMetrics {

pub struct SinkExecutorMetrics {
pub sink_input_row_count: LabelGuardedIntCounter<3>,
pub sink_input_size: LabelGuardedIntCounter<3>,
pub sink_chunk_buffer_size: LabelGuardedIntGauge<3>,
}

Expand Down
1 change: 1 addition & 0 deletions src/stream/src/executor/sink.rs
Original file line number Diff line number Diff line change
Expand Up @@ -200,6 +200,7 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
let input = input.inspect_ok(move |msg| {
if let Message::Chunk(c) = msg {
metrics.sink_input_row_count.inc_by(c.capacity() as u64);
metrics.sink_input_size.inc_by(c.estimated_size() as u64);
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the metrics here will be inaccurate when sink decoupling is on because the metrics here is measuring the throughput of the input channel. When sink decoupling is on, the input will be polled and written to log store regardless of the external sink.

We need to add a metric for log store reader throughput:

.inc_by(chunk.cardinality() as _);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like the metric here is log_store_read_rows, which doesn't match the sink_input_row_count we defined earlier. Should we merge the two or add another log_store_read_rows(mb/s)

}
});

Expand Down