risingwave_frontend/optimizer/plan_node/generic/
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
// 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 educe::Educe;
use enum_as_inner::EnumAsInner;
use pretty_xmlish::{Pretty, Str, XmlNode};
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::types::{DataType, Interval, Timestamptz};

use super::{DistillUnit, GenericPlanNode};
use crate::optimizer::plan_node::utils::childless_record;
use crate::optimizer::property::FunctionalDependencySet;
use crate::OptimizerContextRef;

#[derive(Debug, Clone, Educe)]
#[educe(PartialEq, Eq, Hash)]
pub struct Now {
    #[educe(PartialEq(ignore))]
    #[educe(Hash(ignore))]
    pub ctx: OptimizerContextRef,

    pub mode: Mode,
}

#[derive(Debug, Clone, PartialEq, Eq, Hash, EnumAsInner)]
pub enum Mode {
    /// 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,
    },
}

impl GenericPlanNode for Now {
    fn functional_dependency(&self) -> crate::optimizer::property::FunctionalDependencySet {
        FunctionalDependencySet::new(1) // only one column and no dependency
    }

    fn schema(&self) -> risingwave_common::catalog::Schema {
        Schema::new(vec![Field {
            data_type: DataType::Timestamptz,
            name: String::from(if self.mode.is_update_current() {
                "now"
            } else {
                "ts"
            }),
            sub_fields: vec![],
            type_name: String::default(),
        }])
    }

    fn stream_key(&self) -> Option<Vec<usize>> {
        match self.mode {
            Mode::UpdateCurrent => Some(vec![]),
            Mode::GenerateSeries { .. } => Some(vec![0]),
        }
    }

    fn ctx(&self) -> OptimizerContextRef {
        self.ctx.clone()
    }
}

impl Now {
    pub fn update_current(ctx: OptimizerContextRef) -> Self {
        Self::new_inner(ctx, Mode::UpdateCurrent)
    }

    pub fn generate_series(
        ctx: OptimizerContextRef,
        start_timestamp: Timestamptz,
        interval: Interval,
    ) -> Self {
        Self::new_inner(
            ctx,
            Mode::GenerateSeries {
                start_timestamp,
                interval,
            },
        )
    }

    fn new_inner(ctx: OptimizerContextRef, mode: Mode) -> Self {
        Self { ctx, mode }
    }
}

impl DistillUnit for Now {
    fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
        childless_record(name, vec![("mode", Pretty::debug(&self.mode))])
    }
}