|
19 | 19 |
|
20 | 20 | use super::SendableRecordBatchStream; |
21 | 21 | use crate::error::{DataFusionError, Result}; |
22 | | -use crate::execution::context::TaskContext; |
23 | 22 | use crate::execution::memory_pool::MemoryReservation; |
24 | 23 | use crate::physical_plan::stream::RecordBatchReceiverStream; |
25 | | -use crate::physical_plan::{displayable, ColumnStatistics, ExecutionPlan, Statistics}; |
| 24 | +use crate::physical_plan::{ColumnStatistics, ExecutionPlan, Statistics}; |
26 | 25 | use arrow::datatypes::Schema; |
27 | 26 | use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; |
28 | 27 | use arrow::record_batch::RecordBatch; |
29 | 28 | use datafusion_physical_expr::expressions::{BinaryExpr, Column}; |
30 | 29 | use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; |
31 | 30 | use futures::{Future, StreamExt, TryStreamExt}; |
32 | | -use log::debug; |
33 | 31 | use parking_lot::Mutex; |
34 | 32 | use pin_project_lite::pin_project; |
35 | 33 | use std::fs; |
36 | 34 | use std::fs::{metadata, File}; |
37 | 35 | use std::path::{Path, PathBuf}; |
38 | 36 | use std::sync::Arc; |
39 | 37 | use std::task::{Context, Poll}; |
40 | | -use tokio::sync::mpsc; |
41 | | -use tokio::task::{JoinHandle, JoinSet}; |
| 38 | +use tokio::task::JoinHandle; |
42 | 39 |
|
43 | 40 | /// [`MemoryReservation`] used across query execution streams |
44 | 41 | pub(crate) type SharedMemoryReservation = Arc<Mutex<MemoryReservation>>; |
@@ -96,66 +93,30 @@ fn build_file_list_recurse( |
96 | 93 | Ok(()) |
97 | 94 | } |
98 | 95 |
|
99 | | -/// Spawns a task to the tokio threadpool and writes its outputs to the provided mpsc sender |
100 | | -pub(crate) fn spawn_execution( |
101 | | - join_set: &mut JoinSet<()>, |
102 | | - input: Arc<dyn ExecutionPlan>, |
103 | | - output: mpsc::Sender<Result<RecordBatch>>, |
104 | | - partition: usize, |
105 | | - context: Arc<TaskContext>, |
106 | | -) { |
107 | | - join_set.spawn(async move { |
108 | | - let mut stream = match input.execute(partition, context) { |
109 | | - Err(e) => { |
110 | | - // If send fails, plan being torn down, |
111 | | - // there is no place to send the error. |
112 | | - output.send(Err(e)).await.ok(); |
113 | | - debug!( |
114 | | - "Stopping execution: error executing input: {}", |
115 | | - displayable(input.as_ref()).one_line() |
116 | | - ); |
117 | | - return; |
118 | | - } |
119 | | - Ok(stream) => stream, |
120 | | - }; |
121 | | - |
122 | | - while let Some(item) = stream.next().await { |
123 | | - // If send fails, plan being torn down, |
124 | | - // there is no place to send the error. |
125 | | - if output.send(item).await.is_err() { |
126 | | - debug!( |
127 | | - "Stopping execution: output is gone, plan cancelling: {}", |
128 | | - displayable(input.as_ref()).one_line() |
129 | | - ); |
130 | | - return; |
131 | | - } |
132 | | - } |
133 | | - }); |
134 | | -} |
135 | | - |
136 | 96 | /// If running in a tokio context spawns the execution of `stream` to a separate task |
137 | 97 | /// allowing it to execute in parallel with an intermediate buffer of size `buffer` |
138 | 98 | pub(crate) fn spawn_buffered( |
139 | 99 | mut input: SendableRecordBatchStream, |
140 | 100 | buffer: usize, |
141 | 101 | ) -> SendableRecordBatchStream { |
142 | 102 | // Use tokio only if running from a tokio context (#2201) |
143 | | - let handle = match tokio::runtime::Handle::try_current() { |
144 | | - Ok(handle) => handle, |
145 | | - Err(_) => return input, |
| 103 | + if let Err(_) = tokio::runtime::Handle::try_current() { |
| 104 | + return input; |
146 | 105 | }; |
147 | 106 |
|
148 | | - let schema = input.schema(); |
149 | | - let (sender, receiver) = mpsc::channel(buffer); |
150 | | - let join = handle.spawn(async move { |
| 107 | + let mut builder = RecordBatchReceiverStream::builder(input.schema(), buffer); |
| 108 | + |
| 109 | + let sender = builder.tx(); |
| 110 | + |
| 111 | + builder.spawn(async move { |
151 | 112 | while let Some(item) = input.next().await { |
152 | 113 | if sender.send(item).await.is_err() { |
153 | 114 | return; |
154 | 115 | } |
155 | 116 | } |
156 | 117 | }); |
157 | 118 |
|
158 | | - RecordBatchReceiverStream::create(&schema, receiver, join) |
| 119 | + builder.build() |
159 | 120 | } |
160 | 121 |
|
161 | 122 | /// Computes the statistics for an in-memory RecordBatch |
|
0 commit comments