diff --git a/Cargo.lock b/Cargo.lock index b67e3700387de..1e711a211887b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8685,6 +8685,7 @@ dependencies = [ "strum_macros 0.26.1", "task_stats_alloc", "tempfile", + "thiserror-ext", "tikv-jemallocator", "tracing", "vergen", diff --git a/lints/src/format_error.rs b/lints/src/format_error.rs index 0d1df649460e8..8dcbed8cb520d 100644 --- a/lints/src/format_error.rs +++ b/lints/src/format_error.rs @@ -16,7 +16,7 @@ use clippy_utils::diagnostics::span_lint_and_help; use clippy_utils::macros::{ find_format_arg_expr, find_format_args, is_format_macro, macro_backtrace, }; -use clippy_utils::ty::implements_trait; +use clippy_utils::ty::{implements_trait, match_type}; use clippy_utils::{ is_in_cfg_test, is_in_test_function, is_trait_method, match_def_path, match_function_call, }; @@ -64,6 +64,7 @@ const TRACING_FIELD_DEBUG: [&str; 3] = ["tracing_core", "field", "debug"]; const TRACING_FIELD_DISPLAY: [&str; 3] = ["tracing_core", "field", "display"]; const TRACING_MACROS_EVENT: [&str; 3] = ["tracing", "macros", "event"]; const ANYHOW_MACROS_ANYHOW: [&str; 3] = ["anyhow", "macros", "anyhow"]; +const ANYHOW_ERROR: [&str; 2] = ["anyhow", "Error"]; impl<'tcx> LateLintPass<'tcx> for FormatError { fn check_expr(&mut self, cx: &LateContext<'tcx>, expr: &'tcx Expr<'_>) { @@ -143,7 +144,10 @@ fn check_fmt_arg_in_anyhow_error(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { check_fmt_arg_with_help( cx, arg_expr, - "consider directly wrapping the error with `anyhow::anyhow!(..)` instead of formatting it", + ( + "consider directly wrapping the error with `anyhow::anyhow!(..)` instead of formatting it", + "consider removing the redundant wrapping of `anyhow::anyhow!(..)`", + ), ); } @@ -151,12 +155,16 @@ fn check_fmt_arg_in_anyhow_context(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { check_fmt_arg_with_help( cx, arg_expr, - "consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to \ + ( + "consider using `anyhow::Context::(with_)context` to \ attach additional message to the error and make it an error source instead", + "consider using `.context(..)` to \ + attach additional message to the error and make it an error source instead", + ), ); } -fn check_fmt_arg_with_help(cx: &LateContext<'_>, arg_expr: &Expr<'_>, help: &str) { +fn check_fmt_arg_with_help(cx: &LateContext<'_>, arg_expr: &Expr<'_>, help: impl Help) { check_arg(cx, arg_expr, arg_expr.span, help); } @@ -169,27 +177,56 @@ fn check_to_string_call(cx: &LateContext<'_>, receiver: &Expr<'_>, to_string_spa ); } -fn check_arg(cx: &LateContext<'_>, arg_expr: &Expr<'_>, span: Span, help: &str) { +fn check_arg(cx: &LateContext<'_>, arg_expr: &Expr<'_>, span: Span, help: impl Help) { let Some(error_trait_id) = cx.tcx.get_diagnostic_item(sym::Error) else { return; }; let ty = cx.typeck_results().expr_ty(arg_expr).peel_refs(); - if implements_trait(cx, ty, error_trait_id, &[]) { - if let Some(span) = core::iter::successors(Some(span), |s| s.parent_callsite()) - .find(|s| s.can_be_used_for_suggestions()) - { - // TODO: applicable suggestions - span_lint_and_help( - cx, - FORMAT_ERROR, - span, - "should not format error directly", - None, - help, - ); - } + let help = if implements_trait(cx, ty, error_trait_id, &[]) { + help.normal_help() + } else if match_type(cx, ty, &ANYHOW_ERROR) { + help.anyhow_help() + } else { + return; + }; + + if let Some(span) = core::iter::successors(Some(span), |s| s.parent_callsite()) + .find(|s| s.can_be_used_for_suggestions()) + { + // TODO: applicable suggestions + span_lint_and_help( + cx, + FORMAT_ERROR, + span, + "should not format error directly", + None, + help, + ); + } +} + +trait Help { + fn normal_help(&self) -> &str; + fn anyhow_help(&self) -> &str { + self.normal_help() + } +} + +impl Help for &str { + fn normal_help(&self) -> &str { + self + } +} + +impl Help for (&str, &str) { + fn normal_help(&self) -> &str { + self.0 + } + + fn anyhow_help(&self) -> &str { + self.1 } } diff --git a/lints/src/lib.rs b/lints/src/lib.rs index d2c78515272f4..df77538d3cf17 100644 --- a/lints/src/lib.rs +++ b/lints/src/lib.rs @@ -14,6 +14,7 @@ #![feature(rustc_private)] #![feature(let_chains)] +#![feature(lazy_cell)] #![warn(unused_extern_crates)] extern crate rustc_ast; diff --git a/lints/ui/format_error.rs b/lints/ui/format_error.rs index eeead1306ea3f..0e46c72766157 100644 --- a/lints/ui/format_error.rs +++ b/lints/ui/format_error.rs @@ -55,4 +55,22 @@ fn main() { let _ = anyhow!("{:?}", err); let _ = anyhow!("some error occurred: {}", err); let _ = anyhow!("some error occurred: {:?}", err); + + // `anyhow::Error` does not implement `Error` trait, test the special path here. + let make_anyhow_err = || anyhow!("foobar"); + let anyhow_err = make_anyhow_err(); + + let _ = format!("{}", anyhow_err); + let _ = format!("{}", &anyhow_err); + let _ = format!("{}", &&anyhow_err); + let _ = format!("{}", Box::new(&anyhow_err)); // TODO: fail to lint + + tracing::field::display(&anyhow_err); + tracing::field::debug(make_anyhow_err()); + + let _ = anyhow_err.to_string(); + let _ = (&&anyhow_err).to_string(); + + let _ = anyhow!("{}", anyhow_err); + let _ = anyhow!("some error occurred: {:?}", anyhow_err); } diff --git a/lints/ui/format_error.stderr b/lints/ui/format_error.stderr index 8ec6e69b7fcf4..0eb4786380a79 100644 --- a/lints/ui/format_error.stderr +++ b/lints/ui/format_error.stderr @@ -262,7 +262,7 @@ error: should not format error directly LL | let _ = anyhow!("some error occurred: {}", err); | ^^^ | - = help: consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead + = help: consider using `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead error: should not format error directly --> $DIR/format_error.rs:57:50 @@ -270,7 +270,79 @@ error: should not format error directly LL | let _ = anyhow!("some error occurred: {:?}", err); | ^^^ | - = help: consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead + = help: consider using `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead -error: aborting due to 34 previous errors +error: should not format error directly + --> $DIR/format_error.rs:63:27 + | +LL | let _ = format!("{}", anyhow_err); + | ^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:64:27 + | +LL | let _ = format!("{}", &anyhow_err); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:65:27 + | +LL | let _ = format!("{}", &&anyhow_err); + | ^^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:68:29 + | +LL | tracing::field::display(&anyhow_err); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:69:27 + | +LL | tracing::field::debug(make_anyhow_err()); + | ^^^^^^^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:71:24 + | +LL | let _ = anyhow_err.to_string(); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.to_report_string()` instead + +error: should not format error directly + --> $DIR/format_error.rs:72:28 + | +LL | let _ = (&&anyhow_err).to_string(); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.to_report_string()` instead + +error: should not format error directly + --> $DIR/format_error.rs:74:27 + | +LL | let _ = anyhow!("{}", anyhow_err); + | ^^^^^^^^^^ + | + = help: consider removing the redundant wrapping of `anyhow::anyhow!(..)` + +error: should not format error directly + --> $DIR/format_error.rs:75:50 + | +LL | let _ = anyhow!("some error occurred: {:?}", anyhow_err); + | ^^^^^^^^^^ + | + = help: consider using `.context(..)` to attach additional message to the error and make it an error source instead + +error: aborting due to 43 previous errors diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index ce110c9effc17..13d5fcad5ec8c 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::v2::AsReport as _; use risingwave_compactor::CompactorOpts; use risingwave_compute::ComputeNodeOpts; use risingwave_ctl::CliOpts as CtlOpts; @@ -67,13 +68,12 @@ pub fn ctl(opts: CtlOpts) { // Note: Use a simple current thread runtime for ctl. // When there's a heavy workload, multiple thread runtime seems to respond slowly. May need // further investigation. - tokio::runtime::Builder::new_current_thread() + if let Err(e) = tokio::runtime::Builder::new_current_thread() .enable_all() .build() .unwrap() .block_on(risingwave_ctl::start(opts)) - .inspect_err(|e| { - eprintln!("{:#?}", e); - }) - .unwrap(); + { + eprintln!("Error: {:#?}", e.as_report()); + } } diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index bb57fbfe88a09..c5f193ef8a2a3 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -58,6 +58,7 @@ workspace-hack = { path = "../workspace-hack" } expect-test = "1" [build-dependencies] +thiserror-ext = { workspace = true } vergen = { version = "8", default-features = false, features = [ "build", "git", diff --git a/src/cmd_all/build.rs b/src/cmd_all/build.rs index a4a7c27e65685..38d9f2d7107a6 100644 --- a/src/cmd_all/build.rs +++ b/src/cmd_all/build.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use thiserror_ext::AsReport; use vergen::EmitBuilder; fn main() { if let Err(e) = EmitBuilder::builder().git_sha(true).fail_on_error().emit() { // Leave the environment variable unset if error occurs. - println!("cargo:warning={}", e) + println!("cargo:warning={}", e.as_report()) } } diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 971fb28d208c2..b1415a00b1362 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -391,9 +391,9 @@ impl<'de> Deserialize<'de> for DefaultParallelism { VirtualNode::COUNT )))? } else { - NonZeroUsize::new(i) - .context("default parallelism should be greater than 0") - .map_err(|e| serde::de::Error::custom(e.to_string()))? + NonZeroUsize::new(i).ok_or_else(|| { + serde::de::Error::custom("default parallelism should be greater than 0") + })? })), } } diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 2bc4160e7a263..fc6afc379eb76 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -107,12 +107,9 @@ impl Sink for NatsSink { "Nats sink only support append-only mode" ))); } - let _client = self - .config - .common - .build_client() - .await - .context("validate nats sink error")?; + let _client = (self.config.common.build_client().await) + .context("validate nats sink error") + .map_err(SinkError::Nats)?; Ok(()) } diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 43753dad599c7..902f113526a2e 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -14,7 +14,7 @@ use std::str::FromStr; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use async_trait::async_trait; use futures_async_stream::try_stream; use itertools::Itertools; @@ -79,8 +79,8 @@ impl SplitReader for CdcSplitReader { if matches!(T::source_type(), CdcSourceType::Citus) && let Some(server_addr) = split.server_addr() { - let host_addr = HostAddr::from_str(&server_addr) - .map_err(|err| anyhow!("invalid server address for cdc split. {}", err))?; + let host_addr = + HostAddr::from_str(&server_addr).context("invalid server address for cdc split")?; properties.insert("hostname".to_string(), host_addr.host); properties.insert("port".to_string(), host_addr.port.to_string()); // rewrite table name with suffix to capture all shards in the split @@ -218,7 +218,7 @@ impl CommonSplitReader for CdcSplitReader { GLOBAL_ERROR_METRICS.cdc_source_error.report([ source_type.as_str_name().into(), source_id.clone(), - e.to_string(), + e.to_report_string(), ]); Err(e)?; } diff --git a/src/connector/src/source/kafka/enumerator/client.rs b/src/connector/src/source/kafka/enumerator/client.rs index 16314d21dbc1e..4441be8c9db21 100644 --- a/src/connector/src/source/kafka/enumerator/client.rs +++ b/src/connector/src/source/kafka/enumerator/client.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::time::Duration; -use anyhow::{anyhow, Context as _}; +use anyhow::{anyhow, Context}; use async_trait::async_trait; use rdkafka::consumer::{BaseConsumer, Consumer}; use rdkafka::error::KafkaResult; @@ -112,6 +112,7 @@ impl SplitEnumerator for KafkaSplitEnumerator { self.broker_address ) })?; + let watermarks = self.get_watermarks(topic_partitions.as_ref()).await?; let mut start_offsets = self .fetch_start_offset(topic_partitions.as_ref(), &watermarks) diff --git a/src/error/src/lib.rs b/src/error/src/lib.rs index d3364485e8f2f..f7a2611b84a65 100644 --- a/src/error/src/lib.rs +++ b/src/error/src/lib.rs @@ -23,3 +23,6 @@ pub mod anyhow; pub mod tonic; + +// Re-export the `thiserror-ext` crate. +pub use thiserror_ext::*; diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index f7ce8cc04a950..1578fa5cd0dfe 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -944,7 +944,7 @@ pub async fn run_test_file(file_path: &Path, file_content: &str) -> Result<()> { "Test #{i} (id: {}) failed, SQL:\n{}\nError: {}", c.id().clone().unwrap_or_else(|| "".to_string()), c.sql(), - e + e.as_report() ); failed_num += 1; } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index bbb2d93b21790..c8cfd938c23a2 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -62,6 +62,7 @@ use risingwave_sqlparser::ast::{ ProtobufSchema, SourceWatermark, }; use risingwave_sqlparser::parser::IncludeOption; +use thiserror_ext::AsReport; use super::RwPgResponse; use crate::binder::Binder; @@ -1081,7 +1082,7 @@ pub(super) async fn check_source_schema( } else if connector == ICEBERG_CONNECTOR { Ok(check_iceberg_source(props, columns) .await - .map_err(|err| ProtocolError(err.to_string()))?) + .map_err(|err| ProtocolError(err.to_report_string()))?) } else { Ok(()) } diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 814e14d42af9e..b5104e557a1b2 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -30,6 +30,7 @@ use axum::Router; use hyper::Request; use parking_lot::Mutex; use risingwave_rpc_client::ComputeClientPool; +use thiserror_ext::AsReport; use tower::{ServiceBuilder, ServiceExt}; use tower_http::add_extension::AddExtensionLayer; use tower_http::cors::{self, CorsLayer}; @@ -455,7 +456,7 @@ impl DashboardService { proxy::proxy(req, cache).await.or_else(|err| { Ok(( StatusCode::INTERNAL_SERVER_ERROR, - format!("Unhandled internal error: {}", err), + err.context("Unhandled internal error").to_report_string(), ) .into_response()) }) diff --git a/src/meta/src/manager/sink_coordination/coordinator_worker.rs b/src/meta/src/manager/sink_coordination/coordinator_worker.rs index bebef2d307dcc..e1c096aa3cf98 100644 --- a/src/meta/src/manager/sink_coordination/coordinator_worker.rs +++ b/src/meta/src/manager/sink_coordination/coordinator_worker.rs @@ -110,7 +110,7 @@ impl CoordinatorWorker { .wait_for_writers(first_writer_request.vnode_bitmap) .await { - error!("failed to wait for all writers: {:?}", e); + error!(error = %e.as_report(), "failed to wait for all writers"); worker .send_to_all_sink_writers(|| { Err(Status::cancelled("failed to wait for all writers")) diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index ea9099eae319c..9723ee89fbd51 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -31,6 +31,7 @@ use risedev::{ RISEDEV_SESSION_NAME, }; use tempfile::tempdir; +use thiserror_ext::AsReport; use yaml_rust::YamlEmitter; #[derive(Default)] @@ -444,9 +445,9 @@ fn main() -> Result<()> { } Err(err) => { println!( - "{} - Failed to start: {:?}", // with `Caused by` + "{} - Failed to start: {:#}", // pretty with `Caused by` style("ERROR").red().bold(), - err, + err.as_report(), ); println!(); println!( diff --git a/src/risedevtool/src/preflight_check.rs b/src/risedevtool/src/preflight_check.rs index 17dc48884fea8..9b25d39423566 100644 --- a/src/risedevtool/src/preflight_check.rs +++ b/src/risedevtool/src/preflight_check.rs @@ -17,6 +17,7 @@ use std::process::Command; use anyhow::Result; use console::style; +use thiserror_ext::AsReport; fn preflight_check_proxy() -> Result<()> { if env::var("http_proxy").is_ok() @@ -72,7 +73,7 @@ pub fn preflight_check() -> Result<()> { "[{}] {} - failed to run proxy preflight check: {}", style("risedev-preflight-check").bold(), style("WARN").yellow().bold(), - e + e.as_report() ); } @@ -81,7 +82,7 @@ pub fn preflight_check() -> Result<()> { "[{}] {} - failed to run ulimit preflight check: {}", style("risedev-preflight-check").bold(), style("WARN").yellow().bold(), - e + e.as_report() ); } diff --git a/src/tests/regress/src/lib.rs b/src/tests/regress/src/lib.rs index efdd0f1422c00..a18afdebc843f 100644 --- a/src/tests/regress/src/lib.rs +++ b/src/tests/regress/src/lib.rs @@ -27,6 +27,9 @@ #![deny(rustdoc::broken_intra_doc_links)] #![feature(path_file_prefix)] #![feature(let_chains)] +#![feature(register_tool)] +#![register_tool(rw)] +#![allow(rw::format_error)] mod opts; diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 97c08d098f6c9..27cf4985dc4fd 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -15,6 +15,7 @@ use std::time::Duration; use anyhow::{anyhow, Result}; +use risingwave_common::error::v2::AsReport; use risingwave_simulation::cluster::{Cluster, Configuration, Session}; use tokio::time::sleep; @@ -348,7 +349,7 @@ async fn test_high_barrier_latency_cancel(config: Configuration) -> Result<()> { .run("CREATE MATERIALIZED VIEW mv1 as values(1)") .await { - tracing::info!("Recreate mv failed with {e:?}"); + tracing::info!(error = %e.as_report(), "Recreate mv failed"); continue; } else { tracing::info!("recreated mv");