risingwave_stream/executor/dedup/
append_only_dedup.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
// 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 futures::stream;
use itertools::Itertools;
use risingwave_common::array::Op;
use risingwave_common::bitmap::BitmapBuilder;
use risingwave_common::row::RowExt;

use super::cache::DedupCache;
use crate::common::metrics::MetricsInfo;
use crate::executor::prelude::*;

/// [`AppendOnlyDedupExecutor`] drops any message that has duplicate pk columns with previous
/// messages. It only accepts append-only input, and its output will be append-only as well.
pub struct AppendOnlyDedupExecutor<S: StateStore> {
    ctx: ActorContextRef,

    input: Option<Executor>,
    dedup_cols: Vec<usize>,
    state_table: StateTable<S>,
    cache: DedupCache<OwnedRow>,
}

impl<S: StateStore> AppendOnlyDedupExecutor<S> {
    pub fn new(
        ctx: ActorContextRef,
        input: Executor,
        dedup_cols: Vec<usize>,
        state_table: StateTable<S>,
        watermark_epoch: AtomicU64Ref,
        metrics: Arc<StreamingMetrics>,
    ) -> Self {
        let metrics_info =
            MetricsInfo::new(metrics, state_table.table_id(), ctx.id, "AppendOnly Dedup");
        Self {
            ctx,
            input: Some(input),
            dedup_cols,
            state_table,
            cache: DedupCache::new(watermark_epoch, metrics_info),
        }
    }

    #[try_stream(ok = Message, error = StreamExecutorError)]
    async fn executor_inner(mut self) {
        let mut input = self.input.take().unwrap().execute();

        // Consume the first barrier message and initialize state table.
        let barrier = expect_first_barrier(&mut input).await?;
        let first_epoch = barrier.epoch;
        // The first barrier message should be propagated.
        yield Message::Barrier(barrier);
        self.state_table.init_epoch(first_epoch).await?;

        #[for_await]
        for msg in input {
            self.cache.evict();

            match msg? {
                Message::Chunk(chunk) => {
                    // Append-only dedup executor only receives INSERT messages.
                    debug_assert!(chunk.ops().iter().all(|&op| op == Op::Insert));

                    // Extract pk for all rows (regardless of visibility) in the chunk.
                    let keys = chunk
                        .data_chunk()
                        .rows_with_holes()
                        .map(|row_ref| {
                            row_ref.map(|row| row.project(&self.dedup_cols).to_owned_row())
                        })
                        .collect_vec();

                    // Ensure that if a key for a visible row exists before, then it is in the
                    // cache, by querying the storage.
                    self.populate_cache(keys.iter().flatten()).await?;

                    // Now check for duplication and insert new keys into the cache.
                    let mut vis_builder = BitmapBuilder::with_capacity(chunk.capacity());
                    for key in keys {
                        match key {
                            Some(key) => {
                                if self.cache.dedup_insert(key) {
                                    // The key doesn't exist before. The row should be visible.
                                    vis_builder.append(true);
                                } else {
                                    // The key exists before. The row shouldn't be visible.
                                    vis_builder.append(false);
                                }
                            }
                            None => {
                                // The row is originally invisible.
                                vis_builder.append(false);
                            }
                        }
                    }

                    let vis = vis_builder.finish();
                    if vis.count_ones() > 0 {
                        // Construct the new chunk and write the data to state table.
                        let (ops, columns, _) = chunk.into_inner();
                        let chunk = StreamChunk::with_visibility(ops, columns, vis);
                        self.state_table.write_chunk(chunk.clone());

                        yield Message::Chunk(chunk);
                    }
                    self.state_table.try_flush().await?;
                }

                Message::Barrier(barrier) => {
                    self.state_table.commit(barrier.epoch).await?;

                    if let Some(vnode_bitmap) = barrier.as_update_vnode_bitmap(self.ctx.id) {
                        let (_prev_vnode_bitmap, cache_may_stale) =
                            self.state_table.update_vnode_bitmap(vnode_bitmap);
                        if cache_may_stale {
                            self.cache.clear();
                        }
                    }

                    yield Message::Barrier(barrier);
                }

                Message::Watermark(watermark) => {
                    yield Message::Watermark(watermark);
                }
            }
        }
    }

    /// Populate the cache with keys that exist in storage before.
    pub async fn populate_cache<'a>(
        &mut self,
        keys: impl Iterator<Item = &'a OwnedRow>,
    ) -> StreamExecutorResult<()> {
        let mut read_from_storage = false;
        let mut futures = vec![];
        for key in keys {
            if self.cache.contains(key) {
                continue;
            }
            read_from_storage = true;

            let table = &self.state_table;
            futures.push(async move { (key, table.get_encoded_row(key).await) });
        }

        if read_from_storage {
            let mut buffered = stream::iter(futures).buffer_unordered(10).fuse();
            while let Some(result) = buffered.next().await {
                let (key, value) = result;
                if value?.is_some() {
                    // Only insert into the cache when we have this key in storage.
                    self.cache.insert(key.to_owned());
                }
            }
        }

        Ok(())
    }
}

impl<S: StateStore> Execute for AppendOnlyDedupExecutor<S> {
    fn execute(self: Box<Self>) -> BoxedMessageStream {
        self.executor_inner().boxed()
    }
}

#[cfg(test)]
mod tests {
    use std::sync::atomic::AtomicU64;

    use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, TableId};
    use risingwave_common::test_prelude::StreamChunkTestExt;
    use risingwave_common::util::epoch::test_epoch;
    use risingwave_common::util::sort_util::OrderType;
    use risingwave_storage::memory::MemoryStateStore;

    use super::*;
    use crate::common::table::test_utils::gen_pbtable;
    use crate::executor::test_utils::MockSource;

    #[tokio::test]
    async fn test_dedup_executor() {
        let table_id = TableId::new(1);
        let column_descs = vec![
            ColumnDesc::unnamed(ColumnId::new(0), DataType::Int64),
            ColumnDesc::unnamed(ColumnId::new(1), DataType::Int64),
        ];
        let schema = Schema::new(vec![
            Field::unnamed(DataType::Int64),
            Field::unnamed(DataType::Int64),
        ]);
        let dedup_col_indices = vec![0];
        let pk_indices = dedup_col_indices.clone();
        let order_types = vec![OrderType::ascending()];

        let state_store = MemoryStateStore::new();
        let state_table = StateTable::from_table_catalog(
            &gen_pbtable(table_id, column_descs, order_types, pk_indices.clone(), 0),
            state_store,
            None,
        )
        .await;

        let (mut tx, source) = MockSource::channel();
        let source = source.into_executor(schema, pk_indices);
        let mut dedup_executor = AppendOnlyDedupExecutor::new(
            ActorContext::for_test(123),
            source,
            dedup_col_indices,
            state_table,
            Arc::new(AtomicU64::new(0)),
            Arc::new(StreamingMetrics::unused()),
        )
        .boxed()
        .execute();

        tx.push_barrier(test_epoch(1), false);
        dedup_executor.next().await.unwrap().unwrap();

        let chunk = StreamChunk::from_pretty(
            " I I
            + 1 1
            + 2 2 D
            + 1 7",
        );
        tx.push_chunk(chunk);
        let msg = dedup_executor.next().await.unwrap().unwrap();
        assert_eq!(
            msg.into_chunk().unwrap(),
            StreamChunk::from_pretty(
                " I I
                + 1 1
                + 2 2 D
                + 1 7 D",
            )
        );

        tx.push_barrier(test_epoch(2), false);
        dedup_executor.next().await.unwrap().unwrap();

        let chunk = StreamChunk::from_pretty(
            " I I
            + 3 9
            + 2 5
            + 1 20",
        );
        tx.push_chunk(chunk);
        let msg = dedup_executor.next().await.unwrap().unwrap();
        assert_eq!(
            msg.into_chunk().unwrap(),
            StreamChunk::from_pretty(
                " I I
                + 3 9
                + 2 5
                + 1 20 D",
            )
        );
    }
}