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: Add implementation examples to ExecutionPlan::execute #8013

Merged
merged 2 commits into from
Nov 1, 2023
Merged
Changes from all 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
104 changes: 104 additions & 0 deletions datafusion/physical-plan/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -236,6 +236,110 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync {
}

/// Begin execution of `partition`, returning a stream of [`RecordBatch`]es.
///
/// # Implementation Examples
///
/// ## Return Precomputed Batch
///
/// We can return a precomputed batch as a stream
///
/// ```
/// # use std::sync::Arc;
/// # use arrow_array::RecordBatch;
/// # use arrow_schema::SchemaRef;
/// # use datafusion_common::Result;
/// # use datafusion_execution::{SendableRecordBatchStream, TaskContext};
/// # use datafusion_physical_plan::memory::MemoryStream;
/// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter;
/// struct MyPlan {
/// batch: RecordBatch,
/// }
///
/// impl MyPlan {
/// fn execute(
/// &self,
/// partition: usize,
/// context: Arc<TaskContext>
/// ) -> Result<SendableRecordBatchStream> {
/// let fut = futures::future::ready(Ok(self.batch.clone()));
/// let stream = futures::stream::once(fut);
/// Ok(Box::pin(RecordBatchStreamAdapter::new(self.batch.schema(), stream)))
/// }
/// }
/// ```
///
/// ## Async Compute Batch
///
/// We can also lazily compute a RecordBatch when the returned stream is polled
///
/// ```
/// # use std::sync::Arc;
/// # use arrow_array::RecordBatch;
/// # use arrow_schema::SchemaRef;
/// # use datafusion_common::Result;
/// # use datafusion_execution::{SendableRecordBatchStream, TaskContext};
/// # use datafusion_physical_plan::memory::MemoryStream;
/// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter;
/// struct MyPlan {
/// schema: SchemaRef,
/// }
///
/// async fn get_batch() -> Result<RecordBatch> {
/// todo!()
/// }
///
/// impl MyPlan {
/// fn execute(
/// &self,
/// partition: usize,
/// context: Arc<TaskContext>
/// ) -> Result<SendableRecordBatchStream> {
/// let fut = get_batch();
/// let stream = futures::stream::once(fut);
/// Ok(Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream)))
/// }
/// }
/// ```
///
/// ## Async Compute Batch Stream
///
/// We can lazily compute a RecordBatch stream when the returned stream is polled
/// flattening the result into a single stream
///
/// ```
/// # use std::sync::Arc;
/// # use arrow_array::RecordBatch;
/// # use arrow_schema::SchemaRef;
/// # use futures::TryStreamExt;
/// # use datafusion_common::Result;
/// # use datafusion_execution::{SendableRecordBatchStream, TaskContext};
/// # use datafusion_physical_plan::memory::MemoryStream;
/// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter;
/// struct MyPlan {
/// schema: SchemaRef,
/// }
///
/// async fn get_batch_stream() -> Result<SendableRecordBatchStream> {
/// todo!()
/// }
///
/// impl MyPlan {
/// fn execute(
/// &self,
/// partition: usize,
/// context: Arc<TaskContext>
/// ) -> Result<SendableRecordBatchStream> {
/// // A future that yields a stream
/// let fut = get_batch_stream();
/// // Use TryStreamExt::try_flatten to flatten the stream of streams
/// let stream = futures::stream::once(fut).try_flatten();
/// Ok(Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream)))
/// }
/// }
/// ```
///
/// See [`futures::stream::StreamExt`] and [`futures::stream::TryStreamExt`] for further
/// combinators that can be used with streams
fn execute(
&self,
partition: usize,
Expand Down