Skip to main content

datafusion_physical_plan/windows/
window_agg_exec.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Stream and channel implementations for window function expressions.
19
20use std::any::Any;
21use std::pin::Pin;
22use std::sync::Arc;
23use std::task::{Context, Poll};
24
25use super::utils::create_schema;
26use crate::execution_plan::EmissionType;
27use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
28use crate::windows::{
29    calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs,
30    window_equivalence_properties,
31};
32use crate::{
33    ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan,
34    ExecutionPlanProperties, PhysicalExpr, PlanProperties, RecordBatchStream,
35    SendableRecordBatchStream, Statistics, WindowExpr,
36};
37
38use arrow::array::ArrayRef;
39use arrow::compute::{concat, concat_batches};
40use arrow::datatypes::SchemaRef;
41use arrow::error::ArrowError;
42use arrow::record_batch::RecordBatch;
43use datafusion_common::stats::Precision;
44use datafusion_common::utils::{evaluate_partition_ranges, transpose};
45use datafusion_common::{Result, assert_eq_or_internal_err};
46use datafusion_execution::TaskContext;
47use datafusion_physical_expr_common::sort_expr::{
48    OrderingRequirements, PhysicalSortExpr,
49};
50
51use futures::{Stream, StreamExt, ready};
52
53/// Window execution plan
54#[derive(Debug, Clone)]
55pub struct WindowAggExec {
56    /// Input plan
57    pub(crate) input: Arc<dyn ExecutionPlan>,
58    /// Window function expression
59    window_expr: Vec<Arc<dyn WindowExpr>>,
60    /// Schema after the window is run
61    schema: SchemaRef,
62    /// Execution metrics
63    metrics: ExecutionPlanMetricsSet,
64    /// Partition by indices that defines preset for existing ordering
65    // see `get_ordered_partition_by_indices` for more details.
66    ordered_partition_by_indices: Vec<usize>,
67    /// Cache holding plan properties like equivalences, output partitioning etc.
68    cache: PlanProperties,
69    /// If `can_partition` is false, partition_keys is always empty.
70    can_repartition: bool,
71}
72
73impl WindowAggExec {
74    /// Create a new execution plan for window aggregates
75    pub fn try_new(
76        window_expr: Vec<Arc<dyn WindowExpr>>,
77        input: Arc<dyn ExecutionPlan>,
78        can_repartition: bool,
79    ) -> Result<Self> {
80        let schema = create_schema(&input.schema(), &window_expr)?;
81        let schema = Arc::new(schema);
82
83        let ordered_partition_by_indices =
84            get_ordered_partition_by_indices(window_expr[0].partition_by(), &input)?;
85        let cache = Self::compute_properties(&schema, &input, &window_expr)?;
86        Ok(Self {
87            input,
88            window_expr,
89            schema,
90            metrics: ExecutionPlanMetricsSet::new(),
91            ordered_partition_by_indices,
92            cache,
93            can_repartition,
94        })
95    }
96
97    /// Window expressions
98    pub fn window_expr(&self) -> &[Arc<dyn WindowExpr>] {
99        &self.window_expr
100    }
101
102    /// Input plan
103    pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
104        &self.input
105    }
106
107    /// Return the output sort order of partition keys: For example
108    /// OVER(PARTITION BY a, ORDER BY b) -> would give sorting of the column a
109    // We are sure that partition by columns are always at the beginning of sort_keys
110    // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely
111    // to calculate partition separation points
112    pub fn partition_by_sort_keys(&self) -> Result<Vec<PhysicalSortExpr>> {
113        let partition_by = self.window_expr()[0].partition_by();
114        get_partition_by_sort_exprs(
115            &self.input,
116            partition_by,
117            &self.ordered_partition_by_indices,
118        )
119    }
120
121    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
122    fn compute_properties(
123        schema: &SchemaRef,
124        input: &Arc<dyn ExecutionPlan>,
125        window_exprs: &[Arc<dyn WindowExpr>],
126    ) -> Result<PlanProperties> {
127        // Calculate equivalence properties:
128        let eq_properties = window_equivalence_properties(schema, input, window_exprs)?;
129
130        // Get output partitioning:
131        // Because we can have repartitioning using the partition keys this
132        // would be either 1 or more than 1 depending on the presence of repartitioning.
133        let output_partitioning = input.output_partitioning().clone();
134
135        // Construct properties cache:
136        Ok(PlanProperties::new(
137            eq_properties,
138            output_partitioning,
139            // TODO: Emission type and boundedness information can be enhanced here
140            EmissionType::Final,
141            input.boundedness(),
142        ))
143    }
144
145    pub fn partition_keys(&self) -> Vec<Arc<dyn PhysicalExpr>> {
146        if !self.can_repartition {
147            vec![]
148        } else {
149            let all_partition_keys = self
150                .window_expr()
151                .iter()
152                .map(|expr| expr.partition_by().to_vec())
153                .collect::<Vec<_>>();
154
155            all_partition_keys
156                .into_iter()
157                .min_by_key(|s| s.len())
158                .unwrap_or_else(Vec::new)
159        }
160    }
161}
162
163impl DisplayAs for WindowAggExec {
164    fn fmt_as(
165        &self,
166        t: DisplayFormatType,
167        f: &mut std::fmt::Formatter,
168    ) -> std::fmt::Result {
169        match t {
170            DisplayFormatType::Default | DisplayFormatType::Verbose => {
171                write!(f, "WindowAggExec: ")?;
172                let g: Vec<String> = self
173                    .window_expr
174                    .iter()
175                    .map(|e| {
176                        format!(
177                            "{}: {:?}, frame: {:?}",
178                            e.name().to_owned(),
179                            e.field(),
180                            e.get_window_frame()
181                        )
182                    })
183                    .collect();
184                write!(f, "wdw=[{}]", g.join(", "))?;
185            }
186            DisplayFormatType::TreeRender => {
187                let g: Vec<String> = self
188                    .window_expr
189                    .iter()
190                    .map(|e| e.name().to_owned().to_string())
191                    .collect();
192                writeln!(f, "select_list={}", g.join(", "))?;
193            }
194        }
195        Ok(())
196    }
197}
198
199impl ExecutionPlan for WindowAggExec {
200    fn name(&self) -> &'static str {
201        "WindowAggExec"
202    }
203
204    /// Return a reference to Any that can be used for downcasting
205    fn as_any(&self) -> &dyn Any {
206        self
207    }
208
209    fn properties(&self) -> &PlanProperties {
210        &self.cache
211    }
212
213    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
214        vec![&self.input]
215    }
216
217    fn maintains_input_order(&self) -> Vec<bool> {
218        vec![true]
219    }
220
221    fn required_input_ordering(&self) -> Vec<Option<OrderingRequirements>> {
222        let partition_bys = self.window_expr()[0].partition_by();
223        let order_keys = self.window_expr()[0].order_by();
224        if self.ordered_partition_by_indices.len() < partition_bys.len() {
225            vec![calc_requirements(partition_bys, order_keys)]
226        } else {
227            let partition_bys = self
228                .ordered_partition_by_indices
229                .iter()
230                .map(|idx| &partition_bys[*idx]);
231            vec![calc_requirements(partition_bys, order_keys)]
232        }
233    }
234
235    fn required_input_distribution(&self) -> Vec<Distribution> {
236        if self.partition_keys().is_empty() {
237            vec![Distribution::SinglePartition]
238        } else {
239            vec![Distribution::HashPartitioned(self.partition_keys())]
240        }
241    }
242
243    fn with_new_children(
244        self: Arc<Self>,
245        children: Vec<Arc<dyn ExecutionPlan>>,
246    ) -> Result<Arc<dyn ExecutionPlan>> {
247        Ok(Arc::new(WindowAggExec::try_new(
248            self.window_expr.clone(),
249            Arc::clone(&children[0]),
250            true,
251        )?))
252    }
253
254    fn execute(
255        &self,
256        partition: usize,
257        context: Arc<TaskContext>,
258    ) -> Result<SendableRecordBatchStream> {
259        let input = self.input.execute(partition, context)?;
260        let stream = Box::pin(WindowAggStream::new(
261            Arc::clone(&self.schema),
262            self.window_expr.clone(),
263            input,
264            BaselineMetrics::new(&self.metrics, partition),
265            self.partition_by_sort_keys()?,
266            self.ordered_partition_by_indices.clone(),
267        )?);
268        Ok(stream)
269    }
270
271    fn metrics(&self) -> Option<MetricsSet> {
272        Some(self.metrics.clone_inner())
273    }
274
275    fn statistics(&self) -> Result<Statistics> {
276        self.partition_statistics(None)
277    }
278
279    fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
280        let input_stat = self.input.partition_statistics(partition)?;
281        let win_cols = self.window_expr.len();
282        let input_cols = self.input.schema().fields().len();
283        // TODO stats: some windowing function will maintain invariants such as min, max...
284        let mut column_statistics = Vec::with_capacity(win_cols + input_cols);
285        // copy stats of the input to the beginning of the schema.
286        column_statistics.extend(input_stat.column_statistics);
287        for _ in 0..win_cols {
288            column_statistics.push(ColumnStatistics::new_unknown())
289        }
290        Ok(Statistics {
291            num_rows: input_stat.num_rows,
292            column_statistics,
293            total_byte_size: Precision::Absent,
294        })
295    }
296}
297
298/// Compute the window aggregate columns
299fn compute_window_aggregates(
300    window_expr: &[Arc<dyn WindowExpr>],
301    batch: &RecordBatch,
302) -> Result<Vec<ArrayRef>> {
303    window_expr
304        .iter()
305        .map(|window_expr| window_expr.evaluate(batch))
306        .collect()
307}
308
309/// stream for window aggregation plan
310pub struct WindowAggStream {
311    schema: SchemaRef,
312    input: SendableRecordBatchStream,
313    batches: Vec<RecordBatch>,
314    finished: bool,
315    window_expr: Vec<Arc<dyn WindowExpr>>,
316    partition_by_sort_keys: Vec<PhysicalSortExpr>,
317    baseline_metrics: BaselineMetrics,
318    ordered_partition_by_indices: Vec<usize>,
319}
320
321impl WindowAggStream {
322    /// Create a new WindowAggStream
323    pub fn new(
324        schema: SchemaRef,
325        window_expr: Vec<Arc<dyn WindowExpr>>,
326        input: SendableRecordBatchStream,
327        baseline_metrics: BaselineMetrics,
328        partition_by_sort_keys: Vec<PhysicalSortExpr>,
329        ordered_partition_by_indices: Vec<usize>,
330    ) -> Result<Self> {
331        // In WindowAggExec all partition by columns should be ordered.
332        assert_eq_or_internal_err!(
333            window_expr[0].partition_by().len(),
334            ordered_partition_by_indices.len(),
335            "All partition by columns should have an ordering"
336        );
337        Ok(Self {
338            schema,
339            input,
340            batches: vec![],
341            finished: false,
342            window_expr,
343            baseline_metrics,
344            partition_by_sort_keys,
345            ordered_partition_by_indices,
346        })
347    }
348
349    fn compute_aggregates(&self) -> Result<Option<RecordBatch>> {
350        // record compute time on drop
351        let _timer = self.baseline_metrics.elapsed_compute().timer();
352
353        let batch = concat_batches(&self.input.schema(), &self.batches)?;
354        if batch.num_rows() == 0 {
355            return Ok(None);
356        }
357
358        let partition_by_sort_keys = self
359            .ordered_partition_by_indices
360            .iter()
361            .map(|idx| self.partition_by_sort_keys[*idx].evaluate_to_sort_column(&batch))
362            .collect::<Result<Vec<_>>>()?;
363        let partition_points =
364            evaluate_partition_ranges(batch.num_rows(), &partition_by_sort_keys)?;
365
366        let mut partition_results = vec![];
367        // Calculate window cols
368        for partition_point in partition_points {
369            let length = partition_point.end - partition_point.start;
370            partition_results.push(compute_window_aggregates(
371                &self.window_expr,
372                &batch.slice(partition_point.start, length),
373            )?)
374        }
375        let columns = transpose(partition_results)
376            .iter()
377            .map(|elems| concat(&elems.iter().map(|x| x.as_ref()).collect::<Vec<_>>()))
378            .collect::<Vec<_>>()
379            .into_iter()
380            .collect::<Result<Vec<ArrayRef>, ArrowError>>()?;
381
382        // combine with the original cols
383        // note the setup of window aggregates is that they newly calculated window
384        // expression results are always appended to the columns
385        let mut batch_columns = batch.columns().to_vec();
386        // calculate window cols
387        batch_columns.extend_from_slice(&columns);
388        Ok(Some(RecordBatch::try_new(
389            Arc::clone(&self.schema),
390            batch_columns,
391        )?))
392    }
393}
394
395impl Stream for WindowAggStream {
396    type Item = Result<RecordBatch>;
397
398    fn poll_next(
399        mut self: Pin<&mut Self>,
400        cx: &mut Context<'_>,
401    ) -> Poll<Option<Self::Item>> {
402        let poll = self.poll_next_inner(cx);
403        self.baseline_metrics.record_poll(poll)
404    }
405}
406
407impl WindowAggStream {
408    #[inline]
409    fn poll_next_inner(
410        &mut self,
411        cx: &mut Context<'_>,
412    ) -> Poll<Option<Result<RecordBatch>>> {
413        if self.finished {
414            return Poll::Ready(None);
415        }
416
417        loop {
418            return Poll::Ready(Some(match ready!(self.input.poll_next_unpin(cx)) {
419                Some(Ok(batch)) => {
420                    self.batches.push(batch);
421                    continue;
422                }
423                Some(Err(e)) => Err(e),
424                None => {
425                    let Some(result) = self.compute_aggregates()? else {
426                        return Poll::Ready(None);
427                    };
428                    self.finished = true;
429                    // Empty record batches should not be emitted.
430                    // They need to be treated as  [`Option<RecordBatch>`]es and handled separately
431                    debug_assert!(result.num_rows() > 0);
432                    Ok(result)
433                }
434            }));
435        }
436    }
437}
438
439impl RecordBatchStream for WindowAggStream {
440    /// Get the schema
441    fn schema(&self) -> SchemaRef {
442        Arc::clone(&self.schema)
443    }
444}