Skip to main content

mito2/
read.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15//! Common structs and utilities for reading data.
16
17pub mod batch_adapter;
18pub mod compat;
19pub mod dedup;
20pub mod flat_dedup;
21pub mod flat_merge;
22pub mod flat_projection;
23pub mod last_row;
24pub mod projection;
25pub(crate) mod prune;
26pub(crate) mod pruner;
27pub mod range;
28#[cfg(feature = "test")]
29pub mod range_cache;
30#[cfg(not(feature = "test"))]
31pub(crate) mod range_cache;
32pub(crate) mod read_columns;
33pub mod scan_region;
34pub mod scan_util;
35pub(crate) mod seq_scan;
36pub mod series_scan;
37pub mod stream;
38pub(crate) mod unordered_scan;
39
40use std::collections::HashMap;
41use std::sync::Arc;
42use std::time::Duration;
43
44use api::v1::OpType;
45use arrow_schema::SchemaRef;
46use async_trait::async_trait;
47use common_time::Timestamp;
48use datafusion_common::arrow::array::UInt8Array;
49use datatypes::arrow;
50use datatypes::arrow::array::{Array, ArrayRef};
51use datatypes::arrow::compute::SortOptions;
52use datatypes::arrow::record_batch::RecordBatch;
53use datatypes::arrow::row::{RowConverter, SortField};
54use datatypes::prelude::{ConcreteDataType, DataType, ScalarVector};
55use datatypes::scalars::ScalarVectorBuilder;
56use datatypes::types::TimestampType;
57use datatypes::value::{Value, ValueRef};
58use datatypes::vectors::{
59    BooleanVector, Helper, TimestampMicrosecondVector, TimestampMillisecondVector,
60    TimestampMillisecondVectorBuilder, TimestampNanosecondVector, TimestampSecondVector,
61    UInt8Vector, UInt8VectorBuilder, UInt32Vector, UInt64Vector, UInt64VectorBuilder, Vector,
62    VectorRef,
63};
64use futures::TryStreamExt;
65use futures::stream::BoxStream;
66use mito_codec::row_converter::{CompositeValues, PrimaryKeyCodec};
67use snafu::{OptionExt, ResultExt, ensure};
68use store_api::storage::{ColumnId, SequenceNumber, SequenceRange};
69
70use crate::error::{
71    ComputeArrowSnafu, ComputeVectorSnafu, ConvertVectorSnafu, DecodeSnafu, InvalidBatchSnafu,
72    Result,
73};
74use crate::memtable::{BoxedBatchIterator, BoxedRecordBatchIterator};
75/// Storage internal representation of a batch of rows for a primary key (time series).
76///
77/// Rows are sorted by primary key, timestamp, sequence desc, op_type desc. Fields
78/// always keep the same relative order as fields in [RegionMetadata](store_api::metadata::RegionMetadata).
79#[derive(Debug, PartialEq, Clone)]
80pub struct Batch {
81    /// Primary key encoded in a comparable form.
82    primary_key: Vec<u8>,
83    /// Possibly decoded `primary_key` values. Some places would decode it in advance.
84    pk_values: Option<CompositeValues>,
85    /// Timestamps of rows, should be sorted and not null.
86    timestamps: VectorRef,
87    /// Sequences of rows
88    ///
89    /// UInt64 type, not null.
90    sequences: Arc<UInt64Vector>,
91    /// Op types of rows
92    ///
93    /// UInt8 type, not null.
94    op_types: Arc<UInt8Vector>,
95    /// Fields organized in columnar format.
96    fields: Vec<BatchColumn>,
97    /// Cache for field index lookup.
98    fields_idx: Option<HashMap<ColumnId, usize>>,
99}
100
101impl Batch {
102    /// Creates a new batch.
103    pub fn new(
104        primary_key: Vec<u8>,
105        timestamps: VectorRef,
106        sequences: Arc<UInt64Vector>,
107        op_types: Arc<UInt8Vector>,
108        fields: Vec<BatchColumn>,
109    ) -> Result<Batch> {
110        BatchBuilder::with_required_columns(primary_key, timestamps, sequences, op_types)
111            .with_fields(fields)
112            .build()
113    }
114
115    /// Tries to set fields for the batch.
116    pub fn with_fields(self, fields: Vec<BatchColumn>) -> Result<Batch> {
117        Batch::new(
118            self.primary_key,
119            self.timestamps,
120            self.sequences,
121            self.op_types,
122            fields,
123        )
124    }
125
126    /// Returns primary key of the batch.
127    pub fn primary_key(&self) -> &[u8] {
128        &self.primary_key
129    }
130
131    /// Returns possibly decoded primary-key values.
132    pub fn pk_values(&self) -> Option<&CompositeValues> {
133        self.pk_values.as_ref()
134    }
135
136    /// Sets possibly decoded primary-key values.
137    pub fn set_pk_values(&mut self, pk_values: CompositeValues) {
138        self.pk_values = Some(pk_values);
139    }
140
141    /// Removes possibly decoded primary-key values. For testing only.
142    #[cfg(any(test, feature = "test"))]
143    pub fn remove_pk_values(&mut self) {
144        self.pk_values = None;
145    }
146
147    /// Returns fields in the batch.
148    pub fn fields(&self) -> &[BatchColumn] {
149        &self.fields
150    }
151
152    /// Returns timestamps of the batch.
153    pub fn timestamps(&self) -> &VectorRef {
154        &self.timestamps
155    }
156
157    /// Returns sequences of the batch.
158    pub fn sequences(&self) -> &Arc<UInt64Vector> {
159        &self.sequences
160    }
161
162    /// Returns op types of the batch.
163    pub fn op_types(&self) -> &Arc<UInt8Vector> {
164        &self.op_types
165    }
166
167    /// Returns the number of rows in the batch.
168    pub fn num_rows(&self) -> usize {
169        // All vectors have the same length. We use the length of sequences vector
170        // since it has static type.
171        self.sequences.len()
172    }
173
174    /// Create an empty [`Batch`].
175    #[allow(dead_code)]
176    pub(crate) fn empty() -> Self {
177        Self {
178            primary_key: vec![],
179            pk_values: None,
180            timestamps: Arc::new(TimestampMillisecondVectorBuilder::with_capacity(0).finish()),
181            sequences: Arc::new(UInt64VectorBuilder::with_capacity(0).finish()),
182            op_types: Arc::new(UInt8VectorBuilder::with_capacity(0).finish()),
183            fields: vec![],
184            fields_idx: None,
185        }
186    }
187
188    /// Returns true if the number of rows in the batch is 0.
189    pub fn is_empty(&self) -> bool {
190        self.num_rows() == 0
191    }
192
193    /// Returns the first timestamp in the batch or `None` if the batch is empty.
194    pub fn first_timestamp(&self) -> Option<Timestamp> {
195        if self.timestamps.is_empty() {
196            return None;
197        }
198
199        Some(self.get_timestamp(0))
200    }
201
202    /// Returns the last timestamp in the batch or `None` if the batch is empty.
203    pub fn last_timestamp(&self) -> Option<Timestamp> {
204        if self.timestamps.is_empty() {
205            return None;
206        }
207
208        Some(self.get_timestamp(self.timestamps.len() - 1))
209    }
210
211    /// Returns the first sequence in the batch or `None` if the batch is empty.
212    pub fn first_sequence(&self) -> Option<SequenceNumber> {
213        if self.sequences.is_empty() {
214            return None;
215        }
216
217        Some(self.get_sequence(0))
218    }
219
220    /// Returns the last sequence in the batch or `None` if the batch is empty.
221    pub fn last_sequence(&self) -> Option<SequenceNumber> {
222        if self.sequences.is_empty() {
223            return None;
224        }
225
226        Some(self.get_sequence(self.sequences.len() - 1))
227    }
228
229    /// Replaces the primary key of the batch.
230    ///
231    /// Notice that this [Batch] also contains a maybe-exist `pk_values`.
232    /// Be sure to update that field as well.
233    pub fn set_primary_key(&mut self, primary_key: Vec<u8>) {
234        self.primary_key = primary_key;
235    }
236
237    /// Slice the batch, returning a new batch.
238    ///
239    /// # Panics
240    /// Panics if `offset + length > self.num_rows()`.
241    pub fn slice(&self, offset: usize, length: usize) -> Batch {
242        let fields = self
243            .fields
244            .iter()
245            .map(|column| BatchColumn {
246                column_id: column.column_id,
247                data: column.data.slice(offset, length),
248            })
249            .collect();
250        // We skip using the builder to avoid validating the batch again.
251        Batch {
252            // Now we need to clone the primary key. We could try `Bytes` if
253            // this becomes a bottleneck.
254            primary_key: self.primary_key.clone(),
255            pk_values: self.pk_values.clone(),
256            timestamps: self.timestamps.slice(offset, length),
257            sequences: Arc::new(self.sequences.get_slice(offset, length)),
258            op_types: Arc::new(self.op_types.get_slice(offset, length)),
259            fields,
260            fields_idx: self.fields_idx.clone(),
261        }
262    }
263
264    /// Takes `batches` and concat them into one batch.
265    ///
266    /// All `batches` must have the same primary key.
267    pub fn concat(mut batches: Vec<Batch>) -> Result<Batch> {
268        ensure!(
269            !batches.is_empty(),
270            InvalidBatchSnafu {
271                reason: "empty batches",
272            }
273        );
274        if batches.len() == 1 {
275            // Now we own the `batches` so we could pop it directly.
276            return Ok(batches.pop().unwrap());
277        }
278
279        let primary_key = std::mem::take(&mut batches[0].primary_key);
280        let first = &batches[0];
281        // We took the primary key from the first batch so we don't use `first.primary_key()`.
282        ensure!(
283            batches
284                .iter()
285                .skip(1)
286                .all(|b| b.primary_key() == primary_key),
287            InvalidBatchSnafu {
288                reason: "batches have different primary key",
289            }
290        );
291        for b in batches.iter().skip(1) {
292            ensure!(
293                b.fields.len() == first.fields.len(),
294                InvalidBatchSnafu {
295                    reason: "batches have different field num",
296                }
297            );
298            for (l, r) in b.fields.iter().zip(&first.fields) {
299                ensure!(
300                    l.column_id == r.column_id,
301                    InvalidBatchSnafu {
302                        reason: "batches have different fields",
303                    }
304                );
305            }
306        }
307
308        // We take the primary key from the first batch.
309        let mut builder = BatchBuilder::new(primary_key);
310        // Concat timestamps, sequences, op_types, fields.
311        let array = concat_arrays(batches.iter().map(|b| b.timestamps().to_arrow_array()))?;
312        builder.timestamps_array(array)?;
313        let array = concat_arrays(batches.iter().map(|b| b.sequences().to_arrow_array()))?;
314        builder.sequences_array(array)?;
315        let array = concat_arrays(batches.iter().map(|b| b.op_types().to_arrow_array()))?;
316        builder.op_types_array(array)?;
317        for (i, batch_column) in first.fields.iter().enumerate() {
318            let array = concat_arrays(batches.iter().map(|b| b.fields()[i].data.to_arrow_array()))?;
319            builder.push_field_array(batch_column.column_id, array)?;
320        }
321
322        builder.build()
323    }
324
325    /// Removes rows whose op type is delete.
326    pub fn filter_deleted(&mut self) -> Result<()> {
327        // Safety: op type column is not null.
328        let array = self.op_types.as_arrow();
329        // Find rows with non-delete op type.
330        let rhs = UInt8Array::new_scalar(OpType::Delete as u8);
331        let predicate =
332            arrow::compute::kernels::cmp::neq(array, &rhs).context(ComputeArrowSnafu)?;
333        self.filter(&BooleanVector::from(predicate))
334    }
335
336    // Applies the `predicate` to the batch.
337    // Safety: We know the array type so we unwrap on casting.
338    pub fn filter(&mut self, predicate: &BooleanVector) -> Result<()> {
339        self.timestamps = self
340            .timestamps
341            .filter(predicate)
342            .context(ComputeVectorSnafu)?;
343        self.sequences = Arc::new(
344            UInt64Vector::try_from_arrow_array(
345                arrow::compute::filter(self.sequences.as_arrow(), predicate.as_boolean_array())
346                    .context(ComputeArrowSnafu)?,
347            )
348            .unwrap(),
349        );
350        self.op_types = Arc::new(
351            UInt8Vector::try_from_arrow_array(
352                arrow::compute::filter(self.op_types.as_arrow(), predicate.as_boolean_array())
353                    .context(ComputeArrowSnafu)?,
354            )
355            .unwrap(),
356        );
357        for batch_column in &mut self.fields {
358            batch_column.data = batch_column
359                .data
360                .filter(predicate)
361                .context(ComputeVectorSnafu)?;
362        }
363
364        Ok(())
365    }
366
367    /// Filters rows by the given `sequence`. Only preserves rows with sequence less than or equal to `sequence`.
368    pub fn filter_by_sequence(&mut self, sequence: Option<SequenceRange>) -> Result<()> {
369        let seq_range = match sequence {
370            None => return Ok(()),
371            Some(seq_range) => {
372                let (Some(first), Some(last)) = (self.first_sequence(), self.last_sequence())
373                else {
374                    return Ok(());
375                };
376                let is_subset = match seq_range {
377                    SequenceRange::Gt { min } => min < first,
378                    SequenceRange::LtEq { max } => max >= last,
379                    SequenceRange::GtLtEq { min, max } => min < first && max >= last,
380                };
381                if is_subset {
382                    return Ok(());
383                }
384                seq_range
385            }
386        };
387
388        let seqs = self.sequences.as_arrow();
389        let predicate = seq_range.filter(seqs).context(ComputeArrowSnafu)?;
390
391        let predicate = BooleanVector::from(predicate);
392        self.filter(&predicate)?;
393
394        Ok(())
395    }
396
397    /// Sorts rows in the batch. If `dedup` is true, it also removes
398    /// duplicated rows according to primary keys.
399    ///
400    /// It orders rows by timestamp, sequence desc and only keep the latest
401    /// row for the same timestamp. It doesn't consider op type as sequence
402    /// should already provide uniqueness for a row.
403    pub fn sort(&mut self, dedup: bool) -> Result<()> {
404        // If building a converter each time is costly, we may allow passing a
405        // converter.
406        let converter = RowConverter::new(vec![
407            SortField::new(self.timestamps.data_type().as_arrow_type()),
408            SortField::new_with_options(
409                self.sequences.data_type().as_arrow_type(),
410                SortOptions {
411                    descending: true,
412                    ..Default::default()
413                },
414            ),
415        ])
416        .context(ComputeArrowSnafu)?;
417        // Columns to sort.
418        let columns = [
419            self.timestamps.to_arrow_array(),
420            self.sequences.to_arrow_array(),
421        ];
422        let rows = converter.convert_columns(&columns).unwrap();
423        let mut to_sort: Vec<_> = rows.iter().enumerate().collect();
424
425        let was_sorted = to_sort.is_sorted_by_key(|x| x.1);
426        if !was_sorted {
427            to_sort.sort_unstable_by_key(|x| x.1);
428        }
429
430        let num_rows = to_sort.len();
431        if dedup {
432            // Dedup by timestamps.
433            to_sort.dedup_by(|left, right| {
434                debug_assert_eq!(18, left.1.as_ref().len());
435                debug_assert_eq!(18, right.1.as_ref().len());
436                let (left_key, right_key) = (left.1.as_ref(), right.1.as_ref());
437                // We only compare the timestamp part and ignore sequence.
438                left_key[..TIMESTAMP_KEY_LEN] == right_key[..TIMESTAMP_KEY_LEN]
439            });
440        }
441        let no_dedup = to_sort.len() == num_rows;
442
443        if was_sorted && no_dedup {
444            return Ok(());
445        }
446        let indices = UInt32Vector::from_iter_values(to_sort.iter().map(|v| v.0 as u32));
447        self.take_in_place(&indices)
448    }
449
450    /// Merges duplicated timestamps in the batch by keeping the latest non-null field values.
451    ///
452    /// Rows must already be sorted by timestamp (ascending) and sequence (descending).
453    ///
454    /// This method deduplicates rows with the same timestamp (keeping the first row in each
455    /// timestamp range as the base row) and fills null fields from subsequent rows until all
456    /// fields are filled or a delete operation is encountered.
457    pub(crate) fn merge_last_non_null(&mut self) -> Result<()> {
458        let num_rows = self.num_rows();
459        if num_rows < 2 {
460            return Ok(());
461        }
462
463        let Some(timestamps) = self.timestamps_native() else {
464            return Ok(());
465        };
466
467        // Fast path: check if there are any duplicate timestamps.
468        let mut has_dup = false;
469        let mut group_count = 1;
470        for i in 1..num_rows {
471            has_dup |= timestamps[i] == timestamps[i - 1];
472            group_count += (timestamps[i] != timestamps[i - 1]) as usize;
473        }
474        if !has_dup {
475            return Ok(());
476        }
477
478        let num_fields = self.fields.len();
479        let op_types = self.op_types.as_arrow().values();
480
481        let mut base_indices: Vec<u32> = Vec::with_capacity(group_count);
482        let mut field_indices: Vec<Vec<u32>> = (0..num_fields)
483            .map(|_| Vec::with_capacity(group_count))
484            .collect();
485
486        let mut start = 0;
487        while start < num_rows {
488            let ts = timestamps[start];
489            let mut end = start + 1;
490            while end < num_rows && timestamps[end] == ts {
491                end += 1;
492            }
493
494            let group_pos = base_indices.len();
495            base_indices.push(start as u32);
496
497            if num_fields > 0 {
498                // Default: take the base row for all fields.
499                for idx in &mut field_indices {
500                    idx.push(start as u32);
501                }
502
503                let base_deleted = op_types[start] == OpType::Delete as u8;
504                if !base_deleted {
505                    // Track fields that are null in the base row and try to fill them from older
506                    // rows in the same timestamp range.
507                    let mut missing_fields = Vec::new();
508                    for (field_idx, col) in self.fields.iter().enumerate() {
509                        if col.data.is_null(start) {
510                            missing_fields.push(field_idx);
511                        }
512                    }
513
514                    if !missing_fields.is_empty() {
515                        for row_idx in (start + 1)..end {
516                            if op_types[row_idx] == OpType::Delete as u8 {
517                                break;
518                            }
519
520                            missing_fields.retain(|&field_idx| {
521                                if self.fields[field_idx].data.is_null(row_idx) {
522                                    true
523                                } else {
524                                    field_indices[field_idx][group_pos] = row_idx as u32;
525                                    false
526                                }
527                            });
528
529                            if missing_fields.is_empty() {
530                                break;
531                            }
532                        }
533                    }
534                }
535            }
536
537            start = end;
538        }
539
540        let base_indices = UInt32Vector::from_vec(base_indices);
541        self.timestamps = self
542            .timestamps
543            .take(&base_indices)
544            .context(ComputeVectorSnafu)?;
545        let array = arrow::compute::take(self.sequences.as_arrow(), base_indices.as_arrow(), None)
546            .context(ComputeArrowSnafu)?;
547        // Safety: We know the array and vector type.
548        self.sequences = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap());
549        let array = arrow::compute::take(self.op_types.as_arrow(), base_indices.as_arrow(), None)
550            .context(ComputeArrowSnafu)?;
551        // Safety: We know the array and vector type.
552        self.op_types = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap());
553
554        for (field_idx, batch_column) in self.fields.iter_mut().enumerate() {
555            let idx = UInt32Vector::from_vec(std::mem::take(&mut field_indices[field_idx]));
556            batch_column.data = batch_column.data.take(&idx).context(ComputeVectorSnafu)?;
557        }
558
559        Ok(())
560    }
561
562    /// Returns the estimated memory size of the batch.
563    pub fn memory_size(&self) -> usize {
564        let mut size = std::mem::size_of::<Self>();
565        size += self.primary_key.len();
566        size += self.timestamps.memory_size();
567        size += self.sequences.memory_size();
568        size += self.op_types.memory_size();
569        for batch_column in &self.fields {
570            size += batch_column.data.memory_size();
571        }
572        size
573    }
574
575    /// Returns timestamps in a native slice or `None` if the batch is empty.
576    pub(crate) fn timestamps_native(&self) -> Option<&[i64]> {
577        if self.timestamps.is_empty() {
578            return None;
579        }
580
581        let values = match self.timestamps.data_type() {
582            ConcreteDataType::Timestamp(TimestampType::Second(_)) => self
583                .timestamps
584                .as_any()
585                .downcast_ref::<TimestampSecondVector>()
586                .unwrap()
587                .as_arrow()
588                .values(),
589            ConcreteDataType::Timestamp(TimestampType::Millisecond(_)) => self
590                .timestamps
591                .as_any()
592                .downcast_ref::<TimestampMillisecondVector>()
593                .unwrap()
594                .as_arrow()
595                .values(),
596            ConcreteDataType::Timestamp(TimestampType::Microsecond(_)) => self
597                .timestamps
598                .as_any()
599                .downcast_ref::<TimestampMicrosecondVector>()
600                .unwrap()
601                .as_arrow()
602                .values(),
603            ConcreteDataType::Timestamp(TimestampType::Nanosecond(_)) => self
604                .timestamps
605                .as_any()
606                .downcast_ref::<TimestampNanosecondVector>()
607                .unwrap()
608                .as_arrow()
609                .values(),
610            other => panic!("timestamps in a Batch has other type {:?}", other),
611        };
612
613        Some(values)
614    }
615
616    /// Takes the batch in place.
617    fn take_in_place(&mut self, indices: &UInt32Vector) -> Result<()> {
618        self.timestamps = self.timestamps.take(indices).context(ComputeVectorSnafu)?;
619        let array = arrow::compute::take(self.sequences.as_arrow(), indices.as_arrow(), None)
620            .context(ComputeArrowSnafu)?;
621        // Safety: we know the array and vector type.
622        self.sequences = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap());
623        let array = arrow::compute::take(self.op_types.as_arrow(), indices.as_arrow(), None)
624            .context(ComputeArrowSnafu)?;
625        self.op_types = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap());
626        for batch_column in &mut self.fields {
627            batch_column.data = batch_column
628                .data
629                .take(indices)
630                .context(ComputeVectorSnafu)?;
631        }
632
633        Ok(())
634    }
635
636    /// Gets a timestamp at given `index`.
637    ///
638    /// # Panics
639    /// Panics if `index` is out-of-bound or the timestamp vector returns null.
640    fn get_timestamp(&self, index: usize) -> Timestamp {
641        match self.timestamps.get_ref(index) {
642            ValueRef::Timestamp(timestamp) => timestamp,
643
644            // We have check the data type is timestamp compatible in the [BatchBuilder] so it's safe to panic.
645            value => panic!("{:?} is not a timestamp", value),
646        }
647    }
648
649    /// Gets a sequence at given `index`.
650    ///
651    /// # Panics
652    /// Panics if `index` is out-of-bound or the sequence vector returns null.
653    pub(crate) fn get_sequence(&self, index: usize) -> SequenceNumber {
654        // Safety: sequences is not null so it actually returns Some.
655        self.sequences.get_data(index).unwrap()
656    }
657
658    /// Checks the batch is monotonic by timestamps.
659    #[cfg(debug_assertions)]
660    #[allow(dead_code)]
661    pub(crate) fn check_monotonic(&self) -> Result<(), String> {
662        use std::cmp::Ordering;
663        if self.timestamps_native().is_none() {
664            return Ok(());
665        }
666
667        let timestamps = self.timestamps_native().unwrap();
668        let sequences = self.sequences.as_arrow().values();
669        for (i, window) in timestamps.windows(2).enumerate() {
670            let current = window[0];
671            let next = window[1];
672            let current_sequence = sequences[i];
673            let next_sequence = sequences[i + 1];
674            match current.cmp(&next) {
675                Ordering::Less => {
676                    // The current timestamp is less than the next timestamp.
677                    continue;
678                }
679                Ordering::Equal => {
680                    // The current timestamp is equal to the next timestamp.
681                    if current_sequence < next_sequence {
682                        return Err(format!(
683                            "sequence are not monotonic: ts {} == {} but current sequence {} < {}, index: {}",
684                            current, next, current_sequence, next_sequence, i
685                        ));
686                    }
687                }
688                Ordering::Greater => {
689                    // The current timestamp is greater than the next timestamp.
690                    return Err(format!(
691                        "timestamps are not monotonic: {} > {}, index: {}",
692                        current, next, i
693                    ));
694                }
695            }
696        }
697
698        Ok(())
699    }
700
701    /// Returns Ok if the given batch is behind the current batch.
702    #[cfg(debug_assertions)]
703    #[allow(dead_code)]
704    pub(crate) fn check_next_batch(&self, other: &Batch) -> Result<(), String> {
705        // Checks the primary key
706        if self.primary_key() < other.primary_key() {
707            return Ok(());
708        }
709        if self.primary_key() > other.primary_key() {
710            return Err(format!(
711                "primary key is not monotonic: {:?} > {:?}",
712                self.primary_key(),
713                other.primary_key()
714            ));
715        }
716        // Checks the timestamp.
717        if self.last_timestamp() < other.first_timestamp() {
718            return Ok(());
719        }
720        if self.last_timestamp() > other.first_timestamp() {
721            return Err(format!(
722                "timestamps are not monotonic: {:?} > {:?}",
723                self.last_timestamp(),
724                other.first_timestamp()
725            ));
726        }
727        // Checks the sequence.
728        if self.last_sequence() >= other.first_sequence() {
729            return Ok(());
730        }
731        Err(format!(
732            "sequences are not monotonic: {:?} < {:?}",
733            self.last_sequence(),
734            other.first_sequence()
735        ))
736    }
737
738    /// Returns the value of the column in the primary key.
739    ///
740    /// Lazily decodes the primary key and caches the result.
741    pub fn pk_col_value(
742        &mut self,
743        codec: &dyn PrimaryKeyCodec,
744        col_idx_in_pk: usize,
745        column_id: ColumnId,
746    ) -> Result<Option<&Value>> {
747        if self.pk_values.is_none() {
748            self.pk_values = Some(codec.decode(&self.primary_key).context(DecodeSnafu)?);
749        }
750
751        let pk_values = self.pk_values.as_ref().unwrap();
752        Ok(match pk_values {
753            CompositeValues::Dense(values) => values.get(col_idx_in_pk).map(|(_, v)| v),
754            CompositeValues::Sparse(values) => values.get(&column_id),
755        })
756    }
757
758    /// Returns values of the field in the batch.
759    ///
760    /// Lazily caches the field index.
761    pub fn field_col_value(&mut self, column_id: ColumnId) -> Option<&BatchColumn> {
762        if self.fields_idx.is_none() {
763            self.fields_idx = Some(
764                self.fields
765                    .iter()
766                    .enumerate()
767                    .map(|(i, c)| (c.column_id, i))
768                    .collect(),
769            );
770        }
771
772        self.fields_idx
773            .as_ref()
774            .unwrap()
775            .get(&column_id)
776            .map(|&idx| &self.fields[idx])
777    }
778}
779
780/// A struct to check the batch is monotonic.
781#[cfg(debug_assertions)]
782#[derive(Default)]
783#[allow(dead_code)]
784pub(crate) struct BatchChecker {
785    last_batch: Option<Batch>,
786    start: Option<Timestamp>,
787    end: Option<Timestamp>,
788}
789
790#[cfg(debug_assertions)]
791#[allow(dead_code)]
792impl BatchChecker {
793    /// Attaches the given start timestamp to the checker.
794    pub(crate) fn with_start(mut self, start: Option<Timestamp>) -> Self {
795        self.start = start;
796        self
797    }
798
799    /// Attaches the given end timestamp to the checker.
800    pub(crate) fn with_end(mut self, end: Option<Timestamp>) -> Self {
801        self.end = end;
802        self
803    }
804
805    /// Returns true if the given batch is monotonic and behind
806    /// the last batch.
807    pub(crate) fn check_monotonic(&mut self, batch: &Batch) -> Result<(), String> {
808        batch.check_monotonic()?;
809
810        if let (Some(start), Some(first)) = (self.start, batch.first_timestamp())
811            && start > first
812        {
813            return Err(format!(
814                "batch's first timestamp is before the start timestamp: {:?} > {:?}",
815                start, first
816            ));
817        }
818        if let (Some(end), Some(last)) = (self.end, batch.last_timestamp())
819            && end <= last
820        {
821            return Err(format!(
822                "batch's last timestamp is after the end timestamp: {:?} <= {:?}",
823                end, last
824            ));
825        }
826
827        // Checks the batch is behind the last batch.
828        // Then Updates the last batch.
829        let res = self
830            .last_batch
831            .as_ref()
832            .map(|last| last.check_next_batch(batch))
833            .unwrap_or(Ok(()));
834        self.last_batch = Some(batch.clone());
835        res
836    }
837
838    /// Formats current batch and last batch for debug.
839    pub(crate) fn format_batch(&self, batch: &Batch) -> String {
840        use std::fmt::Write;
841
842        let mut message = String::new();
843        if let Some(last) = &self.last_batch {
844            write!(
845                message,
846                "last_pk: {:?}, last_ts: {:?}, last_seq: {:?}, ",
847                last.primary_key(),
848                last.last_timestamp(),
849                last.last_sequence()
850            )
851            .unwrap();
852        }
853        write!(
854            message,
855            "batch_pk: {:?}, batch_ts: {:?}, batch_seq: {:?}",
856            batch.primary_key(),
857            batch.timestamps(),
858            batch.sequences()
859        )
860        .unwrap();
861
862        message
863    }
864
865    /// Checks batches from the part range are monotonic. Otherwise, panics.
866    pub(crate) fn ensure_part_range_batch(
867        &mut self,
868        scanner: &str,
869        region_id: store_api::storage::RegionId,
870        partition: usize,
871        part_range: store_api::region_engine::PartitionRange,
872        batch: &Batch,
873    ) {
874        if let Err(e) = self.check_monotonic(batch) {
875            let err_msg = format!(
876                "{}: batch is not sorted, {}, region_id: {}, partition: {}, part_range: {:?}",
877                scanner, e, region_id, partition, part_range,
878            );
879            common_telemetry::error!("{err_msg}, {}", self.format_batch(batch));
880            // Only print the number of row in the panic message.
881            panic!("{err_msg}, batch rows: {}", batch.num_rows());
882        }
883    }
884}
885
886/// Len of timestamp in arrow row format.
887const TIMESTAMP_KEY_LEN: usize = 9;
888
889/// Helper function to concat arrays from `iter`.
890fn concat_arrays(iter: impl Iterator<Item = ArrayRef>) -> Result<ArrayRef> {
891    let arrays: Vec<_> = iter.collect();
892    let dyn_arrays: Vec<_> = arrays.iter().map(|array| array.as_ref()).collect();
893    arrow::compute::concat(&dyn_arrays).context(ComputeArrowSnafu)
894}
895
896/// A column in a [Batch].
897#[derive(Debug, PartialEq, Eq, Clone)]
898pub struct BatchColumn {
899    /// Id of the column.
900    pub column_id: ColumnId,
901    /// Data of the column.
902    pub data: VectorRef,
903}
904
905/// Builder to build [Batch].
906pub struct BatchBuilder {
907    primary_key: Vec<u8>,
908    timestamps: Option<VectorRef>,
909    sequences: Option<Arc<UInt64Vector>>,
910    op_types: Option<Arc<UInt8Vector>>,
911    fields: Vec<BatchColumn>,
912}
913
914impl BatchBuilder {
915    /// Creates a new [BatchBuilder] with primary key.
916    pub fn new(primary_key: Vec<u8>) -> BatchBuilder {
917        BatchBuilder {
918            primary_key,
919            timestamps: None,
920            sequences: None,
921            op_types: None,
922            fields: Vec::new(),
923        }
924    }
925
926    /// Creates a new [BatchBuilder] with all required columns.
927    pub fn with_required_columns(
928        primary_key: Vec<u8>,
929        timestamps: VectorRef,
930        sequences: Arc<UInt64Vector>,
931        op_types: Arc<UInt8Vector>,
932    ) -> BatchBuilder {
933        BatchBuilder {
934            primary_key,
935            timestamps: Some(timestamps),
936            sequences: Some(sequences),
937            op_types: Some(op_types),
938            fields: Vec::new(),
939        }
940    }
941
942    /// Set all field columns.
943    pub fn with_fields(mut self, fields: Vec<BatchColumn>) -> Self {
944        self.fields = fields;
945        self
946    }
947
948    /// Push a field column.
949    pub fn push_field(&mut self, column: BatchColumn) -> &mut Self {
950        self.fields.push(column);
951        self
952    }
953
954    /// Push an array as a field.
955    pub fn push_field_array(&mut self, column_id: ColumnId, array: ArrayRef) -> Result<&mut Self> {
956        let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?;
957        self.fields.push(BatchColumn {
958            column_id,
959            data: vector,
960        });
961
962        Ok(self)
963    }
964
965    /// Try to set an array as timestamps.
966    pub fn timestamps_array(&mut self, array: ArrayRef) -> Result<&mut Self> {
967        let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?;
968        ensure!(
969            vector.data_type().is_timestamp(),
970            InvalidBatchSnafu {
971                reason: format!("{:?} is not a timestamp type", vector.data_type()),
972            }
973        );
974
975        self.timestamps = Some(vector);
976        Ok(self)
977    }
978
979    /// Try to set an array as sequences.
980    pub fn sequences_array(&mut self, array: ArrayRef) -> Result<&mut Self> {
981        ensure!(
982            *array.data_type() == arrow::datatypes::DataType::UInt64,
983            InvalidBatchSnafu {
984                reason: "sequence array is not UInt64 type",
985            }
986        );
987        // Safety: The cast must success as we have ensured it is uint64 type.
988        let vector = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap());
989        self.sequences = Some(vector);
990
991        Ok(self)
992    }
993
994    /// Try to set an array as op types.
995    pub fn op_types_array(&mut self, array: ArrayRef) -> Result<&mut Self> {
996        ensure!(
997            *array.data_type() == arrow::datatypes::DataType::UInt8,
998            InvalidBatchSnafu {
999                reason: "sequence array is not UInt8 type",
1000            }
1001        );
1002        // Safety: The cast must success as we have ensured it is uint64 type.
1003        let vector = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap());
1004        self.op_types = Some(vector);
1005
1006        Ok(self)
1007    }
1008
1009    /// Builds the [Batch].
1010    pub fn build(self) -> Result<Batch> {
1011        let timestamps = self.timestamps.context(InvalidBatchSnafu {
1012            reason: "missing timestamps",
1013        })?;
1014        let sequences = self.sequences.context(InvalidBatchSnafu {
1015            reason: "missing sequences",
1016        })?;
1017        let op_types = self.op_types.context(InvalidBatchSnafu {
1018            reason: "missing op_types",
1019        })?;
1020        // Our storage format ensure these columns are not nullable so
1021        // we use assert here.
1022        assert_eq!(0, timestamps.null_count());
1023        assert_eq!(0, sequences.null_count());
1024        assert_eq!(0, op_types.null_count());
1025
1026        let ts_len = timestamps.len();
1027        ensure!(
1028            sequences.len() == ts_len,
1029            InvalidBatchSnafu {
1030                reason: format!(
1031                    "sequence have different len {} != {}",
1032                    sequences.len(),
1033                    ts_len
1034                ),
1035            }
1036        );
1037        ensure!(
1038            op_types.len() == ts_len,
1039            InvalidBatchSnafu {
1040                reason: format!(
1041                    "op type have different len {} != {}",
1042                    op_types.len(),
1043                    ts_len
1044                ),
1045            }
1046        );
1047        for column in &self.fields {
1048            ensure!(
1049                column.data.len() == ts_len,
1050                InvalidBatchSnafu {
1051                    reason: format!(
1052                        "column {} has different len {} != {}",
1053                        column.column_id,
1054                        column.data.len(),
1055                        ts_len
1056                    ),
1057                }
1058            );
1059        }
1060
1061        Ok(Batch {
1062            primary_key: self.primary_key,
1063            pk_values: None,
1064            timestamps,
1065            sequences,
1066            op_types,
1067            fields: self.fields,
1068            fields_idx: None,
1069        })
1070    }
1071}
1072
1073impl From<Batch> for BatchBuilder {
1074    fn from(batch: Batch) -> Self {
1075        Self {
1076            primary_key: batch.primary_key,
1077            timestamps: Some(batch.timestamps),
1078            sequences: Some(batch.sequences),
1079            op_types: Some(batch.op_types),
1080            fields: batch.fields,
1081        }
1082    }
1083}
1084
1085/// Async [Batch] reader and iterator wrapper.
1086///
1087/// This is the data source for SST writers or internal readers.
1088pub enum Source {
1089    /// Source from a [BoxedBatchReader].
1090    Reader(BoxedBatchReader),
1091    /// Source from a [BoxedBatchIterator].
1092    Iter(BoxedBatchIterator),
1093    /// Source from a [BoxedBatchStream].
1094    Stream(BoxedBatchStream),
1095}
1096
1097impl Source {
1098    /// Returns next [Batch] from this data source.
1099    pub async fn next_batch(&mut self) -> Result<Option<Batch>> {
1100        match self {
1101            Source::Reader(reader) => reader.next_batch().await,
1102            Source::Iter(iter) => iter.next().transpose(),
1103            Source::Stream(stream) => stream.try_next().await,
1104        }
1105    }
1106}
1107
1108/// Async [RecordBatch] reader and iterator wrapper for flat format.
1109pub struct FlatSource {
1110    schema: SchemaRef,
1111    inner: FlatSourceInner,
1112}
1113
1114impl FlatSource {
1115    /// Create a [FlatSource] from a [BoxedRecordBatchIterator] and its schema.
1116    pub fn new_iter(schema: SchemaRef, iter: BoxedRecordBatchIterator) -> Self {
1117        Self {
1118            schema,
1119            inner: FlatSourceInner::Iter(iter),
1120        }
1121    }
1122
1123    /// Create a [FlatSource] from a [BoxedRecordBatchStream] and its schema.
1124    pub fn new_stream(schema: SchemaRef, stream: BoxedRecordBatchStream) -> Self {
1125        Self {
1126            schema,
1127            inner: FlatSourceInner::Stream(stream),
1128        }
1129    }
1130
1131    #[expect(unused)]
1132    fn schema(&self) -> &SchemaRef {
1133        &self.schema
1134    }
1135
1136    pub async fn next_batch(&mut self) -> Result<Option<RecordBatch>> {
1137        self.inner.next_batch().await
1138    }
1139
1140    #[cfg(test)]
1141    pub(crate) fn take_iter(self) -> BoxedRecordBatchIterator {
1142        match self.inner {
1143            FlatSourceInner::Iter(iter) => iter,
1144            FlatSourceInner::Stream(_) => unreachable!(),
1145        }
1146    }
1147}
1148
1149enum FlatSourceInner {
1150    /// Source from a [BoxedRecordBatchIterator].
1151    Iter(BoxedRecordBatchIterator),
1152    /// Source from a [BoxedRecordBatchStream].
1153    Stream(BoxedRecordBatchStream),
1154}
1155
1156impl FlatSourceInner {
1157    /// Returns next [RecordBatch] from this data source.
1158    pub async fn next_batch(&mut self) -> Result<Option<RecordBatch>> {
1159        match self {
1160            Self::Iter(iter) => iter.next().transpose(),
1161            Self::Stream(stream) => stream.try_next().await,
1162        }
1163    }
1164}
1165
1166/// Async batch reader.
1167///
1168/// The reader must guarantee [Batch]es returned by it have the same schema.
1169#[async_trait]
1170pub trait BatchReader: Send {
1171    /// Fetch next [Batch].
1172    ///
1173    /// Returns `Ok(None)` when the reader has reached its end and calling `next_batch()`
1174    /// again won't return batch again.
1175    ///
1176    /// If `Err` is returned, caller should not call this method again, the implementor
1177    /// may or may not panic in such case.
1178    async fn next_batch(&mut self) -> Result<Option<Batch>>;
1179}
1180
1181/// Pointer to [BatchReader].
1182pub type BoxedBatchReader = Box<dyn BatchReader>;
1183
1184/// Pointer to a stream that yields [Batch].
1185pub type BoxedBatchStream = BoxStream<'static, Result<Batch>>;
1186
1187/// Pointer to a stream that yields [RecordBatch].
1188pub type BoxedRecordBatchStream = BoxStream<'static, Result<RecordBatch>>;
1189
1190#[async_trait::async_trait]
1191impl<T: BatchReader + ?Sized> BatchReader for Box<T> {
1192    async fn next_batch(&mut self) -> Result<Option<Batch>> {
1193        (**self).next_batch().await
1194    }
1195}
1196
1197/// Local metrics for scanners.
1198#[derive(Debug, Default)]
1199pub(crate) struct ScannerMetrics {
1200    /// Duration to scan data.
1201    scan_cost: Duration,
1202    /// Duration while waiting for `yield`.
1203    yield_cost: Duration,
1204    /// Number of batches returned.
1205    num_batches: usize,
1206    /// Number of rows returned.
1207    num_rows: usize,
1208}
1209
1210#[cfg(test)]
1211mod tests {
1212    use datatypes::arrow::array::{TimestampMillisecondArray, UInt8Array, UInt64Array};
1213    use mito_codec::row_converter::{self, build_primary_key_codec_with_fields};
1214    use store_api::codec::PrimaryKeyEncoding;
1215    use store_api::storage::consts::ReservedColumnId;
1216
1217    use super::*;
1218    use crate::error::Error;
1219    use crate::test_util::new_batch_builder;
1220
1221    fn new_batch(
1222        timestamps: &[i64],
1223        sequences: &[u64],
1224        op_types: &[OpType],
1225        field: &[u64],
1226    ) -> Batch {
1227        new_batch_builder(b"test", timestamps, sequences, op_types, 1, field)
1228            .build()
1229            .unwrap()
1230    }
1231
1232    fn new_batch_with_u64_fields(
1233        timestamps: &[i64],
1234        sequences: &[u64],
1235        op_types: &[OpType],
1236        fields: &[(ColumnId, &[Option<u64>])],
1237    ) -> Batch {
1238        assert_eq!(timestamps.len(), sequences.len());
1239        assert_eq!(timestamps.len(), op_types.len());
1240        for (_, values) in fields {
1241            assert_eq!(timestamps.len(), values.len());
1242        }
1243
1244        let mut builder = BatchBuilder::new(b"test".to_vec());
1245        builder
1246            .timestamps_array(Arc::new(TimestampMillisecondArray::from_iter_values(
1247                timestamps.iter().copied(),
1248            )))
1249            .unwrap()
1250            .sequences_array(Arc::new(UInt64Array::from_iter_values(
1251                sequences.iter().copied(),
1252            )))
1253            .unwrap()
1254            .op_types_array(Arc::new(UInt8Array::from_iter_values(
1255                op_types.iter().map(|v| *v as u8),
1256            )))
1257            .unwrap();
1258
1259        for (col_id, values) in fields {
1260            builder
1261                .push_field_array(*col_id, Arc::new(UInt64Array::from(values.to_vec())))
1262                .unwrap();
1263        }
1264
1265        builder.build().unwrap()
1266    }
1267
1268    fn new_batch_without_fields(
1269        timestamps: &[i64],
1270        sequences: &[u64],
1271        op_types: &[OpType],
1272    ) -> Batch {
1273        assert_eq!(timestamps.len(), sequences.len());
1274        assert_eq!(timestamps.len(), op_types.len());
1275
1276        let mut builder = BatchBuilder::new(b"test".to_vec());
1277        builder
1278            .timestamps_array(Arc::new(TimestampMillisecondArray::from_iter_values(
1279                timestamps.iter().copied(),
1280            )))
1281            .unwrap()
1282            .sequences_array(Arc::new(UInt64Array::from_iter_values(
1283                sequences.iter().copied(),
1284            )))
1285            .unwrap()
1286            .op_types_array(Arc::new(UInt8Array::from_iter_values(
1287                op_types.iter().map(|v| *v as u8),
1288            )))
1289            .unwrap();
1290
1291        builder.build().unwrap()
1292    }
1293
1294    #[test]
1295    fn test_empty_batch() {
1296        let batch = Batch::empty();
1297        assert!(batch.is_empty());
1298        assert_eq!(None, batch.first_timestamp());
1299        assert_eq!(None, batch.last_timestamp());
1300        assert_eq!(None, batch.first_sequence());
1301        assert_eq!(None, batch.last_sequence());
1302        assert!(batch.timestamps_native().is_none());
1303    }
1304
1305    #[test]
1306    fn test_first_last_one() {
1307        let batch = new_batch(&[1], &[2], &[OpType::Put], &[4]);
1308        assert_eq!(
1309            Timestamp::new_millisecond(1),
1310            batch.first_timestamp().unwrap()
1311        );
1312        assert_eq!(
1313            Timestamp::new_millisecond(1),
1314            batch.last_timestamp().unwrap()
1315        );
1316        assert_eq!(2, batch.first_sequence().unwrap());
1317        assert_eq!(2, batch.last_sequence().unwrap());
1318    }
1319
1320    #[test]
1321    fn test_first_last_multiple() {
1322        let batch = new_batch(
1323            &[1, 2, 3],
1324            &[11, 12, 13],
1325            &[OpType::Put, OpType::Put, OpType::Put],
1326            &[21, 22, 23],
1327        );
1328        assert_eq!(
1329            Timestamp::new_millisecond(1),
1330            batch.first_timestamp().unwrap()
1331        );
1332        assert_eq!(
1333            Timestamp::new_millisecond(3),
1334            batch.last_timestamp().unwrap()
1335        );
1336        assert_eq!(11, batch.first_sequence().unwrap());
1337        assert_eq!(13, batch.last_sequence().unwrap());
1338    }
1339
1340    #[test]
1341    fn test_slice() {
1342        let batch = new_batch(
1343            &[1, 2, 3, 4],
1344            &[11, 12, 13, 14],
1345            &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1346            &[21, 22, 23, 24],
1347        );
1348        let batch = batch.slice(1, 2);
1349        let expect = new_batch(
1350            &[2, 3],
1351            &[12, 13],
1352            &[OpType::Delete, OpType::Put],
1353            &[22, 23],
1354        );
1355        assert_eq!(expect, batch);
1356    }
1357
1358    #[test]
1359    fn test_timestamps_native() {
1360        let batch = new_batch(
1361            &[1, 2, 3, 4],
1362            &[11, 12, 13, 14],
1363            &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1364            &[21, 22, 23, 24],
1365        );
1366        assert_eq!(&[1, 2, 3, 4], batch.timestamps_native().unwrap());
1367    }
1368
1369    #[test]
1370    fn test_concat_empty() {
1371        let err = Batch::concat(vec![]).unwrap_err();
1372        assert!(
1373            matches!(err, Error::InvalidBatch { .. }),
1374            "unexpected err: {err}"
1375        );
1376    }
1377
1378    #[test]
1379    fn test_concat_one() {
1380        let batch = new_batch(&[], &[], &[], &[]);
1381        let actual = Batch::concat(vec![batch.clone()]).unwrap();
1382        assert_eq!(batch, actual);
1383
1384        let batch = new_batch(&[1, 2], &[11, 12], &[OpType::Put, OpType::Put], &[21, 22]);
1385        let actual = Batch::concat(vec![batch.clone()]).unwrap();
1386        assert_eq!(batch, actual);
1387    }
1388
1389    #[test]
1390    fn test_concat_multiple() {
1391        let batches = vec![
1392            new_batch(&[1, 2], &[11, 12], &[OpType::Put, OpType::Put], &[21, 22]),
1393            new_batch(
1394                &[3, 4, 5],
1395                &[13, 14, 15],
1396                &[OpType::Put, OpType::Delete, OpType::Put],
1397                &[23, 24, 25],
1398            ),
1399            new_batch(&[], &[], &[], &[]),
1400            new_batch(&[6], &[16], &[OpType::Put], &[26]),
1401        ];
1402        let batch = Batch::concat(batches).unwrap();
1403        let expect = new_batch(
1404            &[1, 2, 3, 4, 5, 6],
1405            &[11, 12, 13, 14, 15, 16],
1406            &[
1407                OpType::Put,
1408                OpType::Put,
1409                OpType::Put,
1410                OpType::Delete,
1411                OpType::Put,
1412                OpType::Put,
1413            ],
1414            &[21, 22, 23, 24, 25, 26],
1415        );
1416        assert_eq!(expect, batch);
1417    }
1418
1419    #[test]
1420    fn test_concat_different() {
1421        let batch1 = new_batch(&[1], &[1], &[OpType::Put], &[1]);
1422        let mut batch2 = new_batch(&[2], &[2], &[OpType::Put], &[2]);
1423        batch2.primary_key = b"hello".to_vec();
1424        let err = Batch::concat(vec![batch1, batch2]).unwrap_err();
1425        assert!(
1426            matches!(err, Error::InvalidBatch { .. }),
1427            "unexpected err: {err}"
1428        );
1429    }
1430
1431    #[test]
1432    fn test_concat_different_fields() {
1433        let batch1 = new_batch(&[1], &[1], &[OpType::Put], &[1]);
1434        let fields = vec![
1435            batch1.fields()[0].clone(),
1436            BatchColumn {
1437                column_id: 2,
1438                data: Arc::new(UInt64Vector::from_slice([2])),
1439            },
1440        ];
1441        // Batch 2 has more fields.
1442        let batch2 = batch1.clone().with_fields(fields).unwrap();
1443        let err = Batch::concat(vec![batch1.clone(), batch2]).unwrap_err();
1444        assert!(
1445            matches!(err, Error::InvalidBatch { .. }),
1446            "unexpected err: {err}"
1447        );
1448
1449        // Batch 2 has different field.
1450        let fields = vec![BatchColumn {
1451            column_id: 2,
1452            data: Arc::new(UInt64Vector::from_slice([2])),
1453        }];
1454        let batch2 = batch1.clone().with_fields(fields).unwrap();
1455        let err = Batch::concat(vec![batch1, batch2]).unwrap_err();
1456        assert!(
1457            matches!(err, Error::InvalidBatch { .. }),
1458            "unexpected err: {err}"
1459        );
1460    }
1461
1462    #[test]
1463    fn test_filter_deleted_empty() {
1464        let mut batch = new_batch(&[], &[], &[], &[]);
1465        batch.filter_deleted().unwrap();
1466        assert!(batch.is_empty());
1467    }
1468
1469    #[test]
1470    fn test_filter_deleted() {
1471        let mut batch = new_batch(
1472            &[1, 2, 3, 4],
1473            &[11, 12, 13, 14],
1474            &[OpType::Delete, OpType::Put, OpType::Delete, OpType::Put],
1475            &[21, 22, 23, 24],
1476        );
1477        batch.filter_deleted().unwrap();
1478        let expect = new_batch(&[2, 4], &[12, 14], &[OpType::Put, OpType::Put], &[22, 24]);
1479        assert_eq!(expect, batch);
1480
1481        let mut batch = new_batch(
1482            &[1, 2, 3, 4],
1483            &[11, 12, 13, 14],
1484            &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1485            &[21, 22, 23, 24],
1486        );
1487        let expect = batch.clone();
1488        batch.filter_deleted().unwrap();
1489        assert_eq!(expect, batch);
1490    }
1491
1492    #[test]
1493    fn test_filter_by_sequence() {
1494        // Filters put only.
1495        let mut batch = new_batch(
1496            &[1, 2, 3, 4],
1497            &[11, 12, 13, 14],
1498            &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1499            &[21, 22, 23, 24],
1500        );
1501        batch
1502            .filter_by_sequence(Some(SequenceRange::LtEq { max: 13 }))
1503            .unwrap();
1504        let expect = new_batch(
1505            &[1, 2, 3],
1506            &[11, 12, 13],
1507            &[OpType::Put, OpType::Put, OpType::Put],
1508            &[21, 22, 23],
1509        );
1510        assert_eq!(expect, batch);
1511
1512        // Filters to empty.
1513        let mut batch = new_batch(
1514            &[1, 2, 3, 4],
1515            &[11, 12, 13, 14],
1516            &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1517            &[21, 22, 23, 24],
1518        );
1519
1520        batch
1521            .filter_by_sequence(Some(SequenceRange::LtEq { max: 10 }))
1522            .unwrap();
1523        assert!(batch.is_empty());
1524
1525        // None filter.
1526        let mut batch = new_batch(
1527            &[1, 2, 3, 4],
1528            &[11, 12, 13, 14],
1529            &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1530            &[21, 22, 23, 24],
1531        );
1532        let expect = batch.clone();
1533        batch.filter_by_sequence(None).unwrap();
1534        assert_eq!(expect, batch);
1535
1536        // Filter a empty batch
1537        let mut batch = new_batch(&[], &[], &[], &[]);
1538        batch
1539            .filter_by_sequence(Some(SequenceRange::LtEq { max: 10 }))
1540            .unwrap();
1541        assert!(batch.is_empty());
1542
1543        // Filter a empty batch with None
1544        let mut batch = new_batch(&[], &[], &[], &[]);
1545        batch.filter_by_sequence(None).unwrap();
1546        assert!(batch.is_empty());
1547
1548        // Test From variant - exclusive lower bound
1549        let mut batch = new_batch(
1550            &[1, 2, 3, 4],
1551            &[11, 12, 13, 14],
1552            &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1553            &[21, 22, 23, 24],
1554        );
1555        batch
1556            .filter_by_sequence(Some(SequenceRange::Gt { min: 12 }))
1557            .unwrap();
1558        let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1559        assert_eq!(expect, batch);
1560
1561        // Test From variant with no matches
1562        let mut batch = new_batch(
1563            &[1, 2, 3, 4],
1564            &[11, 12, 13, 14],
1565            &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1566            &[21, 22, 23, 24],
1567        );
1568        batch
1569            .filter_by_sequence(Some(SequenceRange::Gt { min: 20 }))
1570            .unwrap();
1571        assert!(batch.is_empty());
1572
1573        // Test Range variant - exclusive lower bound, inclusive upper bound
1574        let mut batch = new_batch(
1575            &[1, 2, 3, 4, 5],
1576            &[11, 12, 13, 14, 15],
1577            &[
1578                OpType::Put,
1579                OpType::Put,
1580                OpType::Put,
1581                OpType::Put,
1582                OpType::Put,
1583            ],
1584            &[21, 22, 23, 24, 25],
1585        );
1586        batch
1587            .filter_by_sequence(Some(SequenceRange::GtLtEq { min: 12, max: 14 }))
1588            .unwrap();
1589        let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1590        assert_eq!(expect, batch);
1591
1592        // Test Range variant with mixed operations
1593        let mut batch = new_batch(
1594            &[1, 2, 3, 4, 5],
1595            &[11, 12, 13, 14, 15],
1596            &[
1597                OpType::Put,
1598                OpType::Delete,
1599                OpType::Put,
1600                OpType::Delete,
1601                OpType::Put,
1602            ],
1603            &[21, 22, 23, 24, 25],
1604        );
1605        batch
1606            .filter_by_sequence(Some(SequenceRange::GtLtEq { min: 11, max: 13 }))
1607            .unwrap();
1608        let expect = new_batch(
1609            &[2, 3],
1610            &[12, 13],
1611            &[OpType::Delete, OpType::Put],
1612            &[22, 23],
1613        );
1614        assert_eq!(expect, batch);
1615
1616        // Test Range variant with no matches
1617        let mut batch = new_batch(
1618            &[1, 2, 3, 4],
1619            &[11, 12, 13, 14],
1620            &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1621            &[21, 22, 23, 24],
1622        );
1623        batch
1624            .filter_by_sequence(Some(SequenceRange::GtLtEq { min: 20, max: 25 }))
1625            .unwrap();
1626        assert!(batch.is_empty());
1627    }
1628
1629    #[test]
1630    fn test_merge_last_non_null_no_dup() {
1631        let mut batch = new_batch_with_u64_fields(
1632            &[1, 2],
1633            &[2, 1],
1634            &[OpType::Put, OpType::Put],
1635            &[(1, &[Some(10), None]), (2, &[Some(100), Some(200)])],
1636        );
1637        let expect = batch.clone();
1638        batch.merge_last_non_null().unwrap();
1639        assert_eq!(expect, batch);
1640    }
1641
1642    #[test]
1643    fn test_merge_last_non_null_fill_null_fields() {
1644        // Rows are already sorted by timestamp asc and sequence desc.
1645        let mut batch = new_batch_with_u64_fields(
1646            &[1, 1, 1],
1647            &[3, 2, 1],
1648            &[OpType::Put, OpType::Put, OpType::Put],
1649            &[
1650                (1, &[None, Some(10), Some(11)]),
1651                (2, &[Some(100), Some(200), Some(300)]),
1652            ],
1653        );
1654        batch.merge_last_non_null().unwrap();
1655
1656        // Field 1 is filled from the first older row (seq=2). Field 2 keeps the base value.
1657        // Filled fields must not be overwritten by even older duplicates.
1658        let expect = new_batch_with_u64_fields(
1659            &[1],
1660            &[3],
1661            &[OpType::Put],
1662            &[(1, &[Some(10)]), (2, &[Some(100)])],
1663        );
1664        assert_eq!(expect, batch);
1665    }
1666
1667    #[test]
1668    fn test_merge_last_non_null_stop_at_delete_row() {
1669        // A delete row in older duplicates should stop filling to avoid resurrecting values before
1670        // deletion.
1671        let mut batch = new_batch_with_u64_fields(
1672            &[1, 1, 1],
1673            &[3, 2, 1],
1674            &[OpType::Put, OpType::Delete, OpType::Put],
1675            &[
1676                (1, &[None, Some(10), Some(11)]),
1677                (2, &[Some(100), Some(200), Some(300)]),
1678            ],
1679        );
1680        batch.merge_last_non_null().unwrap();
1681
1682        let expect = new_batch_with_u64_fields(
1683            &[1],
1684            &[3],
1685            &[OpType::Put],
1686            &[(1, &[None]), (2, &[Some(100)])],
1687        );
1688        assert_eq!(expect, batch);
1689    }
1690
1691    #[test]
1692    fn test_merge_last_non_null_base_delete_no_merge() {
1693        let mut batch = new_batch_with_u64_fields(
1694            &[1, 1],
1695            &[3, 2],
1696            &[OpType::Delete, OpType::Put],
1697            &[(1, &[None, Some(10)]), (2, &[None, Some(200)])],
1698        );
1699        batch.merge_last_non_null().unwrap();
1700
1701        // Base row is delete, keep it as is and don't merge fields from older rows.
1702        let expect =
1703            new_batch_with_u64_fields(&[1], &[3], &[OpType::Delete], &[(1, &[None]), (2, &[None])]);
1704        assert_eq!(expect, batch);
1705    }
1706
1707    #[test]
1708    fn test_merge_last_non_null_multiple_timestamp_groups() {
1709        let mut batch = new_batch_with_u64_fields(
1710            &[1, 1, 2, 3, 3],
1711            &[5, 4, 3, 2, 1],
1712            &[
1713                OpType::Put,
1714                OpType::Put,
1715                OpType::Put,
1716                OpType::Put,
1717                OpType::Put,
1718            ],
1719            &[
1720                (1, &[None, Some(10), Some(20), None, Some(30)]),
1721                (2, &[Some(100), Some(110), Some(120), None, Some(130)]),
1722            ],
1723        );
1724        batch.merge_last_non_null().unwrap();
1725
1726        let expect = new_batch_with_u64_fields(
1727            &[1, 2, 3],
1728            &[5, 3, 2],
1729            &[OpType::Put, OpType::Put, OpType::Put],
1730            &[
1731                (1, &[Some(10), Some(20), Some(30)]),
1732                (2, &[Some(100), Some(120), Some(130)]),
1733            ],
1734        );
1735        assert_eq!(expect, batch);
1736    }
1737
1738    #[test]
1739    fn test_merge_last_non_null_no_fields() {
1740        let mut batch = new_batch_without_fields(
1741            &[1, 1, 2],
1742            &[3, 2, 1],
1743            &[OpType::Put, OpType::Put, OpType::Put],
1744        );
1745        batch.merge_last_non_null().unwrap();
1746
1747        let expect = new_batch_without_fields(&[1, 2], &[3, 1], &[OpType::Put, OpType::Put]);
1748        assert_eq!(expect, batch);
1749    }
1750
1751    #[test]
1752    fn test_filter() {
1753        // Filters put only.
1754        let mut batch = new_batch(
1755            &[1, 2, 3, 4],
1756            &[11, 12, 13, 14],
1757            &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1758            &[21, 22, 23, 24],
1759        );
1760        let predicate = BooleanVector::from_vec(vec![false, false, true, true]);
1761        batch.filter(&predicate).unwrap();
1762        let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1763        assert_eq!(expect, batch);
1764
1765        // Filters deletion.
1766        let mut batch = new_batch(
1767            &[1, 2, 3, 4],
1768            &[11, 12, 13, 14],
1769            &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1770            &[21, 22, 23, 24],
1771        );
1772        let predicate = BooleanVector::from_vec(vec![false, false, true, true]);
1773        batch.filter(&predicate).unwrap();
1774        let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1775        assert_eq!(expect, batch);
1776
1777        // Filters to empty.
1778        let predicate = BooleanVector::from_vec(vec![false, false]);
1779        batch.filter(&predicate).unwrap();
1780        assert!(batch.is_empty());
1781    }
1782
1783    #[test]
1784    fn test_sort_and_dedup() {
1785        let original = new_batch(
1786            &[2, 3, 1, 4, 5, 2],
1787            &[1, 2, 3, 4, 5, 6],
1788            &[
1789                OpType::Put,
1790                OpType::Put,
1791                OpType::Put,
1792                OpType::Put,
1793                OpType::Put,
1794                OpType::Put,
1795            ],
1796            &[21, 22, 23, 24, 25, 26],
1797        );
1798
1799        let mut batch = original.clone();
1800        batch.sort(true).unwrap();
1801        // It should only keep one timestamp 2.
1802        assert_eq!(
1803            new_batch(
1804                &[1, 2, 3, 4, 5],
1805                &[3, 6, 2, 4, 5],
1806                &[
1807                    OpType::Put,
1808                    OpType::Put,
1809                    OpType::Put,
1810                    OpType::Put,
1811                    OpType::Put,
1812                ],
1813                &[23, 26, 22, 24, 25],
1814            ),
1815            batch
1816        );
1817
1818        let mut batch = original.clone();
1819        batch.sort(false).unwrap();
1820
1821        // It should only keep one timestamp 2.
1822        assert_eq!(
1823            new_batch(
1824                &[1, 2, 2, 3, 4, 5],
1825                &[3, 6, 1, 2, 4, 5],
1826                &[
1827                    OpType::Put,
1828                    OpType::Put,
1829                    OpType::Put,
1830                    OpType::Put,
1831                    OpType::Put,
1832                    OpType::Put,
1833                ],
1834                &[23, 26, 21, 22, 24, 25],
1835            ),
1836            batch
1837        );
1838
1839        let original = new_batch(
1840            &[2, 2, 1],
1841            &[1, 6, 1],
1842            &[OpType::Delete, OpType::Put, OpType::Put],
1843            &[21, 22, 23],
1844        );
1845
1846        let mut batch = original.clone();
1847        batch.sort(true).unwrap();
1848        let expect = new_batch(&[1, 2], &[1, 6], &[OpType::Put, OpType::Put], &[23, 22]);
1849        assert_eq!(expect, batch);
1850
1851        let mut batch = original.clone();
1852        batch.sort(false).unwrap();
1853        let expect = new_batch(
1854            &[1, 2, 2],
1855            &[1, 6, 1],
1856            &[OpType::Put, OpType::Put, OpType::Delete],
1857            &[23, 22, 21],
1858        );
1859        assert_eq!(expect, batch);
1860    }
1861
1862    #[test]
1863    fn test_get_value() {
1864        let encodings = [PrimaryKeyEncoding::Dense, PrimaryKeyEncoding::Sparse];
1865
1866        for encoding in encodings {
1867            let codec = build_primary_key_codec_with_fields(
1868                encoding,
1869                [
1870                    (
1871                        ReservedColumnId::table_id(),
1872                        row_converter::SortField::new(ConcreteDataType::uint32_datatype()),
1873                    ),
1874                    (
1875                        ReservedColumnId::tsid(),
1876                        row_converter::SortField::new(ConcreteDataType::uint64_datatype()),
1877                    ),
1878                    (
1879                        100,
1880                        row_converter::SortField::new(ConcreteDataType::string_datatype()),
1881                    ),
1882                    (
1883                        200,
1884                        row_converter::SortField::new(ConcreteDataType::string_datatype()),
1885                    ),
1886                ]
1887                .into_iter(),
1888            );
1889
1890            let values = [
1891                Value::UInt32(1000),
1892                Value::UInt64(2000),
1893                Value::String("abcdefgh".into()),
1894                Value::String("zyxwvu".into()),
1895            ];
1896            let mut buf = vec![];
1897            codec
1898                .encode_values(
1899                    &[
1900                        (ReservedColumnId::table_id(), values[0].clone()),
1901                        (ReservedColumnId::tsid(), values[1].clone()),
1902                        (100, values[2].clone()),
1903                        (200, values[3].clone()),
1904                    ],
1905                    &mut buf,
1906                )
1907                .unwrap();
1908
1909            let field_col_id = 2;
1910            let mut batch = new_batch_builder(
1911                &buf,
1912                &[1, 2, 3],
1913                &[1, 1, 1],
1914                &[OpType::Put, OpType::Put, OpType::Put],
1915                field_col_id,
1916                &[42, 43, 44],
1917            )
1918            .build()
1919            .unwrap();
1920
1921            let v = batch
1922                .pk_col_value(&*codec, 0, ReservedColumnId::table_id())
1923                .unwrap()
1924                .unwrap();
1925            assert_eq!(values[0], *v);
1926
1927            let v = batch
1928                .pk_col_value(&*codec, 1, ReservedColumnId::tsid())
1929                .unwrap()
1930                .unwrap();
1931            assert_eq!(values[1], *v);
1932
1933            let v = batch.pk_col_value(&*codec, 2, 100).unwrap().unwrap();
1934            assert_eq!(values[2], *v);
1935
1936            let v = batch.pk_col_value(&*codec, 3, 200).unwrap().unwrap();
1937            assert_eq!(values[3], *v);
1938
1939            let v = batch.field_col_value(field_col_id).unwrap();
1940            assert_eq!(v.data.get(0), Value::UInt64(42));
1941            assert_eq!(v.data.get(1), Value::UInt64(43));
1942            assert_eq!(v.data.get(2), Value::UInt64(44));
1943        }
1944    }
1945}