Skip to content

Commit b1a817c

Browse files
committed
terminate early on panic
1 parent 56a26eb commit b1a817c

1 file changed

Lines changed: 19 additions & 7 deletions

File tree

datafusion/core/src/physical_plan/stream.rs

Lines changed: 19 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -38,9 +38,11 @@ use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream};
3838
/// Builder for [`RecordBatchReceiverStream`] that propagates errors
3939
/// and panic's correctly.
4040
///
41-
/// [`RecordBatchReceiverStream`] can be used when there are one or
42-
/// more tasks spawned which produce RecordBatches and send them to a
43-
/// single `Receiver`.
41+
/// [`RecordBatchReceiverStream`] is used to spawn one or more tasks
42+
/// that produce `RecordBatch`es and send them to a single
43+
/// `Receiver` which can improve parallelism.
44+
///
45+
/// This also handles propagating panic`s and canceling the tasks.
4446
pub struct RecordBatchReceiverStreamBuilder {
4547
tx: Sender<Result<RecordBatch>>,
4648
rx: Receiver<Result<RecordBatch>>,
@@ -94,6 +96,9 @@ impl RecordBatchReceiverStreamBuilder {
9496

9597
/// runs the input_partition of the `input` ExecutionPlan on the
9698
/// tokio threadpool and writes its outputs to this stream
99+
///
100+
/// If the input partition produces an error, the error will be
101+
/// sent to the output stream and no further results are sent.
97102
pub(crate) fn run_input(
98103
&mut self,
99104
input: Arc<dyn ExecutionPlan>,
@@ -105,8 +110,8 @@ impl RecordBatchReceiverStreamBuilder {
105110
self.spawn(async move {
106111
let mut stream = match input.execute(partition, context) {
107112
Err(e) => {
108-
// If send fails, plan being torn down,
109-
// there is no place to send the error.
113+
// If send fails, the plan being torn down, there
114+
// is no place to send the error and no reason to continue.
110115
output.send(Err(e)).await.ok();
111116
debug!(
112117
"Stopping execution: error executing input: {}",
@@ -135,6 +140,10 @@ impl RecordBatchReceiverStreamBuilder {
135140
// stop after the first error is encontered (don't
136141
// drive all streams to completion)
137142
if is_err {
143+
debug!(
144+
"Stopping execution: plan returned error: {}",
145+
displayable(input.as_ref()).one_line()
146+
);
138147
return;
139148
}
140149
}
@@ -153,7 +162,7 @@ impl RecordBatchReceiverStreamBuilder {
153162
// don't need tx
154163
drop(tx);
155164

156-
// future that checks the result of the join set
165+
// future that checks the result of the join set, and propagates panic if seen
157166
let check = async move {
158167
while let Some(result) = join_set.join_next().await {
159168
match result {
@@ -183,7 +192,10 @@ impl RecordBatchReceiverStreamBuilder {
183192
// unwrap Option / only return the error
184193
.filter_map(|item| async move { item });
185194

186-
let inner = ReceiverStream::new(rx).chain(check_stream).boxed();
195+
// Merge the streams together (but futures::stream:StreamExt
196+
// is already in scope, so call it explicitly)
197+
let inner =
198+
tokio_stream::StreamExt::merge(ReceiverStream::new(rx), check_stream).boxed();
187199

188200
Box::pin(RecordBatchReceiverStream { schema, inner })
189201
}

0 commit comments

Comments
 (0)