Skip to content

Commit

Permalink
fix: avoid flush panic when recovery encountered (#15851) (#16126)
Browse files Browse the repository at this point in the history
Co-authored-by: August <pin@singularity-data.com>
  • Loading branch information
github-actions[bot] and yezizp2012 committed Apr 3, 2024
1 parent f0a05f8 commit dc0ea2d
Show file tree
Hide file tree
Showing 2 changed files with 13 additions and 4 deletions.
13 changes: 10 additions & 3 deletions src/meta/src/barrier/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -745,12 +745,19 @@ impl GlobalBarrierManager {

send_latency_timer.observe_duration();

let node_to_collect = self
let node_to_collect = match self
.control_stream_manager
.inject_barrier(command_ctx.clone())
.inspect_err(|_| {
{
Ok(node_to_collect) => node_to_collect,
Err(err) => {
for notifier in notifiers {
notifier.notify_failed(err.clone());
}
fail_point!("inject_barrier_err_success");
})?;
return Err(err);
}
};

// Notify about the injection.
let prev_paused_reason = self.state.paused_reason();
Expand Down
4 changes: 3 additions & 1 deletion src/meta/src/barrier/schedule.rs
Original file line number Diff line number Diff line change
Expand Up @@ -241,7 +241,9 @@ impl BarrierScheduler {
..Default::default()
};
self.attach_notifiers(vec![notifier], checkpoint)?;
rx.await.unwrap()
rx.await
.ok()
.context("failed to wait for barrier collect")?
}

/// Run multiple commands and return when they're all completely finished. It's ensured that
Expand Down

0 comments on commit dc0ea2d

Please sign in to comment.