Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Propagating the error generated by the input stream and continue polling #12642

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
36 changes: 26 additions & 10 deletions datafusion/physical-plan/src/repartition/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -798,7 +798,21 @@ impl RepartitionExec {

// Input is done
let batch = match result {
Some(result) => result?,
Some(Ok(result)) => result,
Some(Err(e)) => {
// Error from running input task. Propagate error to all output partitions
let e = Arc::new(e);

for (tx, _) in output_channels.values() {
// wrap it because need to send error to all output partitions
let err =
Err(DataFusionError::External(Box::new(Arc::clone(&e))));
tx.send(Some(err)).await.ok();
}

// Continue pulling inputs
continue;
}
None => break,
};

Expand Down Expand Up @@ -1242,23 +1256,25 @@ mod tests {
let err = exec_err!("bad data error");

let schema = batch.schema();
let input = MockExec::new(vec![Ok(batch), err], schema);
let input = MockExec::new(vec![err, Ok(batch.clone())], schema);
let partitioning = Partitioning::RoundRobinBatch(1);
let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();

// Note: this should pass (the stream can be created) but the
// error when the input is executed should get passed back
let output_stream = exec.execute(0, task_ctx).unwrap();
let mut output_stream = exec.execute(0, task_ctx).unwrap();

// Expect that an error is returned
let result_string = crate::common::collect(output_stream)
.await
.unwrap_err()
.to_string();
// Ensure the repartition could poll the stream continuously even if error happens
let error_string = output_stream.next().await.unwrap().unwrap_err().to_string();
assert!(
result_string.contains("bad data error"),
"actual: {result_string}"
error_string.contains("bad data error"),
"actual: {error_string}"
);

let result = output_stream.next().await.unwrap().unwrap();
assert_eq!(result, batch);

assert!(output_stream.next().await.is_none());
}

#[tokio::test]
Expand Down
5 changes: 3 additions & 2 deletions datafusion/physical-plan/src/sorts/merge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ impl<C: CursorValues> SortPreservingMergeStream<C> {
for i in remaining_partitions {
match self.maybe_poll_stream(cx, i) {
Poll::Ready(Err(e)) => {
self.aborted = true;
// Propagating the input error
return Poll::Ready(Some(Err(e)));
}
Poll::Pending => {
Expand Down Expand Up @@ -210,7 +210,8 @@ impl<C: CursorValues> SortPreservingMergeStream<C> {
if !self.loser_tree_adjusted {
let winner = self.loser_tree[0];
if let Err(e) = ready!(self.maybe_poll_stream(cx, winner)) {
self.aborted = true;
// Propagate the error from input. The next poll call will poll the
// same stream
return Poll::Ready(Some(Err(e)));
}
self.update_loser_tree();
Expand Down
205 changes: 183 additions & 22 deletions datafusion/physical-plan/src/sorts/sort.rs
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ use datafusion_execution::TaskContext;
use datafusion_physical_expr::LexOrdering;
use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement;

use futures::{StreamExt, TryStreamExt};
use futures::{Future, StreamExt, TryStreamExt};
use log::{debug, trace};

struct ExternalSorterMetrics {
Expand Down Expand Up @@ -887,7 +887,7 @@ impl ExecutionPlan for SortExec {
) -> Result<SendableRecordBatchStream> {
trace!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());

let mut input = self.input.execute(partition, Arc::clone(&context))?;
let input = self.input.execute(partition, Arc::clone(&context))?;

let execution_options = &context.session_config().options().execution;

Expand All @@ -911,7 +911,7 @@ impl ExecutionPlan for SortExec {
))),
(true, None) => Ok(input),
(false, Some(fetch)) => {
let mut topk = TopK::try_new(
let topk = TopK::try_new(
partition,
input.schema(),
self.expr.clone(),
Expand All @@ -923,18 +923,15 @@ impl ExecutionPlan for SortExec {
)?;
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
futures::stream::once(async move {
while let Some(batch) = input.next().await {
let batch = batch?;
topk.insert_batch(batch)?;
}
topk.emit()
})
.try_flatten(),
futures::stream::unfold(
SortStreamState::new(input, topk),
SortStreamState::poll_next,
)
.fuse(),
)))
}
(false, None) => {
let mut sorter = ExternalSorter::new(
let sorter = ExternalSorter::new(
partition,
input.schema(),
self.expr.clone(),
Expand All @@ -947,14 +944,11 @@ impl ExecutionPlan for SortExec {
);
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
futures::stream::once(async move {
while let Some(batch) = input.next().await {
let batch = batch?;
sorter.insert_batch(batch).await?;
}
sorter.sort()
})
.try_flatten(),
futures::stream::unfold(
SortStreamState::new(input, sorter),
SortStreamState::poll_next,
)
.fuse(),
)))
}
}
Expand All @@ -977,6 +971,107 @@ impl ExecutionPlan for SortExec {
}
}

/// Sorter that sorts a stream of batches
trait Sorter: Send + 'static {
/// Sink the record batch into the sorter
fn sink(&mut self, batch: RecordBatch) -> impl Future<Output = Result<()>> + Send;

/// Get the source stream
fn source(self) -> Result<SendableRecordBatchStream>;
}

impl Sorter for TopK {
async fn sink(&mut self, batch: RecordBatch) -> Result<()> {
self.insert_batch(batch)
}

fn source(self) -> Result<SendableRecordBatchStream> {
self.emit()
}
}

impl Sorter for ExternalSorter {
fn sink(&mut self, batch: RecordBatch) -> impl Future<Output = Result<()>> + Send {
self.insert_batch(batch)
}

fn source(mut self) -> Result<SendableRecordBatchStream> {
self.sort()
}
}

enum SortStreamState<T> {
SinkPhase {
input: SendableRecordBatchStream,
sorter: Option<T>,
},
SourcePhase {
output: SendableRecordBatchStream,
},
}

impl<T: Sorter> SortStreamState<T> {
fn new(input: SendableRecordBatchStream, sorter: T) -> Self {
Self::SinkPhase {
input,
sorter: Some(sorter),
}
}

async fn poll_next(self) -> Option<(Result<RecordBatch>, Self)> {
match self {
Self::SinkPhase {
mut input,
mut sorter,
} => {
while let Some(sorter_) = &mut sorter {
if let Some(item) = input.next().await {
match item {
Ok(batch) => {
if let Err(e) = sorter_.sink(batch).await {
return Some((
Err(e),
Self::SinkPhase {
input,
sorter: None,
},
));
}
}
Err(e) => {
// Propagate the error
return Some((Err(e), Self::SinkPhase { input, sorter }));
}
}
} else {
let sorter = sorter.take().unwrap();
match sorter.source() {
Ok(mut output) => {
let rb = output.next().await;
return rb.map(|rb| (rb, Self::SourcePhase { output }));
}
Err(e) => {
return Some((
Err(e),
Self::SinkPhase {
input,
sorter: None,
},
));
}
}
}
}
None
}
Self::SourcePhase { mut output } => {
let rb = output.next().await;
rb.map(|rb| (rb, Self::SourcePhase { output }))
}
}
}
}

#[cfg(test)]
mod tests {
use std::collections::HashMap;
Expand All @@ -990,13 +1085,15 @@ mod tests {
use crate::memory::MemoryExec;
use crate::test;
use crate::test::assert_is_pending;
use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec};
use crate::test::exec::{
assert_strong_count_converges_to_zero, BlockingExec, MockExec,
};

use arrow::array::*;
use arrow::compute::SortOptions;
use arrow::datatypes::*;
use datafusion_common::cast::as_primitive_array;
use datafusion_common::{assert_batches_eq, Result, ScalarValue};
use datafusion_common::{assert_batches_eq, exec_err, Result, ScalarValue};
use datafusion_execution::config::SessionConfig;
use datafusion_execution::runtime_env::RuntimeEnvBuilder;
use datafusion_execution::RecordBatchStream;
Expand Down Expand Up @@ -1589,4 +1686,68 @@ mod tests {
assert_batches_eq!(expected, &batches);
Ok(())
}

#[tokio::test]
async fn test_sort_with_error_in_stream() {
let task_ctx = Arc::new(TaskContext::default());
let schema = Arc::new(Schema::new(vec![
Field::new("key", DataType::Utf8, false),
Field::new("payload", DataType::Int32, false),
]));

let mock_exec = MockExec::new(
vec![
exec_err!("bad data 0"),
Ok(RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(StringArray::from(vec!["datafusion", "arrow"]))
as ArrayRef,
Arc::new(Arc::new(Int32Array::from(vec![10, 2])) as ArrayRef),
],
)
.unwrap()),
exec_err!("bad data 1"),
Ok(RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(StringArray::from(vec!["parquet", "comet"])) as ArrayRef,
Arc::new(Arc::new(Int32Array::from(vec![7, 99])) as ArrayRef),
],
)
.unwrap()),
],
Arc::clone(&schema),
);

let sort_exprs = vec![PhysicalSortExpr {
expr: col("key", &schema).unwrap(),
options: Default::default(),
}];

let sort = SortExec::new(sort_exprs, Arc::new(mock_exec));
let mut stream = sort.execute(0, task_ctx).unwrap();

let err = stream.next().await.unwrap().unwrap_err().to_string();
assert!(err.contains("bad data 0"), "actual: {err}");

let err = stream.next().await.unwrap().unwrap_err().to_string();
assert!(err.contains("bad data 1"), "actual: {err}");

let batch = stream.next().await.unwrap().unwrap();

assert_batches_eq!(
&[
"+------------+---------+",
"| key | payload |",
"+------------+---------+",
"| arrow | 2 |",
"| comet | 99 |",
"| datafusion | 10 |",
"| parquet | 7 |",
"+------------+---------+",
],
&[batch]
);
}
}
Loading