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): pass streaming error score to meta to locate cluster-level root error #17685

Merged
merged 4 commits into from
Jul 19, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
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 e2e_test/error_ui/simple/recovery.slt
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ with error as (
limit 1
)
select
case when error like '%Actor % exited unexpectedly: Executor error: %Numeric out of range%' then 'ok'
case when error like '%get error from control stream, in worker node %: %Actor % exited unexpectedly: Executor error: %Numeric out of range%' then 'ok'
else error
end as result
from error;
Expand Down
16 changes: 16 additions & 0 deletions src/error/src/tonic.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.

pub mod extra;

use std::borrow::Cow;
use std::error::Error;
use std::sync::Arc;
Expand All @@ -24,13 +26,15 @@ use tonic::metadata::{MetadataMap, MetadataValue};
const ERROR_KEY: &str = "risingwave-error-bin";

/// The service name that the error is from. Used to provide better error message.
// TODO: also make it a field of `Extra`?
type ServiceName = Cow<'static, str>;

/// The error produced by the gRPC server and sent to the client on the wire.
#[derive(Debug, Serialize, Deserialize)]
struct ServerError {
error: serde_error::Error,
service_name: Option<ServiceName>,
extra: extra::Extra,
}

impl std::fmt::Display for ServerError {
Expand All @@ -43,6 +47,10 @@ impl std::error::Error for ServerError {
fn source(&self) -> Option<&(dyn Error + 'static)> {
self.error.source()
}

fn provide<'a>(&'a self, request: &mut std::error::Request<'a>) {
self.extra.provide(request);
}
}

fn to_status<T>(error: &T, code: tonic::Code, service_name: Option<ServiceName>) -> tonic::Status
Expand All @@ -55,6 +63,7 @@ where
let source = ServerError {
error: serde_error::Error::new(error),
service_name,
extra: extra::Extra::new(error),
};
let serialized = bincode::serialize(&source).unwrap();

Expand Down Expand Up @@ -204,6 +213,13 @@ impl std::error::Error for TonicStatusWrapper {
// Delegate to `self.inner` as if we're transparent.
self.inner.source()
}

fn provide<'a>(&'a self, request: &mut std::error::Request<'a>) {
// The source error, typically a `ServerError`, may provide additional information through `extra`.
if let Some(source) = self.source() {
source.provide(request);
}
}
}

#[cfg(test)]
Expand Down
50 changes: 50 additions & 0 deletions src/error/src/tonic/extra.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use serde::{Deserialize, Serialize};

/// The score of the error.
///
/// Currently, it's used to identify the root cause of streaming pipeline failures, i.e., which actor
/// led to the failure.
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Serialize, Deserialize)]
pub struct Score(pub i32);

/// Extra fields in errors that can be passed through the gRPC boundary.
///
/// - Field being set to `None` means it is not available.
/// - To add a new field, also update the `provide` method.
#[derive(Debug, Default, Clone, Serialize, Deserialize)]
pub(super) struct Extra {
pub score: Option<Score>,
}

impl Extra {
/// Create a new [`Extra`] by [requesting](std::error::request_ref) each field from the given error.
pub fn new<T>(error: &T) -> Self
where
T: ?Sized + std::error::Error,
{
Self {
score: std::error::request_value(error),
}
}

/// Provide each field to the given [request](std::error::Request).
pub fn provide<'a>(&'a self, request: &mut std::error::Request<'a>) {
if let Some(score) = self.score {
request.provide_value(score);
}
}
}
45 changes: 41 additions & 4 deletions src/meta/src/barrier/rpc.rs
Original file line number Diff line number Diff line change
Expand Up @@ -536,17 +536,54 @@ where
Err(results_err)
}

fn merge_node_rpc_errors<E: Error>(
fn merge_node_rpc_errors<E: Error + Send + Sync + 'static>(
message: &str,
errors: impl IntoIterator<Item = (WorkerId, E)>,
) -> MetaError {
use std::error::request_value;
use std::fmt::Write;

use risingwave_common::error::tonic::extra::Score;

let errors = errors.into_iter().collect_vec();

if errors.is_empty() {
return anyhow!(message.to_owned()).into();
}

// Create the error from the single error.
let single_error = |(worker_id, e)| {
anyhow::Error::from(e)
.context(format!("{message}, in worker node {worker_id}"))
.into()
};

if errors.len() == 1 {
return single_error(errors.into_iter().next().unwrap());
}

// Find the error with the highest score.
let max_score = errors
.iter()
.filter_map(|(_, e)| request_value::<Score>(e))
.max();

if let Some(max_score) = max_score {
let mut errors = errors;
let max_scored = errors
.extract_if(|(_, e)| request_value::<Score>(e) == Some(max_score))
.next()
.unwrap();

return single_error(max_scored);
}

// The errors do not have scores, so simply concatenate them.
let concat: String = errors
.into_iter()
.fold(format!("{message}:"), |mut s, (w, e)| {
write!(&mut s, " worker node {}, {};", w, e.as_report()).unwrap();
.fold(format!("{message}: "), |mut s, (w, e)| {
write!(&mut s, " in worker node {}, {};", w, e.as_report()).unwrap();
s
});
anyhow::anyhow!(concat).into()
anyhow!(concat).into()
}
11 changes: 9 additions & 2 deletions src/rpc_client/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,14 @@ pub enum RpcError {
TransportError(Box<tonic::transport::Error>),

#[error(transparent)]
GrpcStatus(Box<TonicStatusWrapper>),
GrpcStatus(
#[from]
// Typically it does not have a backtrace,
// but this is to let `thiserror` generate `provide` implementation to make `Extra` work.
// See `risingwave_error::tonic::extra`.
#[backtrace]
Box<TonicStatusWrapper>,
),

#[error(transparent)]
MetaAddressParse(#[from] MetaAddressStrategyParseError),
Expand Down Expand Up @@ -61,7 +68,7 @@ macro_rules! impl_from_status {
$(
#[doc = "Convert a gRPC status from " $service " service into an [`RpcError`]."]
pub fn [<from_ $service _status>](s: tonic::Status) -> Self {
Self::grpc_status(s.with_client_side_service_name(stringify!($service)))
Box::new(s.with_client_side_service_name(stringify!($service))).into()
}
)*
}
Expand Down
146 changes: 89 additions & 57 deletions src/stream/src/task/barrier_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ use futures::stream::{BoxStream, FuturesUnordered};
use futures::StreamExt;
use itertools::Itertools;
use parking_lot::Mutex;
use risingwave_common::error::tonic::extra::Score;
use risingwave_pb::stream_service::barrier_complete_response::{
GroupedSstableInfo, PbCreateMviewProgress,
};
Expand Down Expand Up @@ -369,7 +370,7 @@ pub(super) struct LocalBarrierWorker {
actor_failure_rx: UnboundedReceiver<(ActorId, StreamError)>,

/// Cached result of [`Self::try_find_root_failure`].
cached_root_failure: Option<StreamError>,
cached_root_failure: Option<ScoredStreamError>,
}

impl LocalBarrierWorker {
Expand Down Expand Up @@ -793,7 +794,10 @@ impl LocalBarrierWorker {
/// This is similar to [`Self::notify_actor_failure`], but since there's not always an actor failure,
/// the given `err` will be used if there's no root failure found.
async fn notify_other_failure(&mut self, err: StreamError, message: impl Into<String>) {
let root_err = self.try_find_root_failure().await.unwrap_or(err);
let root_err = self
.try_find_root_failure()
.await
.unwrap_or_else(|| ScoredStreamError::new(err));

self.control_stream_handle.reset_stream_with_err(
anyhow!(root_err)
Expand All @@ -815,18 +819,25 @@ impl LocalBarrierWorker {
/// Collect actor errors for a while and find the one that might be the root cause.
///
/// Returns `None` if there's no actor error received.
async fn try_find_root_failure(&mut self) -> Option<StreamError> {
async fn try_find_root_failure(&mut self) -> Option<ScoredStreamError> {
if self.cached_root_failure.is_some() {
return self.cached_root_failure.clone();
}

// fetch more actor errors within a timeout
let _ = tokio::time::timeout(Duration::from_secs(3), async {
while let Some((actor_id, error)) = self.actor_failure_rx.recv().await {
self.add_failure(actor_id, error);
}
})
.await;
self.cached_root_failure = try_find_root_actor_failure(self.failure_actors.values());

// Find the error with highest score.
self.cached_root_failure = self
.failure_actors
.values()
.map(|e| ScoredStreamError::new(e.clone()))
.max_by_key(|e| e.score);

self.cached_root_failure.clone()
}
Expand Down Expand Up @@ -935,63 +946,84 @@ impl LocalBarrierManager {
}
}

/// Tries to find the root cause of actor failures, based on hard-coded rules.
///
/// Returns `None` if the input is empty.
pub fn try_find_root_actor_failure<'a>(
actor_errors: impl IntoIterator<Item = &'a StreamError>,
) -> Option<StreamError> {
// Explicitly list all error kinds here to notice developers to update this function when
// there are changes in error kinds.

fn stream_executor_error_score(e: &StreamExecutorError) -> i32 {
use crate::executor::error::ErrorKind;
match e.inner() {
// `ChannelClosed` or `ExchangeChannelClosed` is likely to be caused by actor exit
// and not the root cause.
ErrorKind::ChannelClosed(_) | ErrorKind::ExchangeChannelClosed(_) => 1,

// Normal errors.
ErrorKind::Uncategorized(_)
| ErrorKind::Storage(_)
| ErrorKind::ArrayError(_)
| ErrorKind::ExprError(_)
| ErrorKind::SerdeError(_)
| ErrorKind::SinkError(_)
| ErrorKind::RpcError(_)
| ErrorKind::AlignBarrier(_, _)
| ErrorKind::ConnectorError(_)
| ErrorKind::DmlError(_)
| ErrorKind::NotImplemented(_) => 999,
}
/// A [`StreamError`] with a score, used to find the root cause of actor failures.
#[derive(Debug, Clone)]
struct ScoredStreamError {
error: StreamError,
score: Score,
}

impl std::fmt::Display for ScoredStreamError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
self.error.fmt(f)
}
}

fn stream_error_score(e: &StreamError) -> i32 {
use crate::error::ErrorKind;
match e.inner() {
// `UnexpectedExit` wraps the original error. Score on the inner error.
ErrorKind::UnexpectedExit { source, .. } => stream_error_score(source),

// `BarrierSend` is likely to be caused by actor exit and not the root cause.
ErrorKind::BarrierSend { .. } => 1,

// Executor errors first.
ErrorKind::Executor(ee) => 2000 + stream_executor_error_score(ee),

// Then other errors.
ErrorKind::Uncategorized(_)
| ErrorKind::Storage(_)
| ErrorKind::Expression(_)
| ErrorKind::Array(_)
| ErrorKind::Sink(_)
| ErrorKind::Secret(_) => 1000,
}
impl std::error::Error for ScoredStreamError {
fn source(&self) -> Option<&(dyn std::error::Error + 'static)> {
self.error.source()
}

fn provide<'a>(&'a self, request: &mut std::error::Request<'a>) {
self.error.provide(request);
// HIGHLIGHT: Provide the score to make it retrievable from meta service.
request.provide_value(self.score);
}
}

impl ScoredStreamError {
/// Score the given error based on hard-coded rules.
fn new(error: StreamError) -> Self {
// Explicitly list all error kinds here to notice developers to update this function when
// there are changes in error kinds.

fn stream_executor_error_score(e: &StreamExecutorError) -> i32 {
use crate::executor::error::ErrorKind;
match e.inner() {
// `ChannelClosed` or `ExchangeChannelClosed` is likely to be caused by actor exit
// and not the root cause.
ErrorKind::ChannelClosed(_) | ErrorKind::ExchangeChannelClosed(_) => 1,

// Normal errors.
ErrorKind::Uncategorized(_)
| ErrorKind::Storage(_)
| ErrorKind::ArrayError(_)
| ErrorKind::ExprError(_)
| ErrorKind::SerdeError(_)
| ErrorKind::SinkError(_)
| ErrorKind::RpcError(_)
| ErrorKind::AlignBarrier(_, _)
| ErrorKind::ConnectorError(_)
| ErrorKind::DmlError(_)
| ErrorKind::NotImplemented(_) => 999,
}
}

actor_errors
.into_iter()
.max_by_key(|&e| stream_error_score(e))
.cloned()
fn stream_error_score(e: &StreamError) -> i32 {
use crate::error::ErrorKind;
match e.inner() {
// `UnexpectedExit` wraps the original error. Score on the inner error.
ErrorKind::UnexpectedExit { source, .. } => stream_error_score(source),

// `BarrierSend` is likely to be caused by actor exit and not the root cause.
ErrorKind::BarrierSend { .. } => 1,

// Executor errors first.
ErrorKind::Executor(ee) => 2000 + stream_executor_error_score(ee),

// Then other errors.
ErrorKind::Uncategorized(_)
| ErrorKind::Storage(_)
| ErrorKind::Expression(_)
| ErrorKind::Array(_)
| ErrorKind::Sink(_)
| ErrorKind::Secret(_) => 1000,
}
}

let score = Score(stream_error_score(&error));
Self { error, score }
}
}

#[cfg(test)]
Expand Down
Loading