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(meta): try to report the root cause of recovery #13441

Merged
merged 25 commits into from
Feb 27, 2024
Merged
Changes from 15 commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
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
88 changes: 80 additions & 8 deletions src/meta/src/barrier/rpc.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,13 @@
use std::collections::HashMap;
use std::future::Future;
use std::sync::Arc;
use std::time::Duration;

use anyhow::anyhow;
use fail::fail_point;
use futures::future::try_join_all;
use futures::future::{select, Either};
use futures::stream::FuturesUnordered;
use futures::{FutureExt, StreamExt};
use futures::{pin_mut, FutureExt, StreamExt};
use itertools::Itertools;
use risingwave_common::bail;
use risingwave_common::hash::ActorId;
Expand All @@ -40,7 +41,7 @@ use uuid::Uuid;
use super::command::CommandContext;
use super::{BarrierCompletion, GlobalBarrierManagerContext};
use crate::manager::{MetaSrvEnv, WorkerId};
use crate::MetaResult;
use crate::{MetaError, MetaResult};

pub(super) struct BarrierRpcManager {
context: GlobalBarrierManagerContext,
Expand Down Expand Up @@ -268,11 +269,18 @@ impl StreamRpcManager {
) -> MetaResult<Vec<RSP>> {
let pool = self.env.stream_client_pool();
let f = &f;
Ok(try_join_all(request.map(|(node, input)| async move {
let client = pool.get(node).await?;
f(client, input).await
}))
.await?)
let iters = request.map(|(node, input)| async move {
let client = pool.get(node).await.map_err(|e| (node.id, e))?;
f(client, input).await.map_err(|e| (node.id, e))
});

let (results_ok, results_err) =
try_join_all_with_error_timeout(iters, Duration::from_secs(3)).await;
if results_err.is_empty() {
return Ok(results_ok);
}
let merged_error = merge_node_rpc_errors("merged RPC Error", results_err);
Copy link
Member

Choose a reason for hiding this comment

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

The message does not look that informative.

Err(merged_error)
}

async fn broadcast<RSP, Fut: Future<Output = Result<RSP, RpcError>> + 'static>(
Expand Down Expand Up @@ -391,3 +399,67 @@ impl StreamRpcManager {
Ok(())
}
}

/// This function is similar to `try_join_all`, but it attempts to collect many error as possible within `error_timeout`.
async fn try_join_all_with_error_timeout<I, RSP, E, F>(
zwang28 marked this conversation as resolved.
Show resolved Hide resolved
iters: I,
error_timeout: Duration,
) -> (Vec<RSP>, Vec<E>)
where
I: IntoIterator<Item = F>,
F: Future<Output = Result<RSP, E>>,
{
let stream = FuturesUnordered::from_iter(iters);
pin_mut!(stream);
let (tx, mut rx) = tokio::sync::mpsc::unbounded_channel();
zwang28 marked this conversation as resolved.
Show resolved Hide resolved
let mut results_ok = vec![];
let mut results_err = vec![];
let mut is_err_timeout = false;
loop {
let rx = rx.recv();
pin_mut!(rx);
match select(rx, stream.next()).await {
Either::Left((_, _)) => {
// error_timeout
break;
}
Either::Right((None, _)) => {
break;
}
Either::Right((Some(Ok(rsp)), _)) => {
results_ok.push(rsp);
}
Either::Right((Some(Err(err)), _)) => {
results_err.push(err);
if is_err_timeout {
continue;
}
// Start `error_timeout` when the first error occurs.
is_err_timeout = true;
let tx = tx.clone();
tokio::spawn(async move {
tokio::time::sleep(error_timeout).await;
let _ = tx.send(());
});
}
}
}
(results_ok, results_err)
}

fn merge_node_rpc_errors(
message: &str,
errors: impl IntoIterator<Item = (WorkerId, RpcError)>,
) -> MetaError {
use std::fmt::Write;

use thiserror_ext::AsReport;

let concat: String = errors
.into_iter()
.fold(format!("{message}:"), |mut s, (w, e)| {
write!(&mut s, " worker node {}, {};", w, e.as_report()).unwrap();
s
});
anyhow::anyhow!(concat).into()
}
Loading