1717
1818//! Stream wrappers for physical operators
1919
20+ use std:: sync:: Arc ;
21+
2022use crate :: error:: Result ;
23+ use crate :: physical_plan:: displayable;
2124use arrow:: { datatypes:: SchemaRef , record_batch:: RecordBatch } ;
2225use datafusion_common:: DataFusionError ;
26+ use datafusion_execution:: TaskContext ;
2327use futures:: stream:: BoxStream ;
2428use futures:: { Future , Stream , StreamExt } ;
29+ use log:: debug;
2530use pin_project_lite:: pin_project;
2631use tokio:: task:: { JoinHandle , JoinSet } ;
2732use tokio_stream:: wrappers:: ReceiverStream ;
2833
2934use super :: common:: AbortOnDropSingle ;
30- use super :: { RecordBatchStream , SendableRecordBatchStream } ;
35+ use super :: { ExecutionPlan , RecordBatchStream , SendableRecordBatchStream } ;
3136
32- /// Builder for [`RecordBatchReceiverStream`]
33- pub struct RecordBatchReceiverStreamBuilder {
37+ /// Builder for [`RecordBatchReceiverStream`] that propagates errors
38+ /// and panic's correctly.
39+ pub ( crate ) struct RecordBatchReceiverStreamBuilder {
3440 tx : tokio:: sync:: mpsc:: Sender < Result < RecordBatch > > ,
3541 rx : tokio:: sync:: mpsc:: Receiver < Result < RecordBatch > > ,
3642 schema : SchemaRef ,
@@ -55,11 +61,6 @@ impl RecordBatchReceiverStreamBuilder {
5561 self . tx . clone ( )
5662 }
5763
58- /// Get a handle to the `JoinSet` on which tasks are launched
59- pub fn join_set_mut ( & mut self ) -> & mut JoinSet < ( ) > {
60- & mut self . join_set
61- }
62-
6364 /// Spawn task that will be aborted if this builder (or the stream
6465 /// built from it) are dropped
6566 ///
@@ -73,6 +74,45 @@ impl RecordBatchReceiverStreamBuilder {
7374 self . join_set . spawn ( task) ;
7475 }
7576
77+ /// runs the input_partition of the `input` ExecutionPlan on the
78+ /// tokio threadpool and writes its outputs to this stream
79+ pub ( crate ) fn run_input (
80+ & mut self ,
81+ input : Arc < dyn ExecutionPlan > ,
82+ partition : usize ,
83+ context : Arc < TaskContext > ,
84+ ) {
85+ let output = self . tx ( ) ;
86+
87+ self . spawn ( async move {
88+ let mut stream = match input. execute ( partition, context) {
89+ Err ( e) => {
90+ // If send fails, plan being torn down,
91+ // there is no place to send the error.
92+ output. send ( Err ( e) ) . await . ok ( ) ;
93+ debug ! (
94+ "Stopping execution: error executing input: {}" ,
95+ displayable( input. as_ref( ) ) . one_line( )
96+ ) ;
97+ return ;
98+ }
99+ Ok ( stream) => stream,
100+ } ;
101+
102+ while let Some ( item) = stream. next ( ) . await {
103+ // If send fails, plan being torn down,
104+ // there is no place to send the error.
105+ if output. send ( item) . await . is_err ( ) {
106+ debug ! (
107+ "Stopping execution: output is gone, plan cancelling: {}" ,
108+ displayable( input. as_ref( ) ) . one_line( )
109+ ) ;
110+ return ;
111+ }
112+ }
113+ } ) ;
114+ }
115+
76116 /// Create a stream of all `RecordBatch`es written to `tx`
77117 pub fn build ( self ) -> SendableRecordBatchStream {
78118 let Self {
0 commit comments