risingwave_stream/executor/
now.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
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
// 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::Bound;
use std::ops::Bound::Unbounded;

use itertools::Itertools;
use risingwave_common::array::Op;
use risingwave_common::row;
use risingwave_common::types::{DefaultOrdered, Interval, Timestamptz, ToDatumRef};
use risingwave_expr::capture_context;
use risingwave_expr::expr::{
    build_func_non_strict, EvalErrorReport, ExpressionBoxExt, InputRefExpression,
    LiteralExpression, NonStrictExpression,
};
use risingwave_expr::expr_context::TIME_ZONE;
use tokio::sync::mpsc::UnboundedReceiver;
use tokio_stream::wrappers::UnboundedReceiverStream;

use crate::executor::prelude::*;
use crate::task::ActorEvalErrorReport;

pub struct NowExecutor<S: StateStore> {
    data_types: Vec<DataType>,

    mode: NowMode,
    eval_error_report: ActorEvalErrorReport,

    /// Receiver of barrier channel.
    barrier_receiver: UnboundedReceiver<Barrier>,

    state_table: StateTable<S>,
}

pub enum NowMode {
    /// Emit current timestamp on startup, update it on barrier.
    UpdateCurrent,
    /// Generate a series of timestamps starting from `start_timestamp` with `interval`.
    /// Keep generating new timestamps on barrier.
    GenerateSeries {
        start_timestamp: Timestamptz,
        interval: Interval,
    },
}

enum ModeVars {
    UpdateCurrent,
    GenerateSeries {
        chunk_builder: StreamChunkBuilder,
        add_interval_expr: NonStrictExpression,
    },
}

impl<S: StateStore> NowExecutor<S> {
    pub fn new(
        data_types: Vec<DataType>,
        mode: NowMode,
        eval_error_report: ActorEvalErrorReport,
        barrier_receiver: UnboundedReceiver<Barrier>,
        state_table: StateTable<S>,
    ) -> Self {
        Self {
            data_types,
            mode,
            eval_error_report,
            barrier_receiver,
            state_table,
        }
    }

    #[try_stream(ok = Message, error = StreamExecutorError)]
    async fn execute_inner(self) {
        let Self {
            data_types,
            mode,
            eval_error_report,
            barrier_receiver,
            mut state_table,
        } = self;

        let max_chunk_size = crate::config::chunk_size();

        // Whether the executor is paused.
        let mut paused = false;
        // The last timestamp **sent** to the downstream.
        let mut last_timestamp: Datum = None;

        // Whether the first barrier is handled and `last_timestamp` is initialized.
        let mut initialized = false;

        let mut mode_vars = match &mode {
            NowMode::UpdateCurrent => ModeVars::UpdateCurrent,
            NowMode::GenerateSeries { interval, .. } => {
                // in most cases there won't be more than one row except for the first time
                let chunk_builder = StreamChunkBuilder::unlimited(data_types.clone(), Some(1));
                let add_interval_expr =
                    build_add_interval_expr_captured(*interval, eval_error_report)?;
                ModeVars::GenerateSeries {
                    chunk_builder,
                    add_interval_expr,
                }
            }
        };

        const MAX_MERGE_BARRIER_SIZE: usize = 64;

        #[for_await]
        for barriers in
            UnboundedReceiverStream::new(barrier_receiver).ready_chunks(MAX_MERGE_BARRIER_SIZE)
        {
            let mut curr_timestamp = None;
            if barriers.len() > 1 {
                warn!(
                    "handle multiple barriers at once in now executor: {}",
                    barriers.len()
                );
            }
            for barrier in barriers {
                let new_timestamp = Some(barrier.get_curr_epoch().as_scalar());
                let pause_mutation =
                    barrier
                        .mutation
                        .as_deref()
                        .and_then(|mutation| match mutation {
                            Mutation::Pause => Some(true),
                            Mutation::Resume => Some(false),
                            _ => None,
                        });

                if !initialized {
                    let first_epoch = barrier.epoch;
                    let is_pause_on_startup = barrier.is_pause_on_startup();
                    yield Message::Barrier(barrier);
                    // Handle the initial barrier.
                    state_table.init_epoch(first_epoch).await?;
                    let state_row = {
                        let sub_range: &(Bound<OwnedRow>, Bound<OwnedRow>) =
                            &(Unbounded, Unbounded);
                        let data_iter = state_table
                            .iter_with_prefix(row::empty(), sub_range, Default::default())
                            .await?;
                        pin_mut!(data_iter);
                        if let Some(keyed_row) = data_iter.next().await {
                            Some(keyed_row?)
                        } else {
                            None
                        }
                    };
                    last_timestamp = state_row.and_then(|row| row[0].clone());
                    paused = is_pause_on_startup;
                    initialized = true;
                } else {
                    state_table.commit(barrier.epoch).await?;
                    yield Message::Barrier(barrier);
                }

                // Extract timestamp from the current epoch.
                curr_timestamp = new_timestamp;

                // Update paused state.
                if let Some(pause_mutation) = pause_mutation {
                    paused = pause_mutation;
                }
            }

            // Do not yield any messages if paused.
            if paused {
                continue;
            }

            match (&mode, &mut mode_vars) {
                (NowMode::UpdateCurrent, ModeVars::UpdateCurrent) => {
                    let chunk = if last_timestamp.is_some() {
                        let last_row = row::once(&last_timestamp);
                        let row = row::once(&curr_timestamp);
                        state_table.update(last_row, row);

                        StreamChunk::from_rows(
                            &[(Op::Delete, last_row), (Op::Insert, row)],
                            &data_types,
                        )
                    } else {
                        let row = row::once(&curr_timestamp);
                        state_table.insert(row);

                        StreamChunk::from_rows(&[(Op::Insert, row)], &data_types)
                    };

                    yield Message::Chunk(chunk);
                    last_timestamp.clone_from(&curr_timestamp)
                }
                (
                    NowMode::GenerateSeries {
                        start_timestamp, ..
                    },
                    ModeVars::GenerateSeries {
                        chunk_builder,
                        ref add_interval_expr,
                    },
                ) => {
                    if last_timestamp.is_none() {
                        // We haven't emit any timestamp yet. Let's emit the first one and populate the state table.
                        let first = Some((*start_timestamp).into());
                        let first_row = row::once(&first);
                        let _ = chunk_builder.append_row(Op::Insert, first_row);
                        state_table.insert(first_row);
                        last_timestamp = first;
                    }

                    // Now let's step through the timestamps from the last timestamp to the current timestamp.
                    // We use `last_row` as a temporary cursor to track the progress, and won't touch `last_timestamp`
                    // until the end of the loop, so that `last_timestamp` is always synced with the state table.
                    let mut last_row = OwnedRow::new(vec![last_timestamp.clone()]);

                    loop {
                        if chunk_builder.size() >= max_chunk_size {
                            // Manually yield the chunk when size exceeds the limit. We don't want to use chunk builder
                            // with limited size here because the initial capacity can be too large for most cases.
                            // Basically only the first several chunks can potentially exceed the `max_chunk_size`.
                            if let Some(chunk) = chunk_builder.take() {
                                yield Message::Chunk(chunk);
                            }
                        }

                        let next = add_interval_expr.eval_row_infallible(&last_row).await;
                        if DefaultOrdered(next.to_datum_ref())
                            > DefaultOrdered(curr_timestamp.to_datum_ref())
                        {
                            // We only increase the timestamp to the current timestamp.
                            break;
                        }

                        let next_row = OwnedRow::new(vec![next]);
                        let _ = chunk_builder.append_row(Op::Insert, &next_row);
                        last_row = next_row;
                    }

                    if let Some(chunk) = chunk_builder.take() {
                        yield Message::Chunk(chunk);
                    }

                    // Update the last timestamp.
                    state_table.update(row::once(&last_timestamp), &last_row);
                    last_timestamp = last_row
                        .into_inner()
                        .into_vec()
                        .into_iter()
                        .exactly_one()
                        .unwrap();
                }
                _ => unreachable!(),
            }

            yield Message::Watermark(Watermark::new(
                0,
                DataType::Timestamptz,
                curr_timestamp.unwrap(),
            ));
        }
    }
}

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

#[capture_context(TIME_ZONE)]
pub fn build_add_interval_expr(
    time_zone: &str,
    interval: Interval,
    eval_error_report: impl EvalErrorReport + 'static,
) -> risingwave_expr::Result<NonStrictExpression> {
    let timestamptz_input = InputRefExpression::new(DataType::Timestamptz, 0);
    let interval = LiteralExpression::new(DataType::Interval, Some(interval.into()));
    let time_zone = LiteralExpression::new(DataType::Varchar, Some(time_zone.into()));

    use risingwave_pb::expr::expr_node::PbType as PbExprType;
    build_func_non_strict(
        PbExprType::AddWithTimeZone,
        DataType::Timestamptz,
        vec![
            timestamptz_input.boxed(),
            interval.boxed(),
            time_zone.boxed(),
        ],
        eval_error_report,
    )
}

#[cfg(test)]
mod tests {
    use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId};
    use risingwave_common::test_prelude::StreamChunkTestExt;
    use risingwave_common::types::test_utils::IntervalTestExt;
    use risingwave_common::util::epoch::test_epoch;
    use risingwave_storage::memory::MemoryStateStore;
    use tokio::sync::mpsc::{unbounded_channel, UnboundedSender};

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

    #[tokio::test]
    async fn test_now() -> StreamExecutorResult<()> {
        let state_store = create_state_store();
        let (tx, mut now) = create_executor(NowMode::UpdateCurrent, &state_store).await;

        // Init barrier
        tx.send(Barrier::new_test_barrier(test_epoch(1))).unwrap();

        // Consume the barrier
        now.next_unwrap_ready_barrier()?;

        // Consume the data chunk
        let chunk_msg = now.next_unwrap_ready_chunk()?;

        assert_eq!(
            chunk_msg.compact(),
            StreamChunk::from_pretty(
                " TZ
                + 2021-04-01T00:00:00.001Z"
            )
        );

        // Consume the watermark
        let watermark = now.next_unwrap_ready_watermark()?;

        assert_eq!(
            watermark,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:00.001Z".parse().unwrap())
            )
        );

        tx.send(Barrier::with_prev_epoch_for_test(
            test_epoch(2),
            test_epoch(1),
        ))
        .unwrap();

        // Consume the barrier
        now.next_unwrap_ready_barrier()?;

        // Consume the data chunk
        let chunk_msg = now.next_unwrap_ready_chunk()?;

        assert_eq!(
            chunk_msg.compact(),
            StreamChunk::from_pretty(
                " TZ
                - 2021-04-01T00:00:00.001Z
                + 2021-04-01T00:00:00.002Z"
            )
        );

        // Consume the watermark
        let watermark = now.next_unwrap_ready_watermark()?;

        assert_eq!(
            watermark,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:00.002Z".parse().unwrap())
            )
        );

        // No more messages until the next barrier
        now.next_unwrap_pending();

        // Recovery
        drop((tx, now));
        let (tx, mut now) = create_executor(NowMode::UpdateCurrent, &state_store).await;
        tx.send(Barrier::with_prev_epoch_for_test(
            test_epoch(3),
            test_epoch(2),
        ))
        .unwrap();

        // Consume the barrier
        now.next_unwrap_ready_barrier()?;

        // Consume the data chunk
        let chunk_msg = now.next_unwrap_ready_chunk()?;
        assert_eq!(
            chunk_msg.compact(),
            // the last chunk was not checkpointed so the deleted old value should be `001`
            StreamChunk::from_pretty(
                " TZ
                - 2021-04-01T00:00:00.001Z
                + 2021-04-01T00:00:00.003Z"
            )
        );

        // Consume the watermark
        let watermark = now.next_unwrap_ready_watermark()?;

        assert_eq!(
            watermark,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:00.003Z".parse().unwrap())
            )
        );

        // Recovery with paused
        drop((tx, now));
        let (tx, mut now) = create_executor(NowMode::UpdateCurrent, &state_store).await;
        tx.send(
            Barrier::with_prev_epoch_for_test(test_epoch(4), test_epoch(3))
                .with_mutation(Mutation::Pause),
        )
        .unwrap();

        // Consume the barrier
        now.next_unwrap_ready_barrier()?;

        // There should be no messages until `Resume`
        now.next_unwrap_pending();

        // Resume barrier
        tx.send(
            Barrier::with_prev_epoch_for_test(test_epoch(5), test_epoch(4))
                .with_mutation(Mutation::Resume),
        )
        .unwrap();

        // Consume the barrier
        now.next_unwrap_ready_barrier()?;

        // Consume the data chunk
        let chunk_msg = now.next_unwrap_ready_chunk()?;
        assert_eq!(
            chunk_msg.compact(),
            StreamChunk::from_pretty(
                " TZ
                - 2021-04-01T00:00:00.001Z
                + 2021-04-01T00:00:00.005Z"
            )
        );

        // Consume the watermark
        let watermark = now.next_unwrap_ready_watermark()?;

        assert_eq!(
            watermark,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:00.005Z".parse().unwrap())
            )
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_now_start_with_paused() -> StreamExecutorResult<()> {
        let state_store = create_state_store();
        let (tx, mut now) = create_executor(NowMode::UpdateCurrent, &state_store).await;

        // Init barrier
        tx.send(Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Pause))
            .unwrap();

        // Consume the barrier
        now.next_unwrap_ready_barrier()?;

        // There should be no messages until `Resume`
        now.next_unwrap_pending();

        // Resume barrier
        tx.send(
            Barrier::with_prev_epoch_for_test(test_epoch(2), test_epoch(1))
                .with_mutation(Mutation::Resume),
        )
        .unwrap();

        // Consume the barrier
        now.next_unwrap_ready_barrier()?;

        // Consume the data chunk
        let chunk_msg = now.next_unwrap_ready_chunk()?;

        assert_eq!(
            chunk_msg.compact(),
            StreamChunk::from_pretty(
                " TZ
                + 2021-04-01T00:00:00.002Z" // <- the timestamp is extracted from the current epoch
            )
        );

        // Consume the watermark
        let watermark = now.next_unwrap_ready_watermark()?;

        assert_eq!(
            watermark,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:00.002Z".parse().unwrap())
            )
        );

        // No more messages until the next barrier
        now.next_unwrap_pending();

        Ok(())
    }

    #[tokio::test]
    async fn test_now_generate_series() -> StreamExecutorResult<()> {
        TIME_ZONE::scope("UTC".to_string(), test_now_generate_series_inner()).await
    }

    async fn test_now_generate_series_inner() -> StreamExecutorResult<()> {
        let start_timestamp = Timestamptz::from_secs(1617235190).unwrap(); // 2021-03-31 23:59:50 UTC
        let interval = Interval::from_millis(1000); // 1s interval

        let state_store = create_state_store();
        let (tx, mut now) = create_executor(
            NowMode::GenerateSeries {
                start_timestamp,
                interval,
            },
            &state_store,
        )
        .await;

        // Init barrier
        tx.send(Barrier::new_test_barrier(test_epoch(1000)))
            .unwrap();
        now.next_unwrap_ready_barrier()?;

        // Initial timestamps
        let chunk = now.next_unwrap_ready_chunk()?;
        assert_eq!(chunk.cardinality(), 12); // seconds from 23:59:50 to 00:00:01 (inclusive)

        assert_eq!(
            now.next_unwrap_ready_watermark()?,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:01.000Z".parse().unwrap())
            )
        );

        tx.send(Barrier::with_prev_epoch_for_test(
            test_epoch(2000),
            test_epoch(1000),
        ))
        .unwrap();
        tx.send(Barrier::with_prev_epoch_for_test(
            test_epoch(3000),
            test_epoch(2000),
        ))
        .unwrap();

        now.next_unwrap_ready_barrier()?;
        now.next_unwrap_ready_barrier()?;

        let chunk = now.next_unwrap_ready_chunk()?;
        assert_eq!(
            chunk.compact(),
            StreamChunk::from_pretty(
                " TZ
                + 2021-04-01T00:00:02.000Z
                + 2021-04-01T00:00:03.000Z"
            )
        );

        let watermark = now.next_unwrap_ready_watermark()?;
        assert_eq!(
            watermark,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:03.000Z".parse().unwrap())
            )
        );

        // Recovery
        drop((tx, now));
        let (tx, mut now) = create_executor(
            NowMode::GenerateSeries {
                start_timestamp,
                interval,
            },
            &state_store,
        )
        .await;

        tx.send(Barrier::with_prev_epoch_for_test(
            test_epoch(4000),
            test_epoch(3000),
        ))
        .unwrap();

        now.next_unwrap_ready_barrier()?;

        let chunk = now.next_unwrap_ready_chunk()?;
        assert_eq!(
            chunk.compact(),
            StreamChunk::from_pretty(
                " TZ
                + 2021-04-01T00:00:02.000Z
                + 2021-04-01T00:00:03.000Z
                + 2021-04-01T00:00:04.000Z"
            )
        );

        let watermark = now.next_unwrap_ready_watermark()?;
        assert_eq!(
            watermark,
            Watermark::new(
                0,
                DataType::Timestamptz,
                ScalarImpl::Timestamptz("2021-04-01T00:00:04.000Z".parse().unwrap())
            )
        );

        Ok(())
    }

    fn create_state_store() -> MemoryStateStore {
        MemoryStateStore::new()
    }

    async fn create_executor(
        mode: NowMode,
        state_store: &MemoryStateStore,
    ) -> (UnboundedSender<Barrier>, BoxedMessageStream) {
        let table_id = TableId::new(1);
        let column_descs = vec![ColumnDesc::unnamed(ColumnId::new(0), DataType::Timestamptz)];
        let state_table = StateTable::from_table_catalog(
            &gen_pbtable(table_id, column_descs, vec![], vec![], 0),
            state_store.clone(),
            None,
        )
        .await;

        let (sender, barrier_receiver) = unbounded_channel();

        let eval_error_report = ActorEvalErrorReport {
            actor_context: ActorContext::for_test(123),
            identity: "NowExecutor".into(),
        };
        let now_executor = NowExecutor::new(
            vec![DataType::Timestamptz],
            mode,
            eval_error_report,
            barrier_receiver,
            state_table,
        );
        (sender, now_executor.boxed().execute())
    }
}