query/
part_sort.rs

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
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
// Copyright 2023 Greptime Team
//
// Licensed 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.

//! Module for sorting input data within each [`PartitionRange`].
//!
//! This module defines the [`PartSortExec`] execution plan, which sorts each
//! partition ([`PartitionRange`]) independently based on the provided physical
//! sort expressions.

use std::any::Any;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};

use arrow::array::ArrayRef;
use arrow::compute::{concat, take_record_batch};
use arrow_schema::SchemaRef;
use common_recordbatch::{DfRecordBatch, DfSendableRecordBatchStream};
use datafusion::common::arrow::compute::sort_to_indices;
use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion::execution::{RecordBatchStream, TaskContext};
use datafusion::physical_plan::coalesce_batches::concat_batches;
use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
use datafusion::physical_plan::{
    DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, TopK,
};
use datafusion_common::{internal_err, DataFusionError};
use datafusion_physical_expr::PhysicalSortExpr;
use futures::{Stream, StreamExt};
use itertools::Itertools;
use snafu::location;
use store_api::region_engine::PartitionRange;

use crate::{array_iter_helper, downcast_ts_array};

/// Sort input within given PartitionRange
///
/// Input is assumed to be segmented by empty RecordBatch, which indicates a new `PartitionRange` is starting
///
/// and this operator will sort each partition independently within the partition.
#[derive(Debug, Clone)]
pub struct PartSortExec {
    /// Physical sort expressions(that is, sort by timestamp)
    expression: PhysicalSortExpr,
    limit: Option<usize>,
    input: Arc<dyn ExecutionPlan>,
    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
    partition_ranges: Vec<Vec<PartitionRange>>,
    properties: PlanProperties,
}

impl PartSortExec {
    pub fn new(
        expression: PhysicalSortExpr,
        limit: Option<usize>,
        partition_ranges: Vec<Vec<PartitionRange>>,
        input: Arc<dyn ExecutionPlan>,
    ) -> Self {
        let metrics = ExecutionPlanMetricsSet::new();
        let properties = PlanProperties::new(
            input.equivalence_properties().clone(),
            input.output_partitioning().clone(),
            input.execution_mode(),
        );

        Self {
            expression,
            limit,
            input,
            metrics,
            partition_ranges,
            properties,
        }
    }

    pub fn to_stream(
        &self,
        context: Arc<TaskContext>,
        partition: usize,
    ) -> datafusion_common::Result<DfSendableRecordBatchStream> {
        let input_stream: DfSendableRecordBatchStream =
            self.input.execute(partition, context.clone())?;

        if partition >= self.partition_ranges.len() {
            internal_err!(
                "Partition index out of range: {} >= {}",
                partition,
                self.partition_ranges.len()
            )?;
        }

        let df_stream = Box::pin(PartSortStream::new(
            context,
            self,
            self.limit,
            input_stream,
            self.partition_ranges[partition].clone(),
            partition,
        )?) as _;

        Ok(df_stream)
    }
}

impl DisplayAs for PartSortExec {
    fn fmt_as(&self, _t: DisplayFormatType, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        write!(
            f,
            "PartSortExec: expr={} num_ranges={}",
            self.expression,
            self.partition_ranges.len(),
        )?;
        if let Some(limit) = self.limit {
            write!(f, " limit={}", limit)?;
        }
        Ok(())
    }
}

impl ExecutionPlan for PartSortExec {
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> SchemaRef {
        self.input.schema()
    }

    fn properties(&self) -> &PlanProperties {
        &self.properties
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        vec![&self.input]
    }

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> datafusion_common::Result<Arc<dyn ExecutionPlan>> {
        let new_input = if let Some(first) = children.first() {
            first
        } else {
            internal_err!("No children found")?
        };
        Ok(Arc::new(Self::new(
            self.expression.clone(),
            self.limit,
            self.partition_ranges.clone(),
            new_input.clone(),
        )))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<TaskContext>,
    ) -> datafusion_common::Result<DfSendableRecordBatchStream> {
        self.to_stream(context, partition)
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    /// # Explain
    ///
    /// This plan needs to be executed on each partition independently,
    /// and is expected to run directly on storage engine's output
    /// distribution / partition.
    fn benefits_from_input_partitioning(&self) -> Vec<bool> {
        vec![false]
    }
}

enum PartSortBuffer {
    All(Vec<DfRecordBatch>),
    /// TopK buffer with row count.
    ///
    /// Given this heap only keeps k element, the capacity of this buffer
    /// is not accurate, and is only used for empty check.
    Top(TopK, usize),
}

impl PartSortBuffer {
    pub fn is_empty(&self) -> bool {
        match self {
            PartSortBuffer::All(v) => v.is_empty(),
            PartSortBuffer::Top(_, cnt) => *cnt == 0,
        }
    }
}

struct PartSortStream {
    /// Memory pool for this stream
    reservation: MemoryReservation,
    buffer: PartSortBuffer,
    expression: PhysicalSortExpr,
    limit: Option<usize>,
    produced: usize,
    input: DfSendableRecordBatchStream,
    input_complete: bool,
    schema: SchemaRef,
    partition_ranges: Vec<PartitionRange>,
    #[allow(dead_code)] // this is used under #[debug_assertions]
    partition: usize,
    cur_part_idx: usize,
    evaluating_batch: Option<DfRecordBatch>,
    metrics: BaselineMetrics,
    context: Arc<TaskContext>,
    root_metrics: ExecutionPlanMetricsSet,
}

impl PartSortStream {
    fn new(
        context: Arc<TaskContext>,
        sort: &PartSortExec,
        limit: Option<usize>,
        input: DfSendableRecordBatchStream,
        partition_ranges: Vec<PartitionRange>,
        partition: usize,
    ) -> datafusion_common::Result<Self> {
        let buffer = if let Some(limit) = limit {
            PartSortBuffer::Top(
                TopK::try_new(
                    partition,
                    sort.schema().clone(),
                    vec![sort.expression.clone()],
                    limit,
                    context.session_config().batch_size(),
                    context.runtime_env(),
                    &sort.metrics,
                    partition,
                )?,
                0,
            )
        } else {
            PartSortBuffer::All(Vec::new())
        };

        Ok(Self {
            reservation: MemoryConsumer::new("PartSortStream".to_string())
                .register(&context.runtime_env().memory_pool),
            buffer,
            expression: sort.expression.clone(),
            limit,
            produced: 0,
            input,
            input_complete: false,
            schema: sort.input.schema(),
            partition_ranges,
            partition,
            cur_part_idx: 0,
            evaluating_batch: None,
            metrics: BaselineMetrics::new(&sort.metrics, partition),
            context,
            root_metrics: sort.metrics.clone(),
        })
    }
}

macro_rules! array_check_helper {
    ($t:ty, $unit:expr, $arr:expr, $cur_range:expr, $min_max_idx:expr) => {{
            if $cur_range.start.unit().as_arrow_time_unit() != $unit
            || $cur_range.end.unit().as_arrow_time_unit() != $unit
        {
            internal_err!(
                "PartitionRange unit mismatch, expect {:?}, found {:?}",
                $cur_range.start.unit(),
                $unit
            )?;
        }
        let arr = $arr
            .as_any()
            .downcast_ref::<arrow::array::PrimitiveArray<$t>>()
            .unwrap();

        let min = arr.value($min_max_idx.0);
        let max = arr.value($min_max_idx.1);
        let (min, max) = if min < max{
            (min, max)
        } else {
            (max, min)
        };
        let cur_min = $cur_range.start.value();
        let cur_max = $cur_range.end.value();
        // note that PartitionRange is left inclusive and right exclusive
        if !(min >= cur_min && max < cur_max) {
            internal_err!(
                "Sort column min/max value out of partition range: sort_column.min_max=[{:?}, {:?}] not in PartitionRange=[{:?}, {:?}]",
                min,
                max,
                cur_min,
                cur_max
            )?;
        }
    }};
}

impl PartSortStream {
    /// check whether the sort column's min/max value is within the partition range
    fn check_in_range(
        &self,
        sort_column: &ArrayRef,
        min_max_idx: (usize, usize),
    ) -> datafusion_common::Result<()> {
        if self.cur_part_idx >= self.partition_ranges.len() {
            internal_err!(
                "Partition index out of range: {} >= {}",
                self.cur_part_idx,
                self.partition_ranges.len()
            )?;
        }
        let cur_range = self.partition_ranges[self.cur_part_idx];

        downcast_ts_array!(
            sort_column.data_type() => (array_check_helper, sort_column, cur_range, min_max_idx),
            _ => internal_err!(
                "Unsupported data type for sort column: {:?}",
                sort_column.data_type()
            )?,
        );

        Ok(())
    }

    /// Try find data whose value exceeds the current partition range.
    ///
    /// Returns `None` if no such data is found, and `Some(idx)` where idx points to
    /// the first data that exceeds the current partition range.
    fn try_find_next_range(
        &self,
        sort_column: &ArrayRef,
    ) -> datafusion_common::Result<Option<usize>> {
        if sort_column.len() == 0 {
            return Ok(Some(0));
        }

        // check if the current partition index is out of range
        if self.cur_part_idx >= self.partition_ranges.len() {
            internal_err!(
                "Partition index out of range: {} >= {}",
                self.cur_part_idx,
                self.partition_ranges.len()
            )?;
        }
        let cur_range = self.partition_ranges[self.cur_part_idx];

        let sort_column_iter = downcast_ts_array!(
            sort_column.data_type() => (array_iter_helper, sort_column),
            _ => internal_err!(
                "Unsupported data type for sort column: {:?}",
                sort_column.data_type()
            )?,
        );

        for (idx, val) in sort_column_iter {
            // ignore vacant time index data
            if let Some(val) = val {
                if val >= cur_range.end.value() || val < cur_range.start.value() {
                    return Ok(Some(idx));
                }
            }
        }

        Ok(None)
    }

    fn push_buffer(&mut self, batch: DfRecordBatch) -> datafusion_common::Result<()> {
        match &mut self.buffer {
            PartSortBuffer::All(v) => v.push(batch),
            PartSortBuffer::Top(top, cnt) => {
                *cnt += batch.num_rows();
                top.insert_batch(batch)?;
            }
        }

        Ok(())
    }

    /// Sort and clear the buffer and return the sorted record batch
    ///
    /// this function will return a empty record batch if the buffer is empty
    fn sort_buffer(&mut self) -> datafusion_common::Result<DfRecordBatch> {
        match &mut self.buffer {
            PartSortBuffer::All(_) => self.sort_all_buffer(),
            PartSortBuffer::Top(_, _) => self.sort_top_buffer(),
        }
    }

    /// Internal method for sorting `All` buffer (without limit).
    fn sort_all_buffer(&mut self) -> datafusion_common::Result<DfRecordBatch> {
        let PartSortBuffer::All(buffer) =
            std::mem::replace(&mut self.buffer, PartSortBuffer::All(Vec::new()))
        else {
            unreachable!("buffer type is checked before and should be All variant")
        };

        if buffer.is_empty() {
            return Ok(DfRecordBatch::new_empty(self.schema.clone()));
        }
        let mut sort_columns = Vec::with_capacity(buffer.len());
        let mut opt = None;
        for batch in buffer.iter() {
            let sort_column = self.expression.evaluate_to_sort_column(batch)?;
            opt = opt.or(sort_column.options);
            sort_columns.push(sort_column.values);
        }

        let sort_column =
            concat(&sort_columns.iter().map(|a| a.as_ref()).collect_vec()).map_err(|e| {
                DataFusionError::ArrowError(
                    e,
                    Some(format!("Fail to concat sort columns at {}", location!())),
                )
            })?;

        let indices = sort_to_indices(&sort_column, opt, self.limit).map_err(|e| {
            DataFusionError::ArrowError(
                e,
                Some(format!("Fail to sort to indices at {}", location!())),
            )
        })?;
        if indices.is_empty() {
            return Ok(DfRecordBatch::new_empty(self.schema.clone()));
        }

        self.check_in_range(
            &sort_column,
            (
                indices.value(0) as usize,
                indices.value(indices.len() - 1) as usize,
            ),
        )
        .inspect_err(|_e| {
            #[cfg(debug_assertions)]
            common_telemetry::error!(
                "Fail to check sort column in range at {}, current_idx: {}, num_rows: {}, err: {}",
                self.partition,
                self.cur_part_idx,
                sort_column.len(),
                _e
            );
        })?;

        // reserve memory for the concat input and sorted output
        let total_mem: usize = buffer.iter().map(|r| r.get_array_memory_size()).sum();
        self.reservation.try_grow(total_mem * 2)?;

        let full_input = concat_batches(
            &self.schema,
            &buffer,
            buffer.iter().map(|r| r.num_rows()).sum(),
        )
        .map_err(|e| {
            DataFusionError::ArrowError(
                e,
                Some(format!(
                    "Fail to concat input batches when sorting at {}",
                    location!()
                )),
            )
        })?;

        let sorted = take_record_batch(&full_input, &indices).map_err(|e| {
            DataFusionError::ArrowError(
                e,
                Some(format!(
                    "Fail to take result record batch when sorting at {}",
                    location!()
                )),
            )
        })?;

        self.produced += sorted.num_rows();
        drop(full_input);
        // here remove both buffer and full_input memory
        self.reservation.shrink(2 * total_mem);
        Ok(sorted)
    }

    /// Internal method for sorting `Top` buffer (with limit).
    fn sort_top_buffer(&mut self) -> datafusion_common::Result<DfRecordBatch> {
        let new_top_buffer = TopK::try_new(
            self.partition,
            self.schema().clone(),
            vec![self.expression.clone()],
            self.limit.unwrap(),
            self.context.session_config().batch_size(),
            self.context.runtime_env(),
            &self.root_metrics,
            self.partition,
        )?;
        let PartSortBuffer::Top(top_k, _) =
            std::mem::replace(&mut self.buffer, PartSortBuffer::Top(new_top_buffer, 0))
        else {
            unreachable!("buffer type is checked before and should be Top variant")
        };

        let mut result_stream = top_k.emit()?;
        let mut placeholder_ctx = std::task::Context::from_waker(futures::task::noop_waker_ref());
        let mut results = vec![];
        let mut row_count = 0;
        // according to the current implementation of `TopK`, the result stream will always be ready
        loop {
            match result_stream.poll_next_unpin(&mut placeholder_ctx) {
                Poll::Ready(Some(batch)) => {
                    let batch = batch?;
                    row_count += batch.num_rows();
                    results.push(batch);
                }
                Poll::Pending => {
                    #[cfg(debug_assertions)]
                    unreachable!("TopK result stream should always be ready")
                }
                Poll::Ready(None) => {
                    break;
                }
            }
        }

        let concat_batch = concat_batches(&self.schema, &results, row_count).map_err(|e| {
            DataFusionError::ArrowError(
                e,
                Some(format!(
                    "Fail to concat top k result record batch when sorting at {}",
                    location!()
                )),
            )
        })?;

        Ok(concat_batch)
    }

    /// Try to split the input batch if it contains data that exceeds the current partition range.
    ///
    /// When the input batch contains data that exceeds the current partition range, this function
    /// will split the input batch into two parts, the first part is within the current partition
    /// range will be merged and sorted with previous buffer, and the second part will be registered
    /// to `evaluating_batch` for next polling.
    ///
    /// Returns `None` if the input batch is empty or fully within the current partition range, and
    /// `Some(batch)` otherwise.
    fn split_batch(
        &mut self,
        batch: DfRecordBatch,
    ) -> datafusion_common::Result<Option<DfRecordBatch>> {
        if batch.num_rows() == 0 {
            return Ok(None);
        }

        let sort_column = self
            .expression
            .expr
            .evaluate(&batch)?
            .into_array(batch.num_rows())?;

        let next_range_idx = self.try_find_next_range(&sort_column)?;
        let Some(idx) = next_range_idx else {
            self.push_buffer(batch)?;
            // keep polling input for next batch
            return Ok(None);
        };

        let this_range = batch.slice(0, idx);
        let remaining_range = batch.slice(idx, batch.num_rows() - idx);
        if this_range.num_rows() != 0 {
            self.push_buffer(this_range)?;
        }
        // mark end of current PartitionRange
        let sorted_batch = self.sort_buffer();
        // step to next proper PartitionRange
        self.cur_part_idx += 1;
        let next_sort_column = sort_column.slice(idx, batch.num_rows() - idx);
        if self.try_find_next_range(&next_sort_column)?.is_some() {
            // remaining batch still contains data that exceeds the current partition range
            // register the remaining batch for next polling
            self.evaluating_batch = Some(remaining_range);
        } else {
            // remaining batch is within the current partition range
            // push to the buffer and continue polling
            if remaining_range.num_rows() != 0 {
                self.push_buffer(remaining_range)?;
            }
        }

        sorted_batch.map(|x| if x.num_rows() == 0 { None } else { Some(x) })
    }

    pub fn poll_next_inner(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<Option<datafusion_common::Result<DfRecordBatch>>> {
        loop {
            // no more input, sort the buffer and return
            if self.input_complete {
                if self.buffer.is_empty() {
                    return Poll::Ready(None);
                } else {
                    return Poll::Ready(Some(self.sort_buffer()));
                }
            }

            // if there is a remaining batch being evaluated from last run,
            // split on it instead of fetching new batch
            if let Some(evaluating_batch) = self.evaluating_batch.take()
                && evaluating_batch.num_rows() != 0
            {
                if let Some(sorted_batch) = self.split_batch(evaluating_batch)? {
                    return Poll::Ready(Some(Ok(sorted_batch)));
                } else {
                    continue;
                }
            }

            // fetch next batch from input
            let res = self.input.as_mut().poll_next(cx);
            match res {
                Poll::Ready(Some(Ok(batch))) => {
                    if let Some(sorted_batch) = self.split_batch(batch)? {
                        return Poll::Ready(Some(Ok(sorted_batch)));
                    } else {
                        continue;
                    }
                }
                // input stream end, mark and continue
                Poll::Ready(None) => {
                    self.input_complete = true;
                    continue;
                }
                Poll::Ready(Some(Err(e))) => return Poll::Ready(Some(Err(e))),
                Poll::Pending => return Poll::Pending,
            }
        }
    }
}

impl Stream for PartSortStream {
    type Item = datafusion_common::Result<DfRecordBatch>;

    fn poll_next(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<Option<datafusion_common::Result<DfRecordBatch>>> {
        let result = self.as_mut().poll_next_inner(cx);
        self.metrics.record_poll(result)
    }
}

impl RecordBatchStream for PartSortStream {
    fn schema(&self) -> SchemaRef {
        self.schema.clone()
    }
}

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

    use arrow::json::ArrayWriter;
    use arrow_schema::{DataType, Field, Schema, SortOptions, TimeUnit};
    use common_time::Timestamp;
    use datafusion_physical_expr::expressions::Column;
    use futures::StreamExt;
    use store_api::region_engine::PartitionRange;

    use super::*;
    use crate::test_util::{new_ts_array, MockInputExec};

    #[tokio::test]
    async fn fuzzy_test() {
        let test_cnt = 100;
        // bound for total count of PartitionRange
        let part_cnt_bound = 100;
        // bound for timestamp range size and offset for each PartitionRange
        let range_size_bound = 100;
        let range_offset_bound = 100;
        // bound for batch count and size within each PartitionRange
        let batch_cnt_bound = 20;
        let batch_size_bound = 100;

        let mut rng = fastrand::Rng::new();
        rng.seed(1337);

        let mut test_cases = Vec::new();

        for case_id in 0..test_cnt {
            let mut bound_val: Option<i64> = None;
            let descending = rng.bool();
            let nulls_first = rng.bool();
            let opt = SortOptions {
                descending,
                nulls_first,
            };
            let limit = if rng.bool() {
                Some(rng.usize(0..batch_cnt_bound * batch_size_bound))
            } else {
                None
            };
            let unit = match rng.u8(0..3) {
                0 => TimeUnit::Second,
                1 => TimeUnit::Millisecond,
                2 => TimeUnit::Microsecond,
                _ => TimeUnit::Nanosecond,
            };

            let schema = Schema::new(vec![Field::new(
                "ts",
                DataType::Timestamp(unit.clone(), None),
                false,
            )]);
            let schema = Arc::new(schema);

            let mut input_ranged_data = vec![];
            let mut output_ranges = vec![];
            let mut output_data = vec![];
            // generate each input `PartitionRange`
            for part_id in 0..rng.usize(0..part_cnt_bound) {
                // generate each `PartitionRange`'s timestamp range
                let (start, end) = if descending {
                    let end = bound_val
                        .map(
                            |i| i
                            .checked_sub(rng.i64(0..range_offset_bound))
                            .expect("Bad luck, fuzzy test generate data that will overflow, change seed and try again")
                        )
                        .unwrap_or_else(|| rng.i64(-100000000..100000000));
                    bound_val = Some(end);
                    let start = end - rng.i64(1..range_size_bound);
                    let start = Timestamp::new(start, unit.clone().into());
                    let end = Timestamp::new(end, unit.clone().into());
                    (start, end)
                } else {
                    let start = bound_val
                        .map(|i| i + rng.i64(0..range_offset_bound))
                        .unwrap_or_else(|| rng.i64(..));
                    bound_val = Some(start);
                    let end = start + rng.i64(1..range_size_bound);
                    let start = Timestamp::new(start, unit.clone().into());
                    let end = Timestamp::new(end, unit.clone().into());
                    (start, end)
                };
                assert!(start < end);

                let mut per_part_sort_data = vec![];
                let mut batches = vec![];
                for _batch_idx in 0..rng.usize(1..batch_cnt_bound) {
                    let cnt = rng.usize(0..batch_size_bound) + 1;
                    let iter = 0..rng.usize(0..cnt);
                    let mut data_gen = iter
                        .map(|_| rng.i64(start.value()..end.value()))
                        .collect_vec();
                    if data_gen.is_empty() {
                        // current batch is empty, skip
                        continue;
                    }
                    // mito always sort on ASC order
                    data_gen.sort();
                    per_part_sort_data.extend(data_gen.clone());
                    let arr = new_ts_array(unit.clone(), data_gen.clone());
                    let batch = DfRecordBatch::try_new(schema.clone(), vec![arr]).unwrap();
                    batches.push(batch);
                }

                let range = PartitionRange {
                    start,
                    end,
                    num_rows: batches.iter().map(|b| b.num_rows()).sum(),
                    identifier: part_id,
                };
                input_ranged_data.push((range, batches));

                output_ranges.push(range);
                if per_part_sort_data.is_empty() {
                    continue;
                }
                output_data.extend_from_slice(&per_part_sort_data);
            }

            // adjust output data with adjacent PartitionRanges
            let mut output_data_iter = output_data.iter().peekable();
            let mut output_data = vec![];
            for range in output_ranges.clone() {
                let mut cur_data = vec![];
                while let Some(val) = output_data_iter.peek() {
                    if **val < range.start.value() || **val >= range.end.value() {
                        break;
                    }
                    cur_data.push(*output_data_iter.next().unwrap());
                }

                if cur_data.is_empty() {
                    continue;
                }

                if descending {
                    cur_data.sort_by(|a, b| b.cmp(a));
                } else {
                    cur_data.sort();
                }
                output_data.push(cur_data);
            }

            let expected_output = output_data
                .into_iter()
                .map(|a| {
                    DfRecordBatch::try_new(schema.clone(), vec![new_ts_array(unit.clone(), a)])
                        .unwrap()
                })
                .map(|rb| {
                    // trim expected output with limit
                    if let Some(limit) = limit
                        && rb.num_rows() > limit
                    {
                        rb.slice(0, limit)
                    } else {
                        rb
                    }
                })
                .collect_vec();

            test_cases.push((
                case_id,
                unit,
                input_ranged_data,
                schema,
                opt,
                limit,
                expected_output,
            ));
        }

        for (case_id, _unit, input_ranged_data, schema, opt, limit, expected_output) in test_cases {
            run_test(
                case_id,
                input_ranged_data,
                schema,
                opt,
                limit,
                expected_output,
            )
            .await;
        }
    }

    #[tokio::test]
    async fn simple_case() {
        let testcases = vec![
            (
                TimeUnit::Millisecond,
                vec![
                    ((0, 10), vec![vec![1, 2, 3], vec![4, 5, 6], vec![7, 8, 9]]),
                    ((5, 10), vec![vec![5, 6], vec![7, 8]]),
                ],
                false,
                None,
                vec![vec![1, 2, 3, 4, 5, 5, 6, 6, 7, 7, 8, 8, 9]],
            ),
            (
                TimeUnit::Millisecond,
                vec![
                    ((5, 10), vec![vec![5, 6], vec![7, 8, 9]]),
                    ((0, 10), vec![vec![1, 2, 3], vec![4, 5, 6], vec![7, 8]]),
                ],
                true,
                None,
                vec![vec![9, 8, 7, 6, 5], vec![8, 7, 6, 5, 4, 3, 2, 1]],
            ),
            (
                TimeUnit::Millisecond,
                vec![
                    ((5, 10), vec![]),
                    ((0, 10), vec![vec![1, 2, 3], vec![4, 5, 6], vec![7, 8]]),
                ],
                true,
                None,
                vec![vec![8, 7, 6, 5, 4, 3, 2, 1]],
            ),
            (
                TimeUnit::Millisecond,
                vec![
                    ((15, 20), vec![vec![17, 18, 19]]),
                    ((10, 15), vec![]),
                    ((5, 10), vec![]),
                    ((0, 10), vec![vec![1, 2, 3], vec![4, 5, 6], vec![7, 8]]),
                ],
                true,
                None,
                vec![vec![19, 18, 17], vec![8, 7, 6, 5, 4, 3, 2, 1]],
            ),
            (
                TimeUnit::Millisecond,
                vec![
                    ((15, 20), vec![]),
                    ((10, 15), vec![]),
                    ((5, 10), vec![]),
                    ((0, 10), vec![]),
                ],
                true,
                None,
                vec![],
            ),
            (
                TimeUnit::Millisecond,
                vec![
                    (
                        (15, 20),
                        vec![vec![15, 17, 19, 10, 11, 12, 5, 6, 7, 8, 9, 1, 2, 3, 4]],
                    ),
                    ((10, 15), vec![]),
                    ((5, 10), vec![]),
                    ((0, 10), vec![]),
                ],
                true,
                None,
                vec![
                    vec![19, 17, 15],
                    vec![12, 11, 10],
                    vec![9, 8, 7, 6, 5],
                    vec![4, 3, 2, 1],
                ],
            ),
            (
                TimeUnit::Millisecond,
                vec![
                    (
                        (15, 20),
                        vec![vec![15, 17, 19, 10, 11, 12, 5, 6, 7, 8, 9, 1, 2, 3, 4]],
                    ),
                    ((10, 15), vec![]),
                    ((5, 10), vec![]),
                    ((0, 10), vec![]),
                ],
                true,
                Some(2),
                vec![vec![19, 17], vec![12, 11], vec![9, 8], vec![4, 3]],
            ),
        ];

        for (identifier, (unit, input_ranged_data, descending, limit, expected_output)) in
            testcases.into_iter().enumerate()
        {
            let schema = Schema::new(vec![Field::new(
                "ts",
                DataType::Timestamp(unit.clone(), None),
                false,
            )]);
            let schema = Arc::new(schema);
            let opt = SortOptions {
                descending,
                ..Default::default()
            };

            let input_ranged_data = input_ranged_data
                .into_iter()
                .map(|(range, data)| {
                    let part = PartitionRange {
                        start: Timestamp::new(range.0, unit.clone().into()),
                        end: Timestamp::new(range.1, unit.clone().into()),
                        num_rows: data.iter().map(|b| b.len()).sum(),
                        identifier,
                    };

                    let batches = data
                        .into_iter()
                        .map(|b| {
                            let arr = new_ts_array(unit.clone(), b);
                            DfRecordBatch::try_new(schema.clone(), vec![arr]).unwrap()
                        })
                        .collect_vec();
                    (part, batches)
                })
                .collect_vec();

            let expected_output = expected_output
                .into_iter()
                .map(|a| {
                    DfRecordBatch::try_new(schema.clone(), vec![new_ts_array(unit.clone(), a)])
                        .unwrap()
                })
                .collect_vec();

            run_test(
                identifier,
                input_ranged_data,
                schema.clone(),
                opt,
                limit,
                expected_output,
            )
            .await;
        }
    }

    #[allow(clippy::print_stdout)]
    async fn run_test(
        case_id: usize,
        input_ranged_data: Vec<(PartitionRange, Vec<DfRecordBatch>)>,
        schema: SchemaRef,
        opt: SortOptions,
        limit: Option<usize>,
        expected_output: Vec<DfRecordBatch>,
    ) {
        for rb in &expected_output {
            if let Some(limit) = limit {
                assert!(
                    rb.num_rows() <= limit,
                    "Expect row count in expected output's batch({}) <= limit({})",
                    rb.num_rows(),
                    limit
                );
            }
        }
        let (ranges, batches): (Vec<_>, Vec<_>) = input_ranged_data.clone().into_iter().unzip();

        let batches = batches
            .into_iter()
            .flat_map(|mut cols| {
                cols.push(DfRecordBatch::new_empty(schema.clone()));
                cols
            })
            .collect_vec();
        let mock_input = MockInputExec::new(batches, schema.clone());

        let exec = PartSortExec::new(
            PhysicalSortExpr {
                expr: Arc::new(Column::new("ts", 0)),
                options: opt,
            },
            limit,
            vec![ranges.clone()],
            Arc::new(mock_input),
        );

        let exec_stream = exec.execute(0, Arc::new(TaskContext::default())).unwrap();

        let real_output = exec_stream.map(|r| r.unwrap()).collect::<Vec<_>>().await;
        // a makeshift solution for compare large data
        if real_output != expected_output {
            let mut first_diff = 0;
            for (idx, (lhs, rhs)) in real_output.iter().zip(expected_output.iter()).enumerate() {
                if lhs != rhs {
                    first_diff = idx;
                    break;
                }
            }
            println!("first diff batch at {}", first_diff);
            println!(
                "ranges: {:?}",
                ranges
                    .into_iter()
                    .map(|r| (r.start.to_chrono_datetime(), r.end.to_chrono_datetime()))
                    .enumerate()
                    .collect::<Vec<_>>()
            );

            let mut full_msg = String::new();
            {
                let mut buf = Vec::with_capacity(10 * real_output.len());
                for batch in real_output.iter().skip(first_diff) {
                    let mut rb_json: Vec<u8> = Vec::new();
                    let mut writer = ArrayWriter::new(&mut rb_json);
                    writer.write(batch).unwrap();
                    writer.finish().unwrap();
                    buf.append(&mut rb_json);
                    buf.push(b',');
                }
                // TODO(discord9): better ways to print buf
                let buf = String::from_utf8_lossy(&buf);
                full_msg += &format!("\ncase_id:{case_id}, real_output \n{buf}\n");
            }
            {
                let mut buf = Vec::with_capacity(10 * real_output.len());
                for batch in expected_output.iter().skip(first_diff) {
                    let mut rb_json: Vec<u8> = Vec::new();
                    let mut writer = ArrayWriter::new(&mut rb_json);
                    writer.write(batch).unwrap();
                    writer.finish().unwrap();
                    buf.append(&mut rb_json);
                    buf.push(b',');
                }
                let buf = String::from_utf8_lossy(&buf);
                full_msg += &format!("case_id:{case_id}, expected_output \n{buf}");
            }
            panic!(
                "case_{} failed, opt: {:?},\n real output has {} batches, {} rows, expected has {} batches with {} rows\nfull msg: {}",
                case_id, opt,
                real_output.len(),
                real_output.iter().map(|x|x.num_rows()).sum::<usize>(),
                expected_output.len(),
                expected_output.iter().map(|x|x.num_rows()).sum::<usize>(), full_msg
            );
        }
    }
}