risingwave_common/array/
stream_chunk.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::fmt::Display;
use std::mem::size_of;
use std::ops::{Deref, DerefMut};
use std::sync::Arc;
use std::{fmt, mem};

use either::Either;
use enum_as_inner::EnumAsInner;
use itertools::Itertools;
use rand::prelude::SmallRng;
use rand::{Rng, SeedableRng};
use risingwave_common_estimate_size::EstimateSize;
use risingwave_pb::data::{PbOp, PbStreamChunk};

use super::stream_chunk_builder::StreamChunkBuilder;
use super::{ArrayImpl, ArrayRef, ArrayResult, DataChunkTestExt, RowRef};
use crate::array::DataChunk;
use crate::bitmap::{Bitmap, BitmapBuilder};
use crate::catalog::Schema;
use crate::field_generator::VarcharProperty;
use crate::row::Row;
use crate::types::{DataType, DefaultOrdered, ToText};

/// `Op` represents three operations in `StreamChunk`.
///
/// `UpdateDelete` and `UpdateInsert` are semantically equivalent to `Delete` and `Insert`
/// but always appear in pairs to represent an update operation.
/// For example, table source, aggregation and outer join can generate updates by themselves,
/// while most of the other operators only pass through updates with best effort.
#[derive(Clone, Copy, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, EnumAsInner)]
pub enum Op {
    Insert,
    Delete,
    UpdateDelete,
    UpdateInsert,
}

impl Op {
    pub fn to_protobuf(self) -> PbOp {
        match self {
            Op::Insert => PbOp::Insert,
            Op::Delete => PbOp::Delete,
            Op::UpdateInsert => PbOp::UpdateInsert,
            Op::UpdateDelete => PbOp::UpdateDelete,
        }
    }

    pub fn from_protobuf(prost: &i32) -> ArrayResult<Op> {
        let op = match PbOp::try_from(*prost) {
            Ok(PbOp::Insert) => Op::Insert,
            Ok(PbOp::Delete) => Op::Delete,
            Ok(PbOp::UpdateInsert) => Op::UpdateInsert,
            Ok(PbOp::UpdateDelete) => Op::UpdateDelete,
            Ok(PbOp::Unspecified) => unreachable!(),
            Err(_) => bail!("No such op type"),
        };
        Ok(op)
    }

    /// convert `UpdateDelete` to `Delete` and `UpdateInsert` to Insert
    pub fn normalize_update(self) -> Op {
        match self {
            Op::Insert => Op::Insert,
            Op::Delete => Op::Delete,
            Op::UpdateDelete => Op::Delete,
            Op::UpdateInsert => Op::Insert,
        }
    }

    pub fn to_i16(self) -> i16 {
        match self {
            Op::Insert => 1,
            Op::Delete => 2,
            Op::UpdateInsert => 3,
            Op::UpdateDelete => 4,
        }
    }

    pub fn to_varchar(self) -> String {
        match self {
            Op::Insert => "Insert",
            Op::Delete => "Delete",
            Op::UpdateInsert => "UpdateInsert",
            Op::UpdateDelete => "UpdateDelete",
        }
        .to_string()
    }
}

pub type Ops<'a> = &'a [Op];

/// `StreamChunk` is used to pass data over the streaming pathway.
#[derive(Clone, PartialEq)]
pub struct StreamChunk {
    // TODO: Optimize using bitmap
    ops: Arc<[Op]>,
    data: DataChunk,
}

impl Default for StreamChunk {
    /// Create a 0-row-0-col `StreamChunk`. Only used in some existing tests.
    /// This is NOT the same as an **empty** chunk, which has 0 rows but with
    /// columns aligned with executor schema.
    fn default() -> Self {
        Self {
            ops: Arc::new([]),
            data: DataChunk::new(vec![], 0),
        }
    }
}

impl StreamChunk {
    /// Create a new `StreamChunk` with given ops and columns.
    pub fn new(ops: impl Into<Arc<[Op]>>, columns: Vec<ArrayRef>) -> Self {
        let ops = ops.into();
        let visibility = Bitmap::ones(ops.len());
        Self::with_visibility(ops, columns, visibility)
    }

    /// Create a new `StreamChunk` with given ops, columns and visibility.
    pub fn with_visibility(
        ops: impl Into<Arc<[Op]>>,
        columns: Vec<ArrayRef>,
        visibility: Bitmap,
    ) -> Self {
        let ops = ops.into();
        for col in &columns {
            assert_eq!(col.len(), ops.len());
        }
        let data = DataChunk::new(columns, visibility);
        StreamChunk { ops, data }
    }

    /// Build a `StreamChunk` from rows.
    ///
    /// Panics if the `rows` is empty.
    ///
    /// Should prefer using [`StreamChunkBuilder`] instead to avoid unnecessary
    /// allocation of rows.
    pub fn from_rows(rows: &[(Op, impl Row)], data_types: &[DataType]) -> Self {
        let mut builder = StreamChunkBuilder::unlimited(data_types.to_vec(), Some(rows.len()));

        for (op, row) in rows {
            let none = builder.append_row(*op, row);
            debug_assert!(none.is_none());
        }

        builder.take().expect("chunk should not be empty")
    }

    /// Get the reference of the underlying data chunk.
    pub fn data_chunk(&self) -> &DataChunk {
        &self.data
    }

    /// compact the `StreamChunk` with its visibility map
    pub fn compact(self) -> Self {
        if self.is_compacted() {
            return self;
        }

        let (ops, columns, visibility) = self.into_inner();

        let cardinality = visibility
            .iter()
            .fold(0, |vis_cnt, vis| vis_cnt + vis as usize);
        let columns: Vec<_> = columns
            .into_iter()
            .map(|col| col.compact(&visibility, cardinality).into())
            .collect();
        let mut new_ops = Vec::with_capacity(cardinality);
        for idx in visibility.iter_ones() {
            new_ops.push(ops[idx]);
        }
        StreamChunk::new(new_ops, columns)
    }

    /// Split the `StreamChunk` into multiple chunks with the given size at most.
    ///
    /// When the total cardinality of all the chunks is not evenly divided by the `size`,
    /// the last new chunk will be the remainder.
    ///
    /// For consecutive `UpdateDelete` and `UpdateInsert`, they will be kept in one chunk.
    /// As a result, some chunks may have `size + 1` rows.
    pub fn split(&self, size: usize) -> Vec<Self> {
        let mut builder = StreamChunkBuilder::new(size, self.data_types());
        let mut outputs = Vec::new();

        // TODO: directly append the chunk.
        for (op, row) in self.rows() {
            if let Some(chunk) = builder.append_row(op, row) {
                outputs.push(chunk);
            }
        }
        if let Some(output) = builder.take() {
            outputs.push(output);
        }

        outputs
    }

    pub fn into_parts(self) -> (DataChunk, Arc<[Op]>) {
        (self.data, self.ops)
    }

    pub fn from_parts(ops: impl Into<Arc<[Op]>>, data_chunk: DataChunk) -> Self {
        let (columns, vis) = data_chunk.into_parts();
        Self::with_visibility(ops, columns, vis)
    }

    pub fn into_inner(self) -> (Arc<[Op]>, Vec<ArrayRef>, Bitmap) {
        let (columns, vis) = self.data.into_parts();
        (self.ops, columns, vis)
    }

    pub fn to_protobuf(&self) -> PbStreamChunk {
        if !self.is_compacted() {
            return self.clone().compact().to_protobuf();
        }
        PbStreamChunk {
            cardinality: self.cardinality() as u32,
            ops: self.ops.iter().map(|op| op.to_protobuf() as i32).collect(),
            columns: self.columns().iter().map(|col| col.to_protobuf()).collect(),
        }
    }

    pub fn from_protobuf(prost: &PbStreamChunk) -> ArrayResult<Self> {
        let cardinality = prost.get_cardinality() as usize;
        let mut ops = Vec::with_capacity(cardinality);
        for op in prost.get_ops() {
            ops.push(Op::from_protobuf(op)?);
        }
        let mut columns = vec![];
        for column in prost.get_columns() {
            columns.push(ArrayImpl::from_protobuf(column, cardinality)?.into());
        }
        Ok(StreamChunk::new(ops, columns))
    }

    pub fn ops(&self) -> &[Op] {
        &self.ops
    }

    /// Returns a table-like text representation of the `StreamChunk`.
    pub fn to_pretty(&self) -> impl Display {
        self.to_pretty_inner(None)
    }

    /// Returns a table-like text representation of the `StreamChunk` with a header of column names
    /// from the given `schema`.
    pub fn to_pretty_with_schema(&self, schema: &Schema) -> impl Display {
        self.to_pretty_inner(Some(schema))
    }

    fn to_pretty_inner(&self, schema: Option<&Schema>) -> impl Display {
        use comfy_table::{Cell, CellAlignment, Table};

        if self.cardinality() == 0 {
            return Either::Left("(empty)");
        }

        let mut table = Table::new();
        table.load_preset(DataChunk::PRETTY_TABLE_PRESET);

        if let Some(schema) = schema {
            assert_eq!(self.dimension(), schema.len());
            let cells = std::iter::once(String::new())
                .chain(schema.fields().iter().map(|f| f.name.clone()));
            table.set_header(cells);
        }

        for (op, row_ref) in self.rows() {
            let mut cells = Vec::with_capacity(row_ref.len() + 1);
            cells.push(
                Cell::new(match op {
                    Op::Insert => "+",
                    Op::Delete => "-",
                    Op::UpdateDelete => "U-",
                    Op::UpdateInsert => "U+",
                })
                .set_alignment(CellAlignment::Right),
            );
            for datum in row_ref.iter() {
                let str = match datum {
                    None => "".to_owned(), // NULL
                    Some(scalar) => scalar.to_text(),
                };
                cells.push(Cell::new(str));
            }
            table.add_row(cells);
        }

        Either::Right(table)
    }

    /// Reorder (and possibly remove) columns.
    ///
    /// e.g. if `indices` is `[2, 1, 0]`, and the chunk contains column `[a, b, c]`, then the output
    /// will be `[c, b, a]`. If `indices` is [2, 0], then the output will be `[c, a]`.
    /// If the input mapping is identity mapping, no reorder will be performed.
    pub fn project(&self, indices: &[usize]) -> Self {
        Self {
            ops: self.ops.clone(),
            data: self.data.project(indices),
        }
    }

    /// Remove the adjacent delete-insert if their row value are the same.
    pub fn eliminate_adjacent_noop_update(self) -> Self {
        let len = self.data_chunk().capacity();
        let mut c: StreamChunkMut = self.into();
        let mut prev_r = None;
        for curr in 0..len {
            if !c.vis(curr) {
                continue;
            }
            if let Some(prev) = prev_r {
                if matches!(c.op(prev), Op::UpdateDelete | Op::Delete)
                    && matches!(c.op(curr), Op::UpdateInsert | Op::Insert)
                    && c.row_ref(prev) == c.row_ref(curr)
                {
                    c.set_vis(prev, false);
                    c.set_vis(curr, false);
                    prev_r = None;
                } else {
                    prev_r = Some(curr)
                }
            } else {
                prev_r = Some(curr);
            }
        }
        c.into()
    }

    /// Reorder columns and set visibility.
    pub fn project_with_vis(&self, indices: &[usize], vis: Bitmap) -> Self {
        Self {
            ops: self.ops.clone(),
            data: self.data.project_with_vis(indices, vis),
        }
    }

    /// Clone the `StreamChunk` with a new visibility.
    pub fn clone_with_vis(&self, vis: Bitmap) -> Self {
        Self {
            ops: self.ops.clone(),
            data: self.data.with_visibility(vis),
        }
    }
}

impl Deref for StreamChunk {
    type Target = DataChunk;

    fn deref(&self) -> &Self::Target {
        &self.data
    }
}

impl DerefMut for StreamChunk {
    fn deref_mut(&mut self) -> &mut Self::Target {
        &mut self.data
    }
}

/// `StreamChunk` can be created from `DataChunk` with all operations set to `Insert`.
impl From<DataChunk> for StreamChunk {
    fn from(data: DataChunk) -> Self {
        Self::from_parts(vec![Op::Insert; data.capacity()], data)
    }
}

impl fmt::Debug for StreamChunk {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        if f.alternate() {
            write!(
                f,
                "StreamChunk {{ cardinality: {}, capacity: {}, data:\n{}\n }}",
                self.cardinality(),
                self.capacity(),
                self.to_pretty()
            )
        } else {
            f.debug_struct("StreamChunk")
                .field("cardinality", &self.cardinality())
                .field("capacity", &self.capacity())
                .finish_non_exhaustive()
        }
    }
}

impl EstimateSize for StreamChunk {
    fn estimated_heap_size(&self) -> usize {
        self.data.estimated_heap_size() + self.ops.len() * size_of::<Op>()
    }
}

enum OpsMutState {
    ArcRef(Arc<[Op]>),
    Mut(Vec<Op>),
}

impl OpsMutState {
    const UNDEFINED: Self = Self::Mut(Vec::new());
}

pub struct OpsMut {
    state: OpsMutState,
}

impl OpsMut {
    pub fn new(ops: Arc<[Op]>) -> Self {
        Self {
            state: OpsMutState::ArcRef(ops),
        }
    }

    pub fn len(&self) -> usize {
        match &self.state {
            OpsMutState::ArcRef(v) => v.len(),
            OpsMutState::Mut(v) => v.len(),
        }
    }

    pub fn is_empty(&self) -> bool {
        self.len() == 0
    }

    pub fn set(&mut self, n: usize, val: Op) {
        debug_assert!(n < self.len());
        if let OpsMutState::Mut(v) = &mut self.state {
            v[n] = val;
        } else {
            let state = mem::replace(&mut self.state, OpsMutState::UNDEFINED); // intermediate state
            let mut v = match state {
                OpsMutState::ArcRef(v) => v.to_vec(),
                OpsMutState::Mut(_) => unreachable!(),
            };
            v[n] = val;
            self.state = OpsMutState::Mut(v);
        }
    }

    pub fn get(&self, n: usize) -> Op {
        debug_assert!(n < self.len());
        match &self.state {
            OpsMutState::ArcRef(v) => v[n],
            OpsMutState::Mut(v) => v[n],
        }
    }
}
impl From<OpsMut> for Arc<[Op]> {
    fn from(v: OpsMut) -> Self {
        match v.state {
            OpsMutState::ArcRef(a) => a,
            OpsMutState::Mut(v) => v.into(),
        }
    }
}

/// A mutable wrapper for `StreamChunk`. can only set the visibilities and ops in place, can not
/// change the length.
pub struct StreamChunkMut {
    columns: Arc<[ArrayRef]>,
    ops: OpsMut,
    vis: BitmapBuilder,
}

impl From<StreamChunk> for StreamChunkMut {
    fn from(c: StreamChunk) -> Self {
        let (c, ops) = c.into_parts();
        let (columns, vis) = c.into_parts_v2();
        Self {
            columns,
            ops: OpsMut::new(ops),
            vis: vis.into(),
        }
    }
}

impl From<StreamChunkMut> for StreamChunk {
    fn from(c: StreamChunkMut) -> Self {
        StreamChunk::from_parts(c.ops, DataChunk::from_parts(c.columns, c.vis.finish()))
    }
}

pub struct OpRowMutRef<'a> {
    c: &'a mut StreamChunkMut,
    i: usize,
}

impl OpRowMutRef<'_> {
    pub fn index(&self) -> usize {
        self.i
    }

    pub fn vis(&self) -> bool {
        self.c.vis.is_set(self.i)
    }

    pub fn op(&self) -> Op {
        self.c.ops.get(self.i)
    }

    pub fn set_vis(&mut self, val: bool) {
        self.c.set_vis(self.i, val);
    }

    pub fn set_op(&mut self, val: Op) {
        self.c.set_op(self.i, val);
    }

    pub fn row_ref(&self) -> RowRef<'_> {
        RowRef::with_columns(self.c.columns(), self.i)
    }

    /// return if the two row ref is in the same chunk
    pub fn same_chunk(&self, other: &Self) -> bool {
        std::ptr::eq(self.c, other.c)
    }
}

impl StreamChunkMut {
    pub fn capacity(&self) -> usize {
        self.vis.len()
    }

    pub fn vis(&self, i: usize) -> bool {
        self.vis.is_set(i)
    }

    pub fn op(&self, i: usize) -> Op {
        self.ops.get(i)
    }

    pub fn row_ref(&self, i: usize) -> RowRef<'_> {
        RowRef::with_columns(self.columns(), i)
    }

    pub fn set_vis(&mut self, n: usize, val: bool) {
        self.vis.set(n, val);
    }

    pub fn set_op(&mut self, n: usize, val: Op) {
        self.ops.set(n, val);
    }

    pub fn columns(&self) -> &[ArrayRef] {
        &self.columns
    }

    /// get the mut reference of the stream chunk.
    pub fn to_rows_mut(&mut self) -> impl Iterator<Item = (RowRef<'_>, OpRowMutRef<'_>)> {
        unsafe {
            (0..self.vis.len())
                .filter(|i| self.vis.is_set(*i))
                .map(|i| {
                    let p = self as *const StreamChunkMut;
                    let p = p as *mut StreamChunkMut;
                    (
                        RowRef::with_columns(self.columns(), i),
                        OpRowMutRef { c: &mut *p, i },
                    )
                })
        }
    }
}

/// Test utilities for [`StreamChunk`].
#[easy_ext::ext(StreamChunkTestExt)]
impl StreamChunk {
    /// Parse a chunk from string.
    ///
    /// See also [`DataChunkTestExt::from_pretty`].
    ///
    /// # Format
    ///
    /// The first line is a header indicating the column types.
    /// The following lines indicate rows within the chunk.
    /// Each line starts with an operation followed by values.
    /// NULL values are represented as `.`.
    ///
    /// # Example
    /// ```
    /// use risingwave_common::array::stream_chunk::StreamChunkTestExt as _;
    /// use risingwave_common::array::StreamChunk;
    /// let chunk = StreamChunk::from_pretty(
    ///     "  I I I I      // type chars
    ///     U- 2 5 . .      // '.' means NULL
    ///     U+ 2 5 2 6 D    // 'D' means deleted in visibility
    ///     +  . . 4 8      // ^ comments are ignored
    ///     -  . . 3 4",
    /// );
    /// //  ^ operations:
    /// //     +: Insert
    /// //     -: Delete
    /// //    U+: UpdateInsert
    /// //    U-: UpdateDelete
    ///
    /// // type chars:
    /// //     I: i64
    /// //     i: i32
    /// //     F: f64
    /// //     f: f32
    /// //     T: str
    /// //    TS: Timestamp
    /// //    TZ: Timestamptz
    /// //   SRL: Serial
    /// //   x[]: array of x
    /// // <i,f>: struct
    /// ```
    pub fn from_pretty(s: &str) -> Self {
        let mut chunk_str = String::new();
        let mut ops = vec![];

        let (header, body) = match s.split_once('\n') {
            Some(pair) => pair,
            None => {
                // empty chunk
                return StreamChunk {
                    ops: Arc::new([]),
                    data: DataChunk::from_pretty(s),
                };
            }
        };
        chunk_str.push_str(header);
        chunk_str.push('\n');

        for line in body.split_inclusive('\n') {
            if line.trim_start().is_empty() {
                continue;
            }
            let (op, row) = line
                .trim_start()
                .split_once(|c: char| c.is_ascii_whitespace())
                .ok_or_else(|| panic!("missing operation: {line:?}"))
                .unwrap();
            ops.push(match op {
                "+" => Op::Insert,
                "-" => Op::Delete,
                "U+" => Op::UpdateInsert,
                "U-" => Op::UpdateDelete,
                t => panic!("invalid op: {t:?}"),
            });
            chunk_str.push_str(row);
        }
        StreamChunk {
            ops: ops.into(),
            data: DataChunk::from_pretty(&chunk_str),
        }
    }

    /// Validate the `StreamChunk` layout.
    pub fn valid(&self) -> bool {
        let len = self.ops.len();
        let data = &self.data;
        data.visibility().len() == len && data.columns().iter().all(|col| col.len() == len)
    }

    /// Concatenate multiple `StreamChunk` into one.
    ///
    /// Panics if `chunks` is empty.
    pub fn concat(chunks: Vec<StreamChunk>) -> StreamChunk {
        let data_types = chunks[0].data_types();
        let size = chunks.iter().map(|c| c.cardinality()).sum::<usize>();

        let mut builder = StreamChunkBuilder::unlimited(data_types, Some(size));

        for chunk in chunks {
            // TODO: directly append chunks.
            for (op, row) in chunk.rows() {
                let none = builder.append_row(op, row);
                debug_assert!(none.is_none());
            }
        }

        builder.take().expect("chunk should not be empty")
    }

    /// Sort rows.
    pub fn sort_rows(self) -> Self {
        if self.capacity() == 0 {
            return self;
        }
        let rows = self.rows().collect_vec();
        let mut idx = (0..self.capacity()).collect_vec();
        idx.sort_by_key(|&i| {
            let (op, row_ref) = rows[i];
            (op, DefaultOrdered(row_ref))
        });
        StreamChunk {
            ops: idx.iter().map(|&i| self.ops[i]).collect(),
            data: self.data.reorder_rows(&idx),
        }
    }

    /// Generate `num_of_chunks` data chunks with type `data_types`,
    /// where each data chunk has cardinality of `chunk_size`.
    /// TODO(kwannoel): Generate different types of op, different vis.
    pub fn gen_stream_chunks(
        num_of_chunks: usize,
        chunk_size: usize,
        data_types: &[DataType],
        varchar_properties: &VarcharProperty,
    ) -> Vec<StreamChunk> {
        Self::gen_stream_chunks_inner(
            num_of_chunks,
            chunk_size,
            data_types,
            varchar_properties,
            1.0,
            1.0,
        )
    }

    pub fn gen_stream_chunks_inner(
        num_of_chunks: usize,
        chunk_size: usize,
        data_types: &[DataType],
        varchar_properties: &VarcharProperty,
        visibility_percent: f64, // % of rows that are visible
        inserts_percent: f64,    // Rest will be deletes.
    ) -> Vec<StreamChunk> {
        let ops = if inserts_percent == 0.0 {
            vec![Op::Delete; chunk_size]
        } else if inserts_percent == 1.0 {
            vec![Op::Insert; chunk_size]
        } else {
            let mut rng = SmallRng::from_seed([0; 32]);
            let mut ops = vec![];
            for _ in 0..chunk_size {
                ops.push(if rng.gen_bool(inserts_percent) {
                    Op::Insert
                } else {
                    Op::Delete
                });
            }
            ops
        };
        DataChunk::gen_data_chunks(
            num_of_chunks,
            chunk_size,
            data_types,
            varchar_properties,
            visibility_percent,
        )
        .into_iter()
        .map(|chunk| StreamChunk::from_parts(ops.clone(), chunk))
        .collect()
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    #[test]
    fn test_to_pretty_string() {
        let chunk = StreamChunk::from_pretty(
            "  I I
             + 1 6
             - 2 .
            U- 3 7
            U+ 4 .",
        );
        assert_eq!(
            chunk.to_pretty().to_string(),
            "\
+----+---+---+
|  + | 1 | 6 |
|  - | 2 |   |
| U- | 3 | 7 |
| U+ | 4 |   |
+----+---+---+"
        );
    }

    #[test]
    fn test_split_1() {
        let chunk = StreamChunk::from_pretty(
            "  I I
             + 1 6
             - 2 .
            U- 3 7
            U+ 4 .",
        );
        let results = chunk.split(2);
        assert_eq!(2, results.len());
        assert_eq!(
            results[0].to_pretty().to_string(),
            "\
+---+---+---+
| + | 1 | 6 |
| - | 2 |   |
+---+---+---+"
        );
        assert_eq!(
            results[1].to_pretty().to_string(),
            "\
+----+---+---+
| U- | 3 | 7 |
| U+ | 4 |   |
+----+---+---+"
        );
    }

    #[test]
    fn test_split_2() {
        let chunk = StreamChunk::from_pretty(
            "  I I
             + 1 6
            U- 3 7
            U+ 4 .
             - 2 .",
        );
        let results = chunk.split(2);
        assert_eq!(2, results.len());
        assert_eq!(
            results[0].to_pretty().to_string(),
            "\
+----+---+---+
|  + | 1 | 6 |
| U- | 3 | 7 |
| U+ | 4 |   |
+----+---+---+"
        );
        assert_eq!(
            results[1].to_pretty().to_string(),
            "\
+---+---+---+
| - | 2 |   |
+---+---+---+"
        );
    }

    #[test]
    fn test_eliminate_adjacent_noop_update() {
        let c = StreamChunk::from_pretty(
            "  I I
            - 1 6 D
            - 2 2
            + 2 3
            - 2 3
            + 1 6
            - 1 7
            + 1 10 D
            + 1 7
            U- 3 7
            U+ 3 7
            + 2 3",
        );
        let c = c.eliminate_adjacent_noop_update();
        assert_eq!(
            c.to_pretty().to_string(),
            "\
+---+---+---+
| - | 2 | 2 |
| + | 2 | 3 |
| - | 2 | 3 |
| + | 1 | 6 |
| + | 2 | 3 |
+---+---+---+"
        );
    }
}