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(metrics): add backpressure metrics #3636

Merged
merged 41 commits into from
Jul 5, 2022
Merged

Conversation

Sunt-ing
Copy link
Contributor

@Sunt-ing Sunt-ing commented Jul 4, 2022

I hereby agree to the terms of the Singularity Data, Inc. Contributor License Agreement.

What's changed and what's your intention?

Add backpressure metrics. Similar to Flink backpressure monitoring, we get the backpressure rate by sampled output buffer occupation rate.

Bench result in 3-CN mode:
image

Checklist

  • I have written necessary docs and comments
  • I have added necessary unit tests and integration tests
  • All checks passed in ./risedev check (or alias, ./risedev c)

Refer to a related PR or issue link (optional)

#3284

@Sunt-ing Sunt-ing requested a review from StrikeW July 4, 2022 12:18
src/stream/src/executor/dispatch.rs Outdated Show resolved Hide resolved
actor_coroutine_monitor_tasks: HashMap<ActorId, JoinHandle<()>>,

/// Stores all actor output buffer montioring tasks.
actor_output_buffer_monitor_tasks: HashMap<ActorId, JoinHandle<()>>,
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we have a single extra coroutine for all metrics collection?

src/stream/src/executor/dispatch.rs Outdated Show resolved Hide resolved
@skyzh
Copy link
Contributor

skyzh commented Jul 4, 2022

thread 'tokio-runtime-worker' panicked at 'called `Option::unwrap()` on a `None` value', src/stream/src/task/stream_manager.rs:792:14
stack backtrace:
   0: rust_begin_unwind
             at /rustc/bb8c2f41174caceec00c28bc6c5c20ae9f9a175c/library/std/src/panicking.rs:584:5
   1: core::panicking::panic_fmt
             at /rustc/bb8c2f41174caceec00c28bc6c5c20ae9f9a175c/library/core/src/panicking.rs:142:14
   2: core::panicking::panic
             at /rustc/bb8c2f41174caceec00c28bc6c5c20ae9f9a175c/library/core/src/panicking.rs:48:5
   3: core::option::Option<T>::unwrap
             at /rustc/bb8c2f41174caceec00c28bc6c5c20ae9f9a175c/library/core/src/option.rs:775:21
   4: risingwave_stream::task::stream_manager::LocalStreamManagerCore::drop_actor
             at ./src/stream/src/task/stream_manager.rs:790:9
   5: risingwave_stream::task::stream_manager::LocalStreamManager::drop_actor
             at ./src/stream/src/task/stream_manager.rs:239:13

Panic because dispatcher metrics future is not started for some actors. Please check.

Copy link
Contributor

@skyzh skyzh left a comment

Choose a reason for hiding this comment

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

https://github.com/apache/flink/blob/16109a31468949f09c2a7bba9003761726e3d61c/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L573-L584

Flink absolutely doesn't count backpressure like this PR. I think we'd better be accurate for this metrics instead of letting future developers be confused about "why this metrics doesn't reflect the real situation".

@StrikeW
Copy link
Contributor

StrikeW commented Jul 4, 2022

Current implementation uses a separate coroutine task to calculate the backpressure rate. The problem would be the visibility is dependent on whether the monitor task can be scheduled on time. A better way would be each actor can report the metrics by itself.

@Sunt-ing
Copy link
Contributor Author

Sunt-ing commented Jul 4, 2022

This PR does not implement accurate back pressure mentoring. If that's necessary, then the implementation can be changed to monitor RemoteOutput::send and LocalOutput::send as follows:

    async fn send(&mut self, message: Message) -> Result<()> {
        // local channel should never fail
        let start_time = if self.ch.capacity() == 0 {
            Some(Instant::now())
        } else {
            None
        };
        self.ch
            .send(message)
            .await
            .map_err(|_| internal_error("failed to send"))?;
        if start_time.is_some(){
            self.metrics
                .output_buffer_blockting_time
                .with_label_values(&[&up_actor_id])
                .inc_by(start_time.unwrap().elapsed().as_nanos() as u64);
        };
        Ok(())
    }

@Sunt-ing
Copy link
Contributor Author

Sunt-ing commented Jul 4, 2022

I benched the overhead with my M1:

#[tokio::main]
async fn main() {
    let buffer_size = 10000;
    let (tx, rx) = tokio::sync::mpsc::channel(buffer_size);
    tokio::spawn(async move {
        let mut cnt = 0;
        for i in 0..buffer_size{
            let start_time = madsim::time::Instant::now();
            tx.send(i).await.unwrap();
            cnt += start_time.elapsed().as_nanos();
        }
        println!("{} ns", cnt as f64 / buffer_size as f64);
    }).await.unwrap();
}

The result is only hundreds of nano seconds.

@Sunt-ing
Copy link
Contributor Author

Sunt-ing commented Jul 4, 2022

The bench result of the updated implementation be like:
image

@codecov
Copy link

codecov bot commented Jul 4, 2022

Codecov Report

Merging #3636 (d14684e) into main (c3a17ce) will decrease coverage by 0.01%.
The diff coverage is 51.47%.

@@            Coverage Diff             @@
##             main    #3636      +/-   ##
==========================================
- Coverage   74.37%   74.36%   -0.02%     
==========================================
  Files         776      776              
  Lines      110163   110213      +50     
==========================================
+ Hits        81937    81958      +21     
- Misses      28226    28255      +29     
Flag Coverage Δ
rust 74.36% <51.47%> (-0.02%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
src/stream/src/task/stream_manager.rs 0.00% <0.00%> (ø)
src/stream/src/executor/dispatch.rs 75.42% <46.93%> (-1.69%) ⬇️
src/stream/src/executor/integration_tests.rs 98.56% <100.00%> (ø)
src/stream/src/executor/monitor/streaming_stats.rs 100.00% <100.00%> (ø)
src/meta/src/hummock/mock_hummock_meta_client.rs 40.56% <0.00%> (-0.95%) ⬇️
src/frontend/src/expr/utils.rs 98.99% <0.00%> (-0.26%) ⬇️
src/storage/src/hummock/local_version_manager.rs 81.38% <0.00%> (-0.12%) ⬇️
src/meta/src/manager/id.rs 95.50% <0.00%> (ø)
src/meta/src/rpc/server.rs 80.51% <0.00%> (+0.23%) ⬆️

📣 Codecov can now indicate which changes are the most critical in Pull Requests. Learn more

grafana/risingwave-dashboard.py Outdated Show resolved Hide resolved
Copy link
Contributor

@StrikeW StrikeW left a comment

Choose a reason for hiding this comment

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

LGTM. there are 39 commits in the PR, please remember to clean the commit log when you merge the PR.

@@ -65,8 +65,10 @@ pub struct LocalStreamManagerCore {

/// Stores all actor information, taken after actor built.
actors: HashMap<ActorId, stream_plan::StreamActor>,
/// Store all actor execution time montioring tasks.

/// Stores all actor tokio runtime montioring tasks.
actor_monitor_tasks: HashMap<ActorId, JoinHandle<()>>,
Copy link
Contributor

Choose a reason for hiding this comment

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

remove this?

Copy link
Contributor

Choose a reason for hiding this comment

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

It's still used for another metrics. I'm thinking to move that metrics into the actor itself, instead of spawning new tasks.

grafana/risingwave-dashboard.py Outdated Show resolved Hide resolved
@Sunt-ing
Copy link
Contributor Author

Sunt-ing commented Jul 5, 2022

Should have fixed all the comments

@Sunt-ing Sunt-ing requested a review from skyzh July 5, 2022 02:41
@Sunt-ing Sunt-ing enabled auto-merge (squash) July 5, 2022 02:45
@Sunt-ing Sunt-ing merged commit be7830e into main Jul 5, 2022
@Sunt-ing Sunt-ing deleted the sunt_backpressure_metrics branch July 5, 2022 02:54
nasnoisaac pushed a commit to nasnoisaac/risingwave that referenced this pull request Aug 9, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants