Skip to content

Commit

Permalink
Minor: add ExecutionPlan.execute() snippet for sync methods
Browse files Browse the repository at this point in the history
  • Loading branch information
comphead committed Oct 31, 2023
1 parent 0d4dc36 commit de81883
Showing 1 changed file with 22 additions and 1 deletion.
23 changes: 22 additions & 1 deletion datafusion/physical-plan/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,28 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync {
Ok(None)
}

/// Begin execution of `partition`, returning a stream of [`RecordBatch`]es.
/// Start plan execution for `partition`, returning a [`SendableRecordBatchStream`] of [`RecordBatch`]es.
///
///
/// Often its needed to call [`.await`](https://doc.rust-lang.org/std/keyword.await.html) just after the stream async combinators.
/// To call it within the sync method please use the snippet:
///
/// ```no_run
/// use datafusion_common::DataFusionError;
///
/// fn print_plan_exec() -> Result<(), DataFusionError> {
/// let stream = node.execute(0, task_ctx)?;
/// futures::stream::once(async move {
/// while let Some(batch) = stream.next().await {
/// let batch = batch?;
/// println!("{}", batch);
/// return Ok(());
/// }
/// Ok(())
/// });
/// Ok(())
/// }
/// ```
fn execute(
&self,
partition: usize,
Expand Down

0 comments on commit de81883

Please sign in to comment.