diff --git a/e2e_test/source/basic/ddl.slt b/e2e_test/source/basic/ddl.slt index 6e640e047d4c..402cf129b86b 100644 --- a/e2e_test/source/basic/ddl.slt +++ b/e2e_test/source/basic/ddl.slt @@ -28,7 +28,8 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: gRPC request to meta service failed: Internal error 2: failed to create source worker - 3: missing field `properties.bootstrap.server` + 3: failed to parse json + 4: missing field `properties.bootstrap.server` statement error diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index 5631707e2f42..f4d7341cbc6d 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -20,6 +20,7 @@ pub use anyhow::anyhow; use risingwave_common::array::ArrayError; use risingwave_common::error::BoxedError; use risingwave_common::util::value_encoding::error::ValueEncodingError; +use risingwave_connector::error::ConnectorError; use risingwave_dml::error::DmlError; use risingwave_expr::ExprError; use risingwave_pb::PbFieldNotFound; @@ -156,3 +157,9 @@ impl From for Status { Self::from(&err) } } + +impl From for BatchError { + fn from(value: ConnectorError) -> Self { + Self::Connector(value.into()) + } +} diff --git a/src/batch/src/executor/iceberg_scan.rs b/src/batch/src/executor/iceberg_scan.rs index caf1289220d4..9c24e554c8e1 100644 --- a/src/batch/src/executor/iceberg_scan.rs +++ b/src/batch/src/executor/iceberg_scan.rs @@ -116,11 +116,7 @@ impl IcebergScanExecutor { #[try_stream(ok = DataChunk, error = BatchError)] async fn do_execute(self: Box) { - let table = self - .iceberg_config - .load_table() - .await - .map_err(BatchError::Internal)?; + let table = self.iceberg_config.load_table().await?; let table_scan: TableScan = table .new_scan_builder() diff --git a/src/common/src/field_generator/mod.rs b/src/common/src/field_generator/mod.rs index 030979806885..679d60ba1f18 100644 --- a/src/common/src/field_generator/mod.rs +++ b/src/common/src/field_generator/mod.rs @@ -18,6 +18,7 @@ mod varchar; use std::time::Duration; +// TODO(error-handling): use a new error type use anyhow::{anyhow, Result}; use chrono::{DateTime, FixedOffset}; pub use numeric::*; diff --git a/src/connector/src/aws_utils.rs b/src/connector/src/aws_utils.rs index cf70a90e07cd..1578c7b84442 100644 --- a/src/connector/src/aws_utils.rs +++ b/src/connector/src/aws_utils.rs @@ -21,6 +21,7 @@ use aws_sdk_s3::{client as s3_client, config as s3_config}; use url::Url; use crate::common::AwsAuthProps; +use crate::error::ConnectorResult; const AWS_CUSTOM_CONFIG_KEY: [&str; 3] = ["retry_times", "conn_timeout", "read_timeout"]; @@ -106,7 +107,7 @@ pub fn s3_client( pub async fn load_file_descriptor_from_s3( location: &Url, config: &AwsAuthProps, -) -> anyhow::Result> { +) -> ConnectorResult> { let bucket = location .domain() .with_context(|| format!("illegal file path {}", location))?; diff --git a/src/connector/src/common.rs b/src/connector/src/common.rs index 418155250e74..d5944eb07fa3 100644 --- a/src/connector/src/common.rs +++ b/src/connector/src/common.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::io::Write; use std::time::Duration; -use anyhow::{anyhow, Context, Ok}; +use anyhow::{anyhow, Context}; use async_nats::jetstream::consumer::DeliverPolicy; use async_nats::jetstream::{self}; use aws_sdk_kinesis::Client as KinesisClient; @@ -35,6 +35,7 @@ use with_options::WithOptions; use crate::aws_utils::load_file_descriptor_from_s3; use crate::deserialize_duration_from_string; +use crate::error::ConnectorResult; use crate::sink::SinkError; use crate::source::nats::source::NatsOffset; // The file describes the common abstractions for each connector and can be used in both source and @@ -72,7 +73,7 @@ pub struct AwsAuthProps { } impl AwsAuthProps { - async fn build_region(&self) -> anyhow::Result { + async fn build_region(&self) -> ConnectorResult { if let Some(region_name) = &self.region { Ok(Region::new(region_name.clone())) } else { @@ -85,11 +86,11 @@ impl AwsAuthProps { .build() .region() .await - .ok_or_else(|| anyhow::format_err!("region should be provided"))?) + .context("region should be provided")?) } } - fn build_credential_provider(&self) -> anyhow::Result { + fn build_credential_provider(&self) -> ConnectorResult { if self.access_key.is_some() && self.secret_key.is_some() { Ok(SharedCredentialsProvider::new( aws_credential_types::Credentials::from_keys( @@ -99,16 +100,14 @@ impl AwsAuthProps { ), )) } else { - Err(anyhow!( - "Both \"access_key\" and \"secret_access\" are required." - )) + bail!("Both \"access_key\" and \"secret_access\" are required.") } } async fn with_role_provider( &self, credential: SharedCredentialsProvider, - ) -> anyhow::Result { + ) -> ConnectorResult { if let Some(role_name) = &self.arn { let region = self.build_region().await?; let mut role = AssumeRoleProvider::builder(role_name) @@ -124,7 +123,7 @@ impl AwsAuthProps { } } - pub async fn build_config(&self) -> anyhow::Result { + pub async fn build_config(&self) -> ConnectorResult { let region = self.build_region().await?; let credentials_provider = self .with_role_provider(self.build_credential_provider()?) @@ -386,12 +385,19 @@ pub struct PulsarOauthCommon { pub scope: Option, } +fn create_credential_temp_file(credentials: &[u8]) -> std::io::Result { + let mut f = NamedTempFile::new()?; + f.write_all(credentials)?; + f.as_file().sync_all()?; + Ok(f) +} + impl PulsarCommon { pub(crate) async fn build_client( &self, oauth: &Option, aws_auth_props: &AwsAuthProps, - ) -> anyhow::Result> { + ) -> ConnectorResult> { let mut pulsar_builder = Pulsar::builder(&self.service_url, TokioExecutor); let mut temp_file = None; if let Some(oauth) = oauth.as_ref() { @@ -399,10 +405,10 @@ impl PulsarCommon { match url.scheme() { "s3" => { let credentials = load_file_descriptor_from_s3(&url, aws_auth_props).await?; - let mut f = NamedTempFile::new()?; - f.write_all(&credentials)?; - f.as_file().sync_all()?; - temp_file = Some(f); + temp_file = Some( + create_credential_temp_file(&credentials) + .context("failed to create temp file for pulsar credentials")?, + ); } "file" => {} _ => { @@ -477,7 +483,7 @@ pub struct KinesisCommon { } impl KinesisCommon { - pub(crate) async fn build_client(&self) -> anyhow::Result { + pub(crate) async fn build_client(&self) -> ConnectorResult { let config = AwsAuthProps { region: Some(self.stream_region.clone()), endpoint: self.endpoint.clone(), @@ -539,7 +545,7 @@ pub struct NatsCommon { } impl NatsCommon { - pub(crate) async fn build_client(&self) -> anyhow::Result { + pub(crate) async fn build_client(&self) -> ConnectorResult { let mut connect_options = async_nats::ConnectOptions::new(); match self.connect_mode.as_str() { "user_and_password" => { @@ -582,7 +588,7 @@ impl NatsCommon { Ok(client) } - pub(crate) async fn build_context(&self) -> anyhow::Result { + pub(crate) async fn build_context(&self) -> ConnectorResult { let client = self.build_client().await?; let jetstream = async_nats::jetstream::new(client); Ok(jetstream) @@ -593,7 +599,7 @@ impl NatsCommon { stream: String, split_id: String, start_sequence: NatsOffset, - ) -> anyhow::Result< + ) -> ConnectorResult< async_nats::jetstream::consumer::Consumer, > { let context = self.build_context().await?; @@ -612,13 +618,16 @@ impl NatsCommon { NatsOffset::Earliest => DeliverPolicy::All, NatsOffset::Latest => DeliverPolicy::Last, NatsOffset::SequenceNumber(v) => { - let parsed = v.parse::()?; + let parsed = v + .parse::() + .context("failed to parse nats offset as sequence number")?; DeliverPolicy::ByStartSequence { start_sequence: 1 + parsed, } } NatsOffset::Timestamp(v) => DeliverPolicy::ByStartTime { - start_time: OffsetDateTime::from_unix_timestamp_nanos(v * 1_000_000)?, + start_time: OffsetDateTime::from_unix_timestamp_nanos(v * 1_000_000) + .context("invalid timestamp for nats offset")?, }, NatsOffset::None => DeliverPolicy::All, }; @@ -635,7 +644,7 @@ impl NatsCommon { &self, jetstream: jetstream::Context, stream: String, - ) -> anyhow::Result { + ) -> ConnectorResult { let subjects: Vec = self.subject.split(',').map(|s| s.to_string()).collect(); let mut config = jetstream::stream::Config { name: stream, @@ -662,7 +671,7 @@ impl NatsCommon { Ok(stream) } - pub(crate) fn create_credential(&self, seed: &str, jwt: &str) -> anyhow::Result { + pub(crate) fn create_credential(&self, seed: &str, jwt: &str) -> ConnectorResult { let creds = format!( "-----BEGIN NATS USER JWT-----\n{}\n------END NATS USER JWT------\n\n\ ************************* IMPORTANT *************************\n\ diff --git a/src/connector/src/error.rs b/src/connector/src/error.rs index 4cf36e9859d3..3dc10af3d8e7 100644 --- a/src/connector/src/error.rs +++ b/src/connector/src/error.rs @@ -13,13 +13,57 @@ // limitations under the License. use risingwave_common::error::v2::def_anyhow_newtype; +use risingwave_pb::PbFieldNotFound; +use risingwave_rpc_client::error::RpcError; + +use crate::parser::AccessError; +use crate::schema::schema_registry::{ConcurrentRequestError, WireFormatError}; +use crate::schema::InvalidOptionError; +use crate::sink::SinkError; def_anyhow_newtype! { pub ConnectorError, + // Common errors + std::io::Error => transparent, + + // Fine-grained connector errors + AccessError => transparent, + WireFormatError => transparent, + ConcurrentRequestError => transparent, + InvalidOptionError => transparent, + SinkError => transparent, + PbFieldNotFound => transparent, + // TODO(error-handling): Remove implicit contexts below and specify ad-hoc context for each conversion. + + // Parsing errors + url::ParseError => "failed to parse url", + serde_json::Error => "failed to parse json", + csv::Error => "failed to parse csv", + + // Connector errors + opendal::Error => transparent, // believed to be self-explanatory + mysql_async::Error => "MySQL error", tokio_postgres::Error => "Postgres error", + apache_avro::Error => "Avro error", + rdkafka::error::KafkaError => "Kafka error", + pulsar::Error => "Pulsar error", + async_nats::jetstream::consumer::StreamError => "Nats error", + async_nats::jetstream::consumer::pull::MessagesError => "Nats error", + async_nats::jetstream::context::CreateStreamError => "Nats error", + async_nats::jetstream::stream::ConsumerError => "Nats error", + icelake::Error => "Iceberg error", + redis::RedisError => "Redis error", + arrow_schema::ArrowError => "Arrow error", + google_cloud_pubsub::client::google_cloud_auth::error::Error => "Google Cloud error", } -pub type ConnectorResult = Result; +pub type ConnectorResult = std::result::Result; + +impl From for RpcError { + fn from(value: ConnectorError) -> Self { + RpcError::Internal(value.0) + } +} diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index e34171717ae6..d4a546c6a00a 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -168,12 +168,12 @@ macro_rules! impl_split { $( impl TryFrom for $split { - type Error = anyhow::Error; + type Error = $crate::error::ConnectorError; fn try_from(split: SplitImpl) -> std::result::Result { match split { SplitImpl::$variant_name(inner) => Ok(inner), - other => Err(anyhow::anyhow!("expect {} but get {:?}", stringify!($split), other)) + other => risingwave_common::bail!("expect {} but get {:?}", stringify!($split), other), } } } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index c1da30f788b3..06cc06156669 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -27,6 +27,7 @@ use risingwave_pb::plan_common::{ AdditionalColumnTimestamp, }; +use crate::error::ConnectorResult; use crate::source::{ GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, @@ -86,7 +87,7 @@ pub fn build_additional_column_catalog( inner_field_name: Option<&str>, data_type: Option<&str>, reject_unknown_connector: bool, -) -> anyhow::Result { +) -> ConnectorResult { let compatible_columns = match ( COMPATIBLE_ADDITIONAL_COLUMNS.get(connector_name), reject_unknown_connector, diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 5e876d2ce932..7343f1c43118 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -23,6 +23,7 @@ use risingwave_pb::plan_common::ColumnDesc; use super::schema_resolver::ConfluentSchemaResolver; use super::util::avro_schema_to_column_descs; +use crate::error::ConnectorResult; use crate::parser::unified::avro::{AvroAccess, AvroParseOptions}; use crate::parser::unified::AccessImpl; use crate::parser::util::bytes_from_url; @@ -40,7 +41,7 @@ pub struct AvroAccessBuilder { } impl AccessBuilder for AvroAccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { self.value = self.parse_avro_value(&payload, Some(&*self.schema)).await?; Ok(AccessImpl::Avro(AvroAccess::new( self.value.as_ref().unwrap(), @@ -50,7 +51,7 @@ impl AccessBuilder for AvroAccessBuilder { } impl AvroAccessBuilder { - pub fn new(config: AvroParserConfig, encoding_type: EncodingType) -> anyhow::Result { + pub fn new(config: AvroParserConfig, encoding_type: EncodingType) -> ConnectorResult { let AvroParserConfig { schema, key_schema, @@ -71,7 +72,7 @@ impl AvroAccessBuilder { &self, payload: &[u8], reader_schema: Option<&Schema>, - ) -> anyhow::Result> { + ) -> ConnectorResult> { // parse payload to avro value // if use confluent schema, get writer schema from confluent schema registry if let Some(resolver) = &self.schema_resolver { @@ -87,9 +88,7 @@ impl AvroAccessBuilder { match reader.next() { Some(Ok(v)) => Ok(Some(v)), Some(Err(e)) => Err(e)?, - None => { - anyhow::bail!("avro parse unexpected eof") - } + None => bail!("avro parse unexpected eof"), } } else { unreachable!("both schema_resolver and reader_schema not exist"); @@ -105,7 +104,7 @@ pub struct AvroParserConfig { } impl AvroParserConfig { - pub async fn new(encoding_properties: EncodingProperties) -> anyhow::Result { + pub async fn new(encoding_properties: EncodingProperties) -> ConnectorResult { let avro_config = try_match_expand!(encoding_properties, EncodingProperties::Avro)?; let schema_location = &avro_config.row_schema_location; let enable_upsert = avro_config.enable_upsert; @@ -160,7 +159,7 @@ impl AvroParserConfig { } } - pub fn extract_pks(&self) -> anyhow::Result> { + pub fn extract_pks(&self) -> ConnectorResult> { avro_schema_to_column_descs( self.key_schema .as_deref() @@ -168,7 +167,7 @@ impl AvroParserConfig { ) } - pub fn map_to_columns(&self) -> anyhow::Result> { + pub fn map_to_columns(&self) -> ConnectorResult> { avro_schema_to_column_descs(self.schema.as_ref()) } } @@ -196,6 +195,7 @@ mod test { use super::*; use crate::common::AwsAuthProps; + use crate::error::ConnectorResult; use crate::parser::plain_parser::PlainParser; use crate::parser::unified::avro::unix_epoch_days; use crate::parser::{ @@ -256,7 +256,7 @@ mod test { println!("schema = {:?}", schema.unwrap()); } - async fn new_avro_conf_from_local(file_name: &str) -> anyhow::Result { + async fn new_avro_conf_from_local(file_name: &str) -> ConnectorResult { let schema_path = "file://".to_owned() + &test_data_path(file_name); let info = StreamSourceInfo { row_schema_location: schema_path.clone(), @@ -269,7 +269,7 @@ mod test { AvroParserConfig::new(parser_config.encoding_config).await } - async fn new_avro_parser_from_local(file_name: &str) -> anyhow::Result { + async fn new_avro_parser_from_local(file_name: &str) -> ConnectorResult { let conf = new_avro_conf_from_local(file_name).await?; Ok(PlainParser { diff --git a/src/connector/src/parser/avro/schema_resolver.rs b/src/connector/src/parser/avro/schema_resolver.rs index ef2dd9fc5f73..cdc52de7acce 100644 --- a/src/connector/src/parser/avro/schema_resolver.rs +++ b/src/connector/src/parser/avro/schema_resolver.rs @@ -18,6 +18,7 @@ use anyhow::Context; use apache_avro::Schema; use moka::future::Cache; +use crate::error::ConnectorResult; use crate::schema::schema_registry::{Client, ConfluentSchema}; #[derive(Debug)] @@ -30,7 +31,7 @@ impl ConfluentSchemaResolver { async fn parse_and_cache_schema( &self, raw_schema: ConfluentSchema, - ) -> anyhow::Result> { + ) -> ConnectorResult> { let schema = Schema::parse_str(&raw_schema.content).context("failed to parse avro schema")?; let schema = Arc::new(schema); @@ -48,7 +49,7 @@ impl ConfluentSchemaResolver { } } - pub async fn get_by_subject_name(&self, subject_name: &str) -> anyhow::Result> { + pub async fn get_by_subject_name(&self, subject_name: &str) -> ConnectorResult> { let raw_schema = self.get_raw_schema_by_subject_name(subject_name).await?; self.parse_and_cache_schema(raw_schema).await } @@ -56,7 +57,7 @@ impl ConfluentSchemaResolver { pub async fn get_raw_schema_by_subject_name( &self, subject_name: &str, - ) -> anyhow::Result { + ) -> ConnectorResult { self.confluent_client .get_schema_by_subject(subject_name) .await @@ -64,7 +65,7 @@ impl ConfluentSchemaResolver { } // get the writer schema by id - pub async fn get(&self, schema_id: i32) -> anyhow::Result> { + pub async fn get(&self, schema_id: i32) -> ConnectorResult> { // TODO: use `get_with` if let Some(schema) = self.writer_schemas.get(&schema_id).await { Ok(schema) diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 8d2d4265883e..ba065b7da4dc 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -16,20 +16,23 @@ use std::sync::LazyLock; use apache_avro::schema::{DecimalSchema, RecordSchema, Schema}; use itertools::Itertools; +use risingwave_common::bail; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{DataType, Decimal}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; -pub fn avro_schema_to_column_descs(schema: &Schema) -> anyhow::Result> { +use crate::error::ConnectorResult; + +pub fn avro_schema_to_column_descs(schema: &Schema) -> ConnectorResult> { if let Schema::Record(RecordSchema { fields, .. }) = schema { let mut index = 0; let fields = fields .iter() .map(|field| avro_field_to_column_desc(&field.name, &field.schema, &mut index)) - .collect::>>()?; + .collect::>>()?; Ok(fields) } else { - anyhow::bail!("schema invalid, record type required at top level of the schema."); + bail!("schema invalid, record type required at top level of the schema."); } } @@ -40,7 +43,7 @@ fn avro_field_to_column_desc( name: &str, schema: &Schema, index: &mut i32, -) -> anyhow::Result { +) -> ConnectorResult { let data_type = avro_type_mapping(schema)?; match schema { Schema::Record(RecordSchema { @@ -51,7 +54,7 @@ fn avro_field_to_column_desc( let vec_column = fields .iter() .map(|f| avro_field_to_column_desc(&f.name, &f.schema, index)) - .collect::>>()?; + .collect::>>()?; *index += 1; Ok(ColumnDesc { column_type: Some(data_type.to_protobuf()), @@ -79,7 +82,7 @@ fn avro_field_to_column_desc( } } -fn avro_type_mapping(schema: &Schema) -> anyhow::Result { +fn avro_type_mapping(schema: &Schema) -> ConnectorResult { let data_type = match schema { Schema::String => DataType::Varchar, Schema::Int => DataType::Int32, @@ -122,7 +125,7 @@ fn avro_type_mapping(schema: &Schema) -> anyhow::Result { let struct_fields = fields .iter() .map(|f| avro_type_mapping(&f.schema)) - .collect::>>()?; + .collect::>>()?; let struct_names = fields.iter().map(|f| f.name.clone()).collect_vec(); DataType::new_struct(struct_fields, struct_names) } @@ -147,18 +150,10 @@ fn avro_type_mapping(schema: &Schema) -> anyhow::Result { { DataType::Decimal } else { - return Err(anyhow::format_err!( - "unsupported type in Avro: {:?}", - schema - )); + bail!("unsupported type in Avro: {:?}", schema); } } - _ => { - return Err(anyhow::format_err!( - "unsupported type in Avro: {:?}", - schema - )); - } + _ => bail!("unsupported type in Avro: {:?}", schema), }; Ok(data_type) diff --git a/src/connector/src/parser/bytes_parser.rs b/src/connector/src/parser/bytes_parser.rs index 44c035fa3ff5..4f353ce2c60e 100644 --- a/src/connector/src/parser/bytes_parser.rs +++ b/src/connector/src/parser/bytes_parser.rs @@ -17,6 +17,7 @@ use risingwave_common::try_match_expand; use super::unified::bytes::BytesAccess; use super::unified::AccessImpl; use super::{AccessBuilder, EncodingProperties}; +use crate::error::ConnectorResult; #[derive(Debug)] pub struct BytesAccessBuilder { @@ -25,7 +26,7 @@ pub struct BytesAccessBuilder { impl AccessBuilder for BytesAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { Ok(AccessImpl::Bytes(BytesAccess::new( &self.column_name, payload, @@ -34,7 +35,7 @@ impl AccessBuilder for BytesAccessBuilder { } impl BytesAccessBuilder { - pub fn new(encoding_properties: EncodingProperties) -> anyhow::Result { + pub fn new(encoding_properties: EncodingProperties) -> ConnectorResult { let config = try_match_expand!(encoding_properties, EncodingProperties::Bytes)?; Ok(Self { column_name: config.column_name, diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index 09a00c490a9f..75e6656fd7a7 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -18,6 +18,7 @@ use risingwave_common::bail; use simd_json::prelude::{MutableObject, ValueAsScalar, ValueObjectAccess}; use simd_json::BorrowedValue; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::canal::operators::*; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; @@ -44,7 +45,7 @@ impl CanalJsonParser { rw_columns: Vec, source_ctx: SourceContextRef, config: &JsonProperties, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self { rw_columns, source_ctx, @@ -57,9 +58,10 @@ impl CanalJsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let mut event: BorrowedValue<'_> = - simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..])?; + simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) + .context("failed to parse canal json payload")?; let is_ddl = event .get(IS_DDL) @@ -123,7 +125,7 @@ impl ByteStreamSourceParser for CanalJsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index 7bb67c9d7b51..8a8bb211da32 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -16,6 +16,7 @@ use risingwave_common::types::{Date, Decimal, Time, Timestamp, Timestamptz}; use super::unified::{AccessError, AccessResult}; use super::{ByteStreamSourceParser, CsvProperties}; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::{ParserFormat, SourceStreamChunkRowWriter}; use crate::source::{DataType, SourceColumnDesc, SourceContext, SourceContextRef}; @@ -44,7 +45,7 @@ impl CsvParser { rw_columns: Vec, csv_props: CsvProperties, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let CsvProperties { delimiter, has_header, @@ -58,7 +59,7 @@ impl CsvParser { }) } - fn read_row(&self, buf: &[u8]) -> anyhow::Result> { + fn read_row(&self, buf: &[u8]) -> ConnectorResult> { let mut reader_builder = csv::ReaderBuilder::default(); reader_builder.delimiter(self.delimiter).has_headers(false); let record = reader_builder @@ -102,7 +103,7 @@ impl CsvParser { &mut self, payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let mut fields = self.read_row(&payload)?; if let Some(headers) = &mut self.headers { @@ -158,7 +159,7 @@ impl ByteStreamSourceParser for CsvParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index 6320ef5cdf3e..ca1574af3d6b 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -21,6 +21,7 @@ use risingwave_common::try_match_expand; use risingwave_pb::catalog::PbSchemaRegistryNameStrategy; use risingwave_pb::plan_common::ColumnDesc; +use crate::error::ConnectorResult; use crate::parser::avro::schema_resolver::ConfluentSchemaResolver; use crate::parser::avro::util::avro_schema_to_column_descs; use crate::parser::unified::avro::{ @@ -48,7 +49,7 @@ pub struct DebeziumAvroAccessBuilder { // TODO: reduce encodingtype match impl AccessBuilder for DebeziumAvroAccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { let (schema_id, mut raw_payload) = extract_schema_id(&payload)?; let schema = self.schema_resolver.get(schema_id).await?; self.value = Some(from_avro_datum(schema.as_ref(), &mut raw_payload, None)?); @@ -70,7 +71,7 @@ impl DebeziumAvroAccessBuilder { pub fn new( config: DebeziumAvroParserConfig, encoding_type: EncodingType, - ) -> anyhow::Result { + ) -> ConnectorResult { let DebeziumAvroParserConfig { outer_schema, schema_resolver, @@ -99,7 +100,7 @@ pub struct DebeziumAvroParserConfig { } impl DebeziumAvroParserConfig { - pub async fn new(encoding_config: EncodingProperties) -> anyhow::Result { + pub async fn new(encoding_config: EncodingProperties) -> ConnectorResult { let avro_config = try_match_expand!(encoding_config, EncodingProperties::Avro)?; let schema_location = &avro_config.row_schema_location; let client_config = &avro_config.client_config; @@ -121,11 +122,11 @@ impl DebeziumAvroParserConfig { }) } - pub fn extract_pks(&self) -> anyhow::Result> { + pub fn extract_pks(&self) -> ConnectorResult> { avro_schema_to_column_descs(&self.key_schema) } - pub fn map_to_columns(&self) -> anyhow::Result> { + pub fn map_to_columns(&self) -> ConnectorResult> { avro_schema_to_column_descs(avro_schema_skip_union(avro_extract_field_schema( &self.outer_schema, Some("before"), @@ -348,7 +349,7 @@ mod tests { #[ignore] #[tokio::test] - async fn test_debezium_avro_parser() -> anyhow::Result<()> { + async fn test_debezium_avro_parser() -> crate::error::ConnectorResult<()> { let props = convert_args!(hashmap!( "kafka.topic" => "dbserver1.inventory.customers" )); diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index f69c19e691e9..d9c824882e1e 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -16,6 +16,7 @@ use risingwave_common::bail; use super::simd_json_parser::DebeziumJsonAccessBuilder; use super::{DebeziumAvroAccessBuilder, DebeziumAvroParserConfig}; +use crate::error::ConnectorResult; use crate::extract_key_config; use crate::parser::unified::debezium::DebeziumChangeEvent; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; @@ -37,7 +38,7 @@ pub struct DebeziumParser { async fn build_accessor_builder( config: EncodingProperties, encoding_type: EncodingType, -) -> anyhow::Result { +) -> ConnectorResult { match config { EncodingProperties::Avro(_) => { let config = DebeziumAvroParserConfig::new(config).await?; @@ -60,7 +61,7 @@ impl DebeziumParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let (key_config, key_type) = extract_key_config!(props); let key_builder = build_accessor_builder(key_config, key_type).await?; let payload_builder = @@ -73,7 +74,7 @@ impl DebeziumParser { }) } - pub async fn new_for_test(rw_columns: Vec) -> anyhow::Result { + pub async fn new_for_test(rw_columns: Vec) -> ConnectorResult { let props = SpecificParserConfig { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { @@ -89,7 +90,7 @@ impl DebeziumParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result { + ) -> ConnectorResult { // tombetone messages are handled implicitly by these accessors let key_accessor = match key { None => None, @@ -137,7 +138,7 @@ impl ByteStreamSourceParser for DebeziumParser { _key: Option>, _payload: Option>, _writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { unreachable!("should call `parse_one_with_txn` instead") } @@ -146,7 +147,7 @@ impl ByteStreamSourceParser for DebeziumParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result { + ) -> ConnectorResult { self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 74c8a4ee8cfb..aa4263ace7cb 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -20,6 +20,7 @@ use risingwave_common::types::DataType; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::unified::debezium::{DebeziumChangeEvent, MongoProjection}; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; @@ -39,7 +40,7 @@ impl DebeziumMongoJsonParser { pub fn new( rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let id_column = rw_columns .iter() .find(|desc| { @@ -78,8 +79,9 @@ impl DebeziumMongoJsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { - let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload)?; + ) -> ConnectorResult<()> { + let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload) + .context("failed to parse debezium mongo json payload")?; // Event can be configured with and without the "payload" field present. // See https://github.com/risingwavelabs/risingwave/issues/10178 @@ -115,7 +117,7 @@ impl ByteStreamSourceParser for DebeziumMongoJsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index 3ad3609c7806..bc516ebce4f2 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -14,9 +14,11 @@ use std::fmt::Debug; +use anyhow::Context; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; +use crate::error::ConnectorResult; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::AccessImpl; use crate::parser::AccessBuilder; @@ -27,17 +29,18 @@ pub struct DebeziumJsonAccessBuilder { } impl DebeziumJsonAccessBuilder { - pub fn new() -> anyhow::Result { + pub fn new() -> ConnectorResult { Ok(Self { value: None }) } } impl AccessBuilder for DebeziumJsonAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { self.value = Some(payload); let mut event: BorrowedValue<'_> = - simd_json::to_borrowed_value(self.value.as_mut().unwrap())?; + simd_json::to_borrowed_value(self.value.as_mut().unwrap()) + .context("failed to parse debezium json payload")?; let payload = if let Some(payload) = event.get_mut("payload") { std::mem::take(payload) @@ -64,12 +67,14 @@ mod tests { DataType, Date, Interval, Scalar, ScalarImpl, StructType, Time, Timestamp, }; use serde_json::Value; + use thiserror_ext::AsReport; use crate::parser::{ DebeziumParser, EncodingProperties, JsonProperties, ProtocolProperties, SourceColumnDesc, SourceStreamChunkBuilder, SpecificParserConfig, }; use crate::source::SourceContextRef; + fn assert_json_eq(parse_result: &Option, json_str: &str) { if let Some(ScalarImpl::Jsonb(json_val)) = parse_result { let mut json_string = String::new(); @@ -491,7 +496,7 @@ mod tests { } else { // For f64 overflow, the parsing fails let e = res.unwrap_err(); - assert!(e.to_string().contains("InvalidNumber"), "{i}: {e}"); + assert!(e.to_report_string().contains("InvalidNumber"), "{i}: {e}"); } } } diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 74c82d14d806..47db36a0b2b8 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -24,6 +24,7 @@ use risingwave_pb::plan_common::ColumnDesc; use super::avro::schema_resolver::ConfluentSchemaResolver; use super::util::{bytes_from_url, get_kafka_topic}; use super::{EncodingProperties, SchemaRegistryAuth, SpecificParserConfig}; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::avro::util::avro_schema_to_column_descs; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; @@ -43,7 +44,7 @@ pub struct JsonAccessBuilder { impl AccessBuilder for JsonAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { if payload.is_empty() { self.value = Some("{}".into()); } else { @@ -51,7 +52,8 @@ impl AccessBuilder for JsonAccessBuilder { } let value = simd_json::to_borrowed_value( &mut self.value.as_mut().unwrap()[self.payload_start_idx..], - )?; + ) + .context("failed to parse json payload")?; Ok(AccessImpl::Json(JsonAccess::new_with_options( value, // Debezium and Canal have their special json access builder and will not @@ -62,7 +64,7 @@ impl AccessBuilder for JsonAccessBuilder { } impl JsonAccessBuilder { - pub fn new(use_schema_registry: bool) -> anyhow::Result { + pub fn new(use_schema_registry: bool) -> ConnectorResult { Ok(Self { value: None, payload_start_idx: if use_schema_registry { 5 } else { 0 }, @@ -84,7 +86,7 @@ impl JsonParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let json_config = try_match_expand!(props.encoding_config, EncodingProperties::Json)?; let payload_start_idx = if json_config.use_schema_registry { 5 @@ -98,7 +100,7 @@ impl JsonParser { }) } - pub fn new_for_test(rw_columns: Vec) -> anyhow::Result { + pub fn new_for_test(rw_columns: Vec) -> ConnectorResult { Ok(Self { rw_columns, source_ctx: Default::default(), @@ -111,8 +113,9 @@ impl JsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { - let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..])?; + ) -> ConnectorResult<()> { + let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) + .context("failed to parse json payload")?; let values = if let simd_json::BorrowedValue::Array(arr) = value { Either::Left(arr.into_iter()) } else { @@ -145,7 +148,7 @@ pub async fn schema_to_columns( schema_location: &str, schema_registry_auth: Option, props: &HashMap, -) -> anyhow::Result> { +) -> ConnectorResult> { let url = handle_sr_list(schema_location)?; let json_schema = if let Some(schema_registry_auth) = schema_registry_auth { let client = Client::new(url, &schema_registry_auth)?; @@ -185,7 +188,7 @@ impl ByteStreamSourceParser for JsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/maxwell/maxwell_parser.rs b/src/connector/src/parser/maxwell/maxwell_parser.rs index aa661585fa1f..8ba95ad21213 100644 --- a/src/connector/src/parser/maxwell/maxwell_parser.rs +++ b/src/connector/src/parser/maxwell/maxwell_parser.rs @@ -14,6 +14,7 @@ use risingwave_common::bail; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::unified::maxwell::MaxwellChangeEvent; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; @@ -35,7 +36,7 @@ impl MaxwellParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { match props.encoding_config { EncodingProperties::Json(_) => { let payload_builder = @@ -55,7 +56,7 @@ impl MaxwellParser { &mut self, payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let payload_accessor = self.payload_builder.generate_accessor(payload).await?; let row_op = MaxwellChangeEvent::new(payload_accessor); @@ -81,7 +82,7 @@ impl ByteStreamSourceParser for MaxwellParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { // restrict the behaviours since there is no corresponding // key/value test for maxwell yet. only_parse_payload!(self, payload, writer) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 952ccd9774d3..721cfa2f241c 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -44,10 +44,11 @@ pub use self::mysql::mysql_row_to_owned_row; use self::plain_parser::PlainParser; pub use self::postgres::postgres_row_to_owned_row; use self::simd_json_parser::DebeziumJsonAccessBuilder; -use self::unified::{AccessImpl, AccessResult}; +use self::unified::AccessImpl; use self::upsert_parser::UpsertParser; use self::util::get_kafka_topic; use crate::common::AwsAuthProps; +use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::maxwell::MaxwellParser; use crate::parser::util::{ extract_header_inner_from_meta, extract_headers_from_meta, extreact_timestamp_from_meta, @@ -76,6 +77,8 @@ mod unified; mod upsert_parser; mod util; +pub use unified::{AccessError, AccessResult}; + /// A builder for building a [`StreamChunk`] from [`SourceColumnDesc`]. pub struct SourceStreamChunkBuilder { descs: Vec, @@ -536,7 +539,7 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a; + ) -> impl Future> + Send + 'a; /// Parse one record from the given `payload`, either write rows to the `writer` or interpret it /// as a transaction control message. @@ -550,13 +553,13 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + ) -> impl Future> + Send + 'a { self.parse_one(key, payload, writer) .map_ok(|_| ParseResult::Rows) } } -#[try_stream(ok = Vec, error = anyhow::Error)] +#[try_stream(ok = Vec, error = ConnectorError)] async fn ensure_largest_at_rate_limit(stream: BoxSourceStream, rate_limit: u32) { #[for_await] for batch in stream { @@ -610,7 +613,7 @@ const MAX_ROWS_FOR_TRANSACTION: usize = 4096; // TODO: when upsert is disabled, how to filter those empty payload // Currently, an err is returned for non upsert with empty payload -#[try_stream(ok = StreamChunk, error = anyhow::Error)] +#[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(mut parser: P, data_stream: BoxSourceStream) { let columns = parser.columns().to_vec(); @@ -700,14 +703,14 @@ async fn into_chunk_stream(mut parser: P, data_stream LazyLock::new(LogSuppresser::default); if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { tracing::error!( - %error, + error = %error.as_report(), split_id = &*msg.split_id, offset = msg.offset, suppressed_count, "failed to parse message, skipping" ); } - parser.source_ctx().report_user_source_error(&*error); + parser.source_ctx().report_user_source_error(&error); } } @@ -750,7 +753,7 @@ async fn into_chunk_stream(mut parser: P, data_stream } pub trait AccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result>; + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult>; } #[derive(Debug)] @@ -770,7 +773,10 @@ pub enum AccessBuilderImpl { } impl AccessBuilderImpl { - pub async fn new_default(config: EncodingProperties, kv: EncodingType) -> anyhow::Result { + pub async fn new_default( + config: EncodingProperties, + kv: EncodingType, + ) -> ConnectorResult { let accessor = match config { EncodingProperties::Avro(_) => { let config = AvroParserConfig::new(config).await?; @@ -794,7 +800,7 @@ impl AccessBuilderImpl { pub async fn generate_accessor( &mut self, payload: Vec, - ) -> anyhow::Result> { + ) -> ConnectorResult> { let accessor = match self { Self::Avro(builder) => builder.generate_accessor(payload).await?, Self::Protobuf(builder) => builder.generate_accessor(payload).await?, @@ -843,7 +849,7 @@ impl ByteStreamSourceParserImpl { pub async fn create( parser_config: ParserConfig, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let CommonParserConfig { rw_columns } = parser_config.common; let protocol = &parser_config.specific.protocol_config; let encode = &parser_config.specific.encoding_config; @@ -990,7 +996,7 @@ impl SpecificParserConfig { pub fn new( info: &StreamSourceInfo, with_properties: &HashMap, - ) -> anyhow::Result { + ) -> ConnectorResult { let source_struct = extract_source_struct(info)?; let format = source_struct.format; let encode = source_struct.encode; diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 5fdb9fbf3c6c..3b5460de1bfb 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -18,6 +18,7 @@ use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; +use crate::error::ConnectorResult; use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; use crate::parser::unified::debezium::parse_transaction_meta; @@ -43,7 +44,7 @@ impl PlainParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { Some(AccessBuilderImpl::Bytes(BytesAccessBuilder::new( EncodingProperties::Bytes(BytesProperties { @@ -81,7 +82,7 @@ impl PlainParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result { + ) -> ConnectorResult { // if the message is transaction metadata, parse it and return if let Some(msg_meta) = writer.row_meta && let SourceMeta::DebeziumCdc(cdc_meta) = msg_meta.meta @@ -145,7 +146,7 @@ impl ByteStreamSourceParser for PlainParser { _key: Option>, _payload: Option>, _writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { unreachable!("should call `parse_one_with_txn` instead") } @@ -154,7 +155,7 @@ impl ByteStreamSourceParser for PlainParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result { + ) -> ConnectorResult { self.parse_inner(key, payload, writer).await } } @@ -262,7 +263,7 @@ mod tests { assert_eq!(1, output.len()); } - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn source_message_stream(transactional: bool) { let begin_msg = r#"{"schema":null,"payload":{"status":"BEGIN","id":"35352:3962948040","event_count":null,"data_collections":null,"ts_ms":1704269323180}}"#; let commit_msg = r#"{"schema":null,"payload":{"status":"END","id":"35352:3962950064","event_count":11,"data_collections":[{"data_collection":"public.orders_tx","event_count":5},{"data_collection":"public.person","event_count":6}],"ts_ms":1704269323180}}"#; diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 922705e3d3f8..d4287a869b22 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -28,6 +28,7 @@ use thiserror::Error; use thiserror_ext::{AsReport, Macro}; use super::schema_resolver::*; +use crate::error::ConnectorResult; use crate::parser::unified::protobuf::ProtobufAccess; use crate::parser::unified::{ bail_uncategorized, uncategorized, AccessError, AccessImpl, AccessResult, @@ -47,7 +48,7 @@ pub struct ProtobufAccessBuilder { impl AccessBuilder for ProtobufAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { let payload = if self.confluent_wire_type { resolve_pb_header(&payload)? } else { @@ -65,7 +66,7 @@ impl AccessBuilder for ProtobufAccessBuilder { } impl ProtobufAccessBuilder { - pub fn new(config: ProtobufParserConfig) -> anyhow::Result { + pub fn new(config: ProtobufParserConfig) -> ConnectorResult { let ProtobufParserConfig { confluent_wire_type, message_descriptor, @@ -89,7 +90,7 @@ pub struct ProtobufParserConfig { } impl ProtobufParserConfig { - pub async fn new(encoding_properties: EncodingProperties) -> anyhow::Result { + pub async fn new(encoding_properties: EncodingProperties) -> ConnectorResult { let protobuf_config = try_match_expand!(encoding_properties, EncodingProperties::Protobuf)?; let location = &protobuf_config.row_schema_location; let message_name = &protobuf_config.message_name; @@ -133,7 +134,7 @@ impl ProtobufParserConfig { } /// Maps the protobuf schema to relational schema. - pub fn map_to_columns(&self) -> anyhow::Result> { + pub fn map_to_columns(&self) -> ConnectorResult> { let mut columns = Vec::with_capacity(self.message_descriptor.fields().len()); let mut index = 0; let mut parse_trace: Vec = vec![]; @@ -153,8 +154,9 @@ impl ProtobufParserConfig { field_descriptor: &FieldDescriptor, index: &mut i32, parse_trace: &mut Vec, - ) -> anyhow::Result { - let field_type = protobuf_type_mapping(field_descriptor, parse_trace)?; + ) -> ConnectorResult { + let field_type = protobuf_type_mapping(field_descriptor, parse_trace) + .context("failed to map protobuf type")?; if let Kind::Message(m) = field_descriptor.kind() { let field_descs = if let DataType::List { .. } = field_type { vec![] @@ -525,7 +527,7 @@ fn protobuf_type_mapping( /// Wire format for Confluent pb header is: /// | 0 | 1-4 | 5-x | x+1-end /// | magic-byte | schema-id | message-indexes | protobuf-payload -pub(crate) fn resolve_pb_header(payload: &[u8]) -> anyhow::Result<&[u8]> { +pub(crate) fn resolve_pb_header(payload: &[u8]) -> ConnectorResult<&[u8]> { // there's a message index array at the front of payload // if it is the first message in proto def, the array is just and `0` // TODO: support parsing more complex index array @@ -575,7 +577,7 @@ mod test { static PRE_GEN_PROTO_DATA: &[u8] = b"\x08\x7b\x12\x0c\x74\x65\x73\x74\x20\x61\x64\x64\x72\x65\x73\x73\x1a\x09\x74\x65\x73\x74\x20\x63\x69\x74\x79\x20\xc8\x03\x2d\x19\x04\x9e\x3f\x32\x0a\x32\x30\x32\x31\x2d\x30\x31\x2d\x30\x31"; #[tokio::test] - async fn test_simple_schema() -> anyhow::Result<()> { + async fn test_simple_schema() -> crate::error::ConnectorResult<()> { let location = schema_dir() + "/simple-schema"; println!("location: {}", location); let message_name = "test.TestRecord"; @@ -620,7 +622,7 @@ mod test { } #[tokio::test] - async fn test_complex_schema() -> anyhow::Result<()> { + async fn test_complex_schema() -> crate::error::ConnectorResult<()> { let location = schema_dir() + "/complex-schema"; let message_name = "test.User"; @@ -912,7 +914,7 @@ mod test { static ANY_GEN_PROTO_DATA: &[u8] = b"\x08\xb9\x60\x12\x32\x0a\x24\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x53\x74\x72\x69\x6e\x67\x56\x61\x6c\x75\x65\x12\x0a\x0a\x08\x4a\x6f\x68\x6e\x20\x44\x6f\x65"; #[tokio::test] - async fn test_any_schema() -> anyhow::Result<()> { + async fn test_any_schema() -> crate::error::ConnectorResult<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); @@ -973,7 +975,7 @@ mod test { static ANY_GEN_PROTO_DATA_1: &[u8] = b"\x08\xb9\x60\x12\x2b\x0a\x23\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x49\x6e\x74\x33\x32\x56\x61\x6c\x75\x65\x12\x04\x08\xd2\xfe\x06"; #[tokio::test] - async fn test_any_schema_1() -> anyhow::Result<()> { + async fn test_any_schema_1() -> crate::error::ConnectorResult<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); @@ -1042,7 +1044,7 @@ mod test { static ANY_RECURSIVE_GEN_PROTO_DATA: &[u8] = b"\x08\xb9\x60\x12\x84\x01\x0a\x21\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x41\x6e\x79\x56\x61\x6c\x75\x65\x12\x5f\x0a\x30\x0a\x24\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x53\x74\x72\x69\x6e\x67\x56\x61\x6c\x75\x65\x12\x08\x0a\x06\x31\x31\x34\x35\x31\x34\x12\x2b\x0a\x23\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x49\x6e\x74\x33\x32\x56\x61\x6c\x75\x65\x12\x04\x08\xd2\xfe\x06"; #[tokio::test] - async fn test_any_recursive() -> anyhow::Result<()> { + async fn test_any_recursive() -> crate::error::ConnectorResult<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); diff --git a/src/connector/src/parser/protobuf/schema_resolver.rs b/src/connector/src/parser/protobuf/schema_resolver.rs index 919413a0c50d..828843842c78 100644 --- a/src/connector/src/parser/protobuf/schema_resolver.rs +++ b/src/connector/src/parser/protobuf/schema_resolver.rs @@ -22,6 +22,7 @@ use protobuf_native::compiler::{ }; use protobuf_native::MessageLite; +use crate::error::ConnectorResult; use crate::schema::schema_registry::Client; macro_rules! embed_wkts { @@ -54,7 +55,7 @@ const WELL_KNOWN_TYPES: &[(&str, &[u8])] = embed_wkts![ pub(super) async fn compile_file_descriptor_from_schema_registry( subject_name: &str, client: &Client, -) -> anyhow::Result> { +) -> ConnectorResult> { let (primary_subject, dependency_subjects) = client .get_subject_and_references(subject_name) .await diff --git a/src/connector/src/parser/unified/avro.rs b/src/connector/src/parser/unified/avro.rs index f00e26a32fae..f0825d9af4bb 100644 --- a/src/connector/src/parser/unified/avro.rs +++ b/src/connector/src/parser/unified/avro.rs @@ -22,6 +22,7 @@ use chrono::Datelike; use itertools::Itertools; use num_bigint::{BigInt, Sign}; use risingwave_common::array::{ListValue, StructValue}; +use risingwave_common::bail; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{ DataType, Date, Datum, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, @@ -29,6 +30,7 @@ use risingwave_common::types::{ use risingwave_common::util::iter_util::ZipEqFast; use super::{bail_uncategorized, uncategorized, Access, AccessError, AccessResult}; +use crate::error::ConnectorResult; #[derive(Clone)] /// Options for parsing an `AvroValue` into Datum, with an optional avro schema. pub struct AvroParseOptions<'a> { @@ -384,7 +386,7 @@ pub(crate) fn extract_decimal(bytes: Vec) -> AccessResult<(u32, u32, u32)> { } } -pub fn avro_schema_skip_union(schema: &Schema) -> anyhow::Result<&Schema> { +pub fn avro_schema_skip_union(schema: &Schema) -> ConnectorResult<&Schema> { match schema { Schema::Union(union_schema) => { let inner_schema = union_schema @@ -403,7 +405,7 @@ pub fn avro_schema_skip_union(schema: &Schema) -> anyhow::Result<&Schema> { pub fn avro_extract_field_schema<'a>( schema: &'a Schema, name: Option<&'a str>, -) -> anyhow::Result<&'a Schema> { +) -> ConnectorResult<&'a Schema> { match schema { Schema::Record(RecordSchema { fields, lookup, .. }) => { let name = @@ -418,7 +420,7 @@ pub fn avro_extract_field_schema<'a>( } Schema::Array(schema) => Ok(schema), Schema::Union(_) => avro_schema_skip_union(schema), - _ => Err(anyhow::format_err!("avro schema is not a record or array")), + _ => bail!("avro schema is not a record or array"), } } @@ -481,7 +483,7 @@ mod tests { value: Value, value_schema: &Schema, shape: &DataType, - ) -> anyhow::Result { + ) -> crate::error::ConnectorResult { AvroParseOptions { schema: Some(value_schema), relax_numeric: true, diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index b8abf4785bb4..048fd0beca3f 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -23,6 +23,7 @@ use super::{ AccessBuilderImpl, ByteStreamSourceParser, BytesProperties, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; +use crate::error::ConnectorResult; use crate::parser::ParserFormat; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; @@ -37,7 +38,7 @@ pub struct UpsertParser { async fn build_accessor_builder( config: EncodingProperties, encoding_type: EncodingType, -) -> anyhow::Result { +) -> ConnectorResult { match config { EncodingProperties::Json(_) | EncodingProperties::Protobuf(_) @@ -66,7 +67,7 @@ impl UpsertParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { // check whether columns has Key as AdditionalColumnType, if so, the key accessor should be // bytes let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { @@ -95,7 +96,7 @@ impl UpsertParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = UpsertChangeEvent::default(); let mut change_event_op = ChangeEventOperation::Delete; @@ -133,7 +134,7 @@ impl ByteStreamSourceParser for UpsertParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/util.rs b/src/connector/src/parser/util.rs index ccb0e47310d9..eeaa09bcc031 100644 --- a/src/connector/src/parser/util.rs +++ b/src/connector/src/parser/util.rs @@ -22,10 +22,11 @@ use risingwave_pb::data::DataType as PbDataType; use crate::aws_utils::load_file_descriptor_from_s3; use crate::common::AwsAuthProps; +use crate::error::ConnectorResult; use crate::source::SourceMeta; /// get kafka topic name -pub(super) fn get_kafka_topic(props: &HashMap) -> anyhow::Result<&String> { +pub(super) fn get_kafka_topic(props: &HashMap) -> ConnectorResult<&String> { const KAFKA_TOPIC_KEY1: &str = "kafka.topic"; const KAFKA_TOPIC_KEY2: &str = "topic"; @@ -45,7 +46,7 @@ pub(super) fn get_kafka_topic(props: &HashMap) -> anyhow::Result } /// download bytes from http(s) url -pub(super) async fn download_from_http(location: &Url) -> anyhow::Result { +pub(super) async fn download_from_http(location: &Url) -> ConnectorResult { let res = reqwest::get(location.clone()) .await .with_context(|| format!("failed to make request to {location}"))? @@ -95,7 +96,7 @@ macro_rules! extract_key_config { pub(super) async fn bytes_from_url( url: &Url, config: Option<&AwsAuthProps>, -) -> anyhow::Result> { +) -> ConnectorResult> { match (url.scheme(), config) { // TODO(Tao): support local file only when it's compiled in debug mode. ("file", _) => { diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 0f64ee85e57d..8d2a9ae78057 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::error::ConnectorError; + pub mod avro; pub mod protobuf; pub mod schema_registry; @@ -46,6 +48,6 @@ pub enum SchemaFetchError { YetToMigrate( #[source] #[backtrace] - anyhow::Error, + ConnectorError, ), } diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index 2a9a2e5a39eb..6f658318f69b 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -30,6 +30,7 @@ use thiserror_ext::AsReport; use with_options::WithOptions; use super::{DummySinkCommitCoordinator, SinkWriterParam}; +use crate::error::ConnectorResult; use crate::sink::catalog::desc::SinkDesc; use crate::sink::log_store::DeliveryFutureManagerAddFuture; use crate::sink::writer::{ @@ -132,7 +133,7 @@ impl ClickHouseEngine { const POOL_IDLE_TIMEOUT: Duration = Duration::from_secs(5); impl ClickHouseCommon { - pub(crate) fn build_client(&self) -> anyhow::Result { + pub(crate) fn build_client(&self) -> ConnectorResult { use hyper_tls::HttpsConnector; let https = HttpsConnector::new(); diff --git a/src/connector/src/sink/iceberg/jni_catalog.rs b/src/connector/src/sink/iceberg/jni_catalog.rs index 11de50e69936..08a876fb62ac 100644 --- a/src/connector/src/sink/iceberg/jni_catalog.rs +++ b/src/connector/src/sink/iceberg/jni_catalog.rs @@ -29,6 +29,8 @@ use jni::JavaVM; use risingwave_jni_core::call_method; use risingwave_jni_core::jvm_runtime::{execute_with_jni_env, jobj_to_str, JVM}; +use crate::error::ConnectorResult; + pub struct JniCatalog { java_catalog: GlobalRef, jvm: &'static JavaVM, @@ -142,7 +144,7 @@ impl JniCatalog { name: impl ToString, catalog_impl: impl ToString, java_catalog_props: HashMap, - ) -> anyhow::Result { + ) -> ConnectorResult { let jvm = JVM.get_or_init()?; execute_with_jni_env(jvm, |env| { @@ -182,5 +184,6 @@ impl JniCatalog { config: base_config, }) as CatalogRef) }) + .map_err(Into::into) } } diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index 326f8586d76e..0c3840af1f3b 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -59,6 +59,7 @@ use super::{ Sink, SinkError, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::deserialize_bool_from_string; +use crate::error::ConnectorResult; use crate::sink::coordinate::CoordinatedSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{Result, SinkCommitCoordinator, SinkParam}; @@ -350,16 +351,19 @@ impl IcebergConfig { Ok((base_catalog_config, java_catalog_configs)) } - async fn create_catalog(&self) -> anyhow::Result { + async fn create_catalog(&self) -> ConnectorResult { match self.catalog_type() { "storage" | "rest" => { let iceberg_configs = self.build_iceberg_configs()?; - let catalog = load_catalog(&iceberg_configs) - .await - .map_err(|e| anyhow!(e))?; + let catalog = load_catalog(&iceberg_configs).await?; Ok(catalog) } - catalog_type if catalog_type == "hive" || catalog_type == "sql" || catalog_type == "glue" || catalog_type == "dynamodb" => { + catalog_type + if catalog_type == "hive" + || catalog_type == "sql" + || catalog_type == "glue" + || catalog_type == "dynamodb" => + { // Create java catalog let (base_catalog_config, java_catalog_props) = self.build_jni_catalog_configs()?; let catalog_impl = match catalog_type { @@ -370,19 +374,24 @@ impl IcebergConfig { _ => unreachable!(), }; - jni_catalog::JniCatalog::build(base_catalog_config, "risingwave", catalog_impl, java_catalog_props) + jni_catalog::JniCatalog::build( + base_catalog_config, + "risingwave", + catalog_impl, + java_catalog_props, + ) } - "mock" => Ok(Arc::new(MockCatalog{})), + "mock" => Ok(Arc::new(MockCatalog {})), _ => { - Err(anyhow!( - "Unsupported catalog type: {}, only support `storage`, `rest`, `hive`, `sql`, `glue`, `dynamodb`", - self.catalog_type() - )) + bail!( + "Unsupported catalog type: {}, only support `storage`, `rest`, `hive`, `sql`, `glue`, `dynamodb`", + self.catalog_type() + ) } } } - pub async fn load_table(&self) -> anyhow::Result { + pub async fn load_table(&self) -> ConnectorResult
{ let catalog = self .create_catalog() .await @@ -395,10 +404,7 @@ impl IcebergConfig { ) .context("Unable to parse table name")?; - catalog - .load_table(&table_id) - .await - .map_err(|err| anyhow!(err)) + catalog.load_table(&table_id).await.map_err(Into::into) } } @@ -428,7 +434,11 @@ impl Debug for IcebergSink { impl IcebergSink { async fn create_and_validate_table(&self) -> Result
{ - let table = self.config.load_table().await.map_err(SinkError::Iceberg)?; + let table = self + .config + .load_table() + .await + .map_err(|err| SinkError::Iceberg(anyhow!(err)))?; let sink_schema = self.param.schema(); let iceberg_schema = table @@ -825,7 +835,7 @@ impl WriteResult { .collect::, icelake::Error>>() .unwrap(); } else { - return Err(anyhow!("icberg sink metadata should have data_files object").into()); + bail!("icberg sink metadata should have data_files object"); } if let serde_json::Value::Array(values) = values .remove(DELETE_FILES) @@ -837,14 +847,14 @@ impl WriteResult { .collect::, icelake::Error>>() .context("Failed to parse data file from json")?; } else { - return Err(anyhow!("icberg sink metadata should have data_files object").into()); + bail!("icberg sink metadata should have data_files object"); } Ok(Self { data_files, delete_files, }) } else { - Err(anyhow!("Can't create iceberg sink write result from empty data!").into()) + bail!("Can't create iceberg sink write result from empty data!") } } } diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index aca370d7ab1b..54e9394b511d 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -176,7 +176,7 @@ impl KinesisSinkWriter { .common .build_client() .await - .map_err(SinkError::Kinesis)?; + .map_err(|err| SinkError::Kinesis(anyhow!(err)))?; Ok(Self { config: config.clone(), formatter, diff --git a/src/connector/src/sink/log_store.rs b/src/connector/src/sink/log_store.rs index f74a22d3b80e..3879d817d1a8 100644 --- a/src/connector/src/sink/log_store.rs +++ b/src/connector/src/sink/log_store.rs @@ -19,9 +19,9 @@ use std::future::{poll_fn, Future}; use std::sync::Arc; use std::task::Poll; -use anyhow::anyhow; use futures::{TryFuture, TryFutureExt}; use risingwave_common::array::StreamChunk; +use risingwave_common::bail; use risingwave_common::buffer::Bitmap; use risingwave_common::util::epoch::{EpochPair, INVALID_EPOCH}; @@ -62,13 +62,13 @@ impl TruncateOffset { } } - pub fn check_next_offset(&self, next_offset: TruncateOffset) -> anyhow::Result<()> { + pub fn check_next_offset(&self, next_offset: TruncateOffset) -> LogStoreResult<()> { if *self >= next_offset { - Err(anyhow!( + bail!( "next offset {:?} should be later than current offset {:?}", next_offset, self - )) + ) } else { Ok(()) } @@ -81,22 +81,22 @@ impl TruncateOffset { .. } => { if epoch != *offset_epoch { - return Err(anyhow!( + bail!( "new item epoch {} not match current chunk offset epoch {}", epoch, offset_epoch - )); + ); } } TruncateOffset::Barrier { epoch: offset_epoch, } => { if epoch <= *offset_epoch { - return Err(anyhow!( + bail!( "new item epoch {} not exceed barrier offset epoch {}", epoch, offset_epoch - )); + ); } } } @@ -534,6 +534,7 @@ mod tests { use tokio::sync::oneshot; use tokio::sync::oneshot::Receiver; + use super::LogStoreResult; use crate::sink::log_store::{DeliveryFutureManager, TruncateOffset}; #[test] @@ -587,7 +588,7 @@ mod tests { } type TestFuture = impl TryFuture + Unpin + 'static; - fn to_test_future(rx: Receiver>) -> TestFuture { + fn to_test_future(rx: Receiver>) -> TestFuture { async move { rx.await.unwrap() }.boxed() } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index fc6712f17604..6abe8d93b595 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -61,6 +61,7 @@ pub use tracing; use self::catalog::{SinkFormatDesc, SinkType}; use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEnum}; +use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; @@ -532,6 +533,12 @@ pub enum SinkError { #[backtrace] anyhow::Error, ), + #[error(transparent)] + Connector( + #[from] + #[backtrace] + ConnectorError, + ), } impl From for SinkError { diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 7a97771dee8e..2bc4160e7a26 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -107,15 +107,12 @@ impl Sink for NatsSink { "Nats sink only support append-only mode" ))); } - match self.config.common.build_client().await { - Ok(_client) => {} - Err(error) => { - return Err(SinkError::Nats(anyhow!( - "validate nats sink error: {:?}", - error - ))); - } - } + let _client = self + .config + .common + .build_client() + .await + .context("validate nats sink error")?; Ok(()) } @@ -134,7 +131,7 @@ impl NatsSinkWriter { .common .build_context() .await - .map_err(|e| SinkError::Nats(anyhow!("nats sink error: {:?}", e)))?; + .map_err(|e| SinkError::Nats(anyhow!(e)))?; Ok::<_, SinkError>(Self { config: config.clone(), context, diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index d79d67e4adc2..f1a07b66cb69 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -30,6 +30,7 @@ use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; use super::{SinkError, SinkParam}; use crate::dispatch_sink_formatter_str_key_impl; +use crate::error::ConnectorResult; use crate::sink::log_store::DeliveryFutureManagerAddFuture; use crate::sink::writer::{ AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, @@ -47,7 +48,7 @@ pub struct RedisCommon { } impl RedisCommon { - pub(crate) fn build_client(&self) -> anyhow::Result { + pub(crate) fn build_client(&self) -> ConnectorResult { let client = RedisClient::open(self.url.clone())?; Ok(client) } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index dfc3bed0e372..4c4a662f8317 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -26,6 +26,7 @@ use itertools::Itertools; use jni::JavaVM; use prost::Message; use risingwave_common::array::StreamChunk; +use risingwave_common::bail; use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::types::DataType; use risingwave_jni_core::jvm_runtime::JVM; @@ -56,9 +57,10 @@ use tokio_stream::wrappers::ReceiverStream; use tracing::warn; use super::elasticsearch::{StreamChunkConverter, ES_OPTION_DELIMITER}; +use crate::error::ConnectorResult; use crate::sink::catalog::desc::SinkDesc; use crate::sink::coordinate::CoordinatedSinkWriter; -use crate::sink::log_store::{LogStoreReadItem, TruncateOffset}; +use crate::sink::log_store::{LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkCommitCoordinator, SinkError, @@ -157,14 +159,12 @@ impl Sink for RemoteSink { } } -async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Result<()> { +async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> ConnectorResult<()> { if sink_name == ElasticSearchSink::SINK_NAME && param.downstream_pk.len() > 1 && param.properties.get(ES_OPTION_DELIMITER).is_none() { - return Err(anyhow!( - "Es sink only support single pk or pk with delimiter option" - )); + bail!("Es sink only support single pk or pk with delimiter option"); } // FIXME: support struct and array in stream sink param.columns.iter().map(|col| { @@ -215,7 +215,7 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Res let jvm = JVM.get_or_init()?; let sink_param = param.to_proto(); - spawn_blocking(move || { + spawn_blocking(move || -> anyhow::Result<()> { let mut env = jvm.attach_current_thread()?; let validate_sink_request = ValidateSinkRequest { sink_param: Some(sink_param), @@ -236,16 +236,13 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Res validate_sink_response.error.map_or_else( || Ok(()), // If there is no error message, return Ok here. - |err| { - Err(anyhow!(format!( - "sink cannot pass validation: {}", - err.error_message - ))) - }, + |err| bail!("sink cannot pass validation: {}", err.error_message), ) }) .await - .context("JoinHandle returns error")? + .context("JoinHandle returns error")??; + + Ok(()) } pub struct RemoteLogSinker { @@ -338,12 +335,11 @@ impl LogSinker for RemoteLogSinker { anyhow!("get unsent offset {:?} in response", persisted_offset) })?; if sent_offset != persisted_offset { - return Err(anyhow!( + bail!( "new response offset {:?} not match the buffer offset {:?}", persisted_offset, sent_offset - ) - .into()); + ); } if let (TruncateOffset::Barrier { .. }, Some(start_time)) = @@ -366,13 +362,13 @@ impl LogSinker for RemoteLogSinker { loop { let either_result: futures::future::Either< Option, - anyhow::Result<(u64, LogStoreReadItem)>, + LogStoreResult<(u64, LogStoreReadItem)>, > = drop_either_future( select(pin!(response_rx.recv()), pin!(log_reader.next_item())).await, ); match either_result { futures::future::Either::Left(opt) => { - let response = opt.ok_or_else(|| anyhow!("end of response stream"))?; + let response = opt.context("end of response stream")?; match response { SinkWriterStreamResponse { response: @@ -569,7 +565,7 @@ impl CoordinatedRemoteSinkWriter { } fn for_test( - response_receiver: Receiver>, + response_receiver: Receiver>, request_sender: Sender, ) -> CoordinatedRemoteSinkWriter { let properties = HashMap::from([("output.path".to_string(), "/tmp/rw".to_string())]); diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index e26bc2dbcb40..625f3dedb30b 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -15,7 +15,7 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; -use anyhow::{anyhow, Result}; +use anyhow::anyhow; use async_trait::async_trait; use aws_sdk_s3::types::Object; use bytes::Bytes; @@ -25,6 +25,7 @@ use futures::Stream; use itertools::Itertools; use parking_lot::Mutex; use risingwave_common::array::StreamChunk; +use risingwave_common::bail; use risingwave_common::catalog::TableId; use risingwave_common::error::ErrorSuppressor; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; @@ -45,6 +46,7 @@ use super::kinesis::KinesisMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use super::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR}; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; pub(crate) use crate::source::common::CommonSplitReader; use crate::source::filesystem::FsPageItem; @@ -69,7 +71,9 @@ pub trait TryFromHashmap: Sized + UnknownFields { /// Each instance should add a `#[derive(with_options::WithOptions)]` marker. pub trait SourceProperties: TryFromHashmap + Clone + WithOptions { const SOURCE_NAME: &'static str; - type Split: SplitMetaData + TryFrom + Into; + type Split: SplitMetaData + + TryFrom + + Into; type SplitEnumerator: SplitEnumerator; type SplitReader: SplitReader; @@ -91,10 +95,10 @@ impl TryFromHashmap for P { if !deny_unknown_fields || res.unknown_fields().is_empty() { Ok(res) } else { - Err(anyhow!( + bail!( "Unknown fields in the WITH clause: {:?}", res.unknown_fields() - )) + ) } } } @@ -340,21 +344,22 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result } (PbFormatType::Plain, PbEncodeType::Bytes) => (SourceFormat::Plain, SourceEncode::Bytes), (format, encode) => { - return Err(anyhow!( + bail!( "Unsupported combination of format {:?} and encode {:?}", format, encode - )); + ); } }; Ok(SourceStruct::new(format, encode)) } -pub type BoxSourceStream = BoxStream<'static, anyhow::Result>>; +pub type BoxSourceStream = BoxStream<'static, crate::error::ConnectorResult>>; -pub trait ChunkSourceStream = Stream> + Send + 'static; -pub type BoxChunkSourceStream = BoxStream<'static, anyhow::Result>; -pub type BoxTryStream = BoxStream<'static, anyhow::Result>; +pub trait ChunkSourceStream = + Stream> + Send + 'static; +pub type BoxChunkSourceStream = BoxStream<'static, crate::error::ConnectorResult>; +pub type BoxTryStream = BoxStream<'static, crate::error::ConnectorResult>; /// [`SplitReader`] is a new abstraction of the external connector read interface which is /// responsible for parsing, it is used to read messages from the outside and transform them into a @@ -370,7 +375,7 @@ pub trait SplitReader: Sized + Send { parser_config: ParserConfig, source_ctx: SourceContextRef, columns: Option>, - ) -> anyhow::Result; + ) -> crate::error::ConnectorResult; fn into_stream(self) -> BoxChunkSourceStream; } @@ -426,7 +431,7 @@ impl ConnectorProperties { PropType, PropType::try_from_hashmap(with_properties, deny_unknown_fields) .map(ConnectorProperties::from), - |other| Err(anyhow!("connector '{}' is not supported", other)) + |other| bail!("connector '{}' is not supported", other) ) } @@ -462,7 +467,7 @@ impl From<&SplitImpl> for ConnectorSplit { } impl TryFrom<&ConnectorSplit> for SplitImpl { - type Error = anyhow::Error; + type Error = crate::error::ConnectorError; fn try_from(split: &ConnectorSplit) -> std::result::Result { match_source_name_str!( @@ -474,7 +479,7 @@ impl TryFrom<&ConnectorSplit> for SplitImpl { ) .map(Into::into) }, - |other| Err(anyhow!("connector '{}' is not supported", other)) + |other| bail!("connector '{}' is not supported", other) ) } } @@ -503,7 +508,7 @@ impl SplitImpl { split_type.to_lowercase().as_str(), PropType, ::Split::restore_from_json(value).map(Into::into), - |other| Err(anyhow!("connector '{}' is not supported", other)) + |other| bail!("connector '{}' is not supported", other) ) } } @@ -621,7 +626,7 @@ pub trait SplitMetaData: Sized { fn encode_to_json(&self) -> JsonbVal; fn restore_from_json(value: JsonbVal) -> Result; - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()>; + fn update_with_offset(&mut self, start_offset: String) -> crate::error::ConnectorResult<()>; } /// [`ConnectorState`] maintains the consuming splits' info. In specific split readers, diff --git a/src/connector/src/source/cdc/enumerator/mod.rs b/src/connector/src/source/cdc/enumerator/mod.rs index 58bc42e53757..b5ac4826921b 100644 --- a/src/connector/src/source/cdc/enumerator/mod.rs +++ b/src/connector/src/source/cdc/enumerator/mod.rs @@ -27,6 +27,7 @@ use risingwave_pb::connector_service::{ SourceCommonParam, SourceType, ValidateSourceRequest, ValidateSourceResponse, }; +use crate::error::ConnectorResult; use crate::source::cdc::{ CdcProperties, CdcSourceTypeTrait, CdcSplitBase, Citus, DebeziumCdcSplit, MySqlCdcSplit, Mysql, Postgres, PostgresCdcSplit, @@ -54,7 +55,7 @@ where async fn new( props: CdcProperties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let server_addrs = props .properties .get(DATABASE_SERVERS_KEY) @@ -72,7 +73,7 @@ where ); let source_id = context.info.source_id; - tokio::task::spawn_blocking(move || { + tokio::task::spawn_blocking(move || -> anyhow::Result<()> { let mut env = JVM.get_or_init()?.attach_current_thread()?; let validate_source_request = ValidateSourceRequest { @@ -100,15 +101,11 @@ where .deref(), )?; - validate_source_response.error.map_or_else( - || Ok(()), - |err| { - Err(anyhow!(format!( - "source cannot pass validation: {}", - err.error_message - ))) - }, - ) + if let Some(error) = validate_source_response.error { + return Err(anyhow!(error.error_message).context("source cannot pass validation")); + } + + Ok(()) }) .await .context("failed to validate source")??; @@ -121,7 +118,7 @@ where }) } - async fn list_splits(&mut self) -> anyhow::Result>> { + async fn list_splits(&mut self) -> ConnectorResult>> { Ok(self.list_cdc_splits()) } } diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index f281d1ecea58..a9808e3a9e1e 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -407,7 +407,7 @@ impl MySqlExternalTableReader { bail!("primary key {} cannot be null", pk); } }) - .try_collect()?; + .try_collect::<_, _, ConnectorError>()?; let rs_stream = sql .with(Params::from(params)) diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index bd8a0b51c04e..9f9a055fd7d8 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -199,7 +199,7 @@ impl PostgresExternalTableReader { let stream = client.query_raw(&sql, ¶ms).await?; let row_stream = stream.map(|row| { let row = row?; - Ok::<_, anyhow::Error>(postgres_row_to_owned_row(row, &self.rw_schema)) + Ok::<_, crate::error::ConnectorError>(postgres_row_to_owned_row(row, &self.rw_schema)) }); pin_mut!(row_stream); diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 5fc6aefdfefd..b663274c3020 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -28,6 +28,7 @@ use risingwave_pb::plan_common::ExternalTableDesc; use simd_json::prelude::ArrayTrait; pub use source::*; +use crate::error::ConnectorResult; use crate::source::{SourceProperties, SplitImpl, TryFromHashmap}; use crate::{for_all_classified_sources, impl_cdc_source_type}; @@ -91,7 +92,7 @@ impl TryFromHashmap for CdcProperties { fn try_from_hashmap( properties: HashMap, _deny_unknown_fields: bool, - ) -> anyhow::Result { + ) -> ConnectorResult { let is_multi_table_shared = properties .get(CDC_SHARING_MODE_KEY) .is_some_and(|v| v == "true"); @@ -107,7 +108,7 @@ impl TryFromHashmap for CdcProperties { impl SourceProperties for CdcProperties where - DebeziumCdcSplit: TryFrom + Into, + DebeziumCdcSplit: TryFrom + Into, DebeziumSplitEnumerator: ListCdcSplits, { type Split = DebeziumCdcSplit; diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index c21d579df777..43753dad599c 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -14,11 +14,12 @@ use std::str::FromStr; -use anyhow::{anyhow, Result}; +use anyhow::anyhow; use async_trait::async_trait; use futures_async_stream::try_stream; use itertools::Itertools; use prost::Message; +use risingwave_common::bail; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::util::addr::HostAddr; use risingwave_jni_core::jvm_runtime::JVM; @@ -29,6 +30,7 @@ use risingwave_pb::connector_service::{ use thiserror_ext::AsReport; use tokio::sync::mpsc; +use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::cdc::{CdcProperties, CdcSourceType, CdcSourceTypeTrait, DebeziumCdcSplit}; @@ -66,7 +68,7 @@ impl SplitReader for CdcSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> Result { + ) -> ConnectorResult { assert_eq!(splits.len(), 1); let split = splits.into_iter().next().unwrap(); let split_id = split.id(); @@ -117,10 +119,8 @@ impl SplitReader for CdcSplitReader { let (mut env, get_event_stream_request_bytes) = match result { Ok(inner) => inner, Err(e) => { - let _ = tx.blocking_send(Err(anyhow!( - "err before calling runJniDbzSourceThread: {:?}", - e - ))); + let _ = tx + .blocking_send(Err(e.context("err before calling runJniDbzSourceThread"))); return; } }; @@ -154,7 +154,7 @@ impl SplitReader for CdcSplitReader { } }; if !inited { - return Err(anyhow!("failed to start cdc connector")); + bail!("failed to start cdc connector"); } } tracing::info!(?source_id, "cdc connector started"); @@ -196,7 +196,7 @@ impl SplitReader for CdcSplitReader { } impl CommonSplitReader for CdcSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = ConnectorError)] async fn into_data_stream(self) { let source_type = T::source_type(); let mut rx = self.rx; @@ -225,6 +225,6 @@ impl CommonSplitReader for CdcSplitReader { } } - Err(anyhow!("all senders are dropped"))?; + bail!("all senders are dropped"); } } diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index a7357d231b78..30165f1939e0 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -14,10 +14,11 @@ use std::marker::PhantomData; -use anyhow::{anyhow, Context}; +use anyhow::Context; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::cdc::external::DebeziumOffset; use crate::source::cdc::CdcSourceTypeTrait; use crate::source::{SplitId, SplitMetaData}; @@ -63,7 +64,7 @@ impl MySqlCdcSplit { Self { inner: split } } - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + pub fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let mut snapshot_done = self.inner.snapshot_done; if !snapshot_done { let dbz_offset: DebeziumOffset = @@ -102,7 +103,7 @@ impl PostgresCdcSplit { } } - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + pub fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let mut snapshot_done = self.inner.snapshot_done; if !snapshot_done { let dbz_offset: DebeziumOffset = @@ -154,11 +155,11 @@ impl SplitMetaData for DebeziumCdcSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { // TODO: may check T to get the specific cdc type assert!(self.mysql_split.is_some() || self.pg_split.is_some()); if let Some(split) = &mut self.mysql_split { diff --git a/src/connector/src/source/common.rs b/src/connector/src/source/common.rs index 5d39e303e759..c145bd7f403d 100644 --- a/src/connector/src/source/common.rs +++ b/src/connector/src/source/common.rs @@ -16,14 +16,15 @@ use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; +use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::ParserConfig; use crate::source::{SourceContextRef, SourceMessage, SplitReader}; pub(crate) trait CommonSplitReader: SplitReader + 'static { - fn into_data_stream(self) -> impl Stream>> + Send; + fn into_data_stream(self) -> impl Stream>> + Send; } -#[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] +#[try_stream(boxed, ok = StreamChunk, error = ConnectorError)] pub(crate) async fn into_chunk_stream( reader: impl CommonSplitReader, parser_config: ParserConfig, diff --git a/src/connector/src/source/datagen/enumerator/mod.rs b/src/connector/src/source/datagen/enumerator/mod.rs index 47eb54cf2400..5b5b473656d0 100644 --- a/src/connector/src/source/datagen/enumerator/mod.rs +++ b/src/connector/src/source/datagen/enumerator/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::Context; use async_trait::async_trait; use crate::source::datagen::{DatagenProperties, DatagenSplit}; @@ -30,13 +31,15 @@ impl SplitEnumerator for DatagenSplitEnumerator { async fn new( properties: DatagenProperties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> crate::error::ConnectorResult { let split_num = properties.split_num.unwrap_or_else(|| "1".to_string()); - let split_num = split_num.parse::()?; + let split_num = split_num + .parse::() + .context("failed to parse datagen split num")?; Ok(Self { split_num }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> crate::error::ConnectorResult> { let mut splits = vec![]; for i in 0..self.split_num { splits.push(DatagenSplit { diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index 18376f3ce73a..1c05c6b4ffc8 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -21,6 +21,7 @@ use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; +use crate::error::ConnectorResult; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use crate::source::{SourceMessage, SourceMeta, SplitId}; @@ -59,7 +60,7 @@ impl DatagenEventGenerator { split_id: SplitId, split_num: u64, split_index: u64, - ) -> anyhow::Result { + ) -> ConnectorResult { let partition_rows_per_second = if rows_per_second % split_num > split_index { rows_per_second / split_num + 1 } else { @@ -76,7 +77,7 @@ impl DatagenEventGenerator { }) } - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn into_msg_stream(mut self) { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; @@ -156,7 +157,7 @@ impl DatagenEventGenerator { } } - #[try_stream(ok = StreamChunk, error = anyhow::Error)] + #[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] pub async fn into_native_stream(mut self) { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index 2e1b5f791726..0b522c4e7c93 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -14,13 +14,14 @@ use std::collections::HashMap; -use anyhow::{Context, Result}; +use anyhow::Context; use async_trait::async_trait; use futures::{Stream, StreamExt, TryStreamExt}; use risingwave_common::field_generator::{FieldGeneratorImpl, VarcharProperty}; use thiserror_ext::AsReport; use super::generator::DatagenEventGenerator; +use crate::error::{ConnectorResult, ConnectorResult as Result}; use crate::parser::{EncodingProperties, ParserConfig, ProtocolProperties}; use crate::source::data_gen_util::spawn_data_generation_stream; use crate::source::datagen::source::SEQUENCE_FIELD_KIND; @@ -183,7 +184,7 @@ impl SplitReader for DatagenSplitReader { } impl CommonSplitReader for DatagenSplitReader { - fn into_data_stream(self) -> impl Stream, anyhow::Error>> { + fn into_data_stream(self) -> impl Stream>> { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; spawn_data_generation_stream(self.generator.into_msg_stream(), BUFFER_SIZE) @@ -253,13 +254,15 @@ fn generator_from_data_type( random_seed, ) } + .map_err(Into::into) } DataType::Varchar => { let length_key = format!("fields.{}.length", name); let length_value = fields_option_map .get(&length_key) .map(|s| s.parse::()) - .transpose()?; + .transpose() + .context("failed to parse the length of varchar field")?; Ok(FieldGeneratorImpl::with_varchar( &VarcharProperty::RandomFixedLength(length_value), random_seed, @@ -280,7 +283,7 @@ fn generator_from_data_type( Ok((field_name.to_string(), gen)) }) .collect::>()?; - FieldGeneratorImpl::with_struct_fields(struct_fields) + FieldGeneratorImpl::with_struct_fields(struct_fields).map_err(Into::into) } DataType::List(datatype) => { let length_key = format!("fields.{}.length", name); @@ -293,7 +296,7 @@ fn generator_from_data_type( split_num, offset, )?; - FieldGeneratorImpl::with_list(generator, length_value) + FieldGeneratorImpl::with_list(generator, length_value).map_err(Into::into) } _ => { let kind_key = format!("fields.{}.kind", name); @@ -312,12 +315,14 @@ fn generator_from_data_type( split_num, offset, ) + .map_err(Into::into) } else { let min_key = format!("fields.{}.min", name); let max_key = format!("fields.{}.max", name); let min_value = fields_option_map.get(&min_key).map(|s| s.to_string()); let max_value = fields_option_map.get(&max_key).map(|s| s.to_string()); FieldGeneratorImpl::with_number_random(data_type, min_value, max_value, random_seed) + .map_err(Into::into) } } } diff --git a/src/connector/src/source/datagen/split.rs b/src/connector/src/source/datagen/split.rs index c2e0bea1f8fb..6d51cfa7d47a 100644 --- a/src/connector/src/source/datagen/split.rs +++ b/src/connector/src/source/datagen/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::base::SplitMetaData; use crate::source::SplitId; @@ -32,15 +32,15 @@ impl SplitMetaData for DatagenSplit { format!("{}-{}", self.split_num, self.split_index).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset.as_str().parse::().unwrap()); Ok(()) } diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 3a537509edb8..ccff7315491b 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -15,12 +15,12 @@ use std::fmt::Debug; use std::hash::Hash; use std::marker::PhantomData; -use anyhow::anyhow; use aws_sdk_s3::types::Object; use risingwave_common::types::{JsonbVal, Timestamptz}; use serde::{Deserialize, Serialize}; use super::opendal_source::OpendalSource; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; /// [`FsSplit`] Describes a file or a split of a file. A file is a generic concept, @@ -47,15 +47,15 @@ impl SplitMetaData for FsSplit { self.name.as_str().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let offset = start_offset.parse().unwrap(); self.offset = offset; Ok(()) @@ -98,15 +98,15 @@ impl SplitMetaData for OpendalFsSplit { self.name.as_str().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let offset = start_offset.parse().unwrap(); self.offset = offset; Ok(()) diff --git a/src/connector/src/source/filesystem/nd_streaming.rs b/src/connector/src/source/filesystem/nd_streaming.rs index 7eb8a84c503a..711e1a15e981 100644 --- a/src/connector/src/source/filesystem/nd_streaming.rs +++ b/src/connector/src/source/filesystem/nd_streaming.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; +use anyhow::Context as _; use bytes::BytesMut; use futures::io::Cursor; use futures::AsyncBufReadExt; @@ -26,7 +26,7 @@ pub fn need_nd_streaming(encode_config: &EncodingProperties) -> bool { || matches!(encode_config, EncodingProperties::Csv(_)) } -#[try_stream(boxed, ok = Vec, error = anyhow::Error)] +#[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] /// This function splits a byte stream by the newline separator "(\r)\n" into a message stream. /// It can be difficult to split and compute offsets correctly when the bytes are received in /// chunks. There are two cases to consider: @@ -50,7 +50,7 @@ pub async fn split_stream(data_stream: BoxSourceStream) { .map(|msg| (msg.offset.clone(), msg.split_id.clone(), msg.meta.clone())) .unwrap(); - let mut offset: usize = offset.parse()?; + let mut offset: usize = offset.parse().context("failed to parse the offset")?; let mut buf = BytesMut::new(); for msg in batch { let payload = msg.payload.unwrap_or_default(); @@ -108,7 +108,7 @@ pub async fn split_stream(data_stream: BoxSourceStream) { last_message = msgs.pop(); } } - Err(e) => return Err(anyhow!(e)), + Err(e) => return Err(e.into()), } line_cnt += 1; diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index d6f7b44bff59..01594af4e4ba 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -21,11 +21,12 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::{GcsProperties, OpendalSource}; +use crate::error::ConnectorResult; use crate::source::filesystem::s3::enumerator::get_prefix; impl OpendalEnumerator { /// create opendal gcs source. - pub fn new_gcs_source(gcs_properties: GcsProperties) -> anyhow::Result { + pub fn new_gcs_source(gcs_properties: GcsProperties) -> ConnectorResult { // Create gcs builder. let mut builder = Gcs::default(); diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index e0c5a22f1fd9..15371a0da90a 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -27,6 +27,7 @@ use self::opendal_enumerator::OpendalEnumerator; use self::opendal_reader::OpendalReader; use super::s3::S3PropertiesCommon; use super::OpendalFsSplit; +use crate::error::ConnectorResult; use crate::source::{SourceProperties, UnknownFields}; pub const GCS_CONNECTOR: &str = "gcs"; @@ -71,7 +72,7 @@ impl SourceProperties for GcsProperties { pub trait OpendalSource: Send + Sync + 'static + Clone + PartialEq { type Properties: SourceProperties + Send + Sync; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result>; + fn new_enumerator(properties: Self::Properties) -> ConnectorResult>; } #[derive(Debug, Clone, Copy, PartialEq, Eq)] @@ -80,7 +81,7 @@ pub struct OpendalS3; impl OpendalSource for OpendalS3 { type Properties = OpendalS3Properties; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + fn new_enumerator(properties: Self::Properties) -> ConnectorResult> { OpendalEnumerator::new_s3_source(properties.s3_properties, properties.assume_role) } } @@ -91,7 +92,7 @@ pub struct OpendalGcs; impl OpendalSource for OpendalGcs { type Properties = GcsProperties; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + fn new_enumerator(properties: Self::Properties) -> ConnectorResult> { OpendalEnumerator::new_gcs_source(properties) } } @@ -102,7 +103,7 @@ pub struct OpendalPosixFs; impl OpendalSource for OpendalPosixFs { type Properties = PosixFsProperties; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + fn new_enumerator(properties: Self::Properties) -> ConnectorResult> { OpendalEnumerator::new_posix_fs_source(properties) } } diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 318467eea606..96646ade0e1d 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -22,6 +22,7 @@ use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamptz; use super::OpendalSource; +use crate::error::ConnectorResult; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -42,11 +43,11 @@ impl SplitEnumerator for OpendalEnumerator { async fn new( properties: Src::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { Src::new_enumerator(properties) } - async fn list_splits(&mut self) -> anyhow::Result>> { + async fn list_splits(&mut self) -> ConnectorResult>> { let empty_split: OpendalFsSplit = OpendalFsSplit::empty_split(); Ok(vec![empty_split]) @@ -54,7 +55,7 @@ impl SplitEnumerator for OpendalEnumerator { } impl OpendalEnumerator { - pub async fn list(&self) -> anyhow::Result { + pub async fn list(&self) -> ConnectorResult { let prefix = match &self.prefix { Some(prefix) => prefix, None => "", @@ -100,4 +101,4 @@ impl OpendalEnumerator { &self.matcher } } -pub type ObjectMetadataIter = BoxStream<'static, anyhow::Result>; +pub type ObjectMetadataIter = BoxStream<'static, ConnectorResult>; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs index 8ed913cad64a..5cb84652fbab 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -22,6 +22,7 @@ use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalEnumerator; use super::OpendalSource; +use crate::error::ConnectorResult; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; @@ -50,7 +51,7 @@ impl SplitReader for OpendalReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let connector = Src::new_enumerator(properties)?; let opendal_reader = OpendalReader { connector, @@ -67,7 +68,7 @@ impl SplitReader for OpendalReader { } impl OpendalReader { - #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] + #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); @@ -107,7 +108,7 @@ impl OpendalReader { } } - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn stream_read_object( op: Operator, split: OpendalFsSplit, diff --git a/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs b/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs index 748230ba5a16..3a4fb7fcfeaa 100644 --- a/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs @@ -21,6 +21,7 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::{OpendalSource, PosixFsProperties}; +use crate::error::ConnectorResult; // Posix fs source should only be used for testing. // For a single-CN cluster, the behavior is well-defined. It will read from the local file system. @@ -28,7 +29,7 @@ use super::{OpendalSource, PosixFsProperties}; impl OpendalEnumerator { /// create opendal posix fs source. - pub fn new_posix_fs_source(posix_fs_properties: PosixFsProperties) -> anyhow::Result { + pub fn new_posix_fs_source(posix_fs_properties: PosixFsProperties) -> ConnectorResult { // Create Fs builder. let mut builder = Fs::default(); diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index ef18ffa4b8fe..f4a548306885 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -21,6 +21,7 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::OpendalSource; +use crate::error::ConnectorResult; use crate::source::filesystem::s3::enumerator::get_prefix; use crate::source::filesystem::s3::S3PropertiesCommon; @@ -29,7 +30,7 @@ impl OpendalEnumerator { pub fn new_s3_source( s3_properties: S3PropertiesCommon, assume_role: Option, - ) -> anyhow::Result { + ) -> ConnectorResult { // Create s3 builder. let mut builder = S3::default(); builder.bucket(&s3_properties.bucket_name); diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index 7033ebdaf91b..63a57b8574b6 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -75,7 +75,7 @@ impl SplitEnumerator for S3SplitEnumerator { async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> crate::error::ConnectorResult { let config = AwsAuthProps::from(&properties); let sdk_config = config.build_config().await?; let s3_client = s3_client(&sdk_config, Some(default_conn_config())); @@ -98,7 +98,7 @@ impl SplitEnumerator for S3SplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> crate::error::ConnectorResult> { let mut objects = Vec::new(); loop { let (files, has_finished) = self.get_next_page::().await?; diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 884f1d19062c..3f485dc22383 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -32,6 +32,7 @@ use tokio_util::io::ReaderStream; use crate::aws_utils::{default_conn_config, s3_client}; use crate::common::AwsAuthProps; +use crate::error::ConnectorResult; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::base::{SplitMetaData, SplitReader}; use crate::source::filesystem::file_common::FsSplit; @@ -54,7 +55,7 @@ pub struct S3FileReader { } impl S3FileReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn stream_read_object( client_for_s3: s3_client::Client, bucket_name: String, @@ -85,7 +86,9 @@ impl S3FileReader { return Ok(()); } Err(e) => { - return Err(anyhow!(e).context(format!("S3 GetObject from {bucket_name} error"))); + return Err(anyhow!(e) + .context(format!("S3 GetObject from {bucket_name} error")) + .into()); } }; @@ -180,7 +183,7 @@ impl SplitReader for S3FileReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let config = AwsAuthProps::from(&props); let sdk_config = config.build_config().await?; @@ -206,7 +209,7 @@ impl SplitReader for S3FileReader { } impl S3FileReader { - #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] + #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); diff --git a/src/connector/src/source/filesystem/s3_v2/lister.rs b/src/connector/src/source/filesystem/s3_v2/lister.rs index d6a6b6c1e68a..ccc40f1ccef8 100644 --- a/src/connector/src/source/filesystem/s3_v2/lister.rs +++ b/src/connector/src/source/filesystem/s3_v2/lister.rs @@ -18,6 +18,7 @@ use aws_sdk_s3::error::DisplayErrorContext; use aws_sdk_s3::types::Object; use itertools::Itertools; +use crate::error::ConnectorResult; use crate::source::filesystem::{FsPageItem, S3SplitEnumerator}; use crate::source::{FsFilterCtrlCtx, FsListInner}; @@ -25,7 +26,7 @@ use crate::source::{FsFilterCtrlCtx, FsListInner}; impl FsListInner for S3SplitEnumerator { async fn get_next_page From<&'a Object>>( &mut self, - ) -> anyhow::Result<(Vec, bool)> { + ) -> ConnectorResult<(Vec, bool)> { let mut has_finished = false; let mut req = self .client diff --git a/src/connector/src/source/google_pubsub/enumerator/client.rs b/src/connector/src/source/google_pubsub/enumerator/client.rs index bc1d9d078b66..25cb28909c47 100644 --- a/src/connector/src/source/google_pubsub/enumerator/client.rs +++ b/src/connector/src/source/google_pubsub/enumerator/client.rs @@ -12,12 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{bail, Context}; +use anyhow::Context; use async_trait::async_trait; use chrono::{TimeZone, Utc}; use google_cloud_pubsub::client::{Client, ClientConfig}; use google_cloud_pubsub::subscription::{SeekTo, SubscriptionConfig}; +use risingwave_common::bail; +use crate::error::ConnectorResult; use crate::source::base::SplitEnumerator; use crate::source::google_pubsub::split::PubsubSplit; use crate::source::google_pubsub::PubsubProperties; @@ -36,7 +38,7 @@ impl SplitEnumerator for PubsubSplitEnumerator { async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let subscription = properties.subscription.to_owned(); if properties.credentials.is_none() && properties.emulator_host.is_none() { @@ -62,7 +64,10 @@ impl SplitEnumerator for PubsubSplitEnumerator { // We need the `retain_acked_messages` configuration to be true to seek back to timestamps // as done in the [`PubsubSplitReader`] and here. - let (_, subscription_config) = sub.config(None).await?; + let (_, subscription_config) = sub + .config(None) + .await + .context("failed to fetch subscription config")?; if let SubscriptionConfig { retain_acked_messages: false, .. @@ -98,7 +103,7 @@ impl SplitEnumerator for PubsubSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { tracing::debug!("enumerating pubsub splits"); let splits: Vec = (0..self.split_count) .map(|i| PubsubSplit { diff --git a/src/connector/src/source/google_pubsub/mod.rs b/src/connector/src/source/google_pubsub/mod.rs index aeec1accd820..0a49fa6467f6 100644 --- a/src/connector/src/source/google_pubsub/mod.rs +++ b/src/connector/src/source/google_pubsub/mod.rs @@ -101,9 +101,8 @@ impl PubsubProperties { #[cfg(test)] mod tests { - use anyhow::Result; - use super::*; + use crate::error::ConnectorResult as Result; const EMULATOR_HOST: &str = "localhost:8081"; const CREDENTIALS: &str = "{}"; diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index fd5fab15ed10..0887cb06594f 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -12,16 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, ensure, Context, Result}; +use anyhow::{anyhow, Context}; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; use futures_async_stream::try_stream; use google_cloud_pubsub::client::{Client, ClientConfig}; use google_cloud_pubsub::subscription::{SeekTo, Subscription}; -use risingwave_common::bail; +use risingwave_common::{bail, ensure}; use tonic::Code; use super::TaggedReceivedMessage; +use crate::error::{ConnectorError, ConnectorResult as Result}; use crate::parser::ParserConfig; use crate::source::google_pubsub::{PubsubProperties, PubsubSplit}; use crate::source::{ @@ -41,7 +42,7 @@ pub struct PubsubSplitReader { } impl CommonSplitReader for PubsubSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = ConnectorError)] async fn into_data_stream(self) { loop { let pull_result = self diff --git a/src/connector/src/source/google_pubsub/split.rs b/src/connector/src/source/google_pubsub/split.rs index e52ffa8ef72a..f150f7f08038 100644 --- a/src/connector/src/source/google_pubsub/split.rs +++ b/src/connector/src/source/google_pubsub/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] @@ -36,8 +36,8 @@ pub struct PubsubSplit { } impl SplitMetaData for PubsubSplit { - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { @@ -48,7 +48,7 @@ impl SplitMetaData for PubsubSplit { format!("{}-{}", self.subscription, self.index).into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset); Ok(()) } diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index e274f639f15b..899828322c81 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -18,6 +18,7 @@ use async_trait::async_trait; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::{ BoxChunkSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, @@ -71,7 +72,7 @@ impl SplitMetaData for IcebergSplit { unimplemented!() } - fn restore_from_json(_value: JsonbVal) -> anyhow::Result { + fn restore_from_json(_value: JsonbVal) -> ConnectorResult { unimplemented!() } @@ -79,7 +80,7 @@ impl SplitMetaData for IcebergSplit { unimplemented!() } - fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, _start_offset: String) -> ConnectorResult<()> { unimplemented!() } } @@ -95,11 +96,11 @@ impl SplitEnumerator for IcebergSplitEnumerator { async fn new( _properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self {}) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { Ok(vec![]) } } @@ -118,7 +119,7 @@ impl SplitReader for IcebergFileReader { _parser_config: ParserConfig, _source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { unimplemented!() } diff --git a/src/connector/src/source/kafka/enumerator/client.rs b/src/connector/src/source/kafka/enumerator/client.rs index 9850a4c24492..16314d21dbc1 100644 --- a/src/connector/src/source/kafka/enumerator/client.rs +++ b/src/connector/src/source/kafka/enumerator/client.rs @@ -15,12 +15,14 @@ use std::collections::HashMap; use std::time::Duration; -use anyhow::anyhow; +use anyhow::{anyhow, Context as _}; use async_trait::async_trait; use rdkafka::consumer::{BaseConsumer, Consumer}; use rdkafka::error::KafkaResult; use rdkafka::{Offset, TopicPartitionList}; +use risingwave_common::bail; +use crate::error::ConnectorResult; use crate::source::base::SplitEnumerator; use crate::source::kafka::split::KafkaSplit; use crate::source::kafka::{KafkaProperties, PrivateLinkConsumerContext, KAFKA_ISOLATION_LEVEL}; @@ -57,7 +59,7 @@ impl SplitEnumerator for KafkaSplitEnumerator { async fn new( properties: KafkaProperties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let mut config = rdkafka::ClientConfig::new(); let common_props = &properties.common; @@ -77,11 +79,9 @@ impl SplitEnumerator for KafkaSplitEnumerator { Some("earliest") => KafkaEnumeratorOffset::Earliest, Some("latest") => KafkaEnumeratorOffset::Latest, None => KafkaEnumeratorOffset::Earliest, - _ => { - return Err(anyhow!( - "properties `scan_startup_mode` only support earliest and latest or leave it empty" - )); - } + _ => bail!( + "properties `scan_startup_mode` only support earliest and latest or leave it empty" + ), }; if let Some(s) = &properties.time_offset { @@ -105,12 +105,12 @@ impl SplitEnumerator for KafkaSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { - let topic_partitions = self.fetch_topic_partition().await.map_err(|e| { - anyhow!(format!( - "failed to fetch metadata from kafka ({}), error: {}", - self.broker_address, e - )) + async fn list_splits(&mut self) -> ConnectorResult> { + let topic_partitions = self.fetch_topic_partition().await.with_context(|| { + format!( + "failed to fetch metadata from kafka ({})", + self.broker_address + ) })?; let watermarks = self.get_watermarks(topic_partitions.as_ref()).await?; let mut start_offsets = self @@ -153,12 +153,12 @@ impl KafkaSplitEnumerator { &mut self, expect_start_timestamp_millis: Option, expect_stop_timestamp_millis: Option, - ) -> anyhow::Result> { - let topic_partitions = self.fetch_topic_partition().await.map_err(|e| { - anyhow!(format!( - "failed to fetch metadata from kafka ({}), error: {}", - self.broker_address, e - )) + ) -> ConnectorResult> { + let topic_partitions = self.fetch_topic_partition().await.with_context(|| { + format!( + "failed to fetch metadata from kafka ({})", + self.broker_address + ) })?; // here we are getting the start offset and end offset for each partition with the given @@ -349,7 +349,7 @@ impl KafkaSplitEnumerator { .is_ok() } - async fn fetch_topic_partition(&self) -> anyhow::Result> { + async fn fetch_topic_partition(&self) -> ConnectorResult> { // for now, we only support one topic let metadata = self .client @@ -358,11 +358,11 @@ impl KafkaSplitEnumerator { let topic_meta = match metadata.topics() { [meta] => meta, - _ => return Err(anyhow!("topic {} not found", self.topic)), + _ => bail!("topic {} not found", self.topic), }; if topic_meta.partitions().is_empty() { - return Err(anyhow!("topic {} not found", self.topic)); + bail!("topic {} not found", self.topic); } Ok(topic_meta diff --git a/src/connector/src/source/kafka/private_link.rs b/src/connector/src/source/kafka/private_link.rs index 645588f45749..3eebacca09f9 100644 --- a/src/connector/src/source/kafka/private_link.rs +++ b/src/connector/src/source/kafka/private_link.rs @@ -16,12 +16,13 @@ use std::collections::{BTreeMap, HashMap}; use std::str::FromStr; use std::sync::Arc; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use itertools::Itertools; use rdkafka::client::BrokerAddr; use rdkafka::consumer::ConsumerContext; use rdkafka::producer::{DeliveryResult, ProducerContext}; use rdkafka::{ClientContext, Statistics}; +use risingwave_common::bail; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::connection::PrivateLinkService; @@ -29,6 +30,7 @@ use risingwave_pb::catalog::connection::PrivateLinkService; use crate::common::{ AwsPrivateLinkItem, PRIVATE_LINK_BROKER_REWRITE_MAP_KEY, PRIVATE_LINK_TARGETS_KEY, }; +use crate::error::ConnectorResult; use crate::source::kafka::stats::RdKafkaStats; use crate::source::kafka::{KAFKA_PROPS_BROKER_KEY, KAFKA_PROPS_BROKER_KEY_ALIAS}; use crate::source::KAFKA_CONNECTOR; @@ -68,9 +70,9 @@ impl BrokerAddrRewriter { pub fn new( role: PrivateLinkContextRole, broker_rewrite_map: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { tracing::info!("[{}] rewrite map {:?}", role, broker_rewrite_map); - let rewrite_map: anyhow::Result> = broker_rewrite_map + let rewrite_map: ConnectorResult> = broker_rewrite_map .map_or(Ok(BTreeMap::new()), |addr_map| { addr_map .into_iter() @@ -109,7 +111,7 @@ impl PrivateLinkConsumerContext { broker_rewrite_map: Option>, identifier: Option, metrics: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let inner = BrokerAddrRewriter::new(PrivateLinkContextRole::Consumer, broker_rewrite_map)?; Ok(Self { inner, @@ -152,7 +154,7 @@ impl PrivateLinkProducerContext { broker_rewrite_map: Option>, identifier: Option, metrics: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let inner = BrokerAddrRewriter::new(PrivateLinkContextRole::Producer, broker_rewrite_map)?; Ok(Self { inner, @@ -195,11 +197,12 @@ fn kafka_props_broker_key(with_properties: &BTreeMap) -> &str { fn get_property_required( with_properties: &BTreeMap, property: &str, -) -> anyhow::Result { +) -> ConnectorResult { with_properties .get(property) .map(|s| s.to_lowercase()) - .ok_or_else(|| anyhow!("Required property \"{property}\" is not provided")) + .with_context(|| format!("Required property \"{property}\" is not provided")) + .map_err(Into::into) } #[inline(always)] @@ -216,7 +219,7 @@ pub fn insert_privatelink_broker_rewrite_map( with_options: &mut BTreeMap, svc: Option<&PrivateLinkService>, privatelink_endpoint: Option, -) -> anyhow::Result<()> { +) -> ConnectorResult<()> { let mut broker_rewrite_map = HashMap::new(); let servers = get_property_required(with_options, kafka_props_broker_key(with_options))?; let broker_addrs = servers.split(',').collect_vec(); @@ -227,11 +230,11 @@ pub fn insert_privatelink_broker_rewrite_map( with_options.remove(PRIVATE_LINK_TARGETS_KEY); if broker_addrs.len() != link_targets.len() { - return Err(anyhow!( + bail!( "The number of broker addrs {} does not match the number of private link targets {}", broker_addrs.len(), link_targets.len() - )); + ); } if let Some(endpoint) = privatelink_endpoint { @@ -241,15 +244,15 @@ pub fn insert_privatelink_broker_rewrite_map( } } else { if svc.is_none() { - return Err(anyhow!("Privatelink endpoint not found.",)); + bail!("Privatelink endpoint not found."); } let svc = svc.unwrap(); for (link, broker) in link_targets.iter().zip_eq_fast(broker_addrs.into_iter()) { if svc.dns_entries.is_empty() { - return Err(anyhow!( + bail!( "No available private link endpoints for Kafka broker {}", broker - )); + ); } // rewrite the broker address to the dns name w/o az // requires the NLB has enabled the cross-zone load balancing diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index bb8e70471282..d67e45fee383 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::mem::swap; use std::time::Duration; -use anyhow::{Context, Result}; +use anyhow::Context; use async_trait::async_trait; use futures::StreamExt; use futures_async_stream::try_stream; @@ -27,6 +27,7 @@ use rdkafka::error::KafkaError; use rdkafka::{ClientConfig, Message, Offset, TopicPartitionList}; use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::kafka::{ @@ -168,7 +169,7 @@ impl KafkaSplitReader { } impl CommonSplitReader for KafkaSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn into_data_stream(self) { if self.offsets.values().all(|(start_offset, stop_offset)| { match (start_offset, stop_offset) { diff --git a/src/connector/src/source/kafka/split.rs b/src/connector/src/source/kafka/split.rs index 1043cdd01f2f..a98707eb4ab3 100644 --- a/src/connector/src/source/kafka/split.rs +++ b/src/connector/src/source/kafka/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] @@ -32,15 +32,15 @@ impl SplitMetaData for KafkaSplit { format!("{}", self.partition).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset.as_str().parse::().unwrap()); Ok(()) } diff --git a/src/connector/src/source/kinesis/enumerator/client.rs b/src/connector/src/source/kinesis/enumerator/client.rs index bbf4e2925826..b8966f99ae1a 100644 --- a/src/connector/src/source/kinesis/enumerator/client.rs +++ b/src/connector/src/source/kinesis/enumerator/client.rs @@ -12,11 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::Result; +use anyhow::Context as _; use async_trait::async_trait; use aws_sdk_kinesis::types::Shard; use aws_sdk_kinesis::Client as kinesis_client; +use risingwave_common::bail; +use crate::error::ConnectorResult as Result; use crate::source::kinesis::split::{KinesisOffset, KinesisSplit}; use crate::source::kinesis::*; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -56,15 +58,11 @@ impl SplitEnumerator for KinesisSplitEnumerator { .set_next_token(next_token) .stream_name(&self.stream_name) .send() - .await?; + .await + .context("failed to list kinesis shards")?; match list_shard_output.shards { Some(shard) => shard_collect.extend(shard), - None => { - return Err(anyhow::Error::msg(format!( - "no shards in stream {}", - &self.stream_name - ))); - } + None => bail!("no shards in stream {}", &self.stream_name), } match list_shard_output.next_token { diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 51b3c7771041..363dfb8777b1 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -14,7 +14,7 @@ use std::time::Duration; -use anyhow::{anyhow, Result}; +use anyhow::anyhow; use async_trait::async_trait; use aws_sdk_kinesis::error::{DisplayErrorContext, SdkError}; use aws_sdk_kinesis::operation::get_records::{GetRecordsError, GetRecordsOutput}; @@ -22,8 +22,11 @@ use aws_sdk_kinesis::primitives::DateTime; use aws_sdk_kinesis::types::ShardIteratorType; use aws_sdk_kinesis::Client as KinesisClient; use futures_async_stream::try_stream; +use risingwave_common::bail; +use thiserror_ext::AsReport; use tokio_retry; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; use crate::source::kinesis::source::message::from_kinesis_record; use crate::source::kinesis::split::{KinesisOffset, KinesisSplit}; @@ -74,13 +77,11 @@ impl SplitReader for KinesisSplitReader { if let Some(ts) = &properties.timestamp_offset { KinesisOffset::Timestamp(*ts) } else { - return Err(anyhow!("scan.startup.timestamp.millis is required")); + bail!("scan.startup.timestamp.millis is required"); } } _ => { - return Err(anyhow!( - "invalid scan_startup_mode, accept earliest/latest/timestamp" - )) + bail!("invalid scan_startup_mode, accept earliest/latest/timestamp") } }, }, @@ -90,9 +91,7 @@ impl SplitReader for KinesisSplitReader { if !matches!(start_position, KinesisOffset::Timestamp(_)) && properties.timestamp_offset.is_some() { - return Err( - anyhow!("scan.startup.mode need to be set to 'timestamp' if you want to start with a specific timestamp") - ); + bail!("scan.startup.mode need to be set to 'timestamp' if you want to start with a specific timestamp"); } let stream_name = properties.common.stream_name.clone(); @@ -121,7 +120,7 @@ impl SplitReader for KinesisSplitReader { } impl CommonSplitReader for KinesisSplitReader { - #[try_stream(ok = Vec < SourceMessage >, error = anyhow::Error)] + #[try_stream(ok = Vec < SourceMessage >, error = crate::error::ConnectorError)] async fn into_data_stream(mut self) { self.new_shard_iter().await?; loop { @@ -189,14 +188,12 @@ impl CommonSplitReader for KinesisSplitReader { continue; } Err(e) => { - let error_msg = format!( - "Kinesis got a unhandled error: {:?}, stream {:?}, shard {:?}", - DisplayErrorContext(e), - self.stream_name, - self.shard_id, - ); - tracing::error!("{}", error_msg); - return Err(anyhow!("{}", error_msg)); + let error = anyhow!(e).context(format!( + "Kinesis got a unhandled error on stream {:?}, shard {:?}", + self.stream_name, self.shard_id + )); + tracing::error!(error = %error.as_report()); + return Err(error.into()); } } } @@ -251,7 +248,7 @@ impl KinesisSplitReader { if let Some(iter) = resp.shard_iterator() { Ok(iter.to_owned()) } else { - Err(anyhow!("shard iterator is none")) + bail!("shard iterator is none") } } diff --git a/src/connector/src/source/kinesis/split.rs b/src/connector/src/source/kinesis/split.rs index c327e6da61e9..1c7bea61f874 100644 --- a/src/connector/src/source/kinesis/split.rs +++ b/src/connector/src/source/kinesis/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize, Hash)] @@ -39,15 +39,15 @@ impl SplitMetaData for KinesisSplit { self.shard_id.clone() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let start_offset = if start_offset.is_empty() { KinesisOffset::Earliest } else { diff --git a/src/connector/src/source/nats/enumerator/mod.rs b/src/connector/src/source/nats/enumerator/mod.rs index c5059fdc8186..557921747b8f 100644 --- a/src/connector/src/source/nats/enumerator/mod.rs +++ b/src/connector/src/source/nats/enumerator/mod.rs @@ -14,11 +14,12 @@ use std::sync::Arc; -use anyhow; use async_trait::async_trait; +use risingwave_common::bail; use super::source::{NatsOffset, NatsSplit}; use super::NatsProperties; +use crate::error::ConnectorResult; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator, SplitId}; #[derive(Debug, Clone)] @@ -36,7 +37,7 @@ impl SplitEnumerator for NatsSplitEnumerator { async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let client = properties.common.build_client().await?; Ok(Self { subject: properties.common.subject, @@ -45,14 +46,14 @@ impl SplitEnumerator for NatsSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { // Nats currently does not support list_splits API, if we simple return the default 0 without checking the client status, will result executor crash let state = self.client.connection_state(); if state != async_nats::connection::State::Connected { - return Err(anyhow::anyhow!( + bail!( "Nats connection status is not connected, current status is {:?}", state - )); + ); } // TODO: to simplify the logic, return 1 split for first version let nats_split = NatsSplit { diff --git a/src/connector/src/source/nats/source/reader.rs b/src/connector/src/source/nats/source/reader.rs index 20a8f9c0dbc0..05954538b91c 100644 --- a/src/connector/src/source/nats/source/reader.rs +++ b/src/connector/src/source/nats/source/reader.rs @@ -12,14 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result}; +use anyhow::Context as _; use async_nats::jetstream::consumer; use async_trait::async_trait; use futures::StreamExt; use futures_async_stream::try_stream; +use risingwave_common::bail; use super::message::NatsMessage; use super::{NatsOffset, NatsSplit}; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::nats::NatsProperties; @@ -60,15 +62,15 @@ impl SplitReader for NatsSplitReader { "earliest" => NatsOffset::Earliest, "timestamp_millis" => { if let Some(time) = &properties.start_time { - NatsOffset::Timestamp(time.parse()?) + NatsOffset::Timestamp(time.parse().context( + "failed to parse the start time as nats offset timestamp", + )?) } else { - return Err(anyhow!("scan_startup_timestamp_millis is required")); + bail!("scan_startup_timestamp_millis is required"); } } _ => { - return Err(anyhow!( - "invalid scan_startup_mode, accept earliest/latest/timestamp_millis" - )) + bail!("invalid scan_startup_mode, accept earliest/latest/timestamp_millis") } }, }, @@ -101,7 +103,7 @@ impl SplitReader for NatsSplitReader { } impl CommonSplitReader for NatsSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn into_data_stream(self) { let capacity = self.source_ctx.source_ctrl_opts.chunk_size; let messages = self.consumer.messages().await?; diff --git a/src/connector/src/source/nats/split.rs b/src/connector/src/source/nats/split.rs index 1a176102efb6..d3b4ded01901 100644 --- a/src/connector/src/source/nats/split.rs +++ b/src/connector/src/source/nats/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Ok}; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize, Hash)] @@ -43,15 +43,15 @@ impl SplitMetaData for NatsSplit { format!("{}", self.split_id).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_sequence: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_sequence: String) -> ConnectorResult<()> { let start_sequence = if start_sequence.is_empty() { NatsOffset::Earliest } else { diff --git a/src/connector/src/source/nexmark/enumerator/mod.rs b/src/connector/src/source/nexmark/enumerator/mod.rs index b67a58f54896..7a53f02489fb 100644 --- a/src/connector/src/source/nexmark/enumerator/mod.rs +++ b/src/connector/src/source/nexmark/enumerator/mod.rs @@ -14,6 +14,7 @@ use async_trait::async_trait; +use crate::error::ConnectorResult; use crate::source::nexmark::split::NexmarkSplit; use crate::source::nexmark::NexmarkProperties; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -32,12 +33,12 @@ impl SplitEnumerator for NexmarkSplitEnumerator { async fn new( properties: NexmarkProperties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let split_num = properties.split_num; Ok(Self { split_num }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { let mut splits = vec![]; for i in 0..self.split_num { splits.push(NexmarkSplit { @@ -52,9 +53,8 @@ impl SplitEnumerator for NexmarkSplitEnumerator { #[cfg(test)] mod tests { - use anyhow::Result; - use super::*; + use crate::error::ConnectorResult as Result; use crate::source::SplitMetaData; #[tokio::test] diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index e7621e532552..fd68348d6faf 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -27,6 +27,7 @@ use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; use tokio::time::Instant; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::data_gen_util::spawn_data_generation_stream; use crate::source::nexmark::source::combined_event::{ @@ -64,7 +65,7 @@ impl SplitReader for NexmarkSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { tracing::debug!("Splits for nexmark found! {:?}", splits); assert!(splits.len() == 1); // TODO: currently, assume there's only one split in one reader @@ -163,7 +164,7 @@ impl NexmarkSplitReader { } } - #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] + #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_native_stream(mut self) { let start_time = Instant::now(); let start_offset = self.generator.global_offset(); @@ -213,7 +214,7 @@ mod tests { use crate::source::{SourceEnumeratorContext, SplitEnumerator}; #[tokio::test] - async fn test_nexmark_split_reader() -> anyhow::Result<()> { + async fn test_nexmark_split_reader() -> crate::error::ConnectorResult<()> { let props = NexmarkProperties { split_num: 2, min_event_gap_in_ns: 0, @@ -247,7 +248,7 @@ mod tests { } #[tokio::test] - async fn test_nexmark_event_num() -> anyhow::Result<()> { + async fn test_nexmark_event_num() -> crate::error::ConnectorResult<()> { let max_chunk_size = 32; let event_num = max_chunk_size * 128 + 1; let props = NexmarkProperties { diff --git a/src/connector/src/source/nexmark/split.rs b/src/connector/src/source/nexmark/split.rs index d68aa2c8e1aa..5150f1b6a1e1 100644 --- a/src/connector/src/source/nexmark/split.rs +++ b/src/connector/src/source/nexmark/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq, Hash)] @@ -31,15 +31,15 @@ impl SplitMetaData for NexmarkSplit { format!("{}-{}", self.split_num, self.split_index).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset.as_str().parse::().unwrap()); Ok(()) } diff --git a/src/connector/src/source/pulsar/enumerator/client.rs b/src/connector/src/source/pulsar/enumerator/client.rs index d92e63306061..dddcf927d0c3 100644 --- a/src/connector/src/source/pulsar/enumerator/client.rs +++ b/src/connector/src/source/pulsar/enumerator/client.rs @@ -12,12 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, bail, Result}; +use anyhow::anyhow; use async_trait::async_trait; use itertools::Itertools; use pulsar::{Pulsar, TokioExecutor}; +use risingwave_common::bail; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::topic::{parse_topic, Topic}; use crate::source::pulsar::PulsarProperties; @@ -45,7 +47,7 @@ impl SplitEnumerator for PulsarSplitEnumerator { async fn new( properties: PulsarProperties, _context: SourceEnumeratorContextRef, - ) -> Result { + ) -> ConnectorResult { let pulsar = properties .common .build_client(&properties.oauth, &properties.aws_auth_props) @@ -80,7 +82,7 @@ impl SplitEnumerator for PulsarSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { let offset = self.start_offset.clone(); // MessageId is only used when recovering from a State assert!(!matches!(offset, PulsarEnumeratorOffset::MessageId(_))); diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 139af839bd16..9ed810dfc933 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -31,7 +31,9 @@ use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioE use risingwave_common::array::{DataChunk, StreamChunk}; use risingwave_common::catalog::ROWID_PREFIX; use risingwave_common::{bail, ensure}; +use thiserror_ext::AsReport; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; @@ -56,7 +58,7 @@ impl SplitReader for PulsarSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { ensure!(splits.len() == 1, "only support single split"); let split = splits.into_iter().next().unwrap(); let topic = split.topic.to_string(); @@ -106,7 +108,7 @@ pub struct PulsarBrokerReader { } // {ledger_id}:{entry_id}:{partition}:{batch_index} -fn parse_message_id(id: &str) -> anyhow::Result { +fn parse_message_id(id: &str) -> ConnectorResult { let splits = id.split(':').collect_vec(); if splits.len() < 2 || splits.len() > 4 { @@ -150,7 +152,7 @@ impl SplitReader for PulsarBrokerReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { ensure!(splits.len() == 1, "only support single split"); let split = splits.into_iter().next().unwrap(); let pulsar = props @@ -233,7 +235,7 @@ impl SplitReader for PulsarBrokerReader { } impl CommonSplitReader for PulsarBrokerReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn into_data_stream(self) { let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; #[for_await] @@ -278,7 +280,7 @@ impl PulsarIcebergReader { } } - async fn scan(&self) -> anyhow::Result { + async fn scan(&self) -> ConnectorResult { let table = self.create_iceberg_table().await?; let schema = table.current_table_metadata().current_schema()?; tracing::debug!("Created iceberg pulsar table, schema is: {:?}", schema,); @@ -321,12 +323,13 @@ impl PulsarIcebergReader { .new_scan_builder() .with_partition_value(partition_value) .with_batch_size(max_chunk_size) - .build()? + .build() + .context("failed to build iceberg table scan")? .scan(&table) .await?) } - async fn create_iceberg_table(&self) -> anyhow::Result
{ + async fn create_iceberg_table(&self) -> ConnectorResult
{ let catalog = load_catalog(&self.build_iceberg_configs()?) .await .context("Unable to load iceberg catalog")?; @@ -340,7 +343,7 @@ impl PulsarIcebergReader { Ok(table) } - #[try_stream(ok = (StreamChunk, HashMap), error = anyhow::Error)] + #[try_stream(ok = (StreamChunk, HashMap), error = crate::error::ConnectorError)] async fn as_stream_chunk_stream(&self) { #[for_await] for file_scan in self.scan().await? { @@ -355,7 +358,7 @@ impl PulsarIcebergReader { } } - #[try_stream(ok = StreamChunk, error = anyhow::Error)] + #[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_stream(self) { let (props, mut split, parser_config, source_ctx) = ( self.props.clone(), @@ -368,8 +371,9 @@ impl PulsarIcebergReader { #[for_await] for msg in self.as_stream_chunk_stream() { - let (_chunk, mapping) = - msg.inspect_err(|e| tracing::error!("Failed to read message from iceberg: {}", e))?; + let (_chunk, mapping) = msg.inspect_err( + |e| tracing::error!(error = %e.as_report(), "Failed to read message from iceberg"), + )?; last_msg_id = mapping.get(self.split.topic.to_string().as_str()).cloned(); } @@ -394,7 +398,7 @@ impl PulsarIcebergReader { } } - fn build_iceberg_configs(&self) -> anyhow::Result> { + fn build_iceberg_configs(&self) -> ConnectorResult> { let mut iceberg_configs = HashMap::new(); let bucket = self @@ -451,7 +455,7 @@ impl PulsarIcebergReader { fn convert_record_batch_to_source_with_state( &self, record_batch: &RecordBatch, - ) -> anyhow::Result<(StreamChunk, HashMap)> { + ) -> ConnectorResult<(StreamChunk, HashMap)> { let mut offsets = Vec::with_capacity(record_batch.num_rows()); let ledger_id_array = record_batch @@ -493,7 +497,8 @@ impl PulsarIcebergReader { .iter() .filter(|col| col.name != ROWID_PREFIX) .map(|col| record_batch.schema().index_of(col.name.as_str())) - .try_collect()?; + .try_collect() + .context("failed to look up column name in arrow record batch")?; for row in 0..record_batch.num_rows() { let offset = format!( @@ -507,7 +512,8 @@ impl PulsarIcebergReader { offsets.push(offset); } - let data_chunk = DataChunk::try_from(&record_batch.project(&field_indices)?)?; + let data_chunk = DataChunk::try_from(&record_batch.project(&field_indices)?) + .context("failed to convert arrow record batch to data chunk")?; let stream_chunk = StreamChunk::from(data_chunk); diff --git a/src/connector/src/source/pulsar/split.rs b/src/connector/src/source/pulsar/split.rs index 36f9bc47e3ec..bf9b63d99d74 100644 --- a/src/connector/src/source/pulsar/split.rs +++ b/src/connector/src/source/pulsar/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::pulsar::topic::Topic; use crate::source::pulsar::PulsarEnumeratorOffset; use crate::source::{SplitId, SplitMetaData}; @@ -32,15 +32,15 @@ impl SplitMetaData for PulsarSplit { self.topic.to_string().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let start_offset = if start_offset.is_empty() { PulsarEnumeratorOffset::Earliest } else { diff --git a/src/connector/src/source/pulsar/topic.rs b/src/connector/src/source/pulsar/topic.rs index 4512662d6252..352c7e47d8da 100644 --- a/src/connector/src/source/pulsar/topic.rs +++ b/src/connector/src/source/pulsar/topic.rs @@ -12,10 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result}; +use anyhow::anyhow; +use risingwave_common::bail; use serde::{Deserialize, Serialize}; use urlencoding::encode; +use crate::error::ConnectorResult as Result; + const PERSISTENT_DOMAIN: &str = "persistent"; const NON_PERSISTENT_DOMAIN: &str = "non-persistent"; const PUBLIC_TENANT: &str = "public"; @@ -59,7 +62,7 @@ impl Topic { pub fn sub_topic(&self, partition: i32) -> Result { if partition < 0 { - return Err(anyhow!("invalid partition index number")); + bail!("invalid partition index number"); } if self.topic.contains(PARTITIONED_TOPIC_SUFFIX) { @@ -119,11 +122,11 @@ pub fn parse_topic(topic: &str) -> Result { ), 3 => format!("{}://{}", PERSISTENT_DOMAIN, topic), _ => { - return Err(anyhow!( + bail!( "Invalid short topic name '{}', \ it should be in the format of // or ", topic - )); + ); } }; } @@ -133,10 +136,10 @@ pub fn parse_topic(topic: &str) -> Result { let domain = match parts[0] { PERSISTENT_DOMAIN | NON_PERSISTENT_DOMAIN => parts[0], _ => { - return Err(anyhow!( + bail!( "The domain only can be specified as 'persistent' or 'non-persistent'. Input domain is '{}'", parts[0] - )); + ); } }; @@ -144,10 +147,10 @@ pub fn parse_topic(topic: &str) -> Result { let parts: Vec<&str> = rest.splitn(3, '/').collect(); if parts.len() != 3 { - return Err(anyhow!( + bail!( "invalid topic name '{}', it should be in the format of //", rest - )); + ); } let parsed_topic = Topic { @@ -159,7 +162,7 @@ pub fn parse_topic(topic: &str) -> Result { }; if parsed_topic.topic.is_empty() { - return Err(anyhow!("topic name cannot be empty".to_string(),)); + bail!("topic name cannot be empty".to_string()); } Ok(parsed_topic) diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index a842b091ab92..46107c2d73d0 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -25,6 +25,7 @@ use risingwave_pb::plan_common::{AdditionalColumn, PbColumnCatalog}; #[expect(deprecated)] use super::fs_reader::FsSourceReader; use super::reader::SourceReader; +use crate::error::ConnectorResult; use crate::parser::additional_columns::{ build_additional_column_catalog, COMMON_COMPATIBLE_ADDITIONAL_COLUMNS, COMPATIBLE_ADDITIONAL_COLUMNS, @@ -176,7 +177,7 @@ impl SourceDescBuilder { columns } - pub fn build(self) -> anyhow::Result { + pub fn build(self) -> ConnectorResult { let columns = self.column_catalogs_to_source_column_descs(); let psrser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; @@ -201,7 +202,7 @@ impl SourceDescBuilder { #[deprecated = "will be replaced by new fs source (list + fetch)"] #[expect(deprecated)] - pub fn build_fs_source_desc(&self) -> anyhow::Result { + pub fn build_fs_source_desc(&self) -> ConnectorResult { let parser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; match ( diff --git a/src/connector/src/source/reader/fs_reader.rs b/src/connector/src/source/reader/fs_reader.rs index f64a9def6aab..93a0bd2c2d6a 100644 --- a/src/connector/src/source/reader/fs_reader.rs +++ b/src/connector/src/source/reader/fs_reader.rs @@ -23,6 +23,7 @@ use futures::StreamExt; use risingwave_common::catalog::ColumnId; use crate::dispatch_source_prop; +use crate::error::ConnectorResult; use crate::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use crate::source::{ create_split_reader, BoxChunkSourceStream, ConnectorProperties, ConnectorState, @@ -44,7 +45,7 @@ impl FsSourceReader { columns: Vec, connector_node_addr: Option, parser_config: SpecificParserConfig, - ) -> anyhow::Result { + ) -> ConnectorResult { // Store the connector node address to properties for later use. let mut source_props: HashMap = HashMap::from_iter(properties.clone()); connector_node_addr @@ -62,7 +63,7 @@ impl FsSourceReader { fn get_target_columns( &self, column_ids: Vec, - ) -> anyhow::Result> { + ) -> ConnectorResult> { column_ids .iter() .map(|id| { @@ -75,6 +76,7 @@ impl FsSourceReader { .cloned() }) .try_collect() + .map_err(Into::into) } pub async fn to_stream( @@ -82,7 +84,7 @@ impl FsSourceReader { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> anyhow::Result { + ) -> ConnectorResult { let config = self.config.clone(); let columns = self.get_target_columns(column_ids)?; diff --git a/src/connector/src/source/reader/reader.rs b/src/connector/src/source/reader/reader.rs index ba9bd4dded4d..833c9661c3ca 100644 --- a/src/connector/src/source/reader/reader.rs +++ b/src/connector/src/source/reader/reader.rs @@ -24,8 +24,10 @@ use itertools::Itertools; use risingwave_common::bail; use risingwave_common::catalog::ColumnId; use rw_futures_util::select_all; +use thiserror_ext::AsReport as _; use crate::dispatch_source_prop; +use crate::error::ConnectorResult; use crate::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use crate::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use crate::source::filesystem::opendal_source::{ @@ -51,7 +53,7 @@ impl SourceReader { columns: Vec, connector_message_buffer_size: usize, parser_config: SpecificParserConfig, - ) -> anyhow::Result { + ) -> ConnectorResult { let config = ConnectorProperties::extract(properties, false)?; Ok(Self { @@ -65,7 +67,7 @@ impl SourceReader { fn get_target_columns( &self, column_ids: Vec, - ) -> anyhow::Result> { + ) -> ConnectorResult> { column_ids .iter() .map(|id| { @@ -78,9 +80,10 @@ impl SourceReader { .cloned() }) .try_collect() + .map_err(Into::into) } - pub fn get_source_list(&self) -> anyhow::Result> { + pub fn get_source_list(&self) -> ConnectorResult> { let config = self.config.clone(); match config { ConnectorProperties::Gcs(prop) => { @@ -107,7 +110,7 @@ impl SourceReader { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> anyhow::Result { + ) -> ConnectorResult { let Some(splits) = state else { return Ok(pending().boxed()); }; @@ -165,7 +168,7 @@ impl SourceReader { } } -#[try_stream(boxed, ok = FsPageItem, error = anyhow::Error)] +#[try_stream(boxed, ok = FsPageItem, error = crate::error::ConnectorError)] async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { let matcher = lister.get_matcher(); let mut object_metadata_iter = lister.list().await?; @@ -185,7 +188,7 @@ async fn build_opendal_fs_list_stream(lister: OpendalEnumera } } Err(err) => { - tracing::error!("list object fail, err {}", err); + tracing::error!(error = %err.as_report(), "list object fail"); return Err(err); } } diff --git a/src/connector/src/source/test_source.rs b/src/connector/src/source/test_source.rs index e0b901ddbf25..6d224593d7a2 100644 --- a/src/connector/src/source/test_source.rs +++ b/src/connector/src/source/test_source.rs @@ -15,13 +15,14 @@ use std::collections::HashMap; use std::sync::{Arc, OnceLock}; -use anyhow::anyhow; use async_trait::async_trait; use parking_lot::Mutex; +use risingwave_common::bail; use risingwave_common::types::JsonbVal; use serde_derive::{Deserialize, Serialize}; use with_options::WithOptions; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::{ BoxChunkSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, @@ -32,7 +33,7 @@ pub type BoxListSplits = Box< dyn FnMut( TestSourceProperties, SourceEnumeratorContextRef, - ) -> anyhow::Result> + ) -> ConnectorResult> + Send + 'static, >; @@ -59,7 +60,7 @@ impl BoxSource { list_splits: impl FnMut( TestSourceProperties, SourceEnumeratorContextRef, - ) -> anyhow::Result> + ) -> ConnectorResult> + Send + 'static, into_source_stream: impl FnMut( @@ -124,11 +125,11 @@ impl TryFromHashmap for TestSourceProperties { fn try_from_hashmap( props: HashMap, _deny_unknown_fields: bool, - ) -> anyhow::Result { + ) -> ConnectorResult { if cfg!(any(madsim, test)) { Ok(TestSourceProperties { properties: props }) } else { - Err(anyhow!("test source only available at test")) + bail!("test source only available at test") } } } @@ -149,11 +150,11 @@ impl SplitMetaData for TestSourceSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.offset = start_offset; Ok(()) } @@ -172,14 +173,14 @@ impl SplitEnumerator for TestSourceSplitEnumerator { async fn new( properties: Self::Properties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self { properties, context, }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { (get_registry() .box_source .lock() @@ -208,7 +209,7 @@ impl SplitReader for TestSourceSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self { properties, state, diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 5c1daa024afb..9898e26a79ba 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -17,6 +17,7 @@ use risingwave_common::array::ArrayError; use risingwave_common::error::{BoxedError, NoFunction, NotImplemented}; use risingwave_common::session_config::SessionConfigError; use risingwave_common::util::value_encoding::error::ValueEncodingError; +use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; use risingwave_expr::ExprError; use risingwave_pb::PbFieldNotFound; @@ -208,6 +209,12 @@ impl From for RwError { } } +impl From for RwError { + fn from(e: ConnectorError) -> Self { + ErrorCode::ConnectorError(e.into()).into() + } +} + impl From for RwError { fn from(err: PbFieldNotFound) -> Self { ErrorCode::InternalError(format!( diff --git a/src/frontend/src/scheduler/error.rs b/src/frontend/src/scheduler/error.rs index f68f72b8727e..590c235e1390 100644 --- a/src/frontend/src/scheduler/error.rs +++ b/src/frontend/src/scheduler/error.rs @@ -14,6 +14,7 @@ use risingwave_batch::error::BatchError; use risingwave_common::session_config::QueryMode; +use risingwave_connector::error::ConnectorError; use risingwave_rpc_client::error::RpcError; use thiserror::Error; use tonic::{Code, Status}; @@ -63,6 +64,13 @@ pub enum SchedulerError { BatchError, ), + #[error(transparent)] + Connector( + #[from] + #[backtrace] + ConnectorError, + ), + #[error(transparent)] Internal( #[from] diff --git a/src/meta/service/src/cloud_service.rs b/src/meta/service/src/cloud_service.rs index 2ee28b1427ed..9c213bd7cb9e 100644 --- a/src/meta/service/src/cloud_service.rs +++ b/src/meta/service/src/cloud_service.rs @@ -18,6 +18,7 @@ use std::sync::LazyLock; use async_trait::async_trait; use regex::Regex; use risingwave_connector::dispatch_source_prop; +use risingwave_connector::error::ConnectorResult; use risingwave_connector::source::kafka::private_link::insert_privatelink_broker_rewrite_map; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, @@ -135,7 +136,7 @@ impl CloudService for CloudServiceImpl { { return Ok(new_rwc_validate_fail_response( ErrorType::PrivatelinkResolveErr, - e.to_string(), + e.to_report_string(), )); } } else { @@ -151,13 +152,13 @@ impl CloudService for CloudServiceImpl { if let Err(e) = props { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaInvalidProperties, - e.to_string(), + e.to_report_string(), )); }; async fn new_enumerator( props: P, - ) -> Result { + ) -> ConnectorResult { P::SplitEnumerator::new(props, SourceEnumeratorContext::default().into()).await } @@ -166,15 +167,15 @@ impl CloudService for CloudServiceImpl { if let Err(e) = enumerator { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaInvalidProperties, - e.to_string(), + e.to_report_string(), )); } if let Err(e) = enumerator.unwrap().list_splits().await { - let error_message = e.to_string(); + let error_message = e.to_report_string(); if error_message.contains("BrokerTransportFailure") { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaBrokerUnreachable, - e.to_string(), + e.to_report_string(), )); } static TOPIC_NOT_FOUND: LazyLock = @@ -182,12 +183,12 @@ impl CloudService for CloudServiceImpl { if TOPIC_NOT_FOUND.is_match(error_message.as_str()) { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaTopicNotFound, - e.to_string(), + e.to_report_string(), )); } return Ok(new_rwc_validate_fail_response( ErrorType::KafkaOther, - e.to_string(), + e.to_report_string(), )); } }); diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index e26e1af0f0cf..a1efaa756bb4 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -845,7 +845,7 @@ impl CatalogController { let ret = src_manager.register_source(&pb_source).await; if let Err(e) = ret { txn.rollback().await?; - return Err(e.into()); + return Err(e); } } txn.commit().await?; diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 9833a51cc193..18230bf74c21 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -14,6 +14,7 @@ use aws_sdk_ec2::error::DisplayErrorContext; use risingwave_common::error::BoxedError; +use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; use risingwave_pb::PbFieldNotFound; use risingwave_rpc_client::error::{RpcError, ToTonicStatus}; @@ -88,6 +89,13 @@ pub enum MetaErrorInner { #[error("SystemParams error: {0}")] SystemParams(String), + #[error(transparent)] + Connector( + #[from] + #[backtrace] + ConnectorError, + ), + #[error("Sink error: {0}")] Sink( #[from] diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 7cff9dc4a9b7..908c62ebdffe 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -31,6 +31,7 @@ use risingwave_common::util::stream_graph_visitor::{ }; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::dispatch_source_prop; +use risingwave_connector::error::ConnectorError; use risingwave_connector::source::cdc::CdcSourceType; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, @@ -436,7 +437,7 @@ impl DdlController { mgr.catalog_manager .cancel_create_source_procedure(&source) .await?; - return Err(e.into()); + return Err(e); } mgr.catalog_manager @@ -804,7 +805,7 @@ impl DdlController { pub(crate) async fn validate_cdc_table( table: &Table, table_fragments: &TableFragments, - ) -> anyhow::Result<()> { + ) -> MetaResult<()> { let stream_scan_fragment = table_fragments .fragments .values() @@ -820,7 +821,7 @@ impl DdlController { async fn new_enumerator_for_validate( source_props: P, - ) -> Result { + ) -> Result { P::SplitEnumerator::new(source_props, SourceEnumeratorContext::default().into()).await } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 8af470ce7df6..eb3d6b3205c4 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -20,10 +20,11 @@ use std::ops::Deref; use std::sync::Arc; use std::time::Duration; -use anyhow::{anyhow, Context}; +use anyhow::Context; use risingwave_common::catalog::TableId; use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_connector::dispatch_source_prop; +use risingwave_connector::error::ConnectorResult; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceEnumeratorInfo, SourceProperties, SplitEnumerator, SplitId, SplitImpl, SplitMetaData, @@ -81,12 +82,12 @@ struct ConnectorSourceWorker { source_is_up: LabelGuardedIntGauge<2>, } -fn extract_prop_from_existing_source(source: &Source) -> anyhow::Result { +fn extract_prop_from_existing_source(source: &Source) -> ConnectorResult { let mut properties = ConnectorProperties::extract(source.with_properties.clone(), false)?; properties.init_from_pb_source(source); Ok(properties) } -fn extract_prop_from_new_source(source: &Source) -> anyhow::Result { +fn extract_prop_from_new_source(source: &Source) -> ConnectorResult { let mut properties = ConnectorProperties::extract(source.with_properties.clone(), true)?; properties.init_from_pb_source(source); Ok(properties) @@ -96,7 +97,7 @@ const DEFAULT_SOURCE_WORKER_TICK_INTERVAL: Duration = Duration::from_secs(30); impl ConnectorSourceWorker

{ /// Recreate the `SplitEnumerator` to establish a new connection to the external source service. - async fn refresh(&mut self) -> anyhow::Result<()> { + async fn refresh(&mut self) -> MetaResult<()> { let enumerator = P::SplitEnumerator::new( self.connector_properties.clone(), Arc::new(SourceEnumeratorContext { @@ -124,7 +125,7 @@ impl ConnectorSourceWorker

{ period: Duration, splits: Arc>, metrics: Arc, - ) -> anyhow::Result { + ) -> MetaResult { let enumerator = P::SplitEnumerator::new( connector_properties.clone(), Arc::new(SourceEnumeratorContext { @@ -711,7 +712,7 @@ impl SourceManager { let handle = core .managed_sources .get(&source_id) - .ok_or_else(|| anyhow!("could not found source {}", source_id))?; + .with_context(|| format!("could not find source {}", source_id))?; if handle.splits.lock().await.splits.is_none() { // force refresh source @@ -758,7 +759,7 @@ impl SourceManager { } /// register connector worker for source. - pub async fn register_source(&self, source: &Source) -> anyhow::Result<()> { + pub async fn register_source(&self, source: &Source) -> MetaResult<()> { let mut core = self.core.lock().await; if core.managed_sources.contains_key(&source.get_id()) { tracing::warn!("source {} already registered", source.get_id()); @@ -823,7 +824,7 @@ impl SourceManager { break worker; } Err(e) => { - tracing::warn!("failed to create source worker: {}", e); + tracing::warn!(error = %e.as_report(), "failed to create source worker"); } } }; @@ -852,7 +853,7 @@ impl SourceManager { source: &Source, managed_sources: &mut HashMap, metrics: Arc, - ) -> anyhow::Result<()> { + ) -> MetaResult<()> { tracing::info!("spawning new watcher for source {}", source.id); let splits = Arc::new(Mutex::new(SharedSplitMap { splits: None })); @@ -879,11 +880,12 @@ impl SourceManager { // in kafka tokio::time::timeout(Self::DEFAULT_SOURCE_TICK_TIMEOUT, worker.tick()) .await - .map_err(|_e| { - anyhow!( - "failed to fetch meta info for source {}, error: timeout {}", + .ok() + .with_context(|| { + format!( + "failed to fetch meta info for source {}, timeout {:?}", source.id, - Self::DEFAULT_SOURCE_TICK_TIMEOUT.as_secs() + Self::DEFAULT_SOURCE_TICK_TIMEOUT ) })??; @@ -984,8 +986,8 @@ pub fn build_actor_split_impls( mod tests { use std::collections::{BTreeMap, HashMap, HashSet}; - use anyhow::anyhow; use risingwave_common::types::JsonbVal; + use risingwave_connector::error::ConnectorResult; use risingwave_connector::source::{SplitId, SplitMetaData}; use serde::{Deserialize, Serialize}; @@ -1006,11 +1008,11 @@ mod tests { serde_json::to_value(*self).unwrap().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, _start_offset: String) -> ConnectorResult<()> { Ok(()) } } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index a2478cdb6bb0..95894429e936 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -25,6 +25,7 @@ use futures_async_stream::try_stream; use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_connector::error::ConnectorError; use risingwave_connector::source::reader::desc::{FsSourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, @@ -442,7 +443,7 @@ impl FsSourceExecutor { self.stream_source_core.latest_split_info.get_mut(id).map( |origin_split| { origin_split.update_in_place(offset.clone())?; - Ok::<_, anyhow::Error>((id.clone(), origin_split.clone())) + Ok::<_, ConnectorError>((id.clone(), origin_split.clone())) }, ) })