risingwave_common/util/value_encoding/
column_aware_row_encoding.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
// 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.

//! Column-aware row encoding is an encoding format which converts row into a binary form that
//! remains explanable after schema changes
//! Current design of flag just contains 1 meaningful information: the 2 LSBs represents
//! the size of offsets: `u8`/`u16`/`u32`
//! We have a `Serializer` and a `Deserializer` for each schema of `Row`, which can be reused
//! until schema changes

use std::collections::HashSet;
use std::sync::Arc;

use ahash::HashMap;
use bitflags::bitflags;

use super::*;
use crate::catalog::ColumnId;

bitflags! {
    #[derive(Debug, Clone, Copy, PartialEq, Eq)]
    struct Flag: u8 {
        const EMPTY = 0b_1000_0000;
        const OFFSET8 = 0b01;
        const OFFSET16 = 0b10;
        const OFFSET32 = 0b11;
    }
}

/// `RowEncoding` holds row-specific information for Column-Aware Encoding
struct RowEncoding {
    flag: Flag,
    offsets: Vec<u8>,
    buf: Vec<u8>,
}

impl RowEncoding {
    fn new() -> Self {
        RowEncoding {
            flag: Flag::EMPTY,
            offsets: vec![],
            buf: vec![],
        }
    }

    fn set_offsets(&mut self, usize_offsets: &[usize], max_offset: usize) {
        debug_assert!(self.offsets.is_empty());
        match max_offset {
            _n @ ..=const { u8::MAX as usize } => {
                self.flag |= Flag::OFFSET8;
                usize_offsets
                    .iter()
                    .for_each(|m| self.offsets.put_u8(*m as u8));
            }
            _n @ ..=const { u16::MAX as usize } => {
                self.flag |= Flag::OFFSET16;
                usize_offsets
                    .iter()
                    .for_each(|m| self.offsets.put_u16_le(*m as u16));
            }
            _n @ ..=const { u32::MAX as usize } => {
                self.flag |= Flag::OFFSET32;
                usize_offsets
                    .iter()
                    .for_each(|m| self.offsets.put_u32_le(*m as u32));
            }
            _ => unreachable!("encoding length exceeds u32"),
        }
    }

    fn encode(&mut self, datum_refs: impl Iterator<Item = impl ToDatumRef>) {
        debug_assert!(
            self.buf.is_empty(),
            "should not encode one RowEncoding object multiple times."
        );
        let mut offset_usize = vec![];
        for datum in datum_refs {
            offset_usize.push(self.buf.len());
            if let Some(v) = datum.to_datum_ref() {
                serialize_scalar(v, &mut self.buf);
            }
        }
        let max_offset = *offset_usize
            .last()
            .expect("should encode at least one column");
        self.set_offsets(&offset_usize, max_offset);
    }

    // TODO: Avoid duplicated code. `encode_slice` is the same as `encode` except it doesn't require column type.
    fn encode_slice<'a>(&mut self, datum_refs: impl Iterator<Item = Option<&'a [u8]>>) {
        debug_assert!(
            self.buf.is_empty(),
            "should not encode one RowEncoding object multiple times."
        );
        let mut offset_usize = vec![];
        for datum in datum_refs {
            offset_usize.push(self.buf.len());
            if let Some(v) = datum {
                self.buf.put_slice(v);
            }
        }
        let max_offset = *offset_usize
            .last()
            .expect("should encode at least one column");
        self.set_offsets(&offset_usize, max_offset);
    }
}

/// Column-Aware `Serializer` holds schema related information, and shall be
/// created again once the schema changes
#[derive(Clone)]
pub struct Serializer {
    encoded_column_ids: Vec<u8>,
    datum_num: u32,
}

impl Serializer {
    /// Create a new `Serializer` with current `column_ids`
    pub fn new(column_ids: &[ColumnId]) -> Self {
        // currently we hard-code ColumnId as i32
        let mut encoded_column_ids = Vec::with_capacity(column_ids.len() * 4);
        for id in column_ids {
            encoded_column_ids.put_i32_le(id.get_id());
        }
        let datum_num = column_ids.len() as u32;
        Self {
            encoded_column_ids,
            datum_num,
        }
    }

    fn serialize_inner(&self, encoding: RowEncoding) -> Vec<u8> {
        let mut row_bytes = Vec::with_capacity(
            5 + self.encoded_column_ids.len() + encoding.offsets.len() + encoding.buf.len(), /* 5 comes from u8+u32 */
        );
        row_bytes.put_u8(encoding.flag.bits());
        row_bytes.put_u32_le(self.datum_num);
        row_bytes.extend(&self.encoded_column_ids);
        row_bytes.extend(&encoding.offsets);
        row_bytes.extend(&encoding.buf);

        row_bytes
    }
}

impl ValueRowSerializer for Serializer {
    /// Serialize a row under the schema of the Serializer
    fn serialize(&self, row: impl Row) -> Vec<u8> {
        assert_eq!(row.len(), self.datum_num as usize);
        let mut encoding = RowEncoding::new();
        encoding.encode(row.iter());
        self.serialize_inner(encoding)
    }
}

/// Column-Aware `Deserializer` holds needed `ColumnIds` and their corresponding schema
/// Should non-null default values be specified, a new field could be added to Deserializer
#[derive(Clone)]
pub struct Deserializer {
    required_column_ids: HashMap<i32, usize>,
    schema: Arc<[DataType]>,

    /// A row with default values for each column or `None` if no default value is specified
    default_row: Vec<Datum>,
}

impl Deserializer {
    pub fn new(
        column_ids: &[ColumnId],
        schema: Arc<[DataType]>,
        column_with_default: impl Iterator<Item = (usize, Datum)>,
    ) -> Self {
        assert_eq!(column_ids.len(), schema.len());
        let mut default_row: Vec<Datum> = vec![None; schema.len()];
        for (i, datum) in column_with_default {
            default_row[i] = datum;
        }
        Self {
            required_column_ids: column_ids
                .iter()
                .enumerate()
                .map(|(i, c)| (c.get_id(), i))
                .collect::<HashMap<_, _>>(),
            schema,
            default_row,
        }
    }
}

impl ValueRowDeserializer for Deserializer {
    fn deserialize(&self, mut encoded_bytes: &[u8]) -> Result<Vec<Datum>> {
        let flag = Flag::from_bits(encoded_bytes.get_u8()).expect("should be a valid flag");
        let offset_bytes = match flag - Flag::EMPTY {
            Flag::OFFSET8 => 1,
            Flag::OFFSET16 => 2,
            Flag::OFFSET32 => 4,
            _ => return Err(ValueEncodingError::InvalidFlag(flag.bits())),
        };
        let datum_num = encoded_bytes.get_u32_le() as usize;
        let offsets_start_idx = 4 * datum_num;
        let data_start_idx = offsets_start_idx + datum_num * offset_bytes;
        let offsets = &encoded_bytes[offsets_start_idx..data_start_idx];
        let data = &encoded_bytes[data_start_idx..];

        let mut row = self.default_row.clone();
        for i in 0..datum_num {
            let this_id = encoded_bytes.get_i32_le();
            if let Some(&decoded_idx) = self.required_column_ids.get(&this_id) {
                let this_offset_start_idx = i * offset_bytes;
                let mut this_offset_slice =
                    &offsets[this_offset_start_idx..(this_offset_start_idx + offset_bytes)];
                let this_offset = deserialize_width(offset_bytes, &mut this_offset_slice);
                let data = if i + 1 < datum_num {
                    let mut next_offset_slice = &offsets[(this_offset_start_idx + offset_bytes)
                        ..(this_offset_start_idx + 2 * offset_bytes)];
                    let next_offset = deserialize_width(offset_bytes, &mut next_offset_slice);
                    if this_offset == next_offset {
                        None
                    } else {
                        let mut data_slice = &data[this_offset..next_offset];
                        Some(deserialize_value(
                            &self.schema[decoded_idx],
                            &mut data_slice,
                        )?)
                    }
                } else if this_offset == data.len() {
                    None
                } else {
                    let mut data_slice = &data[this_offset..];
                    Some(deserialize_value(
                        &self.schema[decoded_idx],
                        &mut data_slice,
                    )?)
                };
                row[decoded_idx] = data;
            }
        }
        Ok(row)
    }
}

fn deserialize_width(len: usize, data: &mut impl Buf) -> usize {
    match len {
        1 => data.get_u8() as usize,
        2 => data.get_u16_le() as usize,
        4 => data.get_u32_le() as usize,
        _ => unreachable!("Width's len should be either 1, 2, or 4"),
    }
}

/// Combined column-aware `Serializer` and `Deserializer` given the same
/// `column_ids` and `schema`
#[derive(Clone)]
pub struct ColumnAwareSerde {
    pub serializer: Serializer,
    pub deserializer: Deserializer,
}

impl ValueRowSerializer for ColumnAwareSerde {
    fn serialize(&self, row: impl Row) -> Vec<u8> {
        self.serializer.serialize(row)
    }
}

impl ValueRowDeserializer for ColumnAwareSerde {
    fn deserialize(&self, encoded_bytes: &[u8]) -> Result<Vec<Datum>> {
        self.deserializer.deserialize(encoded_bytes)
    }
}

/// Deserializes row `encoded_bytes`, drops columns not in `valid_column_ids`, serializes and returns.
/// If no column is dropped, returns None.
// TODO: Avoid duplicated code. The current code combines`Serializer` and `Deserializer` with unavailable parameter removed, e.g. `Deserializer::schema`.
pub fn try_drop_invalid_columns(
    mut encoded_bytes: &[u8],
    valid_column_ids: &HashSet<i32>,
) -> Option<Vec<u8>> {
    let flag = Flag::from_bits(encoded_bytes.get_u8()).expect("should be a valid flag");
    let datum_num = encoded_bytes.get_u32_le() as usize;
    let mut is_column_dropped = false;
    let mut encoded_bytes_copy = encoded_bytes;
    for _ in 0..datum_num {
        let this_id = encoded_bytes_copy.get_i32_le();
        if !valid_column_ids.contains(&this_id) {
            is_column_dropped = true;
            break;
        }
    }
    if !is_column_dropped {
        return None;
    }

    // Slow path that drops columns. Should be rare.
    let offset_bytes = match flag - Flag::EMPTY {
        Flag::OFFSET8 => 1,
        Flag::OFFSET16 => 2,
        Flag::OFFSET32 => 4,
        _ => panic!("invalid flag {}", flag.bits()),
    };
    let offsets_start_idx = 4 * datum_num;
    let data_start_idx = offsets_start_idx + datum_num * offset_bytes;
    let offsets = &encoded_bytes[offsets_start_idx..data_start_idx];
    let data = &encoded_bytes[data_start_idx..];
    let mut datums: Vec<Option<&[u8]>> = Vec::with_capacity(valid_column_ids.len());
    let mut column_ids = Vec::with_capacity(valid_column_ids.len());
    for i in 0..datum_num {
        let this_id = encoded_bytes.get_i32_le();
        if valid_column_ids.contains(&this_id) {
            column_ids.push(this_id);
            let this_offset_start_idx = i * offset_bytes;
            let mut this_offset_slice =
                &offsets[this_offset_start_idx..(this_offset_start_idx + offset_bytes)];
            let this_offset = deserialize_width(offset_bytes, &mut this_offset_slice);
            let data = if i + 1 < datum_num {
                let mut next_offset_slice = &offsets[(this_offset_start_idx + offset_bytes)
                    ..(this_offset_start_idx + 2 * offset_bytes)];
                let next_offset = deserialize_width(offset_bytes, &mut next_offset_slice);
                if this_offset == next_offset {
                    None
                } else {
                    let data_slice = &data[this_offset..next_offset];
                    Some(data_slice)
                }
            } else if this_offset == data.len() {
                None
            } else {
                let data_slice = &data[this_offset..];
                Some(data_slice)
            };
            datums.push(data);
        }
    }
    if column_ids.is_empty() {
        // According to `RowEncoding::encode`, at least one column is required.
        return None;
    }

    let mut encoding = RowEncoding::new();
    encoding.encode_slice(datums.into_iter());
    let mut encoded_column_ids = Vec::with_capacity(column_ids.len() * 4);
    let datum_num = column_ids.len() as u32;
    for id in column_ids {
        encoded_column_ids.put_i32_le(id);
    }
    let mut row_bytes = Vec::with_capacity(
        5 + encoded_column_ids.len() + encoding.offsets.len() + encoding.buf.len(), /* 5 comes from u8+u32 */
    );
    row_bytes.put_u8(encoding.flag.bits());
    row_bytes.put_u32_le(datum_num);
    row_bytes.extend(&encoded_column_ids);
    row_bytes.extend(&encoding.offsets);
    row_bytes.extend(&encoding.buf);

    Some(row_bytes)
}