risingwave_frontend/optimizer/plan_node/
stream_dml.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
// 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 fixedbitset::FixedBitSet;
use pretty_xmlish::{Pretty, XmlNode};
use risingwave_common::catalog::{ColumnDesc, INITIAL_TABLE_VERSION_ID};
use risingwave_pb::stream_plan::stream_node::PbNodeBody;

use super::stream::prelude::*;
use super::utils::{childless_record, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::MonotonicityMap;
use crate::stream_fragmenter::BuildFragmentGraphState;

#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct StreamDml {
    pub base: PlanBase<Stream>,
    input: PlanRef,
    column_descs: Vec<ColumnDesc>,
}

impl StreamDml {
    pub fn new(input: PlanRef, append_only: bool, column_descs: Vec<ColumnDesc>) -> Self {
        let base = PlanBase::new_stream(
            input.ctx(),
            input.schema().clone(),
            input.stream_key().map(|v| v.to_vec()),
            input.functional_dependency().clone(),
            input.distribution().clone(),
            append_only,
            false,                                            // TODO(rc): decide EOWC property
            FixedBitSet::with_capacity(input.schema().len()), // no watermark if dml is allowed
            MonotonicityMap::new(),                           // TODO: derive monotonicity
        );

        Self {
            base,
            input,
            column_descs,
        }
    }

    fn column_names(&self) -> Vec<&str> {
        self.column_descs
            .iter()
            .map(|column_desc| column_desc.name.as_str())
            .collect()
    }
}

impl Distill for StreamDml {
    fn distill<'a>(&self) -> XmlNode<'a> {
        let col = self
            .column_names()
            .iter()
            .map(|n| Pretty::from(n.to_string()))
            .collect();
        let col = Pretty::Array(col);
        childless_record("StreamDml", vec![("columns", col)])
    }
}

impl PlanTreeNodeUnary for StreamDml {
    fn input(&self) -> PlanRef {
        self.input.clone()
    }

    fn clone_with_input(&self, input: PlanRef) -> Self {
        Self::new(input, self.append_only(), self.column_descs.clone())
    }
}

impl_plan_tree_node_for_unary! {StreamDml}

impl StreamNode for StreamDml {
    fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody {
        use risingwave_pb::stream_plan::*;

        PbNodeBody::Dml(DmlNode {
            table_id: 0,                                // Meta will fill this table id.
            table_version_id: INITIAL_TABLE_VERSION_ID, // Meta will fill this version id.
            column_descs: self.column_descs.iter().map(Into::into).collect(),
        })
    }
}

impl ExprRewritable for StreamDml {}

impl ExprVisitable for StreamDml {}