risingwave_stream/executor/lookup/
sides.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
// 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::pin::pin;

use anyhow::Context;
use either::Either;
use futures::future::select;
use futures::{future, StreamExt};
use futures_async_stream::try_stream;
use risingwave_common::array::StreamChunk;
use risingwave_common::bail;
use risingwave_common::catalog::ColumnDesc;
use risingwave_common::types::DataType;
use risingwave_common::util::sort_util::ColumnOrder;
use risingwave_storage::table::batch_table::storage_table::StorageTable;
use risingwave_storage::StateStore;

use crate::executor::error::StreamExecutorError;
use crate::executor::{Barrier, BoxedMessageStream, Executor, Message, MessageStream};

/// Join side of Lookup Executor's stream
pub(crate) struct StreamJoinSide {
    /// Indices of the join key columns
    pub key_indices: Vec<usize>,

    /// The primary key indices of this side, used for state store
    pub pk_indices: Vec<usize>,

    /// The date type of each columns to join on
    pub col_types: Vec<DataType>,
}

impl std::fmt::Debug for StreamJoinSide {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ArrangeJoinSide")
            .field("key_indices", &self.key_indices)
            .field("pk_indices", &self.pk_indices)
            .field("col_types", &self.col_types)
            .finish()
    }
}

/// Join side of Arrange Executor's stream
pub(crate) struct ArrangeJoinSide<S: StateStore> {
    /// The primary key indices of this side, used for state store
    pub pk_indices: Vec<usize>,

    /// The datatype of columns in arrangement
    pub col_types: Vec<DataType>,

    /// The column descriptors of columns in arrangement
    pub col_descs: Vec<ColumnDesc>,

    /// Order rules of the arrangement (only join key is needed, pk should not be included, used
    /// for lookup)
    pub order_rules: Vec<ColumnOrder>,

    /// Key indices for the join
    ///
    /// The key indices of the arrange side won't be used for the lookup process, but we still
    /// record it here in case anyone would use it in the future.
    pub key_indices: Vec<usize>,

    /// Whether to join with the arrangement of the current epoch
    pub use_current_epoch: bool,

    pub storage_table: StorageTable<S>,
}

/// Message from the `arrange_join_stream`.
#[derive(Debug)]
pub enum ArrangeMessage {
    /// Arrangement sides' update in this epoch. There will be only one arrange batch message
    /// within epoch. Once the executor receives an arrange batch message, it can start doing
    /// joins.
    ArrangeReady(Vec<StreamChunk>, Barrier),

    /// There's a message from stream side.
    Stream(StreamChunk),

    /// Barrier (once every epoch).
    Barrier(Barrier),
}

pub type BarrierAlignedMessage = Either<Message, Message>;

#[try_stream(ok = Message, error = StreamExecutorError)]
pub async fn poll_until_barrier(stream: impl MessageStream, expected_barrier: Barrier) {
    #[for_await]
    for item in stream {
        match item? {
            Message::Watermark(_) => {
                // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
            }
            c @ Message::Chunk(_) => yield c,
            Message::Barrier(b) => {
                if b.epoch != expected_barrier.epoch {
                    return Err(StreamExecutorError::align_barrier(expected_barrier, b));
                } else {
                    yield Message::Barrier(b);
                    break;
                }
            }
        }
    }
}

/// A biased barrier aligner which prefers message from the right side. Barrier message will be
/// available for both left and right side, instead of being combined.
#[try_stream(ok = BarrierAlignedMessage, error = StreamExecutorError)]
pub async fn align_barrier(mut left: BoxedMessageStream, mut right: BoxedMessageStream) {
    enum SideStatus {
        LeftBarrier,
        RightBarrier,
    }

    'outer: loop {
        let (side_status, side_barrier) = 'inner: loop {
            // Prefer right
            let select_result = match select(right.next(), left.next()).await {
                future::Either::Left(x) => future::Either::Right(x),
                future::Either::Right(x) => future::Either::Left(x),
            };
            match select_result {
                future::Either::Left((None, _)) => {
                    // left stream end, passthrough right chunks
                    while let Some(msg) = right.next().await {
                        match msg? {
                            w @ Message::Watermark(_) => yield Either::Left(w),
                            c @ Message::Chunk(_) => yield Either::Left(c),
                            Message::Barrier(_) => {
                                bail!("right barrier received while left stream end");
                            }
                        }
                    }
                    break 'outer;
                }
                future::Either::Right((None, _)) => {
                    // right stream end, passthrough left chunks
                    while let Some(msg) = left.next().await {
                        match msg? {
                            w @ Message::Watermark(_) => yield Either::Right(w),
                            c @ Message::Chunk(_) => yield Either::Right(c),
                            Message::Barrier(_) => {
                                bail!("left barrier received while right stream end");
                            }
                        }
                    }
                    break 'outer;
                }
                future::Either::Left((Some(msg), _)) => match msg? {
                    w @ Message::Watermark(_) => yield Either::Left(w),
                    c @ Message::Chunk(_) => yield Either::Left(c),
                    Message::Barrier(b) => {
                        yield Either::Left(Message::Barrier(b.clone()));
                        break 'inner (SideStatus::LeftBarrier, b);
                    }
                },
                future::Either::Right((Some(msg), _)) => match msg? {
                    w @ Message::Watermark(_) => yield Either::Right(w),
                    c @ Message::Chunk(_) => yield Either::Right(c),
                    Message::Barrier(b) => {
                        yield Either::Right(Message::Barrier(b.clone()));
                        break 'inner (SideStatus::RightBarrier, b);
                    }
                },
            }
        };

        match side_status {
            SideStatus::LeftBarrier => {
                #[for_await]
                for item in poll_until_barrier(right.by_ref(), side_barrier) {
                    yield Either::Right(item?);
                }
            }
            SideStatus::RightBarrier => {
                #[for_await]
                for item in poll_until_barrier(left.by_ref(), side_barrier) {
                    yield Either::Left(item?);
                }
            }
        }
    }
}

/// Join the stream with the previous stable snapshot of the arrangement.
///
/// For example, the executor will receive the following message sequence from
/// `stream_lookup_arrange_prev_epoch`:
///
/// * `[Msg`] Barrier (prev = `[1`], current = `[2`])
/// * `[Msg`] Stream (key = a)
/// * `[Do`] lookup `a` in arrangement of epoch `[1`] (prev epoch)
/// * `[Msg`] Arrangement (batch)
/// * `[Msg`] Stream (key = b)
/// * `[Do`] lookup `b` in arrangement of epoch `[1`] (prev epoch)
/// * `[Do`] update cache with epoch `[2`]
/// * Barrier (prev = `[2`], current = `[3`])
/// * `[Msg`] Arrangement (batch)
#[try_stream(ok = ArrangeMessage, error = StreamExecutorError)]
pub async fn stream_lookup_arrange_prev_epoch(stream: Executor, arrangement: Executor) {
    let mut input = pin!(align_barrier(stream.execute(), arrangement.execute()));
    let mut arrange_buf = vec![];
    let mut stream_side_end = false;

    loop {
        let mut arrange_barrier = None;

        while let Some(item) = input.next().await {
            match item? {
                Either::Left(Message::Chunk(msg)) => {
                    // As prev epoch is already available, we can directly forward messages from the
                    // stream side.
                    yield ArrangeMessage::Stream(msg);
                }
                Either::Right(Message::Chunk(chunk)) => {
                    // For message from the arrangement side, put it in a buf
                    arrange_buf.push(chunk);
                }
                Either::Left(Message::Barrier(barrier)) => {
                    yield ArrangeMessage::Barrier(barrier);
                    stream_side_end = true;
                }
                Either::Right(Message::Barrier(barrier)) => {
                    if stream_side_end {
                        yield ArrangeMessage::ArrangeReady(
                            std::mem::take(&mut arrange_buf),
                            barrier,
                        );
                        stream_side_end = false;
                    } else {
                        arrange_barrier = Some(barrier);
                        break;
                    }
                }
                Either::Left(Message::Watermark(_)) => {
                    // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                }
                Either::Right(Message::Watermark(_)) => {
                    // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                }
            }
        }

        loop {
            match input
                .next()
                .await
                .context("unexpected close of barrier aligner")??
            {
                Either::Left(Message::Watermark(_)) => {
                    // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                }
                Either::Left(Message::Chunk(msg)) => yield ArrangeMessage::Stream(msg),
                Either::Left(Message::Barrier(b)) => {
                    yield ArrangeMessage::Barrier(b);
                    break;
                }
                Either::Right(_) => unreachable!(),
            }
        }

        yield ArrangeMessage::ArrangeReady(
            std::mem::take(&mut arrange_buf),
            arrange_barrier.take().unwrap(),
        );
    }
}

/// Join the stream with the current state of the arrangement.
///
/// For example, the executor will receive the following message sequence from
/// `stream_lookup_arrange_this_epoch`:
///
/// * `[Msg`] Barrier (prev = `[1`], current = `[2`])
/// * `[Msg`] Arrangement (batch)
/// * `[Do`] update cache with epoch `[2`]
/// * `[Msg`] Stream (key = a)
/// * `[Do`] lookup `a` in arrangement of epoch `[2`] (current epoch)
/// * Barrier (prev = `[2`], current = `[3`])
#[try_stream(ok = ArrangeMessage, error = StreamExecutorError)]
pub async fn stream_lookup_arrange_this_epoch(stream: Executor, arrangement: Executor) {
    let mut input = pin!(align_barrier(stream.execute(), arrangement.execute()));
    let mut stream_buf = vec![];
    let mut arrange_buf = vec![];

    enum Status {
        ArrangeReady,
        StreamReady(Barrier),
    }

    loop {
        let status = 'inner: loop {
            match input
                .next()
                .await
                .context("unexpected close of barrier aligner")??
            {
                Either::Left(Message::Chunk(msg)) => {
                    // Should wait until arrangement from this epoch is available.
                    stream_buf.push(msg);
                }
                Either::Right(Message::Chunk(chunk)) => {
                    // For message from the arrangement side, put it in buf.
                    arrange_buf.push(chunk);
                }
                Either::Left(Message::Barrier(barrier)) => {
                    break 'inner Status::StreamReady(barrier);
                }
                Either::Right(Message::Barrier(barrier)) => {
                    yield ArrangeMessage::ArrangeReady(std::mem::take(&mut arrange_buf), barrier);
                    for msg in std::mem::take(&mut stream_buf) {
                        yield ArrangeMessage::Stream(msg);
                    }
                    break 'inner Status::ArrangeReady;
                }
                Either::Left(Message::Watermark(_)) => {
                    // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                }
                Either::Right(Message::Watermark(_)) => {
                    // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                }
            }
        };
        match status {
            // Arrangement is ready, but still stream message in this epoch -- we directly forward
            // message from the stream side.
            Status::ArrangeReady => loop {
                match input
                    .next()
                    .await
                    .context("unexpected close of barrier aligner")??
                {
                    Either::Left(Message::Chunk(msg)) => yield ArrangeMessage::Stream(msg),
                    Either::Left(Message::Barrier(b)) => {
                        yield ArrangeMessage::Barrier(b);
                        break;
                    }
                    Either::Left(Message::Watermark(_)) => {
                        // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                    }
                    Either::Right(Message::Watermark(_)) => {
                        // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                    }
                    Either::Right(_) => unreachable!(),
                }
            },
            // Stream is done in this epoch, but arrangement is not ready -- we wait for the
            // arrangement ready and pipe out all buffered stream messages.
            Status::StreamReady(stream_barrier) => loop {
                match input
                    .next()
                    .await
                    .context("unexpected close of barrier aligner")??
                {
                    Either::Left(_) => unreachable!(),
                    Either::Right(Message::Chunk(chunk)) => {
                        arrange_buf.push(chunk);
                    }
                    Either::Right(Message::Barrier(barrier)) => {
                        yield ArrangeMessage::ArrangeReady(
                            std::mem::take(&mut arrange_buf),
                            barrier,
                        );
                        for msg in std::mem::take(&mut stream_buf) {
                            yield ArrangeMessage::Stream(msg);
                        }
                        yield ArrangeMessage::Barrier(stream_barrier);
                        break;
                    }
                    Either::Right(Message::Watermark(_)) => {
                        // TODO: https://github.com/risingwavelabs/risingwave/issues/6042
                    }
                }
            },
        }
    }
}

impl<S: StateStore> std::fmt::Debug for ArrangeJoinSide<S> {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ArrangeJoinSide")
            .field("pk_indices", &self.pk_indices)
            .field("col_types", &self.col_types)
            .field("col_descs", &self.col_descs)
            .field("order_rules", &self.order_rules)
            .field("use_current_epoch", &self.use_current_epoch)
            .finish()
    }
}

#[cfg(test)]
mod tests {
    use futures::StreamExt;
    use risingwave_common::array::{StreamChunk, StreamChunkTestExt};
    use risingwave_common::catalog::{Field, Schema};
    use risingwave_common::types::DataType;
    use risingwave_common::util::epoch::test_epoch;

    use crate::executor::lookup::sides::stream_lookup_arrange_this_epoch;
    use crate::executor::test_utils::MockSource;
    use crate::executor::StreamExecutorResult;

    #[tokio::test]
    async fn test_stream_lookup_arrange_this_epoch() -> StreamExecutorResult<()> {
        let chunk_l1 = StreamChunk::from_pretty(
            "  I I
             + 1 1",
        );

        let schema = Schema {
            fields: vec![
                Field::unnamed(DataType::Int64), // join key
                Field::unnamed(DataType::Int64),
            ],
        };
        let (mut tx_l, source_l) = MockSource::channel();
        let source_l = source_l
            .stop_on_finish(false)
            .into_executor(schema.clone(), vec![1]);
        let (tx_r, source_r) = MockSource::channel();
        let source_r = source_r
            .stop_on_finish(false)
            .into_executor(schema, vec![1]);

        let mut stream = stream_lookup_arrange_this_epoch(source_l, source_r).boxed();

        // Simulate recovery test
        drop(tx_r);

        tx_l.push_barrier(test_epoch(1), false);

        tx_l.push_chunk(chunk_l1);

        // It should throw an error instead of panic.
        stream.next().await.unwrap().unwrap_err();

        Ok(())
    }
}