1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

use std::any::Any;
use std::fmt::Debug;
use std::ops::Range;
use std::sync::Arc;

use crate::{LexOrderingRef, PhysicalExpr, PhysicalSortExpr};

use arrow::array::{new_empty_array, Array, ArrayRef};
use arrow::compute::kernels::sort::SortColumn;
use arrow::compute::SortOptions;
use arrow::datatypes::Field;
use arrow::record_batch::RecordBatch;
use datafusion_common::utils::compare_rows;
use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue};
use datafusion_expr::window_state::{
    PartitionBatchState, WindowAggState, WindowFrameContext, WindowFrameStateGroups,
};
use datafusion_expr::{Accumulator, PartitionEvaluator, WindowFrame, WindowFrameBound};

use indexmap::IndexMap;

/// Common trait for [window function] implementations
///
/// # Aggregate Window Expressions
///
/// These expressions take the form
///
/// ```text
/// OVER({ROWS | RANGE| GROUPS} BETWEEN UNBOUNDED PRECEDING AND ...)
/// ```
///
/// For example, cumulative window frames uses `PlainAggregateWindowExpr`.
///
/// # Non Aggregate Window Expressions
///
/// The expressions have the form
///
/// ```text
/// OVER({ROWS | RANGE| GROUPS} BETWEEN M {PRECEDING| FOLLOWING} AND ...)
/// ```
///
/// For example, sliding window frames use [`SlidingAggregateWindowExpr`].
///
/// [window function]: https://en.wikipedia.org/wiki/Window_function_(SQL)
/// [`PlainAggregateWindowExpr`]: crate::window::PlainAggregateWindowExpr
/// [`SlidingAggregateWindowExpr`]: crate::window::SlidingAggregateWindowExpr
pub trait WindowExpr: Send + Sync + Debug {
    /// Returns the window expression as [`Any`] so that it can be
    /// downcast to a specific implementation.
    fn as_any(&self) -> &dyn Any;

    /// The field of the final result of this window function.
    fn field(&self) -> Result<Field>;

    /// Human readable name such as `"MIN(c2)"` or `"RANK()"`. The default
    /// implementation returns placeholder text.
    fn name(&self) -> &str {
        "WindowExpr: default name"
    }

    /// Expressions that are passed to the WindowAccumulator.
    /// Functions which take a single input argument, such as `sum`, return a single [`datafusion_expr::expr::Expr`],
    /// others (e.g. `cov`) return many.
    fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>>;

    /// Evaluate the window function arguments against the batch and return
    /// array ref, normally the resulting `Vec` is a single element one.
    fn evaluate_args(&self, batch: &RecordBatch) -> Result<Vec<ArrayRef>> {
        self.expressions()
            .iter()
            .map(|e| {
                e.evaluate(batch)
                    .and_then(|v| v.into_array(batch.num_rows()))
            })
            .collect()
    }

    /// Evaluate the window function values against the batch
    fn evaluate(&self, batch: &RecordBatch) -> Result<ArrayRef>;

    /// Evaluate the window function against the batch. This function facilitates
    /// stateful, bounded-memory implementations.
    fn evaluate_stateful(
        &self,
        _partition_batches: &PartitionBatches,
        _window_agg_state: &mut PartitionWindowAggStates,
    ) -> Result<()> {
        internal_err!("evaluate_stateful is not implemented for {}", self.name())
    }

    /// Expressions that's from the window function's partition by clause, empty if absent
    fn partition_by(&self) -> &[Arc<dyn PhysicalExpr>];

    /// Expressions that's from the window function's order by clause, empty if absent
    fn order_by(&self) -> &[PhysicalSortExpr];

    /// Get order by columns, empty if absent
    fn order_by_columns(&self, batch: &RecordBatch) -> Result<Vec<SortColumn>> {
        self.order_by()
            .iter()
            .map(|e| e.evaluate_to_sort_column(batch))
            .collect::<Result<Vec<SortColumn>>>()
    }

    /// Get the window frame of this [WindowExpr].
    fn get_window_frame(&self) -> &Arc<WindowFrame>;

    /// Return a flag indicating whether this [WindowExpr] can run with
    /// bounded memory.
    fn uses_bounded_memory(&self) -> bool;

    /// Get the reverse expression of this [WindowExpr].
    fn get_reverse_expr(&self) -> Option<Arc<dyn WindowExpr>>;

    /// Returns all expressions used in the [`WindowExpr`].
    /// These expressions are (1) function arguments, (2) partition by expressions, (3) order by expressions.
    fn all_expressions(&self) -> WindowPhysicalExpressions {
        let args = self.expressions();
        let partition_by_exprs = self.partition_by().to_vec();
        let order_by_exprs = self
            .order_by()
            .iter()
            .map(|sort_expr| Arc::clone(&sort_expr.expr))
            .collect::<Vec<_>>();
        WindowPhysicalExpressions {
            args,
            partition_by_exprs,
            order_by_exprs,
        }
    }

    /// Rewrites [`WindowExpr`], with new expressions given. The argument should be consistent
    /// with the return value of the [`WindowExpr::all_expressions`] method.
    /// Returns `Some(Arc<dyn WindowExpr>)` if re-write is supported, otherwise returns `None`.
    fn with_new_expressions(
        &self,
        _args: Vec<Arc<dyn PhysicalExpr>>,
        _partition_bys: Vec<Arc<dyn PhysicalExpr>>,
        _order_by_exprs: Vec<Arc<dyn PhysicalExpr>>,
    ) -> Option<Arc<dyn WindowExpr>> {
        None
    }
}

/// Stores the physical expressions used inside the `WindowExpr`.
pub struct WindowPhysicalExpressions {
    /// Window function arguments
    pub args: Vec<Arc<dyn PhysicalExpr>>,
    /// PARTITION BY expressions
    pub partition_by_exprs: Vec<Arc<dyn PhysicalExpr>>,
    /// ORDER BY expressions
    pub order_by_exprs: Vec<Arc<dyn PhysicalExpr>>,
}

/// Extension trait that adds common functionality to [`AggregateWindowExpr`]s
pub trait AggregateWindowExpr: WindowExpr {
    /// Get the accumulator for the window expression. Note that distinct
    /// window expressions may return distinct accumulators; e.g. sliding
    /// (non-sliding) expressions will return sliding (normal) accumulators.
    fn get_accumulator(&self) -> Result<Box<dyn Accumulator>>;

    /// Given current range and the last range, calculates the accumulator
    /// result for the range of interest.
    fn get_aggregate_result_inside_range(
        &self,
        last_range: &Range<usize>,
        cur_range: &Range<usize>,
        value_slice: &[ArrayRef],
        accumulator: &mut Box<dyn Accumulator>,
    ) -> Result<ScalarValue>;

    /// Evaluates the window function against the batch.
    fn aggregate_evaluate(&self, batch: &RecordBatch) -> Result<ArrayRef> {
        let mut accumulator = self.get_accumulator()?;
        let mut last_range = Range { start: 0, end: 0 };
        let sort_options: Vec<SortOptions> =
            self.order_by().iter().map(|o| o.options).collect();
        let mut window_frame_ctx =
            WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options);
        self.get_result_column(
            &mut accumulator,
            batch,
            None,
            &mut last_range,
            &mut window_frame_ctx,
            0,
            false,
        )
    }

    /// Statefully evaluates the window function against the batch. Maintains
    /// state so that it can work incrementally over multiple chunks.
    fn aggregate_evaluate_stateful(
        &self,
        partition_batches: &PartitionBatches,
        window_agg_state: &mut PartitionWindowAggStates,
    ) -> Result<()> {
        let field = self.field()?;
        let out_type = field.data_type();
        for (partition_row, partition_batch_state) in partition_batches.iter() {
            if !window_agg_state.contains_key(partition_row) {
                let accumulator = self.get_accumulator()?;
                window_agg_state.insert(
                    partition_row.clone(),
                    WindowState {
                        state: WindowAggState::new(out_type)?,
                        window_fn: WindowFn::Aggregate(accumulator),
                    },
                );
            };
            let window_state =
                window_agg_state.get_mut(partition_row).ok_or_else(|| {
                    DataFusionError::Execution("Cannot find state".to_string())
                })?;
            let accumulator = match &mut window_state.window_fn {
                WindowFn::Aggregate(accumulator) => accumulator,
                _ => unreachable!(),
            };
            let state = &mut window_state.state;
            let record_batch = &partition_batch_state.record_batch;
            let most_recent_row = partition_batch_state.most_recent_row.as_ref();

            // If there is no window state context, initialize it.
            let window_frame_ctx = state.window_frame_ctx.get_or_insert_with(|| {
                let sort_options: Vec<SortOptions> =
                    self.order_by().iter().map(|o| o.options).collect();
                WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options)
            });
            let out_col = self.get_result_column(
                accumulator,
                record_batch,
                most_recent_row,
                // Start search from the last range
                &mut state.window_frame_range,
                window_frame_ctx,
                state.last_calculated_index,
                !partition_batch_state.is_end,
            )?;
            state.update(&out_col, partition_batch_state)?;
        }
        Ok(())
    }

    /// Calculates the window expression result for the given record batch.
    /// Assumes that `record_batch` belongs to a single partition.
    #[allow(clippy::too_many_arguments)]
    fn get_result_column(
        &self,
        accumulator: &mut Box<dyn Accumulator>,
        record_batch: &RecordBatch,
        most_recent_row: Option<&RecordBatch>,
        last_range: &mut Range<usize>,
        window_frame_ctx: &mut WindowFrameContext,
        mut idx: usize,
        not_end: bool,
    ) -> Result<ArrayRef> {
        let values = self.evaluate_args(record_batch)?;
        let order_bys = get_orderby_values(self.order_by_columns(record_batch)?);

        let most_recent_row_order_bys = most_recent_row
            .map(|batch| self.order_by_columns(batch))
            .transpose()?
            .map(get_orderby_values);

        // We iterate on each row to perform a running calculation.
        let length = values[0].len();
        let mut row_wise_results: Vec<ScalarValue> = vec![];
        let is_causal = self.get_window_frame().is_causal();
        while idx < length {
            // Start search from the last_range. This squeezes searched range.
            let cur_range =
                window_frame_ctx.calculate_range(&order_bys, last_range, length, idx)?;
            // Exit if the range is non-causal and extends all the way:
            if cur_range.end == length
                && !is_causal
                && not_end
                && !is_end_bound_safe(
                    window_frame_ctx,
                    &order_bys,
                    most_recent_row_order_bys.as_deref(),
                    self.order_by(),
                    idx,
                )?
            {
                break;
            }
            let value = self.get_aggregate_result_inside_range(
                last_range,
                &cur_range,
                &values,
                accumulator,
            )?;
            // Update last range
            *last_range = cur_range;
            row_wise_results.push(value);
            idx += 1;
        }

        if row_wise_results.is_empty() {
            let field = self.field()?;
            let out_type = field.data_type();
            Ok(new_empty_array(out_type))
        } else {
            ScalarValue::iter_to_array(row_wise_results)
        }
    }
}

/// Determines whether the end bound calculation for a window frame context is
/// safe, meaning that the end bound stays the same, regardless of future data,
/// based on the current sort expressions and ORDER BY columns. This function
/// delegates work to specific functions for each frame type.
///
/// # Parameters
///
/// * `window_frame_ctx`: The context of the window frame being evaluated.
/// * `order_bys`: A slice of `ArrayRef` representing the ORDER BY columns.
/// * `most_recent_order_bys`: An optional reference to the most recent ORDER BY
///   columns.
/// * `sort_exprs`: Defines the lexicographical ordering in question.
/// * `idx`: The current index in the window frame.
///
/// # Returns
///
/// A `Result` which is `Ok(true)` if the end bound is safe, `Ok(false)` otherwise.
pub(crate) fn is_end_bound_safe(
    window_frame_ctx: &WindowFrameContext,
    order_bys: &[ArrayRef],
    most_recent_order_bys: Option<&[ArrayRef]>,
    sort_exprs: LexOrderingRef,
    idx: usize,
) -> Result<bool> {
    if sort_exprs.is_empty() {
        // Early return if no sort expressions are present:
        return Ok(false);
    }

    match window_frame_ctx {
        WindowFrameContext::Rows(window_frame) => {
            is_end_bound_safe_for_rows(&window_frame.end_bound)
        }
        WindowFrameContext::Range { window_frame, .. } => is_end_bound_safe_for_range(
            &window_frame.end_bound,
            &order_bys[0],
            most_recent_order_bys.map(|items| &items[0]),
            &sort_exprs[0].options,
            idx,
        ),
        WindowFrameContext::Groups {
            window_frame,
            state,
        } => is_end_bound_safe_for_groups(
            &window_frame.end_bound,
            state,
            &order_bys[0],
            most_recent_order_bys.map(|items| &items[0]),
            &sort_exprs[0].options,
        ),
    }
}

/// For row-based window frames, determines whether the end bound calculation
/// is safe, which is trivially the case for `Preceding` and `CurrentRow` bounds.
/// For 'Following' bounds, it compares the bound value to zero to ensure that
/// it doesn't extend beyond the current row.
///
/// # Parameters
///
/// * `end_bound`: Reference to the window frame bound in question.
///
/// # Returns
///
/// A `Result` indicating whether the end bound is safe for row-based window frames.
fn is_end_bound_safe_for_rows(end_bound: &WindowFrameBound) -> Result<bool> {
    if let WindowFrameBound::Following(value) = end_bound {
        let zero = ScalarValue::new_zero(&value.data_type());
        Ok(zero.map(|zero| value.eq(&zero)).unwrap_or(false))
    } else {
        Ok(true)
    }
}

/// For row-based window frames, determines whether the end bound calculation
/// is safe by comparing it against specific values (zero, current row). It uses
/// the `is_row_ahead` helper function to determine if the current row is ahead
/// of the most recent row based on the ORDER BY column and sorting options.
///
/// # Parameters
///
/// * `end_bound`: Reference to the window frame bound in question.
/// * `orderby_col`: Reference to the column used for ordering.
/// * `most_recent_ob_col`: Optional reference to the most recent order-by column.
/// * `sort_options`: The sorting options used in the window frame.
/// * `idx`: The current index in the window frame.
///
/// # Returns
///
/// A `Result` indicating whether the end bound is safe for range-based window frames.
fn is_end_bound_safe_for_range(
    end_bound: &WindowFrameBound,
    orderby_col: &ArrayRef,
    most_recent_ob_col: Option<&ArrayRef>,
    sort_options: &SortOptions,
    idx: usize,
) -> Result<bool> {
    match end_bound {
        WindowFrameBound::Preceding(value) => {
            let zero = ScalarValue::new_zero(&value.data_type())?;
            if value.eq(&zero) {
                is_row_ahead(orderby_col, most_recent_ob_col, sort_options)
            } else {
                Ok(true)
            }
        }
        WindowFrameBound::CurrentRow => {
            is_row_ahead(orderby_col, most_recent_ob_col, sort_options)
        }
        WindowFrameBound::Following(delta) => {
            let Some(most_recent_ob_col) = most_recent_ob_col else {
                return Ok(false);
            };
            let most_recent_row_value =
                ScalarValue::try_from_array(most_recent_ob_col, 0)?;
            let current_row_value = ScalarValue::try_from_array(orderby_col, idx)?;

            if sort_options.descending {
                current_row_value
                    .sub(delta)
                    .map(|value| value > most_recent_row_value)
            } else {
                current_row_value
                    .add(delta)
                    .map(|value| most_recent_row_value > value)
            }
        }
    }
}

/// For group-based window frames, determines whether the end bound calculation
/// is safe by considering the group offset and whether the current row is ahead
/// of the most recent row in terms of sorting. It checks if the end bound is
/// within the bounds of the current group based on group end indices.
///
/// # Parameters
///
/// * `end_bound`: Reference to the window frame bound in question.
/// * `state`: The state of the window frame for group calculations.
/// * `orderby_col`: Reference to the column used for ordering.
/// * `most_recent_ob_col`: Optional reference to the most recent order-by column.
/// * `sort_options`: The sorting options used in the window frame.
///
/// # Returns
///
/// A `Result` indicating whether the end bound is safe for group-based window frames.
fn is_end_bound_safe_for_groups(
    end_bound: &WindowFrameBound,
    state: &WindowFrameStateGroups,
    orderby_col: &ArrayRef,
    most_recent_ob_col: Option<&ArrayRef>,
    sort_options: &SortOptions,
) -> Result<bool> {
    match end_bound {
        WindowFrameBound::Preceding(value) => {
            let zero = ScalarValue::new_zero(&value.data_type())?;
            if value.eq(&zero) {
                is_row_ahead(orderby_col, most_recent_ob_col, sort_options)
            } else {
                Ok(true)
            }
        }
        WindowFrameBound::CurrentRow => {
            is_row_ahead(orderby_col, most_recent_ob_col, sort_options)
        }
        WindowFrameBound::Following(ScalarValue::UInt64(Some(offset))) => {
            let delta = state.group_end_indices.len() - state.current_group_idx;
            if delta == (*offset as usize) + 1 {
                is_row_ahead(orderby_col, most_recent_ob_col, sort_options)
            } else {
                Ok(false)
            }
        }
        _ => Ok(false),
    }
}

/// This utility function checks whether `current_cols` is ahead of the `old_cols`
/// in terms of `sort_options`.
fn is_row_ahead(
    old_col: &ArrayRef,
    current_col: Option<&ArrayRef>,
    sort_options: &SortOptions,
) -> Result<bool> {
    let Some(current_col) = current_col else {
        return Ok(false);
    };
    if old_col.is_empty() || current_col.is_empty() {
        return Ok(false);
    }
    let last_value = ScalarValue::try_from_array(old_col, old_col.len() - 1)?;
    let current_value = ScalarValue::try_from_array(current_col, 0)?;
    let cmp = compare_rows(&[current_value], &[last_value], &[*sort_options])?;
    Ok(cmp.is_gt())
}

/// Get order by expression results inside `order_by_columns`.
pub(crate) fn get_orderby_values(order_by_columns: Vec<SortColumn>) -> Vec<ArrayRef> {
    order_by_columns.into_iter().map(|s| s.values).collect()
}

#[derive(Debug)]
pub enum WindowFn {
    Builtin(Box<dyn PartitionEvaluator>),
    Aggregate(Box<dyn Accumulator>),
}

/// State for the RANK(percent_rank, rank, dense_rank) built-in window function.
#[derive(Debug, Clone, Default)]
pub struct RankState {
    /// The last values for rank as these values change, we increase n_rank
    pub last_rank_data: Option<Vec<ScalarValue>>,
    /// The index where last_rank_boundary is started
    pub last_rank_boundary: usize,
    /// Keep the number of entries in current rank
    pub current_group_count: usize,
    /// Rank number kept from the start
    pub n_rank: usize,
}

/// State for the 'ROW_NUMBER' built-in window function.
#[derive(Debug, Clone, Default)]
pub struct NumRowsState {
    pub n_rows: usize,
}

/// Tag to differentiate special use cases of the NTH_VALUE built-in window function.
#[derive(Debug, Copy, Clone)]
pub enum NthValueKind {
    First,
    Last,
    Nth(i64),
}

#[derive(Debug, Clone)]
pub struct NthValueState {
    // In certain cases, we can finalize the result early. Consider this usage:
    // ```
    //  FIRST_VALUE(increasing_col) OVER window AS my_first_value
    //  WINDOW (ORDER BY ts ASC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) AS window
    // ```
    // The result will always be the first entry in the table. We can store such
    // early-finalizing results and then just reuse them as necessary. This opens
    // opportunities to prune our datasets.
    pub finalized_result: Option<ScalarValue>,
    pub kind: NthValueKind,
}

/// Key for IndexMap for each unique partition
///
/// For instance, if window frame is `OVER(PARTITION BY a,b)`,
/// PartitionKey would consist of unique `[a,b]` pairs
pub type PartitionKey = Vec<ScalarValue>;

#[derive(Debug)]
pub struct WindowState {
    pub state: WindowAggState,
    pub window_fn: WindowFn,
}
pub type PartitionWindowAggStates = IndexMap<PartitionKey, WindowState>;

/// The IndexMap (i.e. an ordered HashMap) where record batches are separated for each partition.
pub type PartitionBatches = IndexMap<PartitionKey, PartitionBatchState>;

#[cfg(test)]
mod tests {
    use std::sync::Arc;

    use crate::window::window_expr::is_row_ahead;

    use arrow_array::{ArrayRef, Float64Array};
    use arrow_schema::SortOptions;
    use datafusion_common::Result;

    #[test]
    fn test_is_row_ahead() -> Result<()> {
        let old_values: ArrayRef =
            Arc::new(Float64Array::from(vec![5.0, 7.0, 8.0, 9., 10.]));

        let new_values1: ArrayRef = Arc::new(Float64Array::from(vec![11.0]));
        let new_values2: ArrayRef = Arc::new(Float64Array::from(vec![10.0]));

        assert!(is_row_ahead(
            &old_values,
            Some(&new_values1),
            &SortOptions {
                descending: false,
                nulls_first: false
            }
        )?);
        assert!(!is_row_ahead(
            &old_values,
            Some(&new_values2),
            &SortOptions {
                descending: false,
                nulls_first: false
            }
        )?);

        Ok(())
    }
}