risingwave_storage/table/
mod.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
// 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.

pub mod batch_table;
pub mod merge_sort;

use std::ops::Deref;

use futures::{Stream, StreamExt};
use risingwave_common::array::DataChunk;
use risingwave_common::catalog::Schema;
pub use risingwave_common::hash::table_distribution::*;
use risingwave_common::hash::VirtualNode;
use risingwave_common::row::{OwnedRow, Row};
use risingwave_common::util::chunk_coalesce::DataChunkBuilder;
use risingwave_common::util::iter_util::ZipEqDebug;
use risingwave_hummock_sdk::key::TableKey;

use crate::error::StorageResult;
use crate::row_serde::value_serde::ValueRowSerde;
use crate::store::{ChangeLogValue, StateStoreIterExt, StateStoreReadLogItem};
use crate::StateStoreIter;

// TODO: GAT-ify this trait or remove this trait
#[async_trait::async_trait]
pub trait TableIter: Send {
    async fn next_row(&mut self) -> StorageResult<Option<OwnedRow>>;
}

pub async fn collect_data_chunk<E, S, R>(
    stream: &mut S,
    schema: &Schema,
    chunk_size: Option<usize>,
) -> Result<Option<DataChunk>, E>
where
    S: Stream<Item = Result<R, E>> + Unpin,
    R: Row,
{
    let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0));
    let mut row_count = 0;
    for _ in 0..chunk_size.unwrap_or(usize::MAX) {
        match stream.next().await.transpose()? {
            Some(row) => {
                for (datum, builder) in row.iter().zip_eq_debug(builders.iter_mut()) {
                    builder.append(datum);
                }
            }
            None => break,
        }

        row_count += 1;
    }

    let chunk = {
        let columns: Vec<_> = builders
            .into_iter()
            .map(|builder| builder.finish().into())
            .collect();
        DataChunk::new(columns, row_count)
    };

    if chunk.cardinality() == 0 {
        Ok(None)
    } else {
        Ok(Some(chunk))
    }
}

/// Collects data chunks from stream of rows.
pub async fn collect_data_chunk_with_builder<E, S, R>(
    stream: &mut S,
    builder: &mut DataChunkBuilder,
) -> Result<Option<DataChunk>, E>
where
    R: Row,
    S: Stream<Item = Result<R, E>> + Unpin,
{
    // TODO(kwannoel): If necessary, we can optimize it in the future.
    // This can be done by moving the check if builder is full from `append_one_row` to here,
    while let Some(row) = stream.next().await.transpose()? {
        let result = builder.append_one_row(row);
        if let Some(chunk) = result {
            return Ok(Some(chunk));
        }
    }

    let chunk = builder.consume_all();
    Ok(chunk)
}

pub fn get_second<T, U, E>(arg: Result<(T, U), E>) -> Result<U, E> {
    arg.map(|x| x.1)
}

#[derive(Debug)]
pub struct KeyedRow<T: AsRef<[u8]>> {
    vnode_prefixed_key: TableKey<T>,
    row: OwnedRow,
}

impl<T: AsRef<[u8]>> KeyedRow<T> {
    pub fn new(table_key: TableKey<T>, row: OwnedRow) -> Self {
        Self {
            vnode_prefixed_key: table_key,
            row,
        }
    }

    pub fn into_owned_row(self) -> OwnedRow {
        self.row
    }

    pub fn into_owned_row_key(self) -> (TableKey<T>, OwnedRow) {
        (self.vnode_prefixed_key, self.row)
    }

    pub fn vnode(&self) -> VirtualNode {
        self.vnode_prefixed_key.vnode_part()
    }

    pub fn key(&self) -> &[u8] {
        self.vnode_prefixed_key.key_part()
    }

    pub fn row(&self) -> &OwnedRow {
        &self.row
    }

    pub fn into_parts(self) -> (TableKey<T>, OwnedRow) {
        (self.vnode_prefixed_key, self.row)
    }
}

impl<T: AsRef<[u8]>> Deref for KeyedRow<T> {
    type Target = OwnedRow;

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

#[derive(Debug)]
pub struct KeyedChangeLogRow<T: AsRef<[u8]>> {
    vnode_prefixed_key: TableKey<T>,
    row: ChangeLogRow,
}

impl<T: AsRef<[u8]>> KeyedChangeLogRow<T> {
    pub fn new(table_key: TableKey<T>, row: ChangeLogRow) -> Self {
        Self {
            vnode_prefixed_key: table_key,
            row,
        }
    }

    pub fn into_owned_row(self) -> ChangeLogRow {
        self.row
    }

    pub fn key(&self) -> &[u8] {
        self.vnode_prefixed_key.key_part()
    }
}

pub type ChangeLogRow = ChangeLogValue<OwnedRow>;

pub fn deserialize_log_stream<'a>(
    iter: impl StateStoreIter<StateStoreReadLogItem> + 'a,
    deserializer: &'a impl ValueRowSerde,
) -> impl Stream<Item = StorageResult<ChangeLogRow>> + 'a {
    iter.into_stream(|(_key, log_value)| {
        log_value.try_map(|slice| Ok(OwnedRow::new(deserializer.deserialize(slice)?)))
    })
}