From 2157ed6c8b008dc5b88b102dfa00ff66b466c97a Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 27 Feb 2024 16:31:27 +0800 Subject: [PATCH] refactor(error): eliminate usage of `DisplayErrorContext` (#15225) Signed-off-by: Bugen Zhao --- clippy.toml | 1 + src/connector/src/sink/kinesis.rs | 23 ++++--------------- .../src/source/filesystem/s3_v2/lister.rs | 5 ++-- .../src/source/kinesis/source/reader.rs | 6 ++--- src/meta/src/error.rs | 3 +-- src/object_store/src/object/error.rs | 3 +-- 6 files changed, 13 insertions(+), 28 deletions(-) diff --git a/clippy.toml b/clippy.toml index bcc3c789ae35a..bbda73f322593 100644 --- a/clippy.toml +++ b/clippy.toml @@ -19,6 +19,7 @@ disallowed-types = [ { path = "num_traits::FromPrimitive", reason = "Please use `From` or `TryFrom` with `OrderedFloat` instead." }, { path = "num_traits::ToPrimitive", reason = "Please use `From` or `TryFrom` with `OrderedFloat` instead." }, { path = "num_traits::NumCast", reason = "Please use `From` or `TryFrom` with `OrderedFloat` instead." }, + { path = "aws_smithy_types::error::display::DisplayErrorContext", reason = "Please use `thiserror_ext::AsReport` instead." }, ] disallowed-macros = [ { path = "lazy_static::lazy_static", reason = "Please use `std::sync::LazyLock` instead." }, diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index 54e9394b511db..04127a373dd28 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -14,8 +14,7 @@ use std::collections::HashMap; -use anyhow::anyhow; -use aws_sdk_kinesis::error::DisplayErrorContext; +use anyhow::{anyhow, Context}; use aws_sdk_kinesis::operation::put_record::PutRecordOutput; use aws_sdk_kinesis::primitives::Blob; use aws_sdk_kinesis::Client as KinesisClient; @@ -106,10 +105,8 @@ impl Sink for KinesisSink { .stream_name(&self.config.common.stream_name) .send() .await - .map_err(|e| { - tracing::warn!("failed to list shards: {}", DisplayErrorContext(&e)); - SinkError::Kinesis(anyhow!("failed to list shards: {}", DisplayErrorContext(e))) - })?; + .context("failed to list shards") + .map_err(SinkError::Kinesis)?; Ok(()) } @@ -201,18 +198,8 @@ impl KinesisSinkPayloadWriter { }, ) .await - .map_err(|e| { - tracing::warn!( - "failed to put record: {} to {}", - DisplayErrorContext(&e), - self.config.common.stream_name - ); - SinkError::Kinesis(anyhow!( - "failed to put record: {} to {}", - DisplayErrorContext(e), - self.config.common.stream_name - )) - }) + .with_context(|| format!("failed to put record to {}", self.config.common.stream_name)) + .map_err(SinkError::Kinesis) } } diff --git a/src/connector/src/source/filesystem/s3_v2/lister.rs b/src/connector/src/source/filesystem/s3_v2/lister.rs index ccc40f1ccef80..03a6c0878de43 100644 --- a/src/connector/src/source/filesystem/s3_v2/lister.rs +++ b/src/connector/src/source/filesystem/s3_v2/lister.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; +use anyhow::Context; use async_trait::async_trait; -use aws_sdk_s3::error::DisplayErrorContext; use aws_sdk_s3::types::Object; use itertools::Itertools; @@ -39,7 +38,7 @@ impl FsListInner for S3SplitEnumerator { let mut res = req .send() .await - .map_err(|e| anyhow!(DisplayErrorContext(e)))?; + .with_context(|| format!("failed to list objects in bucket `{}`", self.bucket_name))?; if res.is_truncated().unwrap_or_default() { self.next_continuation_token = res.next_continuation_token.clone(); } else { diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 363dfb8777b12..8b8b2f104cec2 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -14,9 +14,9 @@ use std::time::Duration; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use async_trait::async_trait; -use aws_sdk_kinesis::error::{DisplayErrorContext, SdkError}; +use aws_sdk_kinesis::error::SdkError; use aws_sdk_kinesis::operation::get_records::{GetRecordsError, GetRecordsOutput}; use aws_sdk_kinesis::primitives::DateTime; use aws_sdk_kinesis::types::ShardIteratorType; @@ -243,7 +243,7 @@ impl KinesisSplitReader { .set_timestamp(starting_timestamp) .send() .await - .map_err(|e| anyhow!(DisplayErrorContext(e)))?; + .context("failed to get kinesis shard iterator")?; if let Some(iter) = resp.shard_iterator() { Ok(iter.to_owned()) diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 18230bf74c213..3583e89934548 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use aws_sdk_ec2::error::DisplayErrorContext; use risingwave_common::error::BoxedError; use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; @@ -103,7 +102,7 @@ pub enum MetaErrorInner { SinkError, ), - #[error("AWS SDK error: {}", DisplayErrorContext(& * *.0))] + #[error("AWS SDK error: {0}")] Aws(#[source] BoxedError), #[error(transparent)] diff --git a/src/object_store/src/object/error.rs b/src/object_store/src/object/error.rs index d4927a1595fca..f13ae2b4d8f51 100644 --- a/src/object_store/src/object/error.rs +++ b/src/object_store/src/object/error.rs @@ -15,7 +15,6 @@ use std::io; use std::marker::{Send, Sync}; -use aws_sdk_s3::error::DisplayErrorContext; use aws_sdk_s3::operation::get_object::GetObjectError; use aws_sdk_s3::operation::head_object::HeadObjectError; use aws_sdk_s3::primitives::ByteStreamError; @@ -28,7 +27,7 @@ use tokio::sync::oneshot::error::RecvError; #[derive(Error, Debug, thiserror_ext::Box, thiserror_ext::Construct)] #[thiserror_ext(newtype(name = ObjectError, backtrace, report_debug))] pub enum ObjectErrorInner { - #[error("s3 error: {}", DisplayErrorContext(&**.0))] + #[error("s3 error: {0}")] S3(#[source] BoxedError), #[error("disk error: {msg}")] Disk {