risingwave_stream/from_proto/
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
// 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 anyhow::Context;
use risingwave_common::types::{DataType, Datum};
use risingwave_common::util::value_encoding::DatumFromProtoExt;
use risingwave_pb::stream_plan::now_node::PbMode as PbNowMode;
use risingwave_pb::stream_plan::{NowNode, PbNowModeGenerateSeries};
use risingwave_storage::StateStore;

use super::ExecutorBuilder;
use crate::common::table::state_table::StateTable;
use crate::error::StreamResult;
use crate::executor::{Executor, NowExecutor, NowMode};
use crate::task::ExecutorParams;

pub struct NowExecutorBuilder;

impl ExecutorBuilder for NowExecutorBuilder {
    type Node = NowNode;

    async fn new_boxed_executor(
        params: ExecutorParams,
        node: &NowNode,
        store: impl StateStore,
    ) -> StreamResult<Executor> {
        let barrier_receiver = params
            .shared_context
            .local_barrier_manager
            .subscribe_barrier(params.actor_context.id);

        let mode = if let Ok(pb_mode) = node.get_mode() {
            match pb_mode {
                PbNowMode::UpdateCurrent(_) => NowMode::UpdateCurrent,
                PbNowMode::GenerateSeries(PbNowModeGenerateSeries {
                    start_timestamp,
                    interval,
                }) => {
                    let start_timestamp = Datum::from_protobuf(
                        start_timestamp.as_ref().unwrap(),
                        &DataType::Timestamptz,
                    )
                    .context("`start_timestamp` field is not decodable")?
                    .context("`start_timestamp` field should not be NULL")?
                    .into_timestamptz();
                    let interval =
                        Datum::from_protobuf(interval.as_ref().unwrap(), &DataType::Interval)
                            .context("`interval` field is not decodable")?
                            .context("`interval` field should not be NULL")?
                            .into_interval();
                    NowMode::GenerateSeries {
                        start_timestamp,
                        interval,
                    }
                }
            }
        } else {
            // default to `UpdateCurrent` for backward-compatibility
            NowMode::UpdateCurrent
        };

        let state_table =
            StateTable::from_table_catalog(node.get_state_table()?, store, None).await;

        let exec = NowExecutor::new(
            params.info.schema.data_types(),
            mode,
            params.eval_error_report,
            barrier_receiver,
            state_table,
        );
        Ok((params.info, exec).into())
    }
}