risingwave_common/array/
stream_chunk_iter.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
// 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::ops::Range;

use super::data_chunk_iter::DataChunkRefIter;
use super::stream_record::Record;
use super::RowRef;
use crate::array::{Op, StreamChunk};

impl StreamChunk {
    /// Return an iterator on stream records of this stream chunk.
    pub fn records(&self) -> StreamChunkRefIter<'_> {
        StreamChunkRefIter {
            chunk: self,
            inner: self.data_chunk().rows(),
        }
    }

    /// Return an iterator on rows of this stream chunk.
    ///
    /// Should consider using [`StreamChunk::records`] if possible.
    pub fn rows(&self) -> impl Iterator<Item = (Op, RowRef<'_>)> {
        self.rows_in(0..self.capacity())
    }

    /// Return an iterator on rows of this stream chunk in a range.
    pub fn rows_in(&self, range: Range<usize>) -> impl Iterator<Item = (Op, RowRef<'_>)> {
        self.data_chunk().rows_in(range).map(|row| {
            (
                // SAFETY: index is checked since we are in the iterator.
                unsafe { *self.ops().get_unchecked(row.index()) },
                row,
            )
        })
    }

    /// Random access a row at `pos`. Return the op, data and whether the row is visible.
    pub fn row_at(&self, pos: usize) -> (Op, RowRef<'_>, bool) {
        let op = self.ops()[pos];
        let (row, visible) = self.data_chunk().row_at(pos);
        (op, row, visible)
    }

    pub fn rows_with_holes(&self) -> impl Iterator<Item = Option<(Op, RowRef<'_>)>> {
        self.data_chunk().rows_with_holes().map(|row| {
            row.map(|row| {
                (
                    // SAFETY: index is checked since we are in the iterator.
                    unsafe { *self.ops().get_unchecked(row.index()) },
                    row,
                )
            })
        })
    }
}

pub struct StreamChunkRefIter<'a> {
    chunk: &'a StreamChunk,

    inner: DataChunkRefIter<'a>,
}

impl<'a> Iterator for StreamChunkRefIter<'a> {
    type Item = Record<RowRef<'a>>;

    fn next(&mut self) -> Option<Self::Item> {
        let row = self.inner.next()?;
        // SAFETY: index is checked since `row` is `Some`.
        let op = unsafe { self.chunk.ops().get_unchecked(row.index()) };

        match op {
            Op::Insert => Some(Record::Insert { new_row: row }),
            Op::Delete => Some(Record::Delete { old_row: row }),
            Op::UpdateDelete => {
                let insert_row = self.inner.next().expect("expect a row after U-");
                // SAFETY: index is checked since `insert_row` is `Some`.
                let op = unsafe { *self.chunk.ops().get_unchecked(insert_row.index()) };
                debug_assert_eq!(op, Op::UpdateInsert, "expect a U+ after U-");

                Some(Record::Update {
                    old_row: row,
                    new_row: insert_row,
                })
            }
            Op::UpdateInsert => panic!("expect a U- before U+"),
        }
    }

    fn size_hint(&self) -> (usize, Option<usize>) {
        let (lower, upper) = self.inner.size_hint();
        (lower / 2, upper)
    }
}

#[cfg(test)]
mod tests {
    extern crate test;
    use itertools::Itertools;
    use test::Bencher;

    use crate::array::stream_record::Record;
    use crate::row::Row;
    use crate::test_utils::test_stream_chunk::{
        BigStreamChunk, TestStreamChunk, WhatEverStreamChunk,
    };

    #[test]
    fn test_chunk_rows() {
        let test = WhatEverStreamChunk;
        let chunk = test.stream_chunk();
        let mut rows = chunk.rows().map(|(op, row)| (op, row.into_owned_row()));
        assert_eq!(Some(test.row_with_op_at(0)), rows.next());
        assert_eq!(Some(test.row_with_op_at(1)), rows.next());
        assert_eq!(Some(test.row_with_op_at(2)), rows.next());
        assert_eq!(Some(test.row_with_op_at(3)), rows.next());
    }

    #[test]
    fn test_chunk_records() {
        let test = WhatEverStreamChunk;
        let chunk = test.stream_chunk();
        let mut rows = chunk
            .records()
            .flat_map(Record::into_rows)
            .map(|(op, row)| (op, row.into_owned_row()));
        assert_eq!(Some(test.row_with_op_at(0)), rows.next());
        assert_eq!(Some(test.row_with_op_at(1)), rows.next());
        assert_eq!(Some(test.row_with_op_at(2)), rows.next());
        assert_eq!(Some(test.row_with_op_at(3)), rows.next());
    }

    #[bench]
    fn bench_rows_iterator_from_records(b: &mut Bencher) {
        let chunk = BigStreamChunk::new(10000).stream_chunk();
        b.iter(|| {
            for (_op, row) in chunk.records().flat_map(Record::into_rows) {
                test::black_box(row.iter().count());
            }
        })
    }

    #[bench]
    fn bench_rows_iterator(b: &mut Bencher) {
        let chunk = BigStreamChunk::new(10000).stream_chunk();
        b.iter(|| {
            for (_op, row) in chunk.rows() {
                test::black_box(row.iter().count());
            }
        })
    }

    #[bench]
    fn bench_rows_iterator_vec_of_datum_refs(b: &mut Bencher) {
        let chunk = BigStreamChunk::new(10000).stream_chunk();
        b.iter(|| {
            for (_op, row) in chunk.rows() {
                // Mimic the old `RowRef(Vec<DatumRef>)`
                let row = row.iter().collect_vec();
                test::black_box(row);
            }
        })
    }

    #[bench]
    fn bench_record_iterator(b: &mut Bencher) {
        let chunk = BigStreamChunk::new(10000).stream_chunk();
        b.iter(|| {
            for record in chunk.records() {
                match record {
                    Record::Insert { new_row } => test::black_box(new_row.iter().count()),
                    Record::Delete { old_row } => test::black_box(old_row.iter().count()),
                    Record::Update { old_row, new_row } => {
                        test::black_box(old_row.iter().count());
                        test::black_box(new_row.iter().count())
                    }
                };
            }
        })
    }
}