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

Parallelise collect_partitioned #6109

Merged
merged 1 commit into from
Apr 25, 2023
Merged
Show file tree
Hide file tree
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
25 changes: 3 additions & 22 deletions datafusion/core/src/datasource/memory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

//! [`MemTable`] for querying `Vec<RecordBatch>` by DataFusion.

use futures::{StreamExt, TryStreamExt};
use futures::StreamExt;
use std::any::Any;
use std::sync::Arc;

Expand All @@ -26,17 +26,16 @@ use arrow::record_batch::RecordBatch;
use async_trait::async_trait;
use datafusion_expr::LogicalPlan;
use tokio::sync::RwLock;
use tokio::task;

use crate::datasource::{TableProvider, TableType};
use crate::error::{DataFusionError, Result};
use crate::execution::context::SessionState;
use crate::logical_expr::Expr;
use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec;
use crate::physical_plan::common;
use crate::physical_plan::common::AbortOnDropSingle;
use crate::physical_plan::memory::MemoryExec;
use crate::physical_plan::ExecutionPlan;
use crate::physical_plan::{collect_partitioned, common};
use crate::physical_plan::{repartition::RepartitionExec, Partitioning};

/// In-memory data source for presenting a `Vec<RecordBatch>` as a
Expand Down Expand Up @@ -199,25 +198,7 @@ impl TableProvider for MemTable {
)?)
};

// Get the task context from the session state.
let task_ctx = state.task_ctx();

// Execute the plan and collect the results into batches.
let mut tasks = vec![];
for idx in 0..plan.output_partitioning().partition_count() {
let stream = plan.execute(idx, task_ctx.clone())?;
let handle = task::spawn(async move {
stream.try_collect().await.map_err(DataFusionError::from)
});
tasks.push(AbortOnDropSingle::new(handle));
}
let results = futures::future::join_all(tasks)
.await
.into_iter()
.map(|result| {
result.map_err(|e| DataFusionError::Execution(format!("{e}")))?
})
.collect::<Result<Vec<Vec<RecordBatch>>>>()?;
let results = collect_partitioned(plan, state.task_ctx()).await?;

// Write the results into the table.
let mut all_batches = self.batches.write().await;
Expand Down
23 changes: 17 additions & 6 deletions datafusion/core/src/physical_plan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ pub use datafusion_expr::Accumulator;
pub use datafusion_expr::ColumnarValue;
pub use datafusion_physical_expr::aggregate::row_accumulator::RowAccumulator;
pub use display::DisplayFormatType;
use futures::stream::Stream;
use futures::stream::{Stream, TryStreamExt};
use std::fmt;
use std::fmt::Debug;

Expand Down Expand Up @@ -443,11 +443,21 @@ pub async fn collect_partitioned(
context: Arc<TaskContext>,
) -> Result<Vec<Vec<RecordBatch>>> {
let streams = execute_stream_partitioned(plan, context)?;
let mut batches = Vec::with_capacity(streams.len());
for stream in streams {
batches.push(common::collect(stream).await?);
}
Ok(batches)

// Execute the plan and collect the results into batches.
let handles = streams
.into_iter()
.enumerate()
.map(|(idx, stream)| async move {
let handle = tokio::task::spawn(stream.try_collect());
AbortOnDropSingle::new(handle).await.map_err(|e| {
DataFusionError::Execution(format!(
"collect_partitioned partition {idx} panicked: {e}"
))
})?
});

futures::future::try_join_all(handles).await
Copy link
Contributor Author

@tustvold tustvold Apr 24, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Using try_join_all will abort on first error as opposed to join_all which would run the execution to completion, and then discard everything but the first error if any

}

/// Execute the [ExecutionPlan] and return a vec with one stream per output partition
Expand Down Expand Up @@ -665,6 +675,7 @@ pub mod values;
pub mod windows;

use crate::execution::context::TaskContext;
use crate::physical_plan::common::AbortOnDropSingle;
use crate::physical_plan::repartition::RepartitionExec;
use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec;
pub use datafusion_physical_expr::{
Expand Down