Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Minor: Improve documentation on execution error handling #12651

Merged
merged 4 commits into from
Sep 29, 2024
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
24 changes: 22 additions & 2 deletions datafusion/execution/src/stream.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@ use datafusion_common::Result;
use futures::Stream;
use std::pin::Pin;

/// Trait for types that stream [arrow::record_batch::RecordBatch]
/// Trait for types that stream [RecordBatch]
///
/// See [`SendableRecordBatchStream`] for more details.
pub trait RecordBatchStream: Stream<Item = Result<RecordBatch>> {
/// Returns the schema of this `RecordBatchStream`.
///
Expand All @@ -29,5 +31,23 @@ pub trait RecordBatchStream: Stream<Item = Result<RecordBatch>> {
fn schema(&self) -> SchemaRef;
}

/// Trait for a [`Stream`] of [`RecordBatch`]es
/// Trait for a [`Stream`] of [`RecordBatch`]es that can be passed between threads
///
/// This trait is used to retrieve the results of DataFusion execution plans.
alamb marked this conversation as resolved.
Show resolved Hide resolved
///
/// The trait is a specialized Rust Async [`Stream`] that also knows the schema
/// of the data it will return (even if the stream has no data). Every
/// `RecordBatch` returned by the stream should have the same schema as returned
/// by [`schema`](`RecordBatchStream::schema`).
///
/// # Error Handling
///
/// One a stream returns an error, it should not be polled again (the caller
alamb marked this conversation as resolved.
Show resolved Hide resolved
/// should stop calling `next`) and handle the error.
///
/// However, returning `Ready(None)` (end of stream) is likely the safest
/// behavior after an error. Like [`Stream`]s, `RecordBatchStream`s should not
/// be polled after end of stream or returning an error. However, also like
/// [`Stream`]s there is no mechanism to prevent callers polling so returning
/// `Ready(None)` is recommended.
pub type SendableRecordBatchStream = Pin<Box<dyn RecordBatchStream + Send>>;
10 changes: 10 additions & 0 deletions datafusion/physical-plan/src/execution_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -228,6 +228,16 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync {
/// [`TryStreamExt`]: futures::stream::TryStreamExt
/// [`RecordBatchStreamAdapter`]: crate::stream::RecordBatchStreamAdapter
///
/// # Error handling
///
/// Any error that occurs during execution is sent as an `Err` in the output
/// stream.
///
/// `ExecutionPlan` implementations in DataFusion cancel additional work
/// immediately once an error occurs. The rationale is that if the overall
/// query will return an error, any additional work such as continued
/// polling of inputs will be wasted as it will be thrown away.
///
/// # Cancellation / Aborting Execution
///
/// The [`Stream`] that is returned must ensure that any allocated resources
Expand Down
5 changes: 5 additions & 0 deletions datafusion/physical-plan/src/repartition/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -385,6 +385,11 @@ impl BatchPartitioner {
/// `───────' `───────'
///```
///
/// # Error Handling
///
/// If any of the input partitions return an error, the error is propagated to
/// all output partitions and inputs are not polled again.
///
/// # Output Ordering
///
/// If more than one stream is being repartitioned, the output will be some
Expand Down
1 change: 1 addition & 0 deletions datafusion/physical-plan/src/sorts/merge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ use futures::Stream;
/// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`]
type CursorStream<C> = Box<dyn PartitionedStream<Output = Result<(C, RecordBatch)>>>;

/// Merges a stream of sorted cursors and record batches into a single sorted stream
#[derive(Debug)]
pub(crate) struct SortPreservingMergeStream<C: CursorValues> {
in_progress: BatchBuilder,
Expand Down
5 changes: 5 additions & 0 deletions datafusion/physical-plan/src/sorts/sort_preserving_merge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,11 @@ use log::{debug, trace};
/// Input Streams Output stream
/// (sorted) (sorted)
/// ```
///
/// # Error Handling
///
/// If any of the input partitions return an error, the error is propagated to
/// the output and inputs are not polled again.
#[derive(Debug)]
pub struct SortPreservingMergeExec {
/// Input plan
Expand Down
Loading