1use 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#[derive(Debug, Clone)]
77pub struct BoundedWindowAggExec {
78 input: Arc<dyn ExecutionPlan>,
80 window_expr: Vec<Arc<dyn WindowExpr>>,
82 schema: SchemaRef,
84 metrics: ExecutionPlanMetricsSet,
86 pub input_order_mode: InputOrderMode,
88 ordered_partition_by_indices: Vec<usize>,
95 cache: PlanProperties,
97 can_repartition: bool,
99}
100
101impl BoundedWindowAggExec {
102 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 pub fn window_expr(&self) -> &[Arc<dyn WindowExpr>] {
144 &self.window_expr
145 }
146
147 pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
149 &self.input
150 }
151
152 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 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 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 fn compute_properties(
194 input: &Arc<dyn ExecutionPlan>,
195 schema: &SchemaRef,
196 window_exprs: &[Arc<dyn WindowExpr>],
197 ) -> Result<PlanProperties> {
198 let eq_properties = window_equivalence_properties(schema, input, window_exprs)?;
200
201 let output_partitioning = input.output_partitioning().clone();
205
206 Ok(PlanProperties::new(
208 eq_properties,
209 output_partitioning,
210 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 let mut column_statistics = Vec::with_capacity(win_cols + input_cols);
238 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 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
379trait PartitionSearcher: Send {
382 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 fn is_mode_linear(&self) -> bool {
401 false
402 }
403
404 fn evaluate_partition_batches(
406 &mut self,
407 record_batch: &RecordBatch,
408 window_expr: &[Arc<dyn WindowExpr>],
409 ) -> Result<Vec<(PartitionKey, RecordBatch)>>;
410
411 fn prune(&mut self, _n_out: usize) {}
413
414 fn mark_partition_end(&self, partition_buffers: &mut PartitionBatches);
417
418 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 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 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
486pub struct LinearSearch {
489 input_buffer_hashes: VecDeque<u64>,
492 random_state: RandomState,
494 ordered_partition_by_indices: Vec<usize>,
499 row_map_batch: HashTable<(u64, usize)>,
503 row_map_out: HashTable<(u64, usize, usize)>,
509 input_schema: SchemaRef,
510}
511
512impl PartitionSearcher for LinearSearch {
513 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 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 partition_batch_state.n_out_row = length;
574 all_indices.append_slice(&indices);
576 }
577 let all_indices = all_indices.finish();
578 if all_indices.is_empty() {
579 return Ok(None);
581 }
582
583 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 let sorted_indices = sort_to_indices(&all_indices, None, None)?;
594 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 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 self.input_buffer_hashes.drain(0..n_out);
625 }
626
627 fn mark_partition_end(&self, partition_buffers: &mut PartitionBatches) {
628 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 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 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 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 self.row_map_batch.clear();
685 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 let row = get_row_at_idx(columns, row_idx as usize).unwrap();
692 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 result.push((row, vec![row_idx]));
706 }
707 }
708 Ok(result)
709 }
710
711 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 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
765pub struct SortedSearch {
768 partition_by_sort_keys: Vec<PhysicalSortExpr>,
770 ordered_partition_by_indices: Vec<usize>,
775 input_schema: SchemaRef,
776}
777
778impl PartitionSearcher for SortedSearch {
779 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 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 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 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 fn calculate_n_out_row(
850 &mut self,
851 window_agg_states: &[PartitionWindowAggStates],
852 partition_buffers: &mut PartitionBatches,
853 ) -> usize {
854 let mut counts = vec![];
857 let out_col_counts = window_agg_states.iter().map(|window_agg_state| {
858 let mut cur_window_expr_out_result_len = 0;
861 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 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
893fn 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
911pub struct BoundedWindowAggStream {
913 schema: SchemaRef,
914 input: SendableRecordBatchStream,
915 input_buffer: RecordBatch,
918 partition_buffers: PartitionBatches,
927 window_agg_states: Vec<PartitionWindowAggStates>,
931 finished: bool,
932 window_expr: Vec<Arc<dyn WindowExpr>>,
933 baseline_metrics: BaselineMetrics,
934 search_mode: Box<dyn PartitionSearcher>,
937}
938
939impl BoundedWindowAggStream {
940 fn prune_state(&mut self, n_out: usize) -> Result<()> {
948 self.prune_out_columns();
950 self.prune_partition_batches();
952 self.prune_input_batch(n_out)?;
954 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 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 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 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 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 fn prune_partition_batches(&mut self) {
1075 self.partition_buffers
1079 .retain(|_, partition_batch_state| !partition_batch_state.is_end);
1080
1081 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 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 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 fn prune_input_batch(&mut self, n_out: usize) -> Result<()> {
1126 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 fn prune_out_columns(&mut self) {
1144 for partition_window_agg_states in self.window_agg_states.iter_mut() {
1148 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 fn schema(&self) -> SchemaRef {
1173 Arc::clone(&self.schema)
1174 }
1175}
1176
1177fn 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
1183fn 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 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 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
1229pub(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 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 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 return Poll::Ready(None);
1339 } else {
1340 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 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 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 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 #[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 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 let schema = test_schema();
1565 let orderings = schema_orders(&schema)?;
1566
1567 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 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 #[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 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 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 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 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 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 #[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 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}