1pub 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
76pub(crate) fn timestamp_array_to_i64_slice(arr: &ArrayRef) -> &[i64] {
77 use datatypes::arrow::array::{
78 TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray,
79 TimestampSecondArray,
80 };
81 use datatypes::arrow::datatypes::{DataType, TimeUnit};
82
83 match arr.data_type() {
84 DataType::Timestamp(t, _) => match t {
85 TimeUnit::Second => arr
86 .as_any()
87 .downcast_ref::<TimestampSecondArray>()
88 .unwrap()
89 .values(),
90 TimeUnit::Millisecond => arr
91 .as_any()
92 .downcast_ref::<TimestampMillisecondArray>()
93 .unwrap()
94 .values(),
95 TimeUnit::Microsecond => arr
96 .as_any()
97 .downcast_ref::<TimestampMicrosecondArray>()
98 .unwrap()
99 .values(),
100 TimeUnit::Nanosecond => arr
101 .as_any()
102 .downcast_ref::<TimestampNanosecondArray>()
103 .unwrap()
104 .values(),
105 },
106 _ => unreachable!(),
107 }
108}
109
110#[derive(Debug, PartialEq, Clone)]
115pub struct Batch {
116 primary_key: Vec<u8>,
118 pk_values: Option<CompositeValues>,
120 timestamps: VectorRef,
122 sequences: Arc<UInt64Vector>,
126 op_types: Arc<UInt8Vector>,
130 fields: Vec<BatchColumn>,
132 fields_idx: Option<HashMap<ColumnId, usize>>,
134}
135
136impl Batch {
137 pub fn new(
139 primary_key: Vec<u8>,
140 timestamps: VectorRef,
141 sequences: Arc<UInt64Vector>,
142 op_types: Arc<UInt8Vector>,
143 fields: Vec<BatchColumn>,
144 ) -> Result<Batch> {
145 BatchBuilder::with_required_columns(primary_key, timestamps, sequences, op_types)
146 .with_fields(fields)
147 .build()
148 }
149
150 pub fn with_fields(self, fields: Vec<BatchColumn>) -> Result<Batch> {
152 Batch::new(
153 self.primary_key,
154 self.timestamps,
155 self.sequences,
156 self.op_types,
157 fields,
158 )
159 }
160
161 pub fn primary_key(&self) -> &[u8] {
163 &self.primary_key
164 }
165
166 pub fn pk_values(&self) -> Option<&CompositeValues> {
168 self.pk_values.as_ref()
169 }
170
171 pub fn set_pk_values(&mut self, pk_values: CompositeValues) {
173 self.pk_values = Some(pk_values);
174 }
175
176 #[cfg(any(test, feature = "test"))]
178 pub fn remove_pk_values(&mut self) {
179 self.pk_values = None;
180 }
181
182 pub fn fields(&self) -> &[BatchColumn] {
184 &self.fields
185 }
186
187 pub fn timestamps(&self) -> &VectorRef {
189 &self.timestamps
190 }
191
192 pub fn sequences(&self) -> &Arc<UInt64Vector> {
194 &self.sequences
195 }
196
197 pub fn op_types(&self) -> &Arc<UInt8Vector> {
199 &self.op_types
200 }
201
202 pub fn num_rows(&self) -> usize {
204 self.sequences.len()
207 }
208
209 #[allow(dead_code)]
211 pub(crate) fn empty() -> Self {
212 Self {
213 primary_key: vec![],
214 pk_values: None,
215 timestamps: Arc::new(TimestampMillisecondVectorBuilder::with_capacity(0).finish()),
216 sequences: Arc::new(UInt64VectorBuilder::with_capacity(0).finish()),
217 op_types: Arc::new(UInt8VectorBuilder::with_capacity(0).finish()),
218 fields: vec![],
219 fields_idx: None,
220 }
221 }
222
223 pub fn is_empty(&self) -> bool {
225 self.num_rows() == 0
226 }
227
228 pub fn first_timestamp(&self) -> Option<Timestamp> {
230 if self.timestamps.is_empty() {
231 return None;
232 }
233
234 Some(self.get_timestamp(0))
235 }
236
237 pub fn last_timestamp(&self) -> Option<Timestamp> {
239 if self.timestamps.is_empty() {
240 return None;
241 }
242
243 Some(self.get_timestamp(self.timestamps.len() - 1))
244 }
245
246 pub fn first_sequence(&self) -> Option<SequenceNumber> {
248 if self.sequences.is_empty() {
249 return None;
250 }
251
252 Some(self.get_sequence(0))
253 }
254
255 pub fn last_sequence(&self) -> Option<SequenceNumber> {
257 if self.sequences.is_empty() {
258 return None;
259 }
260
261 Some(self.get_sequence(self.sequences.len() - 1))
262 }
263
264 pub fn set_primary_key(&mut self, primary_key: Vec<u8>) {
269 self.primary_key = primary_key;
270 }
271
272 pub fn slice(&self, offset: usize, length: usize) -> Batch {
277 let fields = self
278 .fields
279 .iter()
280 .map(|column| BatchColumn {
281 column_id: column.column_id,
282 data: column.data.slice(offset, length),
283 })
284 .collect();
285 Batch {
287 primary_key: self.primary_key.clone(),
290 pk_values: self.pk_values.clone(),
291 timestamps: self.timestamps.slice(offset, length),
292 sequences: Arc::new(self.sequences.get_slice(offset, length)),
293 op_types: Arc::new(self.op_types.get_slice(offset, length)),
294 fields,
295 fields_idx: self.fields_idx.clone(),
296 }
297 }
298
299 pub fn concat(mut batches: Vec<Batch>) -> Result<Batch> {
303 ensure!(
304 !batches.is_empty(),
305 InvalidBatchSnafu {
306 reason: "empty batches",
307 }
308 );
309 if batches.len() == 1 {
310 return Ok(batches.pop().unwrap());
312 }
313
314 let primary_key = std::mem::take(&mut batches[0].primary_key);
315 let first = &batches[0];
316 ensure!(
318 batches
319 .iter()
320 .skip(1)
321 .all(|b| b.primary_key() == primary_key),
322 InvalidBatchSnafu {
323 reason: "batches have different primary key",
324 }
325 );
326 for b in batches.iter().skip(1) {
327 ensure!(
328 b.fields.len() == first.fields.len(),
329 InvalidBatchSnafu {
330 reason: "batches have different field num",
331 }
332 );
333 for (l, r) in b.fields.iter().zip(&first.fields) {
334 ensure!(
335 l.column_id == r.column_id,
336 InvalidBatchSnafu {
337 reason: "batches have different fields",
338 }
339 );
340 }
341 }
342
343 let mut builder = BatchBuilder::new(primary_key);
345 let array = concat_arrays(batches.iter().map(|b| b.timestamps().to_arrow_array()))?;
347 builder.timestamps_array(array)?;
348 let array = concat_arrays(batches.iter().map(|b| b.sequences().to_arrow_array()))?;
349 builder.sequences_array(array)?;
350 let array = concat_arrays(batches.iter().map(|b| b.op_types().to_arrow_array()))?;
351 builder.op_types_array(array)?;
352 for (i, batch_column) in first.fields.iter().enumerate() {
353 let array = concat_arrays(batches.iter().map(|b| b.fields()[i].data.to_arrow_array()))?;
354 builder.push_field_array(batch_column.column_id, array)?;
355 }
356
357 builder.build()
358 }
359
360 pub fn filter_deleted(&mut self) -> Result<()> {
362 let array = self.op_types.as_arrow();
364 let rhs = UInt8Array::new_scalar(OpType::Delete as u8);
366 let predicate =
367 arrow::compute::kernels::cmp::neq(array, &rhs).context(ComputeArrowSnafu)?;
368 self.filter(&BooleanVector::from(predicate))
369 }
370
371 pub fn filter(&mut self, predicate: &BooleanVector) -> Result<()> {
374 self.timestamps = self
375 .timestamps
376 .filter(predicate)
377 .context(ComputeVectorSnafu)?;
378 self.sequences = Arc::new(
379 UInt64Vector::try_from_arrow_array(
380 arrow::compute::filter(self.sequences.as_arrow(), predicate.as_boolean_array())
381 .context(ComputeArrowSnafu)?,
382 )
383 .unwrap(),
384 );
385 self.op_types = Arc::new(
386 UInt8Vector::try_from_arrow_array(
387 arrow::compute::filter(self.op_types.as_arrow(), predicate.as_boolean_array())
388 .context(ComputeArrowSnafu)?,
389 )
390 .unwrap(),
391 );
392 for batch_column in &mut self.fields {
393 batch_column.data = batch_column
394 .data
395 .filter(predicate)
396 .context(ComputeVectorSnafu)?;
397 }
398
399 Ok(())
400 }
401
402 pub fn filter_by_sequence(&mut self, sequence: Option<SequenceRange>) -> Result<()> {
404 let seq_range = match sequence {
405 None => return Ok(()),
406 Some(seq_range) => {
407 let (Some(first), Some(last)) = (self.first_sequence(), self.last_sequence())
408 else {
409 return Ok(());
410 };
411 let is_subset = match seq_range {
412 SequenceRange::Gt { min } => min < first,
413 SequenceRange::LtEq { max } => max >= last,
414 SequenceRange::GtLtEq { min, max } => min < first && max >= last,
415 };
416 if is_subset {
417 return Ok(());
418 }
419 seq_range
420 }
421 };
422
423 let seqs = self.sequences.as_arrow();
424 let predicate = seq_range.filter(seqs).context(ComputeArrowSnafu)?;
425
426 let predicate = BooleanVector::from(predicate);
427 self.filter(&predicate)?;
428
429 Ok(())
430 }
431
432 pub fn sort(&mut self, dedup: bool) -> Result<()> {
439 let converter = RowConverter::new(vec![
442 SortField::new(self.timestamps.data_type().as_arrow_type()),
443 SortField::new_with_options(
444 self.sequences.data_type().as_arrow_type(),
445 SortOptions {
446 descending: true,
447 ..Default::default()
448 },
449 ),
450 ])
451 .context(ComputeArrowSnafu)?;
452 let columns = [
454 self.timestamps.to_arrow_array(),
455 self.sequences.to_arrow_array(),
456 ];
457 let rows = converter.convert_columns(&columns).unwrap();
458 let mut to_sort: Vec<_> = rows.iter().enumerate().collect();
459
460 let was_sorted = to_sort.is_sorted_by_key(|x| x.1);
461 if !was_sorted {
462 to_sort.sort_unstable_by_key(|x| x.1);
463 }
464
465 let num_rows = to_sort.len();
466 if dedup {
467 to_sort.dedup_by(|left, right| {
469 debug_assert_eq!(18, left.1.as_ref().len());
470 debug_assert_eq!(18, right.1.as_ref().len());
471 let (left_key, right_key) = (left.1.as_ref(), right.1.as_ref());
472 left_key[..TIMESTAMP_KEY_LEN] == right_key[..TIMESTAMP_KEY_LEN]
474 });
475 }
476 let no_dedup = to_sort.len() == num_rows;
477
478 if was_sorted && no_dedup {
479 return Ok(());
480 }
481 let indices = UInt32Vector::from_iter_values(to_sort.iter().map(|v| v.0 as u32));
482 self.take_in_place(&indices)
483 }
484
485 pub(crate) fn merge_last_non_null(&mut self) -> Result<()> {
493 let num_rows = self.num_rows();
494 if num_rows < 2 {
495 return Ok(());
496 }
497
498 let Some(timestamps) = self.timestamps_native() else {
499 return Ok(());
500 };
501
502 let mut has_dup = false;
504 let mut group_count = 1;
505 for i in 1..num_rows {
506 has_dup |= timestamps[i] == timestamps[i - 1];
507 group_count += (timestamps[i] != timestamps[i - 1]) as usize;
508 }
509 if !has_dup {
510 return Ok(());
511 }
512
513 let num_fields = self.fields.len();
514 let op_types = self.op_types.as_arrow().values();
515
516 let mut base_indices: Vec<u32> = Vec::with_capacity(group_count);
517 let mut field_indices: Vec<Vec<u32>> = (0..num_fields)
518 .map(|_| Vec::with_capacity(group_count))
519 .collect();
520
521 let mut start = 0;
522 while start < num_rows {
523 let ts = timestamps[start];
524 let mut end = start + 1;
525 while end < num_rows && timestamps[end] == ts {
526 end += 1;
527 }
528
529 let group_pos = base_indices.len();
530 base_indices.push(start as u32);
531
532 if num_fields > 0 {
533 for idx in &mut field_indices {
535 idx.push(start as u32);
536 }
537
538 let base_deleted = op_types[start] == OpType::Delete as u8;
539 if !base_deleted {
540 let mut missing_fields = Vec::new();
543 for (field_idx, col) in self.fields.iter().enumerate() {
544 if col.data.is_null(start) {
545 missing_fields.push(field_idx);
546 }
547 }
548
549 if !missing_fields.is_empty() {
550 for row_idx in (start + 1)..end {
551 if op_types[row_idx] == OpType::Delete as u8 {
552 break;
553 }
554
555 missing_fields.retain(|&field_idx| {
556 if self.fields[field_idx].data.is_null(row_idx) {
557 true
558 } else {
559 field_indices[field_idx][group_pos] = row_idx as u32;
560 false
561 }
562 });
563
564 if missing_fields.is_empty() {
565 break;
566 }
567 }
568 }
569 }
570 }
571
572 start = end;
573 }
574
575 let base_indices = UInt32Vector::from_vec(base_indices);
576 self.timestamps = self
577 .timestamps
578 .take(&base_indices)
579 .context(ComputeVectorSnafu)?;
580 let array = arrow::compute::take(self.sequences.as_arrow(), base_indices.as_arrow(), None)
581 .context(ComputeArrowSnafu)?;
582 self.sequences = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap());
584 let array = arrow::compute::take(self.op_types.as_arrow(), base_indices.as_arrow(), None)
585 .context(ComputeArrowSnafu)?;
586 self.op_types = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap());
588
589 for (field_idx, batch_column) in self.fields.iter_mut().enumerate() {
590 let idx = UInt32Vector::from_vec(std::mem::take(&mut field_indices[field_idx]));
591 batch_column.data = batch_column.data.take(&idx).context(ComputeVectorSnafu)?;
592 }
593
594 Ok(())
595 }
596
597 pub fn memory_size(&self) -> usize {
599 let mut size = std::mem::size_of::<Self>();
600 size += self.primary_key.len();
601 size += self.timestamps.memory_size();
602 size += self.sequences.memory_size();
603 size += self.op_types.memory_size();
604 for batch_column in &self.fields {
605 size += batch_column.data.memory_size();
606 }
607 size
608 }
609
610 pub(crate) fn timestamps_native(&self) -> Option<&[i64]> {
612 if self.timestamps.is_empty() {
613 return None;
614 }
615
616 let values = match self.timestamps.data_type() {
617 ConcreteDataType::Timestamp(TimestampType::Second(_)) => self
618 .timestamps
619 .as_any()
620 .downcast_ref::<TimestampSecondVector>()
621 .unwrap()
622 .as_arrow()
623 .values(),
624 ConcreteDataType::Timestamp(TimestampType::Millisecond(_)) => self
625 .timestamps
626 .as_any()
627 .downcast_ref::<TimestampMillisecondVector>()
628 .unwrap()
629 .as_arrow()
630 .values(),
631 ConcreteDataType::Timestamp(TimestampType::Microsecond(_)) => self
632 .timestamps
633 .as_any()
634 .downcast_ref::<TimestampMicrosecondVector>()
635 .unwrap()
636 .as_arrow()
637 .values(),
638 ConcreteDataType::Timestamp(TimestampType::Nanosecond(_)) => self
639 .timestamps
640 .as_any()
641 .downcast_ref::<TimestampNanosecondVector>()
642 .unwrap()
643 .as_arrow()
644 .values(),
645 other => panic!("timestamps in a Batch has other type {:?}", other),
646 };
647
648 Some(values)
649 }
650
651 fn take_in_place(&mut self, indices: &UInt32Vector) -> Result<()> {
653 self.timestamps = self.timestamps.take(indices).context(ComputeVectorSnafu)?;
654 let array = arrow::compute::take(self.sequences.as_arrow(), indices.as_arrow(), None)
655 .context(ComputeArrowSnafu)?;
656 self.sequences = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap());
658 let array = arrow::compute::take(self.op_types.as_arrow(), indices.as_arrow(), None)
659 .context(ComputeArrowSnafu)?;
660 self.op_types = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap());
661 for batch_column in &mut self.fields {
662 batch_column.data = batch_column
663 .data
664 .take(indices)
665 .context(ComputeVectorSnafu)?;
666 }
667
668 Ok(())
669 }
670
671 fn get_timestamp(&self, index: usize) -> Timestamp {
676 match self.timestamps.get_ref(index) {
677 ValueRef::Timestamp(timestamp) => timestamp,
678
679 value => panic!("{:?} is not a timestamp", value),
681 }
682 }
683
684 pub(crate) fn get_sequence(&self, index: usize) -> SequenceNumber {
689 self.sequences.get_data(index).unwrap()
691 }
692
693 #[cfg(debug_assertions)]
695 #[allow(dead_code)]
696 pub(crate) fn check_monotonic(&self) -> Result<(), String> {
697 use std::cmp::Ordering;
698 if self.timestamps_native().is_none() {
699 return Ok(());
700 }
701
702 let timestamps = self.timestamps_native().unwrap();
703 let sequences = self.sequences.as_arrow().values();
704 for (i, window) in timestamps.windows(2).enumerate() {
705 let current = window[0];
706 let next = window[1];
707 let current_sequence = sequences[i];
708 let next_sequence = sequences[i + 1];
709 match current.cmp(&next) {
710 Ordering::Less => {
711 continue;
713 }
714 Ordering::Equal => {
715 if current_sequence < next_sequence {
717 return Err(format!(
718 "sequence are not monotonic: ts {} == {} but current sequence {} < {}, index: {}",
719 current, next, current_sequence, next_sequence, i
720 ));
721 }
722 }
723 Ordering::Greater => {
724 return Err(format!(
726 "timestamps are not monotonic: {} > {}, index: {}",
727 current, next, i
728 ));
729 }
730 }
731 }
732
733 Ok(())
734 }
735
736 #[cfg(debug_assertions)]
738 #[allow(dead_code)]
739 pub(crate) fn check_next_batch(&self, other: &Batch) -> Result<(), String> {
740 if self.primary_key() < other.primary_key() {
742 return Ok(());
743 }
744 if self.primary_key() > other.primary_key() {
745 return Err(format!(
746 "primary key is not monotonic: {:?} > {:?}",
747 self.primary_key(),
748 other.primary_key()
749 ));
750 }
751 if self.last_timestamp() < other.first_timestamp() {
753 return Ok(());
754 }
755 if self.last_timestamp() > other.first_timestamp() {
756 return Err(format!(
757 "timestamps are not monotonic: {:?} > {:?}",
758 self.last_timestamp(),
759 other.first_timestamp()
760 ));
761 }
762 if self.last_sequence() >= other.first_sequence() {
764 return Ok(());
765 }
766 Err(format!(
767 "sequences are not monotonic: {:?} < {:?}",
768 self.last_sequence(),
769 other.first_sequence()
770 ))
771 }
772
773 pub fn pk_col_value(
777 &mut self,
778 codec: &dyn PrimaryKeyCodec,
779 col_idx_in_pk: usize,
780 column_id: ColumnId,
781 ) -> Result<Option<&Value>> {
782 if self.pk_values.is_none() {
783 self.pk_values = Some(codec.decode(&self.primary_key).context(DecodeSnafu)?);
784 }
785
786 let pk_values = self.pk_values.as_ref().unwrap();
787 Ok(match pk_values {
788 CompositeValues::Dense(values) => values.get(col_idx_in_pk).map(|(_, v)| v),
789 CompositeValues::Sparse(values) => values.get(&column_id),
790 })
791 }
792
793 pub fn field_col_value(&mut self, column_id: ColumnId) -> Option<&BatchColumn> {
797 if self.fields_idx.is_none() {
798 self.fields_idx = Some(
799 self.fields
800 .iter()
801 .enumerate()
802 .map(|(i, c)| (c.column_id, i))
803 .collect(),
804 );
805 }
806
807 self.fields_idx
808 .as_ref()
809 .unwrap()
810 .get(&column_id)
811 .map(|&idx| &self.fields[idx])
812 }
813}
814
815#[cfg(debug_assertions)]
817#[derive(Default)]
818#[allow(dead_code)]
819pub(crate) struct BatchChecker {
820 last_batch: Option<Batch>,
821 start: Option<Timestamp>,
822 end: Option<Timestamp>,
823}
824
825#[cfg(debug_assertions)]
826#[allow(dead_code)]
827impl BatchChecker {
828 pub(crate) fn with_start(mut self, start: Option<Timestamp>) -> Self {
830 self.start = start;
831 self
832 }
833
834 pub(crate) fn with_end(mut self, end: Option<Timestamp>) -> Self {
836 self.end = end;
837 self
838 }
839
840 pub(crate) fn check_monotonic(&mut self, batch: &Batch) -> Result<(), String> {
843 batch.check_monotonic()?;
844
845 if let (Some(start), Some(first)) = (self.start, batch.first_timestamp())
846 && start > first
847 {
848 return Err(format!(
849 "batch's first timestamp is before the start timestamp: {:?} > {:?}",
850 start, first
851 ));
852 }
853 if let (Some(end), Some(last)) = (self.end, batch.last_timestamp())
854 && end <= last
855 {
856 return Err(format!(
857 "batch's last timestamp is after the end timestamp: {:?} <= {:?}",
858 end, last
859 ));
860 }
861
862 let res = self
865 .last_batch
866 .as_ref()
867 .map(|last| last.check_next_batch(batch))
868 .unwrap_or(Ok(()));
869 self.last_batch = Some(batch.clone());
870 res
871 }
872
873 pub(crate) fn format_batch(&self, batch: &Batch) -> String {
875 use std::fmt::Write;
876
877 let mut message = String::new();
878 if let Some(last) = &self.last_batch {
879 write!(
880 message,
881 "last_pk: {:?}, last_ts: {:?}, last_seq: {:?}, ",
882 last.primary_key(),
883 last.last_timestamp(),
884 last.last_sequence()
885 )
886 .unwrap();
887 }
888 write!(
889 message,
890 "batch_pk: {:?}, batch_ts: {:?}, batch_seq: {:?}",
891 batch.primary_key(),
892 batch.timestamps(),
893 batch.sequences()
894 )
895 .unwrap();
896
897 message
898 }
899
900 pub(crate) fn ensure_part_range_batch(
902 &mut self,
903 scanner: &str,
904 region_id: store_api::storage::RegionId,
905 partition: usize,
906 part_range: store_api::region_engine::PartitionRange,
907 batch: &Batch,
908 ) {
909 if let Err(e) = self.check_monotonic(batch) {
910 let err_msg = format!(
911 "{}: batch is not sorted, {}, region_id: {}, partition: {}, part_range: {:?}",
912 scanner, e, region_id, partition, part_range,
913 );
914 common_telemetry::error!("{err_msg}, {}", self.format_batch(batch));
915 panic!("{err_msg}, batch rows: {}", batch.num_rows());
917 }
918 }
919}
920
921const TIMESTAMP_KEY_LEN: usize = 9;
923
924fn concat_arrays(iter: impl Iterator<Item = ArrayRef>) -> Result<ArrayRef> {
926 let arrays: Vec<_> = iter.collect();
927 let dyn_arrays: Vec<_> = arrays.iter().map(|array| array.as_ref()).collect();
928 arrow::compute::concat(&dyn_arrays).context(ComputeArrowSnafu)
929}
930
931#[derive(Debug, PartialEq, Eq, Clone)]
933pub struct BatchColumn {
934 pub column_id: ColumnId,
936 pub data: VectorRef,
938}
939
940pub struct BatchBuilder {
942 primary_key: Vec<u8>,
943 timestamps: Option<VectorRef>,
944 sequences: Option<Arc<UInt64Vector>>,
945 op_types: Option<Arc<UInt8Vector>>,
946 fields: Vec<BatchColumn>,
947}
948
949impl BatchBuilder {
950 pub fn new(primary_key: Vec<u8>) -> BatchBuilder {
952 BatchBuilder {
953 primary_key,
954 timestamps: None,
955 sequences: None,
956 op_types: None,
957 fields: Vec::new(),
958 }
959 }
960
961 pub fn with_required_columns(
963 primary_key: Vec<u8>,
964 timestamps: VectorRef,
965 sequences: Arc<UInt64Vector>,
966 op_types: Arc<UInt8Vector>,
967 ) -> BatchBuilder {
968 BatchBuilder {
969 primary_key,
970 timestamps: Some(timestamps),
971 sequences: Some(sequences),
972 op_types: Some(op_types),
973 fields: Vec::new(),
974 }
975 }
976
977 pub fn with_fields(mut self, fields: Vec<BatchColumn>) -> Self {
979 self.fields = fields;
980 self
981 }
982
983 pub fn push_field(&mut self, column: BatchColumn) -> &mut Self {
985 self.fields.push(column);
986 self
987 }
988
989 pub fn push_field_array(&mut self, column_id: ColumnId, array: ArrayRef) -> Result<&mut Self> {
991 let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?;
992 self.fields.push(BatchColumn {
993 column_id,
994 data: vector,
995 });
996
997 Ok(self)
998 }
999
1000 pub fn timestamps_array(&mut self, array: ArrayRef) -> Result<&mut Self> {
1002 let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?;
1003 ensure!(
1004 vector.data_type().is_timestamp(),
1005 InvalidBatchSnafu {
1006 reason: format!("{:?} is not a timestamp type", vector.data_type()),
1007 }
1008 );
1009
1010 self.timestamps = Some(vector);
1011 Ok(self)
1012 }
1013
1014 pub fn sequences_array(&mut self, array: ArrayRef) -> Result<&mut Self> {
1016 ensure!(
1017 *array.data_type() == arrow::datatypes::DataType::UInt64,
1018 InvalidBatchSnafu {
1019 reason: "sequence array is not UInt64 type",
1020 }
1021 );
1022 let vector = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap());
1024 self.sequences = Some(vector);
1025
1026 Ok(self)
1027 }
1028
1029 pub fn op_types_array(&mut self, array: ArrayRef) -> Result<&mut Self> {
1031 ensure!(
1032 *array.data_type() == arrow::datatypes::DataType::UInt8,
1033 InvalidBatchSnafu {
1034 reason: "sequence array is not UInt8 type",
1035 }
1036 );
1037 let vector = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap());
1039 self.op_types = Some(vector);
1040
1041 Ok(self)
1042 }
1043
1044 pub fn build(self) -> Result<Batch> {
1046 let timestamps = self.timestamps.context(InvalidBatchSnafu {
1047 reason: "missing timestamps",
1048 })?;
1049 let sequences = self.sequences.context(InvalidBatchSnafu {
1050 reason: "missing sequences",
1051 })?;
1052 let op_types = self.op_types.context(InvalidBatchSnafu {
1053 reason: "missing op_types",
1054 })?;
1055 assert_eq!(0, timestamps.null_count());
1058 assert_eq!(0, sequences.null_count());
1059 assert_eq!(0, op_types.null_count());
1060
1061 let ts_len = timestamps.len();
1062 ensure!(
1063 sequences.len() == ts_len,
1064 InvalidBatchSnafu {
1065 reason: format!(
1066 "sequence have different len {} != {}",
1067 sequences.len(),
1068 ts_len
1069 ),
1070 }
1071 );
1072 ensure!(
1073 op_types.len() == ts_len,
1074 InvalidBatchSnafu {
1075 reason: format!(
1076 "op type have different len {} != {}",
1077 op_types.len(),
1078 ts_len
1079 ),
1080 }
1081 );
1082 for column in &self.fields {
1083 ensure!(
1084 column.data.len() == ts_len,
1085 InvalidBatchSnafu {
1086 reason: format!(
1087 "column {} has different len {} != {}",
1088 column.column_id,
1089 column.data.len(),
1090 ts_len
1091 ),
1092 }
1093 );
1094 }
1095
1096 Ok(Batch {
1097 primary_key: self.primary_key,
1098 pk_values: None,
1099 timestamps,
1100 sequences,
1101 op_types,
1102 fields: self.fields,
1103 fields_idx: None,
1104 })
1105 }
1106}
1107
1108impl From<Batch> for BatchBuilder {
1109 fn from(batch: Batch) -> Self {
1110 Self {
1111 primary_key: batch.primary_key,
1112 timestamps: Some(batch.timestamps),
1113 sequences: Some(batch.sequences),
1114 op_types: Some(batch.op_types),
1115 fields: batch.fields,
1116 }
1117 }
1118}
1119
1120pub enum Source {
1124 Reader(BoxedBatchReader),
1126 Iter(BoxedBatchIterator),
1128 Stream(BoxedBatchStream),
1130}
1131
1132impl Source {
1133 pub async fn next_batch(&mut self) -> Result<Option<Batch>> {
1135 match self {
1136 Source::Reader(reader) => reader.next_batch().await,
1137 Source::Iter(iter) => iter.next().transpose(),
1138 Source::Stream(stream) => stream.try_next().await,
1139 }
1140 }
1141}
1142
1143pub struct FlatSource {
1145 schema: SchemaRef,
1146 inner: FlatSourceInner,
1147}
1148
1149impl FlatSource {
1150 pub fn new_iter(schema: SchemaRef, iter: BoxedRecordBatchIterator) -> Self {
1152 Self {
1153 schema,
1154 inner: FlatSourceInner::Iter(iter),
1155 }
1156 }
1157
1158 pub fn new_stream(schema: SchemaRef, stream: BoxedRecordBatchStream) -> Self {
1160 Self {
1161 schema,
1162 inner: FlatSourceInner::Stream(stream),
1163 }
1164 }
1165
1166 #[expect(unused)]
1167 fn schema(&self) -> &SchemaRef {
1168 &self.schema
1169 }
1170
1171 pub async fn next_batch(&mut self) -> Result<Option<RecordBatch>> {
1172 self.inner.next_batch().await
1173 }
1174
1175 #[cfg(test)]
1176 pub(crate) fn take_iter(self) -> BoxedRecordBatchIterator {
1177 match self.inner {
1178 FlatSourceInner::Iter(iter) => iter,
1179 FlatSourceInner::Stream(_) => unreachable!(),
1180 }
1181 }
1182}
1183
1184enum FlatSourceInner {
1185 Iter(BoxedRecordBatchIterator),
1187 Stream(BoxedRecordBatchStream),
1189}
1190
1191impl FlatSourceInner {
1192 pub async fn next_batch(&mut self) -> Result<Option<RecordBatch>> {
1194 match self {
1195 Self::Iter(iter) => iter.next().transpose(),
1196 Self::Stream(stream) => stream.try_next().await,
1197 }
1198 }
1199}
1200
1201#[async_trait]
1205pub trait BatchReader: Send {
1206 async fn next_batch(&mut self) -> Result<Option<Batch>>;
1214}
1215
1216pub type BoxedBatchReader = Box<dyn BatchReader>;
1218
1219pub type BoxedBatchStream = BoxStream<'static, Result<Batch>>;
1221
1222pub type BoxedRecordBatchStream = BoxStream<'static, Result<RecordBatch>>;
1224
1225#[async_trait::async_trait]
1226impl<T: BatchReader + ?Sized> BatchReader for Box<T> {
1227 async fn next_batch(&mut self) -> Result<Option<Batch>> {
1228 (**self).next_batch().await
1229 }
1230}
1231
1232#[derive(Debug, Default)]
1234pub(crate) struct ScannerMetrics {
1235 scan_cost: Duration,
1237 yield_cost: Duration,
1239 num_batches: usize,
1241 num_rows: usize,
1243}
1244
1245#[cfg(test)]
1246mod tests {
1247 use datatypes::arrow::array::{TimestampMillisecondArray, UInt8Array, UInt64Array};
1248 use mito_codec::row_converter::{self, build_primary_key_codec_with_fields};
1249 use store_api::codec::PrimaryKeyEncoding;
1250 use store_api::storage::consts::ReservedColumnId;
1251
1252 use super::*;
1253 use crate::error::Error;
1254 use crate::test_util::new_batch_builder;
1255
1256 fn new_batch(
1257 timestamps: &[i64],
1258 sequences: &[u64],
1259 op_types: &[OpType],
1260 field: &[u64],
1261 ) -> Batch {
1262 new_batch_builder(b"test", timestamps, sequences, op_types, 1, field)
1263 .build()
1264 .unwrap()
1265 }
1266
1267 fn new_batch_with_u64_fields(
1268 timestamps: &[i64],
1269 sequences: &[u64],
1270 op_types: &[OpType],
1271 fields: &[(ColumnId, &[Option<u64>])],
1272 ) -> Batch {
1273 assert_eq!(timestamps.len(), sequences.len());
1274 assert_eq!(timestamps.len(), op_types.len());
1275 for (_, values) in fields {
1276 assert_eq!(timestamps.len(), values.len());
1277 }
1278
1279 let mut builder = BatchBuilder::new(b"test".to_vec());
1280 builder
1281 .timestamps_array(Arc::new(TimestampMillisecondArray::from_iter_values(
1282 timestamps.iter().copied(),
1283 )))
1284 .unwrap()
1285 .sequences_array(Arc::new(UInt64Array::from_iter_values(
1286 sequences.iter().copied(),
1287 )))
1288 .unwrap()
1289 .op_types_array(Arc::new(UInt8Array::from_iter_values(
1290 op_types.iter().map(|v| *v as u8),
1291 )))
1292 .unwrap();
1293
1294 for (col_id, values) in fields {
1295 builder
1296 .push_field_array(*col_id, Arc::new(UInt64Array::from(values.to_vec())))
1297 .unwrap();
1298 }
1299
1300 builder.build().unwrap()
1301 }
1302
1303 fn new_batch_without_fields(
1304 timestamps: &[i64],
1305 sequences: &[u64],
1306 op_types: &[OpType],
1307 ) -> Batch {
1308 assert_eq!(timestamps.len(), sequences.len());
1309 assert_eq!(timestamps.len(), op_types.len());
1310
1311 let mut builder = BatchBuilder::new(b"test".to_vec());
1312 builder
1313 .timestamps_array(Arc::new(TimestampMillisecondArray::from_iter_values(
1314 timestamps.iter().copied(),
1315 )))
1316 .unwrap()
1317 .sequences_array(Arc::new(UInt64Array::from_iter_values(
1318 sequences.iter().copied(),
1319 )))
1320 .unwrap()
1321 .op_types_array(Arc::new(UInt8Array::from_iter_values(
1322 op_types.iter().map(|v| *v as u8),
1323 )))
1324 .unwrap();
1325
1326 builder.build().unwrap()
1327 }
1328
1329 #[test]
1330 fn test_empty_batch() {
1331 let batch = Batch::empty();
1332 assert!(batch.is_empty());
1333 assert_eq!(None, batch.first_timestamp());
1334 assert_eq!(None, batch.last_timestamp());
1335 assert_eq!(None, batch.first_sequence());
1336 assert_eq!(None, batch.last_sequence());
1337 assert!(batch.timestamps_native().is_none());
1338 }
1339
1340 #[test]
1341 fn test_first_last_one() {
1342 let batch = new_batch(&[1], &[2], &[OpType::Put], &[4]);
1343 assert_eq!(
1344 Timestamp::new_millisecond(1),
1345 batch.first_timestamp().unwrap()
1346 );
1347 assert_eq!(
1348 Timestamp::new_millisecond(1),
1349 batch.last_timestamp().unwrap()
1350 );
1351 assert_eq!(2, batch.first_sequence().unwrap());
1352 assert_eq!(2, batch.last_sequence().unwrap());
1353 }
1354
1355 #[test]
1356 fn test_first_last_multiple() {
1357 let batch = new_batch(
1358 &[1, 2, 3],
1359 &[11, 12, 13],
1360 &[OpType::Put, OpType::Put, OpType::Put],
1361 &[21, 22, 23],
1362 );
1363 assert_eq!(
1364 Timestamp::new_millisecond(1),
1365 batch.first_timestamp().unwrap()
1366 );
1367 assert_eq!(
1368 Timestamp::new_millisecond(3),
1369 batch.last_timestamp().unwrap()
1370 );
1371 assert_eq!(11, batch.first_sequence().unwrap());
1372 assert_eq!(13, batch.last_sequence().unwrap());
1373 }
1374
1375 #[test]
1376 fn test_slice() {
1377 let batch = new_batch(
1378 &[1, 2, 3, 4],
1379 &[11, 12, 13, 14],
1380 &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1381 &[21, 22, 23, 24],
1382 );
1383 let batch = batch.slice(1, 2);
1384 let expect = new_batch(
1385 &[2, 3],
1386 &[12, 13],
1387 &[OpType::Delete, OpType::Put],
1388 &[22, 23],
1389 );
1390 assert_eq!(expect, batch);
1391 }
1392
1393 #[test]
1394 fn test_timestamps_native() {
1395 let batch = new_batch(
1396 &[1, 2, 3, 4],
1397 &[11, 12, 13, 14],
1398 &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1399 &[21, 22, 23, 24],
1400 );
1401 assert_eq!(&[1, 2, 3, 4], batch.timestamps_native().unwrap());
1402 }
1403
1404 #[test]
1405 fn test_concat_empty() {
1406 let err = Batch::concat(vec![]).unwrap_err();
1407 assert!(
1408 matches!(err, Error::InvalidBatch { .. }),
1409 "unexpected err: {err}"
1410 );
1411 }
1412
1413 #[test]
1414 fn test_concat_one() {
1415 let batch = new_batch(&[], &[], &[], &[]);
1416 let actual = Batch::concat(vec![batch.clone()]).unwrap();
1417 assert_eq!(batch, actual);
1418
1419 let batch = new_batch(&[1, 2], &[11, 12], &[OpType::Put, OpType::Put], &[21, 22]);
1420 let actual = Batch::concat(vec![batch.clone()]).unwrap();
1421 assert_eq!(batch, actual);
1422 }
1423
1424 #[test]
1425 fn test_concat_multiple() {
1426 let batches = vec![
1427 new_batch(&[1, 2], &[11, 12], &[OpType::Put, OpType::Put], &[21, 22]),
1428 new_batch(
1429 &[3, 4, 5],
1430 &[13, 14, 15],
1431 &[OpType::Put, OpType::Delete, OpType::Put],
1432 &[23, 24, 25],
1433 ),
1434 new_batch(&[], &[], &[], &[]),
1435 new_batch(&[6], &[16], &[OpType::Put], &[26]),
1436 ];
1437 let batch = Batch::concat(batches).unwrap();
1438 let expect = new_batch(
1439 &[1, 2, 3, 4, 5, 6],
1440 &[11, 12, 13, 14, 15, 16],
1441 &[
1442 OpType::Put,
1443 OpType::Put,
1444 OpType::Put,
1445 OpType::Delete,
1446 OpType::Put,
1447 OpType::Put,
1448 ],
1449 &[21, 22, 23, 24, 25, 26],
1450 );
1451 assert_eq!(expect, batch);
1452 }
1453
1454 #[test]
1455 fn test_concat_different() {
1456 let batch1 = new_batch(&[1], &[1], &[OpType::Put], &[1]);
1457 let mut batch2 = new_batch(&[2], &[2], &[OpType::Put], &[2]);
1458 batch2.primary_key = b"hello".to_vec();
1459 let err = Batch::concat(vec![batch1, batch2]).unwrap_err();
1460 assert!(
1461 matches!(err, Error::InvalidBatch { .. }),
1462 "unexpected err: {err}"
1463 );
1464 }
1465
1466 #[test]
1467 fn test_concat_different_fields() {
1468 let batch1 = new_batch(&[1], &[1], &[OpType::Put], &[1]);
1469 let fields = vec![
1470 batch1.fields()[0].clone(),
1471 BatchColumn {
1472 column_id: 2,
1473 data: Arc::new(UInt64Vector::from_slice([2])),
1474 },
1475 ];
1476 let batch2 = batch1.clone().with_fields(fields).unwrap();
1478 let err = Batch::concat(vec![batch1.clone(), batch2]).unwrap_err();
1479 assert!(
1480 matches!(err, Error::InvalidBatch { .. }),
1481 "unexpected err: {err}"
1482 );
1483
1484 let fields = vec![BatchColumn {
1486 column_id: 2,
1487 data: Arc::new(UInt64Vector::from_slice([2])),
1488 }];
1489 let batch2 = batch1.clone().with_fields(fields).unwrap();
1490 let err = Batch::concat(vec![batch1, batch2]).unwrap_err();
1491 assert!(
1492 matches!(err, Error::InvalidBatch { .. }),
1493 "unexpected err: {err}"
1494 );
1495 }
1496
1497 #[test]
1498 fn test_filter_deleted_empty() {
1499 let mut batch = new_batch(&[], &[], &[], &[]);
1500 batch.filter_deleted().unwrap();
1501 assert!(batch.is_empty());
1502 }
1503
1504 #[test]
1505 fn test_filter_deleted() {
1506 let mut batch = new_batch(
1507 &[1, 2, 3, 4],
1508 &[11, 12, 13, 14],
1509 &[OpType::Delete, OpType::Put, OpType::Delete, OpType::Put],
1510 &[21, 22, 23, 24],
1511 );
1512 batch.filter_deleted().unwrap();
1513 let expect = new_batch(&[2, 4], &[12, 14], &[OpType::Put, OpType::Put], &[22, 24]);
1514 assert_eq!(expect, batch);
1515
1516 let mut batch = new_batch(
1517 &[1, 2, 3, 4],
1518 &[11, 12, 13, 14],
1519 &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1520 &[21, 22, 23, 24],
1521 );
1522 let expect = batch.clone();
1523 batch.filter_deleted().unwrap();
1524 assert_eq!(expect, batch);
1525 }
1526
1527 #[test]
1528 fn test_filter_by_sequence() {
1529 let mut batch = new_batch(
1531 &[1, 2, 3, 4],
1532 &[11, 12, 13, 14],
1533 &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1534 &[21, 22, 23, 24],
1535 );
1536 batch
1537 .filter_by_sequence(Some(SequenceRange::LtEq { max: 13 }))
1538 .unwrap();
1539 let expect = new_batch(
1540 &[1, 2, 3],
1541 &[11, 12, 13],
1542 &[OpType::Put, OpType::Put, OpType::Put],
1543 &[21, 22, 23],
1544 );
1545 assert_eq!(expect, batch);
1546
1547 let mut batch = new_batch(
1549 &[1, 2, 3, 4],
1550 &[11, 12, 13, 14],
1551 &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1552 &[21, 22, 23, 24],
1553 );
1554
1555 batch
1556 .filter_by_sequence(Some(SequenceRange::LtEq { max: 10 }))
1557 .unwrap();
1558 assert!(batch.is_empty());
1559
1560 let mut batch = new_batch(
1562 &[1, 2, 3, 4],
1563 &[11, 12, 13, 14],
1564 &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1565 &[21, 22, 23, 24],
1566 );
1567 let expect = batch.clone();
1568 batch.filter_by_sequence(None).unwrap();
1569 assert_eq!(expect, batch);
1570
1571 let mut batch = new_batch(&[], &[], &[], &[]);
1573 batch
1574 .filter_by_sequence(Some(SequenceRange::LtEq { max: 10 }))
1575 .unwrap();
1576 assert!(batch.is_empty());
1577
1578 let mut batch = new_batch(&[], &[], &[], &[]);
1580 batch.filter_by_sequence(None).unwrap();
1581 assert!(batch.is_empty());
1582
1583 let mut batch = new_batch(
1585 &[1, 2, 3, 4],
1586 &[11, 12, 13, 14],
1587 &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1588 &[21, 22, 23, 24],
1589 );
1590 batch
1591 .filter_by_sequence(Some(SequenceRange::Gt { min: 12 }))
1592 .unwrap();
1593 let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1594 assert_eq!(expect, batch);
1595
1596 let mut batch = new_batch(
1598 &[1, 2, 3, 4],
1599 &[11, 12, 13, 14],
1600 &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1601 &[21, 22, 23, 24],
1602 );
1603 batch
1604 .filter_by_sequence(Some(SequenceRange::Gt { min: 20 }))
1605 .unwrap();
1606 assert!(batch.is_empty());
1607
1608 let mut batch = new_batch(
1610 &[1, 2, 3, 4, 5],
1611 &[11, 12, 13, 14, 15],
1612 &[
1613 OpType::Put,
1614 OpType::Put,
1615 OpType::Put,
1616 OpType::Put,
1617 OpType::Put,
1618 ],
1619 &[21, 22, 23, 24, 25],
1620 );
1621 batch
1622 .filter_by_sequence(Some(SequenceRange::GtLtEq { min: 12, max: 14 }))
1623 .unwrap();
1624 let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1625 assert_eq!(expect, batch);
1626
1627 let mut batch = new_batch(
1629 &[1, 2, 3, 4, 5],
1630 &[11, 12, 13, 14, 15],
1631 &[
1632 OpType::Put,
1633 OpType::Delete,
1634 OpType::Put,
1635 OpType::Delete,
1636 OpType::Put,
1637 ],
1638 &[21, 22, 23, 24, 25],
1639 );
1640 batch
1641 .filter_by_sequence(Some(SequenceRange::GtLtEq { min: 11, max: 13 }))
1642 .unwrap();
1643 let expect = new_batch(
1644 &[2, 3],
1645 &[12, 13],
1646 &[OpType::Delete, OpType::Put],
1647 &[22, 23],
1648 );
1649 assert_eq!(expect, batch);
1650
1651 let mut batch = new_batch(
1653 &[1, 2, 3, 4],
1654 &[11, 12, 13, 14],
1655 &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1656 &[21, 22, 23, 24],
1657 );
1658 batch
1659 .filter_by_sequence(Some(SequenceRange::GtLtEq { min: 20, max: 25 }))
1660 .unwrap();
1661 assert!(batch.is_empty());
1662 }
1663
1664 #[test]
1665 fn test_merge_last_non_null_no_dup() {
1666 let mut batch = new_batch_with_u64_fields(
1667 &[1, 2],
1668 &[2, 1],
1669 &[OpType::Put, OpType::Put],
1670 &[(1, &[Some(10), None]), (2, &[Some(100), Some(200)])],
1671 );
1672 let expect = batch.clone();
1673 batch.merge_last_non_null().unwrap();
1674 assert_eq!(expect, batch);
1675 }
1676
1677 #[test]
1678 fn test_merge_last_non_null_fill_null_fields() {
1679 let mut batch = new_batch_with_u64_fields(
1681 &[1, 1, 1],
1682 &[3, 2, 1],
1683 &[OpType::Put, OpType::Put, OpType::Put],
1684 &[
1685 (1, &[None, Some(10), Some(11)]),
1686 (2, &[Some(100), Some(200), Some(300)]),
1687 ],
1688 );
1689 batch.merge_last_non_null().unwrap();
1690
1691 let expect = new_batch_with_u64_fields(
1694 &[1],
1695 &[3],
1696 &[OpType::Put],
1697 &[(1, &[Some(10)]), (2, &[Some(100)])],
1698 );
1699 assert_eq!(expect, batch);
1700 }
1701
1702 #[test]
1703 fn test_merge_last_non_null_stop_at_delete_row() {
1704 let mut batch = new_batch_with_u64_fields(
1707 &[1, 1, 1],
1708 &[3, 2, 1],
1709 &[OpType::Put, OpType::Delete, OpType::Put],
1710 &[
1711 (1, &[None, Some(10), Some(11)]),
1712 (2, &[Some(100), Some(200), Some(300)]),
1713 ],
1714 );
1715 batch.merge_last_non_null().unwrap();
1716
1717 let expect = new_batch_with_u64_fields(
1718 &[1],
1719 &[3],
1720 &[OpType::Put],
1721 &[(1, &[None]), (2, &[Some(100)])],
1722 );
1723 assert_eq!(expect, batch);
1724 }
1725
1726 #[test]
1727 fn test_merge_last_non_null_base_delete_no_merge() {
1728 let mut batch = new_batch_with_u64_fields(
1729 &[1, 1],
1730 &[3, 2],
1731 &[OpType::Delete, OpType::Put],
1732 &[(1, &[None, Some(10)]), (2, &[None, Some(200)])],
1733 );
1734 batch.merge_last_non_null().unwrap();
1735
1736 let expect =
1738 new_batch_with_u64_fields(&[1], &[3], &[OpType::Delete], &[(1, &[None]), (2, &[None])]);
1739 assert_eq!(expect, batch);
1740 }
1741
1742 #[test]
1743 fn test_merge_last_non_null_multiple_timestamp_groups() {
1744 let mut batch = new_batch_with_u64_fields(
1745 &[1, 1, 2, 3, 3],
1746 &[5, 4, 3, 2, 1],
1747 &[
1748 OpType::Put,
1749 OpType::Put,
1750 OpType::Put,
1751 OpType::Put,
1752 OpType::Put,
1753 ],
1754 &[
1755 (1, &[None, Some(10), Some(20), None, Some(30)]),
1756 (2, &[Some(100), Some(110), Some(120), None, Some(130)]),
1757 ],
1758 );
1759 batch.merge_last_non_null().unwrap();
1760
1761 let expect = new_batch_with_u64_fields(
1762 &[1, 2, 3],
1763 &[5, 3, 2],
1764 &[OpType::Put, OpType::Put, OpType::Put],
1765 &[
1766 (1, &[Some(10), Some(20), Some(30)]),
1767 (2, &[Some(100), Some(120), Some(130)]),
1768 ],
1769 );
1770 assert_eq!(expect, batch);
1771 }
1772
1773 #[test]
1774 fn test_merge_last_non_null_no_fields() {
1775 let mut batch = new_batch_without_fields(
1776 &[1, 1, 2],
1777 &[3, 2, 1],
1778 &[OpType::Put, OpType::Put, OpType::Put],
1779 );
1780 batch.merge_last_non_null().unwrap();
1781
1782 let expect = new_batch_without_fields(&[1, 2], &[3, 1], &[OpType::Put, OpType::Put]);
1783 assert_eq!(expect, batch);
1784 }
1785
1786 #[test]
1787 fn test_filter() {
1788 let mut batch = new_batch(
1790 &[1, 2, 3, 4],
1791 &[11, 12, 13, 14],
1792 &[OpType::Put, OpType::Put, OpType::Put, OpType::Put],
1793 &[21, 22, 23, 24],
1794 );
1795 let predicate = BooleanVector::from_vec(vec![false, false, true, true]);
1796 batch.filter(&predicate).unwrap();
1797 let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1798 assert_eq!(expect, batch);
1799
1800 let mut batch = new_batch(
1802 &[1, 2, 3, 4],
1803 &[11, 12, 13, 14],
1804 &[OpType::Put, OpType::Delete, OpType::Put, OpType::Put],
1805 &[21, 22, 23, 24],
1806 );
1807 let predicate = BooleanVector::from_vec(vec![false, false, true, true]);
1808 batch.filter(&predicate).unwrap();
1809 let expect = new_batch(&[3, 4], &[13, 14], &[OpType::Put, OpType::Put], &[23, 24]);
1810 assert_eq!(expect, batch);
1811
1812 let predicate = BooleanVector::from_vec(vec![false, false]);
1814 batch.filter(&predicate).unwrap();
1815 assert!(batch.is_empty());
1816 }
1817
1818 #[test]
1819 fn test_sort_and_dedup() {
1820 let original = new_batch(
1821 &[2, 3, 1, 4, 5, 2],
1822 &[1, 2, 3, 4, 5, 6],
1823 &[
1824 OpType::Put,
1825 OpType::Put,
1826 OpType::Put,
1827 OpType::Put,
1828 OpType::Put,
1829 OpType::Put,
1830 ],
1831 &[21, 22, 23, 24, 25, 26],
1832 );
1833
1834 let mut batch = original.clone();
1835 batch.sort(true).unwrap();
1836 assert_eq!(
1838 new_batch(
1839 &[1, 2, 3, 4, 5],
1840 &[3, 6, 2, 4, 5],
1841 &[
1842 OpType::Put,
1843 OpType::Put,
1844 OpType::Put,
1845 OpType::Put,
1846 OpType::Put,
1847 ],
1848 &[23, 26, 22, 24, 25],
1849 ),
1850 batch
1851 );
1852
1853 let mut batch = original.clone();
1854 batch.sort(false).unwrap();
1855
1856 assert_eq!(
1858 new_batch(
1859 &[1, 2, 2, 3, 4, 5],
1860 &[3, 6, 1, 2, 4, 5],
1861 &[
1862 OpType::Put,
1863 OpType::Put,
1864 OpType::Put,
1865 OpType::Put,
1866 OpType::Put,
1867 OpType::Put,
1868 ],
1869 &[23, 26, 21, 22, 24, 25],
1870 ),
1871 batch
1872 );
1873
1874 let original = new_batch(
1875 &[2, 2, 1],
1876 &[1, 6, 1],
1877 &[OpType::Delete, OpType::Put, OpType::Put],
1878 &[21, 22, 23],
1879 );
1880
1881 let mut batch = original.clone();
1882 batch.sort(true).unwrap();
1883 let expect = new_batch(&[1, 2], &[1, 6], &[OpType::Put, OpType::Put], &[23, 22]);
1884 assert_eq!(expect, batch);
1885
1886 let mut batch = original.clone();
1887 batch.sort(false).unwrap();
1888 let expect = new_batch(
1889 &[1, 2, 2],
1890 &[1, 6, 1],
1891 &[OpType::Put, OpType::Put, OpType::Delete],
1892 &[23, 22, 21],
1893 );
1894 assert_eq!(expect, batch);
1895 }
1896
1897 #[test]
1898 fn test_get_value() {
1899 let encodings = [PrimaryKeyEncoding::Dense, PrimaryKeyEncoding::Sparse];
1900
1901 for encoding in encodings {
1902 let codec = build_primary_key_codec_with_fields(
1903 encoding,
1904 [
1905 (
1906 ReservedColumnId::table_id(),
1907 row_converter::SortField::new(ConcreteDataType::uint32_datatype()),
1908 ),
1909 (
1910 ReservedColumnId::tsid(),
1911 row_converter::SortField::new(ConcreteDataType::uint64_datatype()),
1912 ),
1913 (
1914 100,
1915 row_converter::SortField::new(ConcreteDataType::string_datatype()),
1916 ),
1917 (
1918 200,
1919 row_converter::SortField::new(ConcreteDataType::string_datatype()),
1920 ),
1921 ]
1922 .into_iter(),
1923 );
1924
1925 let values = [
1926 Value::UInt32(1000),
1927 Value::UInt64(2000),
1928 Value::String("abcdefgh".into()),
1929 Value::String("zyxwvu".into()),
1930 ];
1931 let mut buf = vec![];
1932 codec
1933 .encode_values(
1934 &[
1935 (ReservedColumnId::table_id(), values[0].clone()),
1936 (ReservedColumnId::tsid(), values[1].clone()),
1937 (100, values[2].clone()),
1938 (200, values[3].clone()),
1939 ],
1940 &mut buf,
1941 )
1942 .unwrap();
1943
1944 let field_col_id = 2;
1945 let mut batch = new_batch_builder(
1946 &buf,
1947 &[1, 2, 3],
1948 &[1, 1, 1],
1949 &[OpType::Put, OpType::Put, OpType::Put],
1950 field_col_id,
1951 &[42, 43, 44],
1952 )
1953 .build()
1954 .unwrap();
1955
1956 let v = batch
1957 .pk_col_value(&*codec, 0, ReservedColumnId::table_id())
1958 .unwrap()
1959 .unwrap();
1960 assert_eq!(values[0], *v);
1961
1962 let v = batch
1963 .pk_col_value(&*codec, 1, ReservedColumnId::tsid())
1964 .unwrap()
1965 .unwrap();
1966 assert_eq!(values[1], *v);
1967
1968 let v = batch.pk_col_value(&*codec, 2, 100).unwrap().unwrap();
1969 assert_eq!(values[2], *v);
1970
1971 let v = batch.pk_col_value(&*codec, 3, 200).unwrap().unwrap();
1972 assert_eq!(values[3], *v);
1973
1974 let v = batch.field_col_value(field_col_id).unwrap();
1975 assert_eq!(v.data.get(0), Value::UInt64(42));
1976 assert_eq!(v.data.get(1), Value::UInt64(43));
1977 assert_eq!(v.data.get(2), Value::UInt64(44));
1978 }
1979 }
1980}