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(storage): concurrent checkpoint #2184

Merged
merged 3 commits into from
Jun 23, 2022
Merged

Conversation

xxhZs
Copy link
Contributor

@xxhZs xxhZs commented Apr 28, 2022

What's changed and what's your intention?

concurrent checkpoint

we do the initial version of concurrent barrier

  1. Inject barrier in order, and concurrent barrier collect, commit_epoch in order.
  2. We save managed_barrier_state in map<epoch,state>
  3. if err, we will recovery all epoch in graph.
  4. if send barrier to build actor , We will pause injecting barriers
  5. Add metrics(barrier_nums and barrier_send_latency).
  6. We simply set the upper limit of the max in-flight-barrier-nums. We can change it like checkpoint_interval_ms(deafult :10)

newly introduced structs and fields

We use CheckpointControl to control the injection and save the state of barrier. is_recovery and is_build_actor can pause inject barrier. The state of barrier is saved in queue. It makes the commit orderly
In managed_state we use map to save collected actor for different epoch

Checklist

  • I have written necessary docs and comments
  • I have added necessary unit tests and integration tests

Refer to a related PR or issue link (optional)

#1156

@xxhZs xxhZs linked an issue Apr 28, 2022 that may be closed by this pull request
4 tasks
@codecov
Copy link

codecov bot commented Apr 28, 2022

Codecov Report

Merging #2184 (3f57f61) into main (ac77464) will increase coverage by 0.26%.
The diff coverage is 86.71%.

@@            Coverage Diff             @@
##             main    #2184      +/-   ##
==========================================
+ Coverage   73.60%   73.86%   +0.26%     
==========================================
  Files         765      765              
  Lines      104939   105248     +309     
==========================================
+ Hits        77236    77744     +508     
+ Misses      27703    27504     -199     
Flag Coverage Δ
rust 73.86% <86.71%> (+0.26%) ⬆️

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

Impacted Files Coverage Δ
src/common/src/config.rs 45.37% <0.00%> (-1.30%) ⬇️
src/meta/src/lib.rs 1.29% <0.00%> (-0.04%) ⬇️
src/meta/src/rpc/server.rs 0.00% <0.00%> (ø)
src/meta/src/stream/mod.rs 39.13% <ø> (ø)
src/stream/src/task/barrier_manager.rs 67.67% <0.00%> (-2.85%) ⬇️
src/stream/src/task/stream_manager.rs 0.00% <0.00%> (ø)
...c/stream/src/task/barrier_manager/managed_state.rs 73.00% <75.60%> (-5.52%) ⬇️
src/meta/src/barrier/recovery.rs 87.86% <78.57%> (+87.86%) ⬆️
src/meta/src/barrier/mod.rs 81.19% <86.53%> (+11.60%) ⬆️
src/meta/src/model/barrier.rs 86.66% <91.66%> (+8.09%) ⬆️
... and 19 more

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

@BugenZhao BugenZhao self-requested a review April 28, 2022 03:08
@hzxa21 hzxa21 self-requested a review April 28, 2022 15:26
let heap = self.epoch_heap.clone();
let hummock_manager = self.hummock_manager.clone();

tokio::spawn(async move {
Copy link
Contributor

Choose a reason for hiding this comment

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

How about using multiple coroutines instead of multiple threads here? 🤔

We can collect the futures and await them without thread-overprovisioning, where the latency of RPC is also overlapped.

futures.push(async move{ ... })
...
let res = select_all(futures).await;

If coroutine works, there would be no data race on env and epoch_heap, so RwLock could be mitigated.

    env: Arc<MetaSrvEnv<S>>,

    epoch_heap: Arc<BinaryHeap<u64>>,

Copy link
Contributor Author

@xxhZs xxhZs May 3, 2022

Choose a reason for hiding this comment

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

okoko , I will do it after I finish the basic functions.

@CLAassistant
Copy link

CLAassistant commented May 5, 2022

CLA assistant check
All committers have signed the CLA.

src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
@xxhZs xxhZs force-pushed the 1156-support-concurrent-checkpoint branch 2 times, most recently from 2ded88b to faf343c Compare May 19, 2022 14:13
@xxhZs xxhZs marked this pull request as ready for review May 20, 2022 12:57
Copy link
Member

@BugenZhao BugenZhao left a comment

Choose a reason for hiding this comment

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

Could you please add some docs for newly introduced structs and fields?

src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/stream/src/task/stream_manager.rs Outdated Show resolved Hide resolved
@skyzh skyzh self-requested a review May 23, 2022 09:21
@wenym1 wenym1 self-requested a review May 24, 2022 07:21
@skyzh
Copy link
Contributor

skyzh commented May 24, 2022

I've updated the branch so that we can fairly compare with 944b998 :) May pull this branch before committing new changes.

@skyzh
Copy link
Contributor

skyzh commented May 24, 2022

Just tried to bench this on AWS. However, all create table and create materialized source statement will stuck. Guess there's deadlock, maybe you'll need to debug more 🤣

The bug might be produced easier in 3 compute node release mode. Use ./risedev configure to enable release build, and update the default section of risedev.yml to start 3 compute nodes.

src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/rpc/metrics.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
proto/stream_service.proto Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/manager/env.rs Outdated Show resolved Hide resolved
src/stream/src/task/barrier_manager/managed_state.rs Outdated Show resolved Hide resolved
src/stream/src/task/barrier_manager/managed_state.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
@BugenZhao BugenZhao requested a review from yezizp2012 June 6, 2022 12:25
@skyzh
Copy link
Contributor

skyzh commented Jun 6, 2022

Need to fix conflict. We have a new Grafana generator, and we should adapt to that.

src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/stream/src/task/barrier_manager/managed_state.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/recovery.rs Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
@hzxa21
Copy link
Collaborator

hzxa21 commented Jun 20, 2022

With #2184 (comment) and #2184 (comment), barrier_complete_and_commit implementation can be simplified:

async fn barrier_complete_and_commit(....) {
        let (failed_nodes, res) = match result {
            Ok(resp) => {
                // Try complete barriers
                let succeeded_nodes = checkpoint_control.succeed(prev_epoch);
                let mut failed_nodes = Vec::new();
                let mut res: Result<()> = Ok(());

                // Complete_barriers does the following things:
                // - commit epoches in hummock_manager
                // - trigger post completion work on barriers
                // It returns:
                // - Ok(()) if all succeed
                // - Err((failed_nodes, err)) for the failed nodes and the error.
                if let Err((failed_nodes, err)) = self.complete_barriers(succeeded_nodes) {
                    // Fail all pending barriers as well
                    failed_nodes.concat(checkpoint_control.fail());
                    return (failed_nodes, Err(err));
                }
                Ok(())
            }
            Err(err) => {
                // Fail all barriers
                (checkpoint_control.fail(), Err(err))
            }
        };

        if !failed_nodes.is_empty() {
            if self.enable_recovery {
                // Post collection failure
                for node in failed_nodes {
                    ...
                }
                
                // Trigger recovery
                ...
            } else {
                panic!(...)
            }
        }
}

src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
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.

Generally LGTM, good work!

grafana/risingwave-dashboard.py Show resolved Hide resolved
grafana/risingwave-dashboard.py Outdated Show resolved Hide resolved
grafana/risingwave-dashboard.py Outdated Show resolved Hide resolved
src/common/src/config.rs Show resolved Hide resolved
src/config/risingwave.toml Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
// there's barrier scheduled.
_ = self.scheduled_barriers.wait_one() => {}
_ = self.scheduled_barriers.wait_one(), if checkpoint_control.can_inject_barrier(self.in_flight_barrier_nums) => {}
Copy link
Contributor

Choose a reason for hiding this comment

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

Possible performance regression, need to think twice. select! will evaluate all the conditions before actually doing selects. Is it possible that:

  • can_inject_barrier = false at first
  • no in-flight barrier (barrier_complete_rx will stuck)

At this time, only the shutdown branch and barrier_complete_rx will be activated. Even if can_inject_barrier becomes true sometime later, the thread will still block on polling barrier_complete_rx.

Copy link
Contributor

Choose a reason for hiding this comment

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

Seems that checkpoint_control is a local state and will not be modified by other thread. So while the current thread is blocked, it's not likely to be changed.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

can_inject_barrier will return false in two cases.

  1. the nums of in-flight > max. So there are in-flight barrier
  2. is_build_actor = true. So there is at least one in-flight barrier
    So in-flight barrier returns ==>can_inject_barrier becomes true. ==> continue next loop

Copy link
Contributor

@skyzh skyzh Jun 23, 2022

Choose a reason for hiding this comment

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

Looks like there won't be any deadlock if all related logics are left untouched.

Copy link
Contributor

@skyzh skyzh Jun 23, 2022

Choose a reason for hiding this comment

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

Anyway, you can think of this code like:

loop {
  if can_inject_barrier {
    select! { shutdown, complete_rx }
  } else {
    select! { shutdown, complete_rx, scheduled_barrier }
  }
}

If it is believed that this will always work without deadlock, then everything would be fine :)

src/meta/src/rpc/metrics.rs Outdated Show resolved Hide resolved
@skyzh
Copy link
Contributor

skyzh commented Jun 22, 2022

... who's root in this PR? Maybe you'll need to revisit commit history and remove the unknown user?

Copy link
Member

@BugenZhao BugenZhao left a comment

Choose a reason for hiding this comment

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

Generally LGTM. Good work!!

src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
src/stream/src/task/barrier_manager/managed_state.rs Outdated Show resolved Hide resolved
Comment on lines +178 to +182
/// remove all collect rx less than `prev_epoch`
pub fn drain_collect_rx(&mut self, prev_epoch: u64) {
self.collect_complete_receiver
.drain_filter(|x, _| x < &prev_epoch);
}
Copy link
Member

Choose a reason for hiding this comment

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

Should this be <=?

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 will be used before send_barrier. So there are not prev_epoch in collect_complete_receiver

src/meta/src/barrier/mod.rs Outdated Show resolved Hide resolved
@BugenZhao
Copy link
Member

So there're no necessary changes on the shared buffer? That's really cool. 🥵

@xxhZs
Copy link
Contributor Author

xxhZs commented Jun 22, 2022

... who's root in this PR? Maybe you'll need to revisit commit history and remove the unknown user?

I commit code on another computer with root

@xxhZs xxhZs mentioned this pull request Jun 23, 2022
3 tasks
@xxhZs xxhZs force-pushed the 1156-support-concurrent-checkpoint branch from 82d42d3 to e3a7e34 Compare June 23, 2022 04:24
@xxhZs
Copy link
Contributor Author

xxhZs commented Jun 23, 2022

The result of bench(query 1)
localhost_56975_d_Ecy3uV1nz_risingwave_dashboard_orgId=1 refresh=10s

@skyzh
Copy link
Contributor

skyzh commented Jun 23, 2022

The benchmark looks cool, that's so fast. By the way, would you please wait for all metrics to be fully loaded before taking a screenshot?

Copy link
Collaborator

@hzxa21 hzxa21 left a comment

Choose a reason for hiding this comment

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

LGTM. Good work!

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.

Tracking: Support concurrent checkpoint
7 participants