Skip to content

Commit

Permalink
feat(error): preserve error's source chain across gRPC boundary (#13282)
Browse files Browse the repository at this point in the history
Signed-off-by: Bugen Zhao <[email protected]>
  • Loading branch information
BugenZhao authored Nov 10, 2023
1 parent 1abe9e8 commit 8004505
Show file tree
Hide file tree
Showing 21 changed files with 353 additions and 63 deletions.
28 changes: 28 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ members = [
"src/compute",
"src/connector",
"src/ctl",
"src/error",
"src/expr/core",
"src/expr/impl",
"src/expr/macro",
Expand Down Expand Up @@ -138,6 +139,7 @@ risingwave_compactor = { path = "./src/storage/compactor" }
risingwave_compute = { path = "./src/compute" }
risingwave_ctl = { path = "./src/ctl" }
risingwave_connector = { path = "./src/connector" }
risingwave_error = { path = "./src/error" }
risingwave_expr = { path = "./src/expr/core" }
risingwave_expr_impl = { path = "./src/expr/impl" }
risingwave_frontend = { path = "./src/frontend" }
Expand Down
5 changes: 3 additions & 2 deletions e2e_test/error_ui/main.slt
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,9 @@ alter system set not_exist_key to value;
----
db error: ERROR: QueryError

Caused by this error:
1: internal error: SystemParams error: unrecognized system param "not_exist_key"
Caused by these errors (recent errors listed first):
1: gRPC request to meta service failed: Internal error
2: SystemParams error: unrecognized system param "not_exist_key"


query error
Expand Down
3 changes: 2 additions & 1 deletion src/batch/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ use std::sync::Arc;
pub use anyhow::anyhow;
use risingwave_common::array::ArrayError;
use risingwave_common::error::{ErrorCode, RwError};
use risingwave_rpc_client::error::ToTonicStatus;
use thiserror::Error;
use tonic::Status;

Expand Down Expand Up @@ -79,6 +80,6 @@ impl From<RwError> for BatchError {

impl<'a> From<&'a BatchError> for Status {
fn from(err: &'a BatchError) -> Self {
Status::internal(err.to_string())
err.to_status(tonic::Code::Internal, "batch")
}
}
1 change: 1 addition & 0 deletions src/common/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ rand = "0.8"
regex = "1"
reqwest = { version = "0.11", features = ["json"] }
risingwave_common_proc_macro = { path = "./proc_macro" }
risingwave_error = { workspace = true }
risingwave_pb = { workspace = true }
rust_decimal = { version = "1", features = ["db-postgres", "maths"] }
ryu = "1.0"
Expand Down
66 changes: 38 additions & 28 deletions src/common/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,17 +20,14 @@ use std::io::Error as IoError;
use std::time::{Duration, SystemTime};

use memcomparable::Error as MemComparableError;
use risingwave_error::tonic::{ToTonicStatus, TonicStatusWrapper};
use risingwave_pb::PbFieldNotFound;
use thiserror::Error;
use tokio::task::JoinError;
use tonic::Code;

use crate::array::ArrayError;
use crate::util::value_encoding::error::ValueEncodingError;

/// Header used to store serialized [`RwError`] in grpc status.
pub const RW_ERROR_GRPC_HEADER: &str = "risingwave-error-bin";

const ERROR_SUPPRESSOR_RESET_DURATION: Duration = Duration::from_millis(60 * 60 * 1000); // 1h

pub trait Error = std::error::Error + Send + Sync + 'static;
Expand Down Expand Up @@ -126,10 +123,10 @@ pub enum ErrorCode {
#[source]
BoxedError,
),
#[error("RPC error: {0}")]
#[error(transparent)]
RpcError(
#[source]
#[backtrace]
// #[backtrace] // TODO(error-handling): there's a limitation that `#[transparent]` can't be used with `#[backtrace]` if no `#[from]`
// `tonic::transport::Error`, `TonicStatusWrapper`, or `RpcError`
BoxedError,
),
#[error("Bind error: {0}")]
Expand Down Expand Up @@ -195,12 +192,41 @@ pub struct RwError {

impl From<RwError> for tonic::Status {
fn from(err: RwError) -> Self {
match &*err.inner {
ErrorCode::ExprError(e) => tonic::Status::invalid_argument(e.to_string()),
ErrorCode::PermissionDenied(e) => tonic::Status::permission_denied(e),
ErrorCode::InternalError(e) => tonic::Status::internal(e),
_ => tonic::Status::internal(err.to_string()),
use tonic::Code;

let code = match &*err.inner {
ErrorCode::ExprError(_) => Code::InvalidArgument,
ErrorCode::PermissionDenied(_) => Code::PermissionDenied,
ErrorCode::InternalError(_) => Code::Internal,
_ => Code::Internal,
};

err.to_status_unnamed(code)
}
}

impl From<TonicStatusWrapper> for RwError {
fn from(status: TonicStatusWrapper) -> Self {
use tonic::Code;

let message = status.inner().message();

// TODO(error-handling): `message` loses the source chain.
match status.inner().code() {
Code::InvalidArgument => ErrorCode::InvalidParameterValue(message.to_string()),
Code::NotFound | Code::AlreadyExists => ErrorCode::CatalogError(status.into()),
Code::PermissionDenied => ErrorCode::PermissionDenied(message.to_string()),
Code::Cancelled => ErrorCode::SchedulerError(status.into()),
_ => ErrorCode::RpcError(status.into()),
}
.into()
}
}

impl From<tonic::Status> for RwError {
fn from(status: tonic::Status) -> Self {
// Always wrap the status.
Self::from(TonicStatusWrapper::new(status))
}
}

Expand Down Expand Up @@ -292,22 +318,6 @@ impl From<PbFieldNotFound> for RwError {
}
}

impl From<tonic::Status> for RwError {
fn from(err: tonic::Status) -> Self {
match err.code() {
Code::InvalidArgument => {
ErrorCode::InvalidParameterValue(err.message().to_string()).into()
}
Code::NotFound | Code::AlreadyExists => {
ErrorCode::CatalogError(err.message().to_string().into()).into()
}
Code::PermissionDenied => ErrorCode::PermissionDenied(err.message().to_string()).into(),
Code::Cancelled => ErrorCode::SchedulerError(err.message().to_string().into()).into(),
_ => ErrorCode::InternalError(err.message().to_string()).into(),
}
}
}

impl From<tonic::transport::Error> for RwError {
fn from(err: tonic::transport::Error) -> Self {
ErrorCode::RpcError(err.into()).into()
Expand Down
22 changes: 22 additions & 0 deletions src/error/Cargo.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
[package]
name = "risingwave_error"
version = { workspace = true }
edition = { workspace = true }
homepage = { workspace = true }
keywords = { workspace = true }
license = { workspace = true }
repository = { workspace = true }

[dependencies]
bincode = "1"
bytes = "1"
easy-ext = "1"
serde = "1"
serde-error = "0.1"
thiserror = "1"
thiserror-ext = { workspace = true }
tonic = { workspace = true }
tracing = "0.1"

[lints]
workspace = true
21 changes: 21 additions & 0 deletions src/error/src/lib.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
// Copyright 2023 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.

//! Error handling utilities.
//!
//! This will eventually replace the `RwError` in `risingwave_common`.
#![feature(error_generic_member_access)]

pub mod tonic;
Loading

0 comments on commit 8004505

Please sign in to comment.