Skip to content

Commit d887201

Browse files
committed
consolidate
1 parent 0a96357 commit d887201

4 files changed

Lines changed: 60 additions & 82 deletions

File tree

datafusion/core/src/physical_plan/analyze.rs

Lines changed: 9 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,9 @@ use crate::{
2929
};
3030
use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch};
3131
use futures::StreamExt;
32-
use tokio::task::JoinSet;
3332

3433
use super::expressions::PhysicalSortExpr;
35-
use super::stream::RecordBatchStreamAdapter;
34+
use super::stream::{RecordBatchReceiverStreamBuilder, RecordBatchStreamAdapter};
3635
use super::{Distribution, SendableRecordBatchStream};
3736
use crate::execution::context::TaskContext;
3837

@@ -121,23 +120,15 @@ impl ExecutionPlan for AnalyzeExec {
121120
// Gather futures that will run each input partition in
122121
// parallel (on a separate tokio task) using a JoinSet to
123122
// cancel outstanding futures on drop
124-
let mut set = JoinSet::new();
125123
let num_input_partitions = self.input.output_partitioning().partition_count();
124+
let mut builder =
125+
RecordBatchReceiverStreamBuilder::new(self.schema(), num_input_partitions);
126126

127127
for input_partition in 0..num_input_partitions {
128-
let input_stream = self.input.execute(input_partition, context.clone());
129-
130-
set.spawn(async move {
131-
let mut total_rows = 0;
132-
let mut input_stream = input_stream?;
133-
while let Some(batch) = input_stream.next().await {
134-
let batch = batch?;
135-
total_rows += batch.num_rows();
136-
}
137-
Ok(total_rows) as Result<usize>
138-
});
128+
builder.run_input(self.input.clone(), input_partition, context.clone());
139129
}
140130

131+
// Create future that computes thefinal output
141132
let start = Instant::now();
142133
let captured_input = self.input.clone();
143134
let captured_schema = self.schema.clone();
@@ -146,18 +137,12 @@ impl ExecutionPlan for AnalyzeExec {
146137
// future that gathers the results from all the tasks in the
147138
// JoinSet that computes the overall row count and final
148139
// record batch
140+
let mut input_stream = builder.build();
149141
let output = async move {
150142
let mut total_rows = 0;
151-
while let Some(res) = set.join_next().await {
152-
// translate join errors (aka task panic's) into ExecutionErrors
153-
match res {
154-
Ok(row_count) => total_rows += row_count?,
155-
Err(e) => {
156-
return Err(DataFusionError::Execution(format!(
157-
"Join error in AnalyzeExec: {e}"
158-
)))
159-
}
160-
}
143+
while let Some(batch) = input_stream.next().await {
144+
let batch = batch?;
145+
total_rows += batch.num_rows();
161146
}
162147

163148
let duration = Instant::now() - start;

datafusion/core/src/physical_plan/coalesce_partitions.rs

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@ use crate::physical_plan::{
3434

3535
use super::SendableRecordBatchStream;
3636
use crate::execution::context::TaskContext;
37-
use crate::physical_plan::common::spawn_execution;
3837

3938
/// Merge execution plan executes partitions in parallel and combines them into a single
4039
/// partition. No guarantees are made about the order of the resulting partition.
@@ -140,14 +139,7 @@ impl ExecutionPlan for CoalescePartitionsExec {
140139
// spawn independent tasks whose resulting streams (of batches)
141140
// are sent to the channel for consumption.
142141
for part_i in 0..input_partitions {
143-
let sender = builder.tx();
144-
spawn_execution(
145-
builder.join_set_mut(),
146-
self.input.clone(),
147-
sender,
148-
part_i,
149-
context.clone(),
150-
);
142+
builder.run_input(self.input.clone(), part_i, context.clone());
151143
}
152144

153145
Ok(builder.build())

datafusion/core/src/physical_plan/common.rs

Lines changed: 2 additions & 41 deletions
Original file line numberDiff line numberDiff line change
@@ -19,17 +19,15 @@
1919
2020
use super::SendableRecordBatchStream;
2121
use crate::error::{DataFusionError, Result};
22-
use crate::execution::context::TaskContext;
2322
use crate::execution::memory_pool::MemoryReservation;
2423
use crate::physical_plan::stream::RecordBatchReceiverStream;
25-
use crate::physical_plan::{displayable, ColumnStatistics, ExecutionPlan, Statistics};
24+
use crate::physical_plan::{ColumnStatistics, ExecutionPlan, Statistics};
2625
use arrow::datatypes::Schema;
2726
use arrow::ipc::writer::{FileWriter, IpcWriteOptions};
2827
use arrow::record_batch::RecordBatch;
2928
use datafusion_physical_expr::expressions::{BinaryExpr, Column};
3029
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr};
3130
use futures::{Future, StreamExt, TryStreamExt};
32-
use log::debug;
3331
use parking_lot::Mutex;
3432
use pin_project_lite::pin_project;
3533
use std::fs;
@@ -38,7 +36,7 @@ use std::path::{Path, PathBuf};
3836
use std::sync::Arc;
3937
use std::task::{Context, Poll};
4038
use tokio::sync::mpsc;
41-
use tokio::task::{JoinHandle, JoinSet};
39+
use tokio::task::JoinHandle;
4240

4341
/// [`MemoryReservation`] used across query execution streams
4442
pub(crate) type SharedMemoryReservation = Arc<Mutex<MemoryReservation>>;
@@ -96,43 +94,6 @@ fn build_file_list_recurse(
9694
Ok(())
9795
}
9896

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-
13697
/// If running in a tokio context spawns the execution of `stream` to a separate task
13798
/// allowing it to execute in parallel with an intermediate buffer of size `buffer`
13899
pub(crate) fn spawn_buffered(

datafusion/core/src/physical_plan/stream.rs

Lines changed: 48 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -17,20 +17,26 @@
1717

1818
//! Stream wrappers for physical operators
1919
20+
use std::sync::Arc;
21+
2022
use crate::error::Result;
23+
use crate::physical_plan::displayable;
2124
use arrow::{datatypes::SchemaRef, record_batch::RecordBatch};
2225
use datafusion_common::DataFusionError;
26+
use datafusion_execution::TaskContext;
2327
use futures::stream::BoxStream;
2428
use futures::{Future, Stream, StreamExt};
29+
use log::debug;
2530
use pin_project_lite::pin_project;
2631
use tokio::task::{JoinHandle, JoinSet};
2732
use tokio_stream::wrappers::ReceiverStream;
2833

2934
use 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

Comments
 (0)