Skip to content

Commit

Permalink
fix(batch, source): Propagate user errors (risingwavelabs#8493)
Browse files Browse the repository at this point in the history
Signed-off-by: Runji Wang <wangrunji0408@163.com>
Co-authored-by: jon-chuang <jon-chuang@users.noreply.github.com>
Co-authored-by: Runji Wang <wangrunji0408@163.com>
  • Loading branch information
3 people committed Mar 15, 2023
1 parent 81b4d59 commit 24fe1e8
Show file tree
Hide file tree
Showing 3 changed files with 12 additions and 6 deletions.
3 changes: 2 additions & 1 deletion src/connector/src/macros.rs
Original file line number Diff line number Diff line change
Expand Up @@ -182,8 +182,9 @@ macro_rules! impl_common_parser_logic {
if let Err(e) = self.parse_inner(content.as_ref(), builder.row_writer())
.await
{
self.source_ctx.report_stream_source_error(&e);
tracing::warn!("message parsing failed {}, skipping", e.to_string());
// This will throw an error for batch
self.source_ctx.report_user_source_error(e)?;
continue;
}

Expand Down
9 changes: 5 additions & 4 deletions src/connector/src/source/base.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ use parking_lot::Mutex;
use prost::Message;
use risingwave_common::array::StreamChunk;
use risingwave_common::catalog::TableId;
use risingwave_common::error::{ErrorCode, ErrorSuppressor, RwError};
use risingwave_common::error::{ErrorCode, ErrorSuppressor, Result as RwResult, RwError};
use risingwave_pb::connector_service::TableSchema;
use risingwave_pb::source::ConnectorSplit;
use serde::{Deserialize, Serialize};
Expand Down Expand Up @@ -107,10 +107,10 @@ impl SourceContext {
self.error_suppressor = Some(error_suppressor)
}

pub(crate) fn report_stream_source_error(&self, e: &RwError) {
// Do not report for batch
pub(crate) fn report_user_source_error(&self, e: RwError) -> RwResult<()> {
// Repropagate the error if batch
if self.source_info.fragment_id == u32::MAX {
return;
return Err(e);
}
let mut err_str = e.inner().to_string();
if let Some(suppressor) = &self.error_suppressor &&
Expand All @@ -130,6 +130,7 @@ impl SourceContext {
&self.source_info.source_id.table_id.to_string(),
])
.inc();
Ok(())
}
}

Expand Down
6 changes: 5 additions & 1 deletion src/tests/simulation/src/kafka.rs
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,11 @@ pub async fn producer(broker_addr: &str, datadir: String) {
// binary message data, a file is a message
Box::new(std::iter::once(content.as_slice()))
} else {
Box::new(content.split(|&b| b == b'\n'))
Box::new(
content
.split(|&b| b == b'\n')
.filter(|line| !line.is_empty()),
)
};
for msg in msgs {
loop {
Expand Down

0 comments on commit 24fe1e8

Please sign in to comment.