Skip to content

Commit 74b05fa

Browse files
authored
Use DataFusionError instead of ArrowError in SendableRecordBatchStream (#5101)
* Replace ArrowError with DataFusionError in DF context * fix comments
1 parent 9c8bdfe commit 74b05fa

38 files changed

+213
-254
lines changed

datafusion/core/src/physical_plan/aggregates/mod.rs

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -722,7 +722,6 @@ mod tests {
722722
use crate::{assert_batches_sorted_eq, physical_plan::common};
723723
use arrow::array::{Float64Array, UInt32Array};
724724
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
725-
use arrow::error::Result as ArrowResult;
726725
use arrow::record_batch::RecordBatch;
727726
use datafusion_common::{DataFusionError, Result, ScalarValue};
728727
use datafusion_physical_expr::expressions::{lit, ApproxDistinct, Count, Median};
@@ -1038,7 +1037,7 @@ mod tests {
10381037
}
10391038

10401039
impl Stream for TestYieldingStream {
1041-
type Item = ArrowResult<RecordBatch>;
1040+
type Item = Result<RecordBatch>;
10421041

10431042
fn poll_next(
10441043
mut self: std::pin::Pin<&mut Self>,

datafusion/core/src/physical_plan/aggregates/no_grouping.rs

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@ use crate::physical_plan::aggregates::{
2525
use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput};
2626
use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream};
2727
use arrow::datatypes::SchemaRef;
28-
use arrow::error::{ArrowError, Result as ArrowResult};
2928
use arrow::record_batch::RecordBatch;
3029
use datafusion_common::Result;
3130
use datafusion_physical_expr::{AggregateExpr, PhysicalExpr};
@@ -38,7 +37,7 @@ use futures::stream::{Stream, StreamExt};
3837

3938
/// stream struct for aggregation without grouping columns
4039
pub(crate) struct AggregateStream {
41-
stream: BoxStream<'static, ArrowResult<RecordBatch>>,
40+
stream: BoxStream<'static, Result<RecordBatch>>,
4241
schema: SchemaRef,
4342
}
4443

@@ -112,17 +111,17 @@ impl AggregateStream {
112111
.and_then(|allocated| this.reservation.try_grow(allocated))
113112
{
114113
Ok(_) => continue,
115-
Err(e) => Err(ArrowError::ExternalError(Box::new(e))),
114+
Err(e) => Err(e),
116115
}
117116
}
118117
Some(Err(e)) => Err(e),
119118
None => {
120119
this.finished = true;
121120
let timer = this.baseline_metrics.elapsed_compute().timer();
122121
let result = finalize_aggregation(&this.accumulators, &this.mode)
123-
.map_err(|e| ArrowError::ExternalError(Box::new(e)))
124122
.and_then(|columns| {
125123
RecordBatch::try_new(this.schema.clone(), columns)
124+
.map_err(Into::into)
126125
})
127126
.record_output(&this.baseline_metrics);
128127

@@ -146,7 +145,7 @@ impl AggregateStream {
146145
}
147146

148147
impl Stream for AggregateStream {
149-
type Item = ArrowResult<RecordBatch>;
148+
type Item = Result<RecordBatch>;
150149

151150
fn poll_next(
152151
mut self: std::pin::Pin<&mut Self>,

datafusion/core/src/physical_plan/aggregates/row_hash.rs

Lines changed: 4 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -42,13 +42,10 @@ use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream};
4242

4343
use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation};
4444
use arrow::array::{new_null_array, PrimitiveArray};
45+
use arrow::array::{Array, UInt32Builder};
4546
use arrow::compute::cast;
4647
use arrow::datatypes::{DataType, Schema, UInt32Type};
4748
use arrow::{array::ArrayRef, compute};
48-
use arrow::{
49-
array::{Array, UInt32Builder},
50-
error::{ArrowError, Result as ArrowResult},
51-
};
5249
use arrow::{datatypes::SchemaRef, record_batch::RecordBatch};
5350
use datafusion_common::{DataFusionError, ScalarValue};
5451
use datafusion_expr::Accumulator;
@@ -226,7 +223,7 @@ impl GroupedHashAggregateStream {
226223
}
227224

228225
impl Stream for GroupedHashAggregateStream {
229-
type Item = ArrowResult<RecordBatch>;
226+
type Item = Result<RecordBatch>;
230227

231228
fn poll_next(
232229
mut self: std::pin::Pin<&mut Self>,
@@ -252,9 +249,7 @@ impl Stream for GroupedHashAggregateStream {
252249
});
253250

254251
if let Err(e) = result {
255-
return Poll::Ready(Some(Err(
256-
ArrowError::ExternalError(Box::new(e)),
257-
)));
252+
return Poll::Ready(Some(Err(e)));
258253
}
259254
}
260255
// inner had error, return to caller
@@ -569,7 +564,7 @@ impl std::fmt::Debug for RowAggregationState {
569564

570565
impl GroupedHashAggregateStream {
571566
/// Create a RecordBatch with all group keys and accumulator' states or values.
572-
fn create_batch_from_map(&mut self) -> ArrowResult<Option<RecordBatch>> {
567+
fn create_batch_from_map(&mut self) -> Result<Option<RecordBatch>> {
573568
let skip_items = self.row_group_skip_position;
574569
if skip_items > self.row_aggr_state.group_states.len() {
575570
return Ok(None);
@@ -624,7 +619,6 @@ impl GroupedHashAggregateStream {
624619
// the intermediate GroupByScalar type was not the same as the
625620
// output
626621
cast(&item, field.data_type())
627-
.map_err(DataFusionError::ArrowError)
628622
}?;
629623
results.push(result);
630624
}

datafusion/core/src/physical_plan/analyze.rs

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -200,7 +200,8 @@ impl ExecutionPlan for AnalyzeExec {
200200
Arc::new(type_builder.finish()),
201201
Arc::new(plan_builder.finish()),
202202
],
203-
);
203+
)
204+
.map_err(Into::into);
204205
// again ignore error
205206
tx.send(maybe_batch).await.ok();
206207
});

datafusion/core/src/physical_plan/coalesce_batches.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -180,7 +180,7 @@ struct CoalesceBatchesStream {
180180
}
181181

182182
impl Stream for CoalesceBatchesStream {
183-
type Item = ArrowResult<RecordBatch>;
183+
type Item = Result<RecordBatch>;
184184

185185
fn poll_next(
186186
mut self: Pin<&mut Self>,
@@ -200,7 +200,7 @@ impl CoalesceBatchesStream {
200200
fn poll_next_inner(
201201
self: &mut Pin<&mut Self>,
202202
cx: &mut Context<'_>,
203-
) -> Poll<Option<ArrowResult<RecordBatch>>> {
203+
) -> Poll<Option<Result<RecordBatch>>> {
204204
// Get a clone (uses same underlying atomic) as self gets borrowed below
205205
let cloned_time = self.baseline_metrics.elapsed_compute().clone();
206206

datafusion/core/src/physical_plan/coalesce_partitions.rs

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -25,8 +25,8 @@ use std::task::Poll;
2525
use futures::Stream;
2626
use tokio::sync::mpsc;
2727

28+
use arrow::datatypes::SchemaRef;
2829
use arrow::record_batch::RecordBatch;
29-
use arrow::{datatypes::SchemaRef, error::Result as ArrowResult};
3030

3131
use super::common::AbortOnDropMany;
3232
use super::expressions::PhysicalSortExpr;
@@ -138,7 +138,7 @@ impl ExecutionPlan for CoalescePartitionsExec {
138138
// least one result in an attempt to maximize
139139
// parallelism.
140140
let (sender, receiver) =
141-
mpsc::channel::<ArrowResult<RecordBatch>>(input_partitions);
141+
mpsc::channel::<Result<RecordBatch>>(input_partitions);
142142

143143
// spawn independent tasks whose resulting streams (of batches)
144144
// are sent to the channel for consumption.
@@ -185,14 +185,14 @@ impl ExecutionPlan for CoalescePartitionsExec {
185185

186186
struct MergeStream {
187187
schema: SchemaRef,
188-
input: mpsc::Receiver<ArrowResult<RecordBatch>>,
188+
input: mpsc::Receiver<Result<RecordBatch>>,
189189
baseline_metrics: BaselineMetrics,
190190
#[allow(unused)]
191191
drop_helper: AbortOnDropMany<()>,
192192
}
193193

194194
impl Stream for MergeStream {
195-
type Item = ArrowResult<RecordBatch>;
195+
type Item = Result<RecordBatch>;
196196

197197
fn poll_next(
198198
mut self: std::pin::Pin<&mut Self>,

datafusion/core/src/physical_plan/common.rs

Lines changed: 12 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@ use crate::physical_plan::{displayable, ColumnStatistics, ExecutionPlan, Statist
2525
use arrow::compute::concat;
2626
use arrow::datatypes::{Schema, SchemaRef};
2727
use arrow::error::ArrowError;
28-
use arrow::error::Result as ArrowResult;
2928
use arrow::ipc::writer::{FileWriter, IpcWriteOptions};
3029
use arrow::record_batch::RecordBatch;
3130
use datafusion_physical_expr::utils::ordering_satisfy;
@@ -68,7 +67,7 @@ impl SizedRecordBatchStream {
6867
}
6968

7069
impl Stream for SizedRecordBatchStream {
71-
type Item = ArrowResult<RecordBatch>;
70+
type Item = Result<RecordBatch>;
7271

7372
fn poll_next(
7473
mut self: std::pin::Pin<&mut Self>,
@@ -92,10 +91,7 @@ impl RecordBatchStream for SizedRecordBatchStream {
9291

9392
/// Create a vector of record batches from a stream
9493
pub async fn collect(stream: SendableRecordBatchStream) -> Result<Vec<RecordBatch>> {
95-
stream
96-
.try_collect::<Vec<_>>()
97-
.await
98-
.map_err(DataFusionError::from)
94+
stream.try_collect::<Vec<_>>().await
9995
}
10096

10197
/// Merge two record batch references into a single record batch.
@@ -104,15 +100,16 @@ pub fn merge_batches(
104100
first: &RecordBatch,
105101
second: &RecordBatch,
106102
schema: SchemaRef,
107-
) -> ArrowResult<RecordBatch> {
103+
) -> Result<RecordBatch> {
108104
let columns = (0..schema.fields.len())
109105
.map(|index| {
110106
let first_column = first.column(index).as_ref();
111107
let second_column = second.column(index).as_ref();
112108
concat(&[first_column, second_column])
113109
})
114-
.collect::<ArrowResult<Vec<_>>>()?;
115-
RecordBatch::try_new(schema, columns)
110+
.collect::<Result<Vec<_>, ArrowError>>()
111+
.map_err(Into::<DataFusionError>::into)?;
112+
RecordBatch::try_new(schema, columns).map_err(Into::into)
116113
}
117114

118115
/// Merge a slice of record batch references into a single record batch, or
@@ -121,7 +118,7 @@ pub fn merge_batches(
121118
pub fn merge_multiple_batches(
122119
batches: &[&RecordBatch],
123120
schema: SchemaRef,
124-
) -> ArrowResult<Option<RecordBatch>> {
121+
) -> Result<Option<RecordBatch>> {
125122
Ok(if batches.is_empty() {
126123
None
127124
} else {
@@ -134,7 +131,8 @@ pub fn merge_multiple_batches(
134131
.collect::<Vec<_>>(),
135132
)
136133
})
137-
.collect::<ArrowResult<Vec<_>>>()?;
134+
.collect::<Result<Vec<_>, ArrowError>>()
135+
.map_err(Into::<DataFusionError>::into)?;
138136
Some(RecordBatch::try_new(schema, columns)?)
139137
})
140138
}
@@ -190,7 +188,7 @@ fn build_file_list_recurse(
190188
/// Spawns a task to the tokio threadpool and writes its outputs to the provided mpsc sender
191189
pub(crate) fn spawn_execution(
192190
input: Arc<dyn ExecutionPlan>,
193-
output: mpsc::Sender<ArrowResult<RecordBatch>>,
191+
output: mpsc::Sender<Result<RecordBatch>>,
194192
partition: usize,
195193
context: Arc<TaskContext>,
196194
) -> JoinHandle<()> {
@@ -199,8 +197,7 @@ pub(crate) fn spawn_execution(
199197
Err(e) => {
200198
// If send fails, plan being torn down,
201199
// there is no place to send the error.
202-
let arrow_error = ArrowError::ExternalError(Box::new(e));
203-
output.send(Err(arrow_error)).await.ok();
200+
output.send(Err(e)).await.ok();
204201
debug!(
205202
"Stopping execution: error executing input: {}",
206203
displayable(input.as_ref()).one_line()
@@ -524,7 +521,7 @@ impl IPCWriter {
524521

525522
/// Finish the writer
526523
pub fn finish(&mut self) -> Result<()> {
527-
self.writer.finish().map_err(DataFusionError::ArrowError)
524+
self.writer.finish().map_err(Into::into)
528525
}
529526

530527
/// Path write to

datafusion/core/src/physical_plan/file_format/csv.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -452,7 +452,7 @@ mod tests {
452452
let err = it.next().await.unwrap().unwrap_err().to_string();
453453
assert_eq!(
454454
err,
455-
"Csv error: incorrect number of fields for line 1, expected 14 got 13"
455+
"Arrow error: Csv error: incorrect number of fields for line 1, expected 14 got 13"
456456
);
457457
Ok(())
458458
}

datafusion/core/src/physical_plan/file_format/file_stream.rs

Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,8 @@ use std::task::{Context, Poll};
2727
use std::time::Instant;
2828

2929
use arrow::datatypes::SchemaRef;
30-
use arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
30+
use arrow::error::ArrowError;
31+
use arrow::record_batch::RecordBatch;
3132
use datafusion_common::ScalarValue;
3233
use futures::future::BoxFuture;
3334
use futures::stream::BoxStream;
@@ -45,7 +46,7 @@ use crate::physical_plan::RecordBatchStream;
4546

4647
/// A fallible future that resolves to a stream of [`RecordBatch`]
4748
pub type FileOpenFuture =
48-
BoxFuture<'static, Result<BoxStream<'static, ArrowResult<RecordBatch>>>>;
49+
BoxFuture<'static, Result<BoxStream<'static, Result<RecordBatch, ArrowError>>>>;
4950

5051
/// Generic API for opening a file using an [`ObjectStore`] and resolving to a
5152
/// stream of [`RecordBatch`]
@@ -96,7 +97,7 @@ enum FileStreamState {
9697
/// Partitioning column values for the current batch_iter
9798
partition_values: Vec<ScalarValue>,
9899
/// The reader instance
99-
reader: BoxStream<'static, ArrowResult<RecordBatch>>,
100+
reader: BoxStream<'static, Result<RecordBatch, ArrowError>>,
100101
},
101102
/// Encountered an error
102103
Error,
@@ -201,10 +202,7 @@ impl<F: FileOpener> FileStream<F> {
201202
})
202203
}
203204

204-
fn poll_inner(
205-
&mut self,
206-
cx: &mut Context<'_>,
207-
) -> Poll<Option<ArrowResult<RecordBatch>>> {
205+
fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll<Option<Result<RecordBatch>>> {
208206
loop {
209207
match &mut self.state {
210208
FileStreamState::Idle => {
@@ -230,7 +228,7 @@ impl<F: FileOpener> FileStream<F> {
230228
}
231229
Err(e) => {
232230
self.state = FileStreamState::Error;
233-
return Poll::Ready(Some(Err(e.into())));
231+
return Poll::Ready(Some(Err(e)));
234232
}
235233
}
236234
}
@@ -249,7 +247,7 @@ impl<F: FileOpener> FileStream<F> {
249247
}
250248
Err(e) => {
251249
self.state = FileStreamState::Error;
252-
return Poll::Ready(Some(Err(e.into())));
250+
return Poll::Ready(Some(Err(e)));
253251
}
254252
},
255253
FileStreamState::Scan {
@@ -260,7 +258,11 @@ impl<F: FileOpener> FileStream<F> {
260258
self.file_stream_metrics.time_scanning_until_data.stop();
261259
self.file_stream_metrics.time_scanning_total.stop();
262260
let result = result
263-
.and_then(|b| self.pc_projector.project(b, partition_values))
261+
.and_then(|b| {
262+
self.pc_projector
263+
.project(b, partition_values)
264+
.map_err(|e| ArrowError::ExternalError(e.into()))
265+
})
264266
.map(|batch| match &mut self.remain {
265267
Some(remain) => {
266268
if *remain > batch.num_rows() {
@@ -280,7 +282,7 @@ impl<F: FileOpener> FileStream<F> {
280282
self.state = FileStreamState::Error
281283
}
282284
self.file_stream_metrics.time_scanning_total.start();
283-
return Poll::Ready(Some(result));
285+
return Poll::Ready(Some(result.map_err(Into::into)));
284286
}
285287
None => {
286288
self.file_stream_metrics.time_scanning_until_data.stop();
@@ -297,7 +299,7 @@ impl<F: FileOpener> FileStream<F> {
297299
}
298300

299301
impl<F: FileOpener> Stream for FileStream<F> {
300-
type Item = ArrowResult<RecordBatch>;
302+
type Item = Result<RecordBatch>;
301303

302304
fn poll_next(
303305
mut self: Pin<&mut Self>,

0 commit comments

Comments
 (0)