datafusion_physical_plan/windows/
bounded_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//! The executor given here uses bounded memory (does not maintain all
20//! the input data seen so far), which makes it appropriate when processing
21//! infinite inputs.
22
23use std::any::Any;
24use std::cmp::{min, Ordering};
25use std::collections::VecDeque;
26use std::pin::Pin;
27use std::sync::Arc;
28use std::task::{Context, Poll};
29
30use super::utils::create_schema;
31use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
32use crate::windows::{
33    calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs,
34    window_equivalence_properties,
35};
36use crate::{
37    ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan,
38    ExecutionPlanProperties, InputOrderMode, PlanProperties, RecordBatchStream,
39    SendableRecordBatchStream, Statistics, WindowExpr,
40};
41
42use arrow::compute::take_record_batch;
43use arrow::{
44    array::{Array, ArrayRef, RecordBatchOptions, UInt32Builder},
45    compute::{concat, concat_batches, sort_to_indices, take_arrays},
46    datatypes::SchemaRef,
47    record_batch::RecordBatch,
48};
49use datafusion_common::hash_utils::create_hashes;
50use datafusion_common::stats::Precision;
51use datafusion_common::utils::{
52    evaluate_partition_ranges, get_at_indices, get_row_at_idx,
53};
54use datafusion_common::{
55    arrow_datafusion_err, exec_datafusion_err, exec_err, DataFusionError, HashMap, Result,
56};
57use datafusion_execution::TaskContext;
58use datafusion_expr::window_state::{PartitionBatchState, WindowAggState};
59use datafusion_expr::ColumnarValue;
60use datafusion_physical_expr::window::{
61    PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState,
62};
63use datafusion_physical_expr_common::physical_expr::PhysicalExpr;
64use datafusion_physical_expr_common::sort_expr::{
65    OrderingRequirements, PhysicalSortExpr,
66};
67
68use ahash::RandomState;
69use futures::stream::Stream;
70use futures::{ready, StreamExt};
71use hashbrown::hash_table::HashTable;
72use indexmap::IndexMap;
73use log::debug;
74
75/// Window execution plan
76#[derive(Debug, Clone)]
77pub struct BoundedWindowAggExec {
78    /// Input plan
79    input: Arc<dyn ExecutionPlan>,
80    /// Window function expression
81    window_expr: Vec<Arc<dyn WindowExpr>>,
82    /// Schema after the window is run
83    schema: SchemaRef,
84    /// Execution metrics
85    metrics: ExecutionPlanMetricsSet,
86    /// Describes how the input is ordered relative to the partition keys
87    pub input_order_mode: InputOrderMode,
88    /// Partition by indices that define ordering
89    // For example, if input ordering is ORDER BY a, b and window expression
90    // contains PARTITION BY b, a; `ordered_partition_by_indices` would be 1, 0.
91    // Similarly, if window expression contains PARTITION BY a, b; then
92    // `ordered_partition_by_indices` would be 0, 1.
93    // See `get_ordered_partition_by_indices` for more details.
94    ordered_partition_by_indices: Vec<usize>,
95    /// Cache holding plan properties like equivalences, output partitioning etc.
96    cache: PlanProperties,
97    /// If `can_rerepartition` is false, partition_keys is always empty.
98    can_repartition: bool,
99}
100
101impl BoundedWindowAggExec {
102    /// Create a new execution plan for window aggregates
103    pub fn try_new(
104        window_expr: Vec<Arc<dyn WindowExpr>>,
105        input: Arc<dyn ExecutionPlan>,
106        input_order_mode: InputOrderMode,
107        can_repartition: bool,
108    ) -> Result<Self> {
109        let schema = create_schema(&input.schema(), &window_expr)?;
110        let schema = Arc::new(schema);
111        let partition_by_exprs = window_expr[0].partition_by();
112        let ordered_partition_by_indices = match &input_order_mode {
113            InputOrderMode::Sorted => {
114                let indices = get_ordered_partition_by_indices(
115                    window_expr[0].partition_by(),
116                    &input,
117                )?;
118                if indices.len() == partition_by_exprs.len() {
119                    indices
120                } else {
121                    (0..partition_by_exprs.len()).collect::<Vec<_>>()
122                }
123            }
124            InputOrderMode::PartiallySorted(ordered_indices) => ordered_indices.clone(),
125            InputOrderMode::Linear => {
126                vec![]
127            }
128        };
129        let cache = Self::compute_properties(&input, &schema, &window_expr)?;
130        Ok(Self {
131            input,
132            window_expr,
133            schema,
134            metrics: ExecutionPlanMetricsSet::new(),
135            input_order_mode,
136            ordered_partition_by_indices,
137            cache,
138            can_repartition,
139        })
140    }
141
142    /// Window expressions
143    pub fn window_expr(&self) -> &[Arc<dyn WindowExpr>] {
144        &self.window_expr
145    }
146
147    /// Input plan
148    pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
149        &self.input
150    }
151
152    /// Return the output sort order of partition keys: For example
153    /// OVER(PARTITION BY a, ORDER BY b) -> would give sorting of the column a
154    // We are sure that partition by columns are always at the beginning of sort_keys
155    // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely
156    // to calculate partition separation points
157    pub fn partition_by_sort_keys(&self) -> Result<Vec<PhysicalSortExpr>> {
158        let partition_by = self.window_expr()[0].partition_by();
159        get_partition_by_sort_exprs(
160            &self.input,
161            partition_by,
162            &self.ordered_partition_by_indices,
163        )
164    }
165
166    /// Initializes the appropriate [`PartitionSearcher`] implementation from
167    /// the state.
168    fn get_search_algo(&self) -> Result<Box<dyn PartitionSearcher>> {
169        let partition_by_sort_keys = self.partition_by_sort_keys()?;
170        let ordered_partition_by_indices = self.ordered_partition_by_indices.clone();
171        let input_schema = self.input().schema();
172        Ok(match &self.input_order_mode {
173            InputOrderMode::Sorted => {
174                // In Sorted mode, all partition by columns should be ordered.
175                if self.window_expr()[0].partition_by().len()
176                    != ordered_partition_by_indices.len()
177                {
178                    return exec_err!("All partition by columns should have an ordering in Sorted mode.");
179                }
180                Box::new(SortedSearch {
181                    partition_by_sort_keys,
182                    ordered_partition_by_indices,
183                    input_schema,
184                })
185            }
186            InputOrderMode::Linear | InputOrderMode::PartiallySorted(_) => Box::new(
187                LinearSearch::new(ordered_partition_by_indices, input_schema),
188            ),
189        })
190    }
191
192    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
193    fn compute_properties(
194        input: &Arc<dyn ExecutionPlan>,
195        schema: &SchemaRef,
196        window_exprs: &[Arc<dyn WindowExpr>],
197    ) -> Result<PlanProperties> {
198        // Calculate equivalence properties:
199        let eq_properties = window_equivalence_properties(schema, input, window_exprs)?;
200
201        // As we can have repartitioning using the partition keys, this can
202        // be either one or more than one, depending on the presence of
203        // repartitioning.
204        let output_partitioning = input.output_partitioning().clone();
205
206        // Construct properties cache
207        Ok(PlanProperties::new(
208            eq_properties,
209            output_partitioning,
210            // TODO: Emission type and boundedness information can be enhanced here
211            input.pipeline_behavior(),
212            input.boundedness(),
213        ))
214    }
215
216    pub fn partition_keys(&self) -> Vec<Arc<dyn PhysicalExpr>> {
217        if !self.can_repartition {
218            vec![]
219        } else {
220            let all_partition_keys = self
221                .window_expr()
222                .iter()
223                .map(|expr| expr.partition_by().to_vec())
224                .collect::<Vec<_>>();
225
226            all_partition_keys
227                .into_iter()
228                .min_by_key(|s| s.len())
229                .unwrap_or_else(Vec::new)
230        }
231    }
232
233    fn statistics_helper(&self, statistics: Statistics) -> Result<Statistics> {
234        let win_cols = self.window_expr.len();
235        let input_cols = self.input.schema().fields().len();
236        // TODO stats: some windowing function will maintain invariants such as min, max...
237        let mut column_statistics = Vec::with_capacity(win_cols + input_cols);
238        // copy stats of the input to the beginning of the schema.
239        column_statistics.extend(statistics.column_statistics);
240        for _ in 0..win_cols {
241            column_statistics.push(ColumnStatistics::new_unknown())
242        }
243        Ok(Statistics {
244            num_rows: statistics.num_rows,
245            column_statistics,
246            total_byte_size: Precision::Absent,
247        })
248    }
249}
250
251impl DisplayAs for BoundedWindowAggExec {
252    fn fmt_as(
253        &self,
254        t: DisplayFormatType,
255        f: &mut std::fmt::Formatter,
256    ) -> std::fmt::Result {
257        match t {
258            DisplayFormatType::Default | DisplayFormatType::Verbose => {
259                write!(f, "BoundedWindowAggExec: ")?;
260                let g: Vec<String> = self
261                    .window_expr
262                    .iter()
263                    .map(|e| {
264                        let field = match e.field() {
265                            Ok(f) => f.to_string(),
266                            Err(e) => format!("{e:?}"),
267                        };
268                        format!(
269                            "{}: {}, frame: {}",
270                            e.name().to_owned(),
271                            field,
272                            e.get_window_frame()
273                        )
274                    })
275                    .collect();
276                let mode = &self.input_order_mode;
277                write!(f, "wdw=[{}], mode=[{:?}]", g.join(", "), mode)?;
278            }
279            DisplayFormatType::TreeRender => {
280                let g: Vec<String> = self
281                    .window_expr
282                    .iter()
283                    .map(|e| e.name().to_owned().to_string())
284                    .collect();
285                writeln!(f, "select_list={}", g.join(", "))?;
286
287                let mode = &self.input_order_mode;
288                writeln!(f, "mode={mode:?}")?;
289            }
290        }
291        Ok(())
292    }
293}
294
295impl ExecutionPlan for BoundedWindowAggExec {
296    fn name(&self) -> &'static str {
297        "BoundedWindowAggExec"
298    }
299
300    /// Return a reference to Any that can be used for downcasting
301    fn as_any(&self) -> &dyn Any {
302        self
303    }
304
305    fn properties(&self) -> &PlanProperties {
306        &self.cache
307    }
308
309    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
310        vec![&self.input]
311    }
312
313    fn required_input_ordering(&self) -> Vec<Option<OrderingRequirements>> {
314        let partition_bys = self.window_expr()[0].partition_by();
315        let order_keys = self.window_expr()[0].order_by();
316        let partition_bys = self
317            .ordered_partition_by_indices
318            .iter()
319            .map(|idx| &partition_bys[*idx]);
320        vec![calc_requirements(partition_bys, order_keys)]
321    }
322
323    fn required_input_distribution(&self) -> Vec<Distribution> {
324        if self.partition_keys().is_empty() {
325            debug!("No partition defined for BoundedWindowAggExec!!!");
326            vec![Distribution::SinglePartition]
327        } else {
328            vec![Distribution::HashPartitioned(self.partition_keys().clone())]
329        }
330    }
331
332    fn maintains_input_order(&self) -> Vec<bool> {
333        vec![true]
334    }
335
336    fn with_new_children(
337        self: Arc<Self>,
338        children: Vec<Arc<dyn ExecutionPlan>>,
339    ) -> Result<Arc<dyn ExecutionPlan>> {
340        Ok(Arc::new(BoundedWindowAggExec::try_new(
341            self.window_expr.clone(),
342            Arc::clone(&children[0]),
343            self.input_order_mode.clone(),
344            self.can_repartition,
345        )?))
346    }
347
348    fn execute(
349        &self,
350        partition: usize,
351        context: Arc<TaskContext>,
352    ) -> Result<SendableRecordBatchStream> {
353        let input = self.input.execute(partition, context)?;
354        let search_mode = self.get_search_algo()?;
355        let stream = Box::pin(BoundedWindowAggStream::new(
356            Arc::clone(&self.schema),
357            self.window_expr.clone(),
358            input,
359            BaselineMetrics::new(&self.metrics, partition),
360            search_mode,
361        )?);
362        Ok(stream)
363    }
364
365    fn metrics(&self) -> Option<MetricsSet> {
366        Some(self.metrics.clone_inner())
367    }
368
369    fn statistics(&self) -> Result<Statistics> {
370        self.partition_statistics(None)
371    }
372
373    fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
374        let input_stat = self.input.partition_statistics(partition)?;
375        self.statistics_helper(input_stat)
376    }
377}
378
379/// Trait that specifies how we search for (or calculate) partitions. It has two
380/// implementations: [`SortedSearch`] and [`LinearSearch`].
381trait PartitionSearcher: Send {
382    /// This method constructs output columns using the result of each window expression
383    /// (each entry in the output vector comes from a window expression).
384    /// Executor when producing output concatenates `input_buffer` (corresponding section), and
385    /// result of this function to generate output `RecordBatch`. `input_buffer` is used to determine
386    /// which sections of the window expression results should be used to generate output.
387    /// `partition_buffers` contains corresponding section of the `RecordBatch` for each partition.
388    /// `window_agg_states` stores per partition state for each window expression.
389    /// None case means that no result is generated
390    /// `Some(Vec<ArrayRef>)` is the result of each window expression.
391    fn calculate_out_columns(
392        &mut self,
393        input_buffer: &RecordBatch,
394        window_agg_states: &[PartitionWindowAggStates],
395        partition_buffers: &mut PartitionBatches,
396        window_expr: &[Arc<dyn WindowExpr>],
397    ) -> Result<Option<Vec<ArrayRef>>>;
398
399    /// Determine whether `[InputOrderMode]` is `[InputOrderMode::Linear]` or not.
400    fn is_mode_linear(&self) -> bool {
401        false
402    }
403
404    // Constructs corresponding batches for each partition for the record_batch.
405    fn evaluate_partition_batches(
406        &mut self,
407        record_batch: &RecordBatch,
408        window_expr: &[Arc<dyn WindowExpr>],
409    ) -> Result<Vec<(PartitionKey, RecordBatch)>>;
410
411    /// Prunes the state.
412    fn prune(&mut self, _n_out: usize) {}
413
414    /// Marks the partition as done if we are sure that corresponding partition
415    /// cannot receive any more values.
416    fn mark_partition_end(&self, partition_buffers: &mut PartitionBatches);
417
418    /// Updates `input_buffer` and `partition_buffers` with the new `record_batch`.
419    fn update_partition_batch(
420        &mut self,
421        input_buffer: &mut RecordBatch,
422        record_batch: RecordBatch,
423        window_expr: &[Arc<dyn WindowExpr>],
424        partition_buffers: &mut PartitionBatches,
425    ) -> Result<()> {
426        if record_batch.num_rows() == 0 {
427            return Ok(());
428        }
429        let partition_batches =
430            self.evaluate_partition_batches(&record_batch, window_expr)?;
431        for (partition_row, partition_batch) in partition_batches {
432            if let Some(partition_batch_state) = partition_buffers.get_mut(&partition_row)
433            {
434                partition_batch_state.extend(&partition_batch)?
435            } else {
436                let options = RecordBatchOptions::new()
437                    .with_row_count(Some(partition_batch.num_rows()));
438                // Use input_schema for the buffer schema, not `record_batch.schema()`
439                // as it may not have the "correct" schema in terms of output
440                // nullability constraints. For details, see the following issue:
441                // https://github.com/apache/datafusion/issues/9320
442                let partition_batch = RecordBatch::try_new_with_options(
443                    Arc::clone(self.input_schema()),
444                    partition_batch.columns().to_vec(),
445                    &options,
446                )?;
447                let partition_batch_state =
448                    PartitionBatchState::new_with_batch(partition_batch);
449                partition_buffers.insert(partition_row, partition_batch_state);
450            }
451        }
452
453        if self.is_mode_linear() {
454            // In `Linear` mode, it is guaranteed that the first ORDER BY column
455            // is sorted across partitions. Note that only the first ORDER BY
456            // column is guaranteed to be ordered. As a counter example, consider
457            // the case, `PARTITION BY b, ORDER BY a, c` when the input is sorted
458            // by `[a, b, c]`. In this case, `BoundedWindowAggExec` mode will be
459            // `Linear`. However, we cannot guarantee that the last row of the
460            // input data will be the "last" data in terms of the ordering requirement
461            // `[a, c]` -- it will be the "last" data in terms of `[a, b, c]`.
462            // Hence, only column `a` should be used as a guarantee of the "last"
463            // data across partitions. For other modes (`Sorted`, `PartiallySorted`),
464            // we do not need to keep track of the most recent row guarantee across
465            // partitions. Since leading ordering separates partitions, guaranteed
466            // by the most recent row, already prune the previous partitions completely.
467            let last_row = get_last_row_batch(&record_batch)?;
468            for (_, partition_batch) in partition_buffers.iter_mut() {
469                partition_batch.set_most_recent_row(last_row.clone());
470            }
471        }
472        self.mark_partition_end(partition_buffers);
473
474        *input_buffer = if input_buffer.num_rows() == 0 {
475            record_batch
476        } else {
477            concat_batches(self.input_schema(), [input_buffer, &record_batch])?
478        };
479
480        Ok(())
481    }
482
483    fn input_schema(&self) -> &SchemaRef;
484}
485
486/// This object encapsulates the algorithm state for a simple linear scan
487/// algorithm for computing partitions.
488pub struct LinearSearch {
489    /// Keeps the hash of input buffer calculated from PARTITION BY columns.
490    /// Its length is equal to the `input_buffer` length.
491    input_buffer_hashes: VecDeque<u64>,
492    /// Used during hash value calculation.
493    random_state: RandomState,
494    /// Input ordering and partition by key ordering need not be the same, so
495    /// this vector stores the mapping between them. For instance, if the input
496    /// is ordered by a, b and the window expression contains a PARTITION BY b, a
497    /// clause, this attribute stores [1, 0].
498    ordered_partition_by_indices: Vec<usize>,
499    /// We use this [`HashTable`] to calculate unique partitions for each new
500    /// RecordBatch. First entry in the tuple is the hash value, the second
501    /// entry is the unique ID for each partition (increments from 0 to n).
502    row_map_batch: HashTable<(u64, usize)>,
503    /// We use this [`HashTable`] to calculate the output columns that we can
504    /// produce at each cycle. First entry in the tuple is the hash value, the
505    /// second entry is the unique ID for each partition (increments from 0 to n).
506    /// The third entry stores how many new outputs are calculated for the
507    /// corresponding partition.
508    row_map_out: HashTable<(u64, usize, usize)>,
509    input_schema: SchemaRef,
510}
511
512impl PartitionSearcher for LinearSearch {
513    /// This method constructs output columns using the result of each window expression.
514    // Assume input buffer is         |      Partition Buffers would be (Where each partition and its data is separated)
515    // a, 2                           |      a, 2
516    // b, 2                           |      a, 2
517    // a, 2                           |      a, 2
518    // b, 2                           |
519    // a, 2                           |      b, 2
520    // b, 2                           |      b, 2
521    // b, 2                           |      b, 2
522    //                                |      b, 2
523    // Also assume we happen to calculate 2 new values for a, and 3 for b (To be calculate missing values we may need to consider future values).
524    // Partition buffers effectively will be
525    // a, 2, 1
526    // a, 2, 2
527    // a, 2, (missing)
528    //
529    // b, 2, 1
530    // b, 2, 2
531    // b, 2, 3
532    // b, 2, (missing)
533    // When partition buffers are mapped back to the original record batch. Result becomes
534    // a, 2, 1
535    // b, 2, 1
536    // a, 2, 2
537    // b, 2, 2
538    // a, 2, (missing)
539    // b, 2, 3
540    // b, 2, (missing)
541    // This function calculates the column result of window expression(s) (First 4 entry of 3rd column in the above section.)
542    // 1
543    // 1
544    // 2
545    // 2
546    // Above section corresponds to calculated result which can be emitted without breaking input buffer ordering.
547    fn calculate_out_columns(
548        &mut self,
549        input_buffer: &RecordBatch,
550        window_agg_states: &[PartitionWindowAggStates],
551        partition_buffers: &mut PartitionBatches,
552        window_expr: &[Arc<dyn WindowExpr>],
553    ) -> Result<Option<Vec<ArrayRef>>> {
554        let partition_output_indices = self.calc_partition_output_indices(
555            input_buffer,
556            window_agg_states,
557            window_expr,
558        )?;
559
560        let n_window_col = window_agg_states.len();
561        let mut new_columns = vec![vec![]; n_window_col];
562        // Size of all_indices can be at most input_buffer.num_rows():
563        let mut all_indices = UInt32Builder::with_capacity(input_buffer.num_rows());
564        for (row, indices) in partition_output_indices {
565            let length = indices.len();
566            for (idx, window_agg_state) in window_agg_states.iter().enumerate() {
567                let partition = &window_agg_state[&row];
568                let values = Arc::clone(&partition.state.out_col.slice(0, length));
569                new_columns[idx].push(values);
570            }
571            let partition_batch_state = &mut partition_buffers[&row];
572            // Store how many rows are generated for each partition
573            partition_batch_state.n_out_row = length;
574            // For each row keep corresponding index in the input record batch
575            all_indices.append_slice(&indices);
576        }
577        let all_indices = all_indices.finish();
578        if all_indices.is_empty() {
579            // We couldn't generate any new value, return early:
580            return Ok(None);
581        }
582
583        // Concatenate results for each column by converting `Vec<Vec<ArrayRef>>`
584        // to Vec<ArrayRef> where inner `Vec<ArrayRef>`s are converted to `ArrayRef`s.
585        let new_columns = new_columns
586            .iter()
587            .map(|items| {
588                concat(&items.iter().map(|e| e.as_ref()).collect::<Vec<_>>())
589                    .map_err(|e| arrow_datafusion_err!(e))
590            })
591            .collect::<Result<Vec<_>>>()?;
592        // We should emit columns according to row index ordering.
593        let sorted_indices = sort_to_indices(&all_indices, None, None)?;
594        // Construct new column according to row ordering. This fixes ordering
595        take_arrays(&new_columns, &sorted_indices, None)
596            .map(Some)
597            .map_err(|e| arrow_datafusion_err!(e))
598    }
599
600    fn evaluate_partition_batches(
601        &mut self,
602        record_batch: &RecordBatch,
603        window_expr: &[Arc<dyn WindowExpr>],
604    ) -> Result<Vec<(PartitionKey, RecordBatch)>> {
605        let partition_bys =
606            evaluate_partition_by_column_values(record_batch, window_expr)?;
607        // NOTE: In Linear or PartiallySorted modes, we are sure that
608        //       `partition_bys` are not empty.
609        // Calculate indices for each partition and construct a new record
610        // batch from the rows at these indices for each partition:
611        self.get_per_partition_indices(&partition_bys, record_batch)?
612            .into_iter()
613            .map(|(row, indices)| {
614                let mut new_indices = UInt32Builder::with_capacity(indices.len());
615                new_indices.append_slice(&indices);
616                let indices = new_indices.finish();
617                Ok((row, take_record_batch(record_batch, &indices)?))
618            })
619            .collect()
620    }
621
622    fn prune(&mut self, n_out: usize) {
623        // Delete hashes for the rows that are outputted.
624        self.input_buffer_hashes.drain(0..n_out);
625    }
626
627    fn mark_partition_end(&self, partition_buffers: &mut PartitionBatches) {
628        // We should be in the `PartiallySorted` case, otherwise we can not
629        // tell when we are at the end of a given partition.
630        if !self.ordered_partition_by_indices.is_empty() {
631            if let Some((last_row, _)) = partition_buffers.last() {
632                let last_sorted_cols = self
633                    .ordered_partition_by_indices
634                    .iter()
635                    .map(|idx| last_row[*idx].clone())
636                    .collect::<Vec<_>>();
637                for (row, partition_batch_state) in partition_buffers.iter_mut() {
638                    let sorted_cols = self
639                        .ordered_partition_by_indices
640                        .iter()
641                        .map(|idx| &row[*idx]);
642                    // All the partitions other than `last_sorted_cols` are done.
643                    // We are sure that we will no longer receive values for these
644                    // partitions (arrival of a new value would violate ordering).
645                    partition_batch_state.is_end = !sorted_cols.eq(&last_sorted_cols);
646                }
647            }
648        }
649    }
650
651    fn is_mode_linear(&self) -> bool {
652        self.ordered_partition_by_indices.is_empty()
653    }
654
655    fn input_schema(&self) -> &SchemaRef {
656        &self.input_schema
657    }
658}
659
660impl LinearSearch {
661    /// Initialize a new [`LinearSearch`] partition searcher.
662    fn new(ordered_partition_by_indices: Vec<usize>, input_schema: SchemaRef) -> Self {
663        LinearSearch {
664            input_buffer_hashes: VecDeque::new(),
665            random_state: Default::default(),
666            ordered_partition_by_indices,
667            row_map_batch: HashTable::with_capacity(256),
668            row_map_out: HashTable::with_capacity(256),
669            input_schema,
670        }
671    }
672
673    /// Calculate indices of each partition (according to PARTITION BY expression)
674    /// `columns` contain partition by expression results.
675    fn get_per_partition_indices(
676        &mut self,
677        columns: &[ArrayRef],
678        batch: &RecordBatch,
679    ) -> Result<Vec<(PartitionKey, Vec<u32>)>> {
680        let mut batch_hashes = vec![0; batch.num_rows()];
681        create_hashes(columns, &self.random_state, &mut batch_hashes)?;
682        self.input_buffer_hashes.extend(&batch_hashes);
683        // reset row_map for new calculation
684        self.row_map_batch.clear();
685        // res stores PartitionKey and row indices (indices where these partition occurs in the `batch`) for each partition.
686        let mut result: Vec<(PartitionKey, Vec<u32>)> = vec![];
687        for (hash, row_idx) in batch_hashes.into_iter().zip(0u32..) {
688            let entry = self.row_map_batch.find_mut(hash, |(_, group_idx)| {
689                // We can safely get the first index of the partition indices
690                // since partition indices has one element during initialization.
691                let row = get_row_at_idx(columns, row_idx as usize).unwrap();
692                // Handle hash collusions with an equality check:
693                row.eq(&result[*group_idx].0)
694            });
695            if let Some((_, group_idx)) = entry {
696                result[*group_idx].1.push(row_idx)
697            } else {
698                self.row_map_batch.insert_unique(
699                    hash,
700                    (hash, result.len()),
701                    |(hash, _)| *hash,
702                );
703                let row = get_row_at_idx(columns, row_idx as usize)?;
704                // This is a new partition its only index is row_idx for now.
705                result.push((row, vec![row_idx]));
706            }
707        }
708        Ok(result)
709    }
710
711    /// Calculates partition keys and result indices for each partition.
712    /// The return value is a vector of tuples where the first entry stores
713    /// the partition key (unique for each partition) and the second entry
714    /// stores indices of the rows for which the partition is constructed.
715    fn calc_partition_output_indices(
716        &mut self,
717        input_buffer: &RecordBatch,
718        window_agg_states: &[PartitionWindowAggStates],
719        window_expr: &[Arc<dyn WindowExpr>],
720    ) -> Result<Vec<(PartitionKey, Vec<u32>)>> {
721        let partition_by_columns =
722            evaluate_partition_by_column_values(input_buffer, window_expr)?;
723        // Reset the row_map state:
724        self.row_map_out.clear();
725        let mut partition_indices: Vec<(PartitionKey, Vec<u32>)> = vec![];
726        for (hash, row_idx) in self.input_buffer_hashes.iter().zip(0u32..) {
727            let entry = self.row_map_out.find_mut(*hash, |(_, group_idx, _)| {
728                let row =
729                    get_row_at_idx(&partition_by_columns, row_idx as usize).unwrap();
730                row == partition_indices[*group_idx].0
731            });
732            if let Some((_, group_idx, n_out)) = entry {
733                let (_, indices) = &mut partition_indices[*group_idx];
734                if indices.len() >= *n_out {
735                    break;
736                }
737                indices.push(row_idx);
738            } else {
739                let row = get_row_at_idx(&partition_by_columns, row_idx as usize)?;
740                let min_out = window_agg_states
741                    .iter()
742                    .map(|window_agg_state| {
743                        window_agg_state
744                            .get(&row)
745                            .map(|partition| partition.state.out_col.len())
746                            .unwrap_or(0)
747                    })
748                    .min()
749                    .unwrap_or(0);
750                if min_out == 0 {
751                    break;
752                }
753                self.row_map_out.insert_unique(
754                    *hash,
755                    (*hash, partition_indices.len(), min_out),
756                    |(hash, _, _)| *hash,
757                );
758                partition_indices.push((row, vec![row_idx]));
759            }
760        }
761        Ok(partition_indices)
762    }
763}
764
765/// This object encapsulates the algorithm state for sorted searching
766/// when computing partitions.
767pub struct SortedSearch {
768    /// Stores partition by columns and their ordering information
769    partition_by_sort_keys: Vec<PhysicalSortExpr>,
770    /// Input ordering and partition by key ordering need not be the same, so
771    /// this vector stores the mapping between them. For instance, if the input
772    /// is ordered by a, b and the window expression contains a PARTITION BY b, a
773    /// clause, this attribute stores [1, 0].
774    ordered_partition_by_indices: Vec<usize>,
775    input_schema: SchemaRef,
776}
777
778impl PartitionSearcher for SortedSearch {
779    /// This method constructs new output columns using the result of each window expression.
780    fn calculate_out_columns(
781        &mut self,
782        _input_buffer: &RecordBatch,
783        window_agg_states: &[PartitionWindowAggStates],
784        partition_buffers: &mut PartitionBatches,
785        _window_expr: &[Arc<dyn WindowExpr>],
786    ) -> Result<Option<Vec<ArrayRef>>> {
787        let n_out = self.calculate_n_out_row(window_agg_states, partition_buffers);
788        if n_out == 0 {
789            Ok(None)
790        } else {
791            window_agg_states
792                .iter()
793                .map(|map| get_aggregate_result_out_column(map, n_out).map(Some))
794                .collect()
795        }
796    }
797
798    fn evaluate_partition_batches(
799        &mut self,
800        record_batch: &RecordBatch,
801        _window_expr: &[Arc<dyn WindowExpr>],
802    ) -> Result<Vec<(PartitionKey, RecordBatch)>> {
803        let num_rows = record_batch.num_rows();
804        // Calculate result of partition by column expressions
805        let partition_columns = self
806            .partition_by_sort_keys
807            .iter()
808            .map(|elem| elem.evaluate_to_sort_column(record_batch))
809            .collect::<Result<Vec<_>>>()?;
810        // Reorder `partition_columns` such that its ordering matches input ordering.
811        let partition_columns_ordered =
812            get_at_indices(&partition_columns, &self.ordered_partition_by_indices)?;
813        let partition_points =
814            evaluate_partition_ranges(num_rows, &partition_columns_ordered)?;
815        let partition_bys = partition_columns
816            .into_iter()
817            .map(|arr| arr.values)
818            .collect::<Vec<ArrayRef>>();
819
820        partition_points
821            .iter()
822            .map(|range| {
823                let row = get_row_at_idx(&partition_bys, range.start)?;
824                let len = range.end - range.start;
825                let slice = record_batch.slice(range.start, len);
826                Ok((row, slice))
827            })
828            .collect::<Result<Vec<_>>>()
829    }
830
831    fn mark_partition_end(&self, partition_buffers: &mut PartitionBatches) {
832        // In Sorted case. We can mark all partitions besides last partition as ended.
833        // We are sure that those partitions will never receive any values.
834        // (Otherwise ordering invariant is violated.)
835        let n_partitions = partition_buffers.len();
836        for (idx, (_, partition_batch_state)) in partition_buffers.iter_mut().enumerate()
837        {
838            partition_batch_state.is_end |= idx < n_partitions - 1;
839        }
840    }
841
842    fn input_schema(&self) -> &SchemaRef {
843        &self.input_schema
844    }
845}
846
847impl SortedSearch {
848    /// Calculates how many rows we can output.
849    fn calculate_n_out_row(
850        &mut self,
851        window_agg_states: &[PartitionWindowAggStates],
852        partition_buffers: &mut PartitionBatches,
853    ) -> usize {
854        // Different window aggregators may produce results at different rates.
855        // We produce the overall batch result only as fast as the slowest one.
856        let mut counts = vec![];
857        let out_col_counts = window_agg_states.iter().map(|window_agg_state| {
858            // Store how many elements are generated for the current
859            // window expression:
860            let mut cur_window_expr_out_result_len = 0;
861            // We iterate over `window_agg_state`, which is an IndexMap.
862            // Iterations follow the insertion order, hence we preserve
863            // sorting when partition columns are sorted.
864            let mut per_partition_out_results = HashMap::new();
865            for (row, WindowState { state, .. }) in window_agg_state.iter() {
866                cur_window_expr_out_result_len += state.out_col.len();
867                let count = per_partition_out_results.entry(row).or_insert(0);
868                if *count < state.out_col.len() {
869                    *count = state.out_col.len();
870                }
871                // If we do not generate all results for the current
872                // partition, we do not generate results for next
873                // partition --  otherwise we will lose input ordering.
874                if state.n_row_result_missing > 0 {
875                    break;
876                }
877            }
878            counts.push(per_partition_out_results);
879            cur_window_expr_out_result_len
880        });
881        argmin(out_col_counts).map_or(0, |(min_idx, minima)| {
882            let mut slowest_partition = counts.swap_remove(min_idx);
883            for (partition_key, partition_batch) in partition_buffers.iter_mut() {
884                if let Some(count) = slowest_partition.remove(partition_key) {
885                    partition_batch.n_out_row = count;
886                }
887            }
888            minima
889        })
890    }
891}
892
893/// Calculates partition by expression results for each window expression
894/// on `record_batch`.
895fn evaluate_partition_by_column_values(
896    record_batch: &RecordBatch,
897    window_expr: &[Arc<dyn WindowExpr>],
898) -> Result<Vec<ArrayRef>> {
899    window_expr[0]
900        .partition_by()
901        .iter()
902        .map(|item| match item.evaluate(record_batch)? {
903            ColumnarValue::Array(array) => Ok(array),
904            ColumnarValue::Scalar(scalar) => {
905                scalar.to_array_of_size(record_batch.num_rows())
906            }
907        })
908        .collect()
909}
910
911/// Stream for the bounded window aggregation plan.
912pub struct BoundedWindowAggStream {
913    schema: SchemaRef,
914    input: SendableRecordBatchStream,
915    /// The record batch executor receives as input (i.e. the columns needed
916    /// while calculating aggregation results).
917    input_buffer: RecordBatch,
918    /// We separate `input_buffer` based on partitions (as
919    /// determined by PARTITION BY columns) and store them per partition
920    /// in `partition_batches`. We use this variable when calculating results
921    /// for each window expression. This enables us to use the same batch for
922    /// different window expressions without copying.
923    // Note that we could keep record batches for each window expression in
924    // `PartitionWindowAggStates`. However, this would use more memory (as
925    // many times as the number of window expressions).
926    partition_buffers: PartitionBatches,
927    /// An executor can run multiple window expressions if the PARTITION BY
928    /// and ORDER BY sections are same. We keep state of the each window
929    /// expression inside `window_agg_states`.
930    window_agg_states: Vec<PartitionWindowAggStates>,
931    finished: bool,
932    window_expr: Vec<Arc<dyn WindowExpr>>,
933    baseline_metrics: BaselineMetrics,
934    /// Search mode for partition columns. This determines the algorithm with
935    /// which we group each partition.
936    search_mode: Box<dyn PartitionSearcher>,
937}
938
939impl BoundedWindowAggStream {
940    /// Prunes sections of the state that are no longer needed when calculating
941    /// results (as determined by window frame boundaries and number of results generated).
942    // For instance, if first `n` (not necessarily same with `n_out`) elements are no longer needed to
943    // calculate window expression result (outside the window frame boundary) we retract first `n` elements
944    // from `self.partition_batches` in corresponding partition.
945    // For instance, if `n_out` number of rows are calculated, we can remove
946    // first `n_out` rows from `self.input_buffer`.
947    fn prune_state(&mut self, n_out: usize) -> Result<()> {
948        // Prune `self.window_agg_states`:
949        self.prune_out_columns();
950        // Prune `self.partition_batches`:
951        self.prune_partition_batches();
952        // Prune `self.input_buffer`:
953        self.prune_input_batch(n_out)?;
954        // Prune internal state of search algorithm.
955        self.search_mode.prune(n_out);
956        Ok(())
957    }
958}
959
960impl Stream for BoundedWindowAggStream {
961    type Item = Result<RecordBatch>;
962
963    fn poll_next(
964        mut self: Pin<&mut Self>,
965        cx: &mut Context<'_>,
966    ) -> Poll<Option<Self::Item>> {
967        let poll = self.poll_next_inner(cx);
968        self.baseline_metrics.record_poll(poll)
969    }
970}
971
972impl BoundedWindowAggStream {
973    /// Create a new BoundedWindowAggStream
974    fn new(
975        schema: SchemaRef,
976        window_expr: Vec<Arc<dyn WindowExpr>>,
977        input: SendableRecordBatchStream,
978        baseline_metrics: BaselineMetrics,
979        search_mode: Box<dyn PartitionSearcher>,
980    ) -> Result<Self> {
981        let state = window_expr.iter().map(|_| IndexMap::new()).collect();
982        let empty_batch = RecordBatch::new_empty(Arc::clone(&schema));
983        Ok(Self {
984            schema,
985            input,
986            input_buffer: empty_batch,
987            partition_buffers: IndexMap::new(),
988            window_agg_states: state,
989            finished: false,
990            window_expr,
991            baseline_metrics,
992            search_mode,
993        })
994    }
995
996    fn compute_aggregates(&mut self) -> Result<Option<RecordBatch>> {
997        // calculate window cols
998        for (cur_window_expr, state) in
999            self.window_expr.iter().zip(&mut self.window_agg_states)
1000        {
1001            cur_window_expr.evaluate_stateful(&self.partition_buffers, state)?;
1002        }
1003
1004        let schema = Arc::clone(&self.schema);
1005        let window_expr_out = self.search_mode.calculate_out_columns(
1006            &self.input_buffer,
1007            &self.window_agg_states,
1008            &mut self.partition_buffers,
1009            &self.window_expr,
1010        )?;
1011        if let Some(window_expr_out) = window_expr_out {
1012            let n_out = window_expr_out[0].len();
1013            // right append new columns to corresponding section in the original input buffer.
1014            let columns_to_show = self
1015                .input_buffer
1016                .columns()
1017                .iter()
1018                .map(|elem| elem.slice(0, n_out))
1019                .chain(window_expr_out)
1020                .collect::<Vec<_>>();
1021            let n_generated = columns_to_show[0].len();
1022            self.prune_state(n_generated)?;
1023            Ok(Some(RecordBatch::try_new(schema, columns_to_show)?))
1024        } else {
1025            Ok(None)
1026        }
1027    }
1028
1029    #[inline]
1030    fn poll_next_inner(
1031        &mut self,
1032        cx: &mut Context<'_>,
1033    ) -> Poll<Option<Result<RecordBatch>>> {
1034        if self.finished {
1035            return Poll::Ready(None);
1036        }
1037
1038        let elapsed_compute = self.baseline_metrics.elapsed_compute().clone();
1039        match ready!(self.input.poll_next_unpin(cx)) {
1040            Some(Ok(batch)) => {
1041                // Start the timer for compute time within this operator. It will be
1042                // stopped when dropped.
1043                let _timer = elapsed_compute.timer();
1044
1045                self.search_mode.update_partition_batch(
1046                    &mut self.input_buffer,
1047                    batch,
1048                    &self.window_expr,
1049                    &mut self.partition_buffers,
1050                )?;
1051                if let Some(batch) = self.compute_aggregates()? {
1052                    return Poll::Ready(Some(Ok(batch)));
1053                }
1054                self.poll_next_inner(cx)
1055            }
1056            Some(Err(e)) => Poll::Ready(Some(Err(e))),
1057            None => {
1058                let _timer = elapsed_compute.timer();
1059
1060                self.finished = true;
1061                for (_, partition_batch_state) in self.partition_buffers.iter_mut() {
1062                    partition_batch_state.is_end = true;
1063                }
1064                if let Some(batch) = self.compute_aggregates()? {
1065                    return Poll::Ready(Some(Ok(batch)));
1066                }
1067                Poll::Ready(None)
1068            }
1069        }
1070    }
1071
1072    /// Prunes the sections of the record batch (for each partition)
1073    /// that we no longer need to calculate the window function result.
1074    fn prune_partition_batches(&mut self) {
1075        // Remove partitions which we know already ended (is_end flag is true).
1076        // Since the retain method preserves insertion order, we still have
1077        // ordering in between partitions after removal.
1078        self.partition_buffers
1079            .retain(|_, partition_batch_state| !partition_batch_state.is_end);
1080
1081        // The data in `self.partition_batches` is used by all window expressions.
1082        // Therefore, when removing from `self.partition_batches`, we need to remove
1083        // from the earliest range boundary among all window expressions. Variable
1084        // `n_prune_each_partition` fill the earliest range boundary information for
1085        // each partition. This way, we can delete the no-longer-needed sections from
1086        // `self.partition_batches`.
1087        // For instance, if window frame one uses [10, 20] and window frame two uses
1088        // [5, 15]; we only prune the first 5 elements from the corresponding record
1089        // batch in `self.partition_batches`.
1090
1091        // Calculate how many elements to prune for each partition batch
1092        let mut n_prune_each_partition = HashMap::new();
1093        for window_agg_state in self.window_agg_states.iter_mut() {
1094            window_agg_state.retain(|_, WindowState { state, .. }| !state.is_end);
1095            for (partition_row, WindowState { state: value, .. }) in window_agg_state {
1096                let n_prune =
1097                    min(value.window_frame_range.start, value.last_calculated_index);
1098                if let Some(current) = n_prune_each_partition.get_mut(partition_row) {
1099                    if n_prune < *current {
1100                        *current = n_prune;
1101                    }
1102                } else {
1103                    n_prune_each_partition.insert(partition_row.clone(), n_prune);
1104                }
1105            }
1106        }
1107
1108        // Retract no longer needed parts during window calculations from partition batch:
1109        for (partition_row, n_prune) in n_prune_each_partition.iter() {
1110            let pb_state = &mut self.partition_buffers[partition_row];
1111
1112            let batch = &pb_state.record_batch;
1113            pb_state.record_batch = batch.slice(*n_prune, batch.num_rows() - n_prune);
1114            pb_state.n_out_row = 0;
1115
1116            // Update state indices since we have pruned some rows from the beginning:
1117            for window_agg_state in self.window_agg_states.iter_mut() {
1118                window_agg_state[partition_row].state.prune_state(*n_prune);
1119            }
1120        }
1121    }
1122
1123    /// Prunes the section of the input batch whose aggregate results
1124    /// are calculated and emitted.
1125    fn prune_input_batch(&mut self, n_out: usize) -> Result<()> {
1126        // Prune first n_out rows from the input_buffer
1127        let n_to_keep = self.input_buffer.num_rows() - n_out;
1128        let batch_to_keep = self
1129            .input_buffer
1130            .columns()
1131            .iter()
1132            .map(|elem| elem.slice(n_out, n_to_keep))
1133            .collect::<Vec<_>>();
1134        self.input_buffer = RecordBatch::try_new_with_options(
1135            self.input_buffer.schema(),
1136            batch_to_keep,
1137            &RecordBatchOptions::new().with_row_count(Some(n_to_keep)),
1138        )?;
1139        Ok(())
1140    }
1141
1142    /// Prunes emitted parts from WindowAggState `out_col` field.
1143    fn prune_out_columns(&mut self) {
1144        // We store generated columns for each window expression in the `out_col`
1145        // field of `WindowAggState`. Given how many rows are emitted, we remove
1146        // these sections from state.
1147        for partition_window_agg_states in self.window_agg_states.iter_mut() {
1148            // Remove `n_out` entries from the `out_col` field of `WindowAggState`.
1149            // `n_out` is stored in `self.partition_buffers` for each partition.
1150            // If `is_end` is set, directly remove them; this shrinks the hash map.
1151            partition_window_agg_states
1152                .retain(|_, partition_batch_state| !partition_batch_state.state.is_end);
1153            for (
1154                partition_key,
1155                WindowState {
1156                    state: WindowAggState { out_col, .. },
1157                    ..
1158                },
1159            ) in partition_window_agg_states
1160            {
1161                let partition_batch = &mut self.partition_buffers[partition_key];
1162                let n_to_del = partition_batch.n_out_row;
1163                let n_to_keep = out_col.len() - n_to_del;
1164                *out_col = out_col.slice(n_to_del, n_to_keep);
1165            }
1166        }
1167    }
1168}
1169
1170impl RecordBatchStream for BoundedWindowAggStream {
1171    /// Get the schema
1172    fn schema(&self) -> SchemaRef {
1173        Arc::clone(&self.schema)
1174    }
1175}
1176
1177// Gets the index of minimum entry, returns None if empty.
1178fn argmin<T: PartialOrd>(data: impl Iterator<Item = T>) -> Option<(usize, T)> {
1179    data.enumerate()
1180        .min_by(|(_, a), (_, b)| a.partial_cmp(b).unwrap_or(Ordering::Equal))
1181}
1182
1183/// Calculates the section we can show results for expression
1184fn get_aggregate_result_out_column(
1185    partition_window_agg_states: &PartitionWindowAggStates,
1186    len_to_show: usize,
1187) -> Result<ArrayRef> {
1188    let mut result = None;
1189    let mut running_length = 0;
1190    let mut batches_to_concat = vec![];
1191    // We assume that iteration order is according to insertion order
1192    for (
1193        _,
1194        WindowState {
1195            state: WindowAggState { out_col, .. },
1196            ..
1197        },
1198    ) in partition_window_agg_states
1199    {
1200        if running_length < len_to_show {
1201            let n_to_use = min(len_to_show - running_length, out_col.len());
1202            let slice_to_use = if n_to_use == out_col.len() {
1203                // avoid slice when the entire column is used
1204                Arc::clone(out_col)
1205            } else {
1206                out_col.slice(0, n_to_use)
1207            };
1208            batches_to_concat.push(slice_to_use);
1209            running_length += n_to_use;
1210        } else {
1211            break;
1212        }
1213    }
1214
1215    if !batches_to_concat.is_empty() {
1216        let array_refs: Vec<&dyn Array> =
1217            batches_to_concat.iter().map(|a| a.as_ref()).collect();
1218        result = Some(concat(&array_refs)?);
1219    }
1220
1221    if running_length != len_to_show {
1222        return exec_err!(
1223            "Generated row number should be {len_to_show}, it is {running_length}"
1224        );
1225    }
1226    result.ok_or_else(|| exec_datafusion_err!("Should contain something"))
1227}
1228
1229/// Constructs a batch from the last row of batch in the argument.
1230pub(crate) fn get_last_row_batch(batch: &RecordBatch) -> Result<RecordBatch> {
1231    if batch.num_rows() == 0 {
1232        return exec_err!("Latest batch should have at least 1 row");
1233    }
1234    Ok(batch.slice(batch.num_rows() - 1, 1))
1235}
1236
1237#[cfg(test)]
1238mod tests {
1239    use std::pin::Pin;
1240    use std::sync::Arc;
1241    use std::task::{Context, Poll};
1242    use std::time::Duration;
1243
1244    use crate::common::collect;
1245    use crate::expressions::PhysicalSortExpr;
1246    use crate::projection::{ProjectionExec, ProjectionExpr};
1247    use crate::streaming::{PartitionStream, StreamingTableExec};
1248    use crate::test::TestMemoryExec;
1249    use crate::windows::{
1250        create_udwf_window_expr, create_window_expr, BoundedWindowAggExec, InputOrderMode,
1251    };
1252    use crate::{displayable, execute_stream, ExecutionPlan};
1253
1254    use arrow::array::{
1255        builder::{Int64Builder, UInt64Builder},
1256        RecordBatch,
1257    };
1258    use arrow::compute::SortOptions;
1259    use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
1260    use datafusion_common::test_util::batches_to_string;
1261    use datafusion_common::{exec_datafusion_err, Result, ScalarValue};
1262    use datafusion_execution::config::SessionConfig;
1263    use datafusion_execution::{
1264        RecordBatchStream, SendableRecordBatchStream, TaskContext,
1265    };
1266    use datafusion_expr::{
1267        WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition,
1268    };
1269    use datafusion_functions_aggregate::count::count_udaf;
1270    use datafusion_functions_window::nth_value::last_value_udwf;
1271    use datafusion_functions_window::nth_value::nth_value_udwf;
1272    use datafusion_physical_expr::expressions::{col, Column, Literal};
1273    use datafusion_physical_expr::window::StandardWindowExpr;
1274    use datafusion_physical_expr::{LexOrdering, PhysicalExpr};
1275
1276    use futures::future::Shared;
1277    use futures::{pin_mut, ready, FutureExt, Stream, StreamExt};
1278    use insta::assert_snapshot;
1279    use itertools::Itertools;
1280    use tokio::time::timeout;
1281
1282    #[derive(Debug, Clone)]
1283    struct TestStreamPartition {
1284        schema: SchemaRef,
1285        batches: Vec<RecordBatch>,
1286        idx: usize,
1287        state: PolingState,
1288        sleep_duration: Duration,
1289        send_exit: bool,
1290    }
1291
1292    impl PartitionStream for TestStreamPartition {
1293        fn schema(&self) -> &SchemaRef {
1294            &self.schema
1295        }
1296
1297        fn execute(&self, _ctx: Arc<TaskContext>) -> SendableRecordBatchStream {
1298            // We create an iterator from the record batches and map them into Ok values,
1299            // converting the iterator into a futures::stream::Stream
1300            Box::pin(self.clone())
1301        }
1302    }
1303
1304    impl Stream for TestStreamPartition {
1305        type Item = Result<RecordBatch>;
1306
1307        fn poll_next(
1308            mut self: Pin<&mut Self>,
1309            cx: &mut Context<'_>,
1310        ) -> Poll<Option<Self::Item>> {
1311            self.poll_next_inner(cx)
1312        }
1313    }
1314
1315    #[derive(Debug, Clone)]
1316    enum PolingState {
1317        Sleep(Shared<futures::future::BoxFuture<'static, ()>>),
1318        BatchReturn,
1319    }
1320
1321    impl TestStreamPartition {
1322        fn poll_next_inner(
1323            self: &mut Pin<&mut Self>,
1324            cx: &mut Context<'_>,
1325        ) -> Poll<Option<Result<RecordBatch>>> {
1326            loop {
1327                match &mut self.state {
1328                    PolingState::BatchReturn => {
1329                        // Wait for self.sleep_duration before sending any new data
1330                        let f = tokio::time::sleep(self.sleep_duration).boxed().shared();
1331                        self.state = PolingState::Sleep(f);
1332                        let input_batch = if let Some(batch) =
1333                            self.batches.clone().get(self.idx)
1334                        {
1335                            batch.clone()
1336                        } else if self.send_exit {
1337                            // Send None to signal end of data
1338                            return Poll::Ready(None);
1339                        } else {
1340                            // Go to sleep mode
1341                            let f =
1342                                tokio::time::sleep(self.sleep_duration).boxed().shared();
1343                            self.state = PolingState::Sleep(f);
1344                            continue;
1345                        };
1346                        self.idx += 1;
1347                        return Poll::Ready(Some(Ok(input_batch)));
1348                    }
1349                    PolingState::Sleep(future) => {
1350                        pin_mut!(future);
1351                        ready!(future.poll_unpin(cx));
1352                        self.state = PolingState::BatchReturn;
1353                    }
1354                }
1355            }
1356        }
1357    }
1358
1359    impl RecordBatchStream for TestStreamPartition {
1360        fn schema(&self) -> SchemaRef {
1361            Arc::clone(&self.schema)
1362        }
1363    }
1364
1365    fn bounded_window_exec_pb_latent_range(
1366        input: Arc<dyn ExecutionPlan>,
1367        n_future_range: usize,
1368        hash: &str,
1369        order_by: &str,
1370    ) -> Result<Arc<dyn ExecutionPlan>> {
1371        let schema = input.schema();
1372        let window_fn = WindowFunctionDefinition::AggregateUDF(count_udaf());
1373        let col_expr =
1374            Arc::new(Column::new(schema.fields[0].name(), 0)) as Arc<dyn PhysicalExpr>;
1375        let args = vec![col_expr];
1376        let partitionby_exprs = vec![col(hash, &schema)?];
1377        let orderby_exprs = vec![PhysicalSortExpr {
1378            expr: col(order_by, &schema)?,
1379            options: SortOptions::default(),
1380        }];
1381        let window_frame = WindowFrame::new_bounds(
1382            WindowFrameUnits::Range,
1383            WindowFrameBound::CurrentRow,
1384            WindowFrameBound::Following(ScalarValue::UInt64(Some(n_future_range as u64))),
1385        );
1386        let fn_name = format!(
1387            "{window_fn}({args:?}) PARTITION BY: [{partitionby_exprs:?}], ORDER BY: [{orderby_exprs:?}]"
1388        );
1389        let input_order_mode = InputOrderMode::Linear;
1390        Ok(Arc::new(BoundedWindowAggExec::try_new(
1391            vec![create_window_expr(
1392                &window_fn,
1393                fn_name,
1394                &args,
1395                &partitionby_exprs,
1396                &orderby_exprs,
1397                Arc::new(window_frame),
1398                input.schema(),
1399                false,
1400                false,
1401                None,
1402            )?],
1403            input,
1404            input_order_mode,
1405            true,
1406        )?))
1407    }
1408
1409    fn projection_exec(input: Arc<dyn ExecutionPlan>) -> Result<Arc<dyn ExecutionPlan>> {
1410        let schema = input.schema();
1411        let exprs = input
1412            .schema()
1413            .fields
1414            .iter()
1415            .enumerate()
1416            .map(|(idx, field)| {
1417                let name = if field.name().len() > 20 {
1418                    format!("col_{idx}")
1419                } else {
1420                    field.name().clone()
1421                };
1422                let expr = col(field.name(), &schema).unwrap();
1423                (expr, name)
1424            })
1425            .collect::<Vec<_>>();
1426        let proj_exprs: Vec<ProjectionExpr> = exprs
1427            .into_iter()
1428            .map(|(expr, alias)| ProjectionExpr { expr, alias })
1429            .collect();
1430        Ok(Arc::new(ProjectionExec::try_new(proj_exprs, input)?))
1431    }
1432
1433    fn task_context_helper() -> TaskContext {
1434        let task_ctx = TaskContext::default();
1435        // Create session context with config
1436        let session_config = SessionConfig::new()
1437            .with_batch_size(1)
1438            .with_target_partitions(2)
1439            .with_round_robin_repartition(false);
1440        task_ctx.with_session_config(session_config)
1441    }
1442
1443    fn task_context() -> Arc<TaskContext> {
1444        Arc::new(task_context_helper())
1445    }
1446
1447    pub async fn collect_stream(
1448        mut stream: SendableRecordBatchStream,
1449        results: &mut Vec<RecordBatch>,
1450    ) -> Result<()> {
1451        while let Some(item) = stream.next().await {
1452            results.push(item?);
1453        }
1454        Ok(())
1455    }
1456
1457    /// Execute the [ExecutionPlan] and collect the results in memory
1458    pub async fn collect_with_timeout(
1459        plan: Arc<dyn ExecutionPlan>,
1460        context: Arc<TaskContext>,
1461        timeout_duration: Duration,
1462    ) -> Result<Vec<RecordBatch>> {
1463        let stream = execute_stream(plan, context)?;
1464        let mut results = vec![];
1465
1466        // Execute the asynchronous operation with a timeout
1467        if timeout(timeout_duration, collect_stream(stream, &mut results))
1468            .await
1469            .is_ok()
1470        {
1471            return Err(exec_datafusion_err!("shouldn't have completed"));
1472        };
1473
1474        Ok(results)
1475    }
1476
1477    /// Execute the [ExecutionPlan] and collect the results in memory
1478    #[allow(dead_code)]
1479    pub async fn collect_bonafide(
1480        plan: Arc<dyn ExecutionPlan>,
1481        context: Arc<TaskContext>,
1482    ) -> Result<Vec<RecordBatch>> {
1483        let stream = execute_stream(plan, context)?;
1484        let mut results = vec![];
1485
1486        collect_stream(stream, &mut results).await?;
1487
1488        Ok(results)
1489    }
1490
1491    fn test_schema() -> SchemaRef {
1492        Arc::new(Schema::new(vec![
1493            Field::new("sn", DataType::UInt64, true),
1494            Field::new("hash", DataType::Int64, true),
1495        ]))
1496    }
1497
1498    fn schema_orders(schema: &SchemaRef) -> Result<Vec<LexOrdering>> {
1499        let orderings = vec![[PhysicalSortExpr {
1500            expr: col("sn", schema)?,
1501            options: SortOptions {
1502                descending: false,
1503                nulls_first: false,
1504            },
1505        }]
1506        .into()];
1507        Ok(orderings)
1508    }
1509
1510    fn is_integer_division_safe(lhs: usize, rhs: usize) -> bool {
1511        let res = lhs / rhs;
1512        res * rhs == lhs
1513    }
1514    fn generate_batches(
1515        schema: &SchemaRef,
1516        n_row: usize,
1517        n_chunk: usize,
1518    ) -> Result<Vec<RecordBatch>> {
1519        let mut batches = vec![];
1520        assert!(n_row > 0);
1521        assert!(n_chunk > 0);
1522        assert!(is_integer_division_safe(n_row, n_chunk));
1523        let hash_replicate = 4;
1524
1525        let chunks = (0..n_row)
1526            .chunks(n_chunk)
1527            .into_iter()
1528            .map(|elem| elem.into_iter().collect::<Vec<_>>())
1529            .collect::<Vec<_>>();
1530
1531        // Send 2 RecordBatches at the source
1532        for sn_values in chunks {
1533            let mut sn1_array = UInt64Builder::with_capacity(sn_values.len());
1534            let mut hash_array = Int64Builder::with_capacity(sn_values.len());
1535
1536            for sn in sn_values {
1537                sn1_array.append_value(sn as u64);
1538                let hash_value = (2 - (sn / hash_replicate)) as i64;
1539                hash_array.append_value(hash_value);
1540            }
1541
1542            let batch = RecordBatch::try_new(
1543                Arc::clone(schema),
1544                vec![Arc::new(sn1_array.finish()), Arc::new(hash_array.finish())],
1545            )?;
1546            batches.push(batch);
1547        }
1548        Ok(batches)
1549    }
1550
1551    fn generate_never_ending_source(
1552        n_rows: usize,
1553        chunk_length: usize,
1554        n_partition: usize,
1555        is_infinite: bool,
1556        send_exit: bool,
1557        per_batch_wait_duration_in_millis: u64,
1558    ) -> Result<Arc<dyn ExecutionPlan>> {
1559        assert!(n_partition > 0);
1560
1561        // We use same hash value in the table. This makes sure that
1562        // After hashing computation will continue in only in one of the output partitions
1563        // In this case, data flow should still continue
1564        let schema = test_schema();
1565        let orderings = schema_orders(&schema)?;
1566
1567        // Source waits per_batch_wait_duration_in_millis ms before sending other batch
1568        let per_batch_wait_duration =
1569            Duration::from_millis(per_batch_wait_duration_in_millis);
1570
1571        let batches = generate_batches(&schema, n_rows, chunk_length)?;
1572
1573        // Source has 2 partitions
1574        let partitions = vec![
1575            Arc::new(TestStreamPartition {
1576                schema: Arc::clone(&schema),
1577                batches,
1578                idx: 0,
1579                state: PolingState::BatchReturn,
1580                sleep_duration: per_batch_wait_duration,
1581                send_exit,
1582            }) as _;
1583            n_partition
1584        ];
1585        let source = Arc::new(StreamingTableExec::try_new(
1586            Arc::clone(&schema),
1587            partitions,
1588            None,
1589            orderings,
1590            is_infinite,
1591            None,
1592        )?) as _;
1593        Ok(source)
1594    }
1595
1596    // Tests NTH_VALUE(negative index) with memoize feature
1597    // To be able to trigger memoize feature for NTH_VALUE we need to
1598    // - feed BoundedWindowAggExec with batch stream data.
1599    // - Window frame should contain UNBOUNDED PRECEDING.
1600    // It hard to ensure these conditions are met, from the sql query.
1601    #[tokio::test]
1602    async fn test_window_nth_value_bounded_memoize() -> Result<()> {
1603        let config = SessionConfig::new().with_target_partitions(1);
1604        let task_ctx = Arc::new(TaskContext::default().with_session_config(config));
1605
1606        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]));
1607        // Create a new batch of data to insert into the table
1608        let batch = RecordBatch::try_new(
1609            Arc::clone(&schema),
1610            vec![Arc::new(arrow::array::Int32Array::from(vec![1, 2, 3]))],
1611        )?;
1612
1613        let memory_exec = TestMemoryExec::try_new_exec(
1614            &[vec![batch.clone(), batch.clone(), batch.clone()]],
1615            Arc::clone(&schema),
1616            None,
1617        )?;
1618        let col_a = col("a", &schema)?;
1619        let nth_value_func1 = create_udwf_window_expr(
1620            &nth_value_udwf(),
1621            &[
1622                Arc::clone(&col_a),
1623                Arc::new(Literal::new(ScalarValue::Int32(Some(1)))),
1624            ],
1625            &schema,
1626            "nth_value(-1)".to_string(),
1627            false,
1628        )?
1629        .reverse_expr()
1630        .unwrap();
1631        let nth_value_func2 = create_udwf_window_expr(
1632            &nth_value_udwf(),
1633            &[
1634                Arc::clone(&col_a),
1635                Arc::new(Literal::new(ScalarValue::Int32(Some(2)))),
1636            ],
1637            &schema,
1638            "nth_value(-2)".to_string(),
1639            false,
1640        )?
1641        .reverse_expr()
1642        .unwrap();
1643
1644        let last_value_func = create_udwf_window_expr(
1645            &last_value_udwf(),
1646            &[Arc::clone(&col_a)],
1647            &schema,
1648            "last".to_string(),
1649            false,
1650        )?;
1651
1652        let window_exprs = vec![
1653            // LAST_VALUE(a)
1654            Arc::new(StandardWindowExpr::new(
1655                last_value_func,
1656                &[],
1657                &[],
1658                Arc::new(WindowFrame::new_bounds(
1659                    WindowFrameUnits::Rows,
1660                    WindowFrameBound::Preceding(ScalarValue::UInt64(None)),
1661                    WindowFrameBound::CurrentRow,
1662                )),
1663            )) as _,
1664            // NTH_VALUE(a, -1)
1665            Arc::new(StandardWindowExpr::new(
1666                nth_value_func1,
1667                &[],
1668                &[],
1669                Arc::new(WindowFrame::new_bounds(
1670                    WindowFrameUnits::Rows,
1671                    WindowFrameBound::Preceding(ScalarValue::UInt64(None)),
1672                    WindowFrameBound::CurrentRow,
1673                )),
1674            )) as _,
1675            // NTH_VALUE(a, -2)
1676            Arc::new(StandardWindowExpr::new(
1677                nth_value_func2,
1678                &[],
1679                &[],
1680                Arc::new(WindowFrame::new_bounds(
1681                    WindowFrameUnits::Rows,
1682                    WindowFrameBound::Preceding(ScalarValue::UInt64(None)),
1683                    WindowFrameBound::CurrentRow,
1684                )),
1685            )) as _,
1686        ];
1687        let physical_plan = BoundedWindowAggExec::try_new(
1688            window_exprs,
1689            memory_exec,
1690            InputOrderMode::Sorted,
1691            true,
1692        )
1693        .map(|e| Arc::new(e) as Arc<dyn ExecutionPlan>)?;
1694
1695        let batches = collect(physical_plan.execute(0, task_ctx)?).await?;
1696
1697        // Get string representation of the plan
1698        assert_snapshot!(displayable(physical_plan.as_ref()).indent(true), @r#"
1699        BoundedWindowAggExec: wdw=[last: Field { "last": nullable Int32 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, nth_value(-1): Field { "nth_value(-1)": nullable Int32 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, nth_value(-2): Field { "nth_value(-2)": nullable Int32 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted]
1700          DataSourceExec: partitions=1, partition_sizes=[3]
1701        "#);
1702
1703        assert_snapshot!(batches_to_string(&batches), @r#"
1704            +---+------+---------------+---------------+
1705            | a | last | nth_value(-1) | nth_value(-2) |
1706            +---+------+---------------+---------------+
1707            | 1 | 1    | 1             |               |
1708            | 2 | 2    | 2             | 1             |
1709            | 3 | 3    | 3             | 2             |
1710            | 1 | 1    | 1             | 3             |
1711            | 2 | 2    | 2             | 1             |
1712            | 3 | 3    | 3             | 2             |
1713            | 1 | 1    | 1             | 3             |
1714            | 2 | 2    | 2             | 1             |
1715            | 3 | 3    | 3             | 2             |
1716            +---+------+---------------+---------------+
1717            "#);
1718        Ok(())
1719    }
1720
1721    // This test, tests whether most recent row guarantee by the input batch of the `BoundedWindowAggExec`
1722    // helps `BoundedWindowAggExec` to generate low latency result in the `Linear` mode.
1723    // Input data generated at the source is
1724    //       "+----+------+",
1725    //       "| sn | hash |",
1726    //       "+----+------+",
1727    //       "| 0  | 2    |",
1728    //       "| 1  | 2    |",
1729    //       "| 2  | 2    |",
1730    //       "| 3  | 2    |",
1731    //       "| 4  | 1    |",
1732    //       "| 5  | 1    |",
1733    //       "| 6  | 1    |",
1734    //       "| 7  | 1    |",
1735    //       "| 8  | 0    |",
1736    //       "| 9  | 0    |",
1737    //       "+----+------+",
1738    //
1739    // Effectively following query is run on this data
1740    //
1741    //   SELECT *, count(*) OVER(PARTITION BY duplicated_hash ORDER BY sn RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)
1742    //   FROM test;
1743    //
1744    // partition `duplicated_hash=2` receives following data from the input
1745    //
1746    //       "+----+------+",
1747    //       "| sn | hash |",
1748    //       "+----+------+",
1749    //       "| 0  | 2    |",
1750    //       "| 1  | 2    |",
1751    //       "| 2  | 2    |",
1752    //       "| 3  | 2    |",
1753    //       "+----+------+",
1754    // normally `BoundedWindowExec` can only generate following result from the input above
1755    //
1756    //       "+----+------+---------+",
1757    //       "| sn | hash |  count  |",
1758    //       "+----+------+---------+",
1759    //       "| 0  | 2    |  2      |",
1760    //       "| 1  | 2    |  2      |",
1761    //       "| 2  | 2    |<not yet>|",
1762    //       "| 3  | 2    |<not yet>|",
1763    //       "+----+------+---------+",
1764    // where result of last 2 row is missing. Since window frame end is not may change with future data
1765    // since window frame end is determined by 1 following (To generate result for row=3[where sn=2] we
1766    // need to received sn=4 to make sure window frame end bound won't change with future data).
1767    //
1768    // With the ability of different partitions to use global ordering at the input (where most up-to date
1769    //   row is
1770    //      "| 9  | 0    |",
1771    //   )
1772    //
1773    // `BoundedWindowExec` should be able to generate following result in the test
1774    //
1775    //       "+----+------+-------+",
1776    //       "| sn | hash | col_2 |",
1777    //       "+----+------+-------+",
1778    //       "| 0  | 2    | 2     |",
1779    //       "| 1  | 2    | 2     |",
1780    //       "| 2  | 2    | 2     |",
1781    //       "| 3  | 2    | 1     |",
1782    //       "| 4  | 1    | 2     |",
1783    //       "| 5  | 1    | 2     |",
1784    //       "| 6  | 1    | 2     |",
1785    //       "| 7  | 1    | 1     |",
1786    //       "+----+------+-------+",
1787    //
1788    // where result for all rows except last 2 is calculated (To calculate result for row 9 where sn=8
1789    //   we need to receive sn=10 value to calculate it result.).
1790    // In this test, out aim is to test for which portion of the input data `BoundedWindowExec` can generate
1791    // a result. To test this behaviour, we generated the data at the source infinitely (no `None` signal
1792    //    is sent to output from source). After, row:
1793    //
1794    //       "| 9  | 0    |",
1795    //
1796    // is sent. Source stops sending data to output. We collect, result emitted by the `BoundedWindowExec` at the
1797    // end of the pipeline with a timeout (Since no `None` is sent from source. Collection never ends otherwise).
1798    #[tokio::test]
1799    async fn bounded_window_exec_linear_mode_range_information() -> Result<()> {
1800        let n_rows = 10;
1801        let chunk_length = 2;
1802        let n_future_range = 1;
1803
1804        let timeout_duration = Duration::from_millis(2000);
1805
1806        let source =
1807            generate_never_ending_source(n_rows, chunk_length, 1, true, false, 5)?;
1808
1809        let window =
1810            bounded_window_exec_pb_latent_range(source, n_future_range, "hash", "sn")?;
1811
1812        let plan = projection_exec(window)?;
1813
1814        // Get string representation of the plan
1815        assert_snapshot!(displayable(plan.as_ref()).indent(true), @r#"
1816        ProjectionExec: expr=[sn@0 as sn, hash@1 as hash, count([Column { name: "sn", index: 0 }]) PARTITION BY: [[Column { name: "hash", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: "sn", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]@2 as col_2]
1817          BoundedWindowAggExec: wdw=[count([Column { name: "sn", index: 0 }]) PARTITION BY: [[Column { name: "hash", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: "sn", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]: Field { "count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]": Int64 }, frame: RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING], mode=[Linear]
1818            StreamingTableExec: partition_sizes=1, projection=[sn, hash], infinite_source=true, output_ordering=[sn@0 ASC NULLS LAST]
1819        "#);
1820
1821        let task_ctx = task_context();
1822        let batches = collect_with_timeout(plan, task_ctx, timeout_duration).await?;
1823
1824        assert_snapshot!(batches_to_string(&batches), @r#"
1825            +----+------+-------+
1826            | sn | hash | col_2 |
1827            +----+------+-------+
1828            | 0  | 2    | 2     |
1829            | 1  | 2    | 2     |
1830            | 2  | 2    | 2     |
1831            | 3  | 2    | 1     |
1832            | 4  | 1    | 2     |
1833            | 5  | 1    | 2     |
1834            | 6  | 1    | 2     |
1835            | 7  | 1    | 1     |
1836            +----+------+-------+
1837            "#);
1838
1839        Ok(())
1840    }
1841}