Skip to content
Closed
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
30 changes: 24 additions & 6 deletions datafusion/src/physical_plan/windows.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ use std::ops::Range;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use tokio::task;

/// Window execution plan
#[derive(Debug)]
Expand Down Expand Up @@ -467,14 +468,29 @@ pin_project! {
}

/// Compute the window aggregate columns
fn compute_window_aggregates(
async fn compute_window_aggregates(
window_expr: Vec<Arc<dyn WindowExpr>>,
batch: &RecordBatch,
batch: Arc<RecordBatch>,
) -> Result<Vec<ArrayRef>> {
window_expr
let handles = window_expr
.iter()
.map(|window_expr| window_expr.evaluate(batch))
.collect()
.map(|window_expr| {
let batch = batch.clone();
let window_expr = window_expr.clone();
task::spawn_blocking(move || window_expr.evaluate(&batch))
})
.collect::<Vec<_>>();
let mut result = vec![];
for handle in handles {
let arr = handle.await.map_err(|e| {
DataFusionError::Execution(format!(
"Failed to join window aggregation handle {}",
e
))
})??;
result.push(arr);
}
Ok(result)
}

impl WindowAggStream {
Expand Down Expand Up @@ -510,8 +526,10 @@ impl WindowAggStream {
.map_err(DataFusionError::into_arrow_external_error)?;
let batch = common::combine_batches(&batches, input_schema.clone())?;
if let Some(batch) = batch {
let batch = Arc::new(batch);
// calculate window cols
let mut columns = compute_window_aggregates(window_expr, &batch)
let mut columns = compute_window_aggregates(window_expr, batch.clone())
.await
.map_err(DataFusionError::into_arrow_external_error)?;
// combine with the original cols
// note the setup of window aggregates is that they newly calculated window
Expand Down