risingwave_common/util/
stream_graph_visitor.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
// 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 itertools::Itertools;
use risingwave_pb::catalog::Table;
use risingwave_pb::stream_plan::stream_fragment_graph::StreamFragment;
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::{agg_call_state, StreamNode};

/// A utility for visiting and mutating the [`NodeBody`] of the [`StreamNode`]s recursively.
pub fn visit_stream_node<F>(stream_node: &mut StreamNode, mut f: F)
where
    F: FnMut(&mut NodeBody),
{
    fn visit_inner<F>(stream_node: &mut StreamNode, f: &mut F)
    where
        F: FnMut(&mut NodeBody),
    {
        f(stream_node.node_body.as_mut().unwrap());
        for input in &mut stream_node.input {
            visit_inner(input, f);
        }
    }

    visit_inner(stream_node, &mut f)
}

/// A utility for to accessing the [`StreamNode`] mutably. The returned bool is used to determine whether the access needs to continue.
pub fn visit_stream_node_cont_mut<F>(stream_node: &mut StreamNode, mut f: F)
where
    F: FnMut(&mut StreamNode) -> bool,
{
    fn visit_inner<F>(stream_node: &mut StreamNode, f: &mut F)
    where
        F: FnMut(&mut StreamNode) -> bool,
    {
        if !f(stream_node) {
            return;
        }
        for input in &mut stream_node.input {
            visit_inner(input, f);
        }
    }

    visit_inner(stream_node, &mut f)
}

/// A utility for to accessing the [`StreamNode`] immutably. The returned bool is used to determine whether the access needs to continue.
pub fn visit_stream_node_cont<F>(stream_node: &StreamNode, mut f: F)
where
    F: FnMut(&StreamNode) -> bool,
{
    fn visit_inner<F>(stream_node: &StreamNode, f: &mut F)
    where
        F: FnMut(&StreamNode) -> bool,
    {
        if !f(stream_node) {
            return;
        }
        for input in &stream_node.input {
            visit_inner(input, f);
        }
    }

    visit_inner(stream_node, &mut f)
}

/// A utility for visiting and mutating the [`NodeBody`] of the [`StreamNode`]s in a
/// [`StreamFragment`] recursively.
pub fn visit_fragment<F>(fragment: &mut StreamFragment, f: F)
where
    F: FnMut(&mut NodeBody),
{
    visit_stream_node(fragment.node.as_mut().unwrap(), f)
}

/// Visit the tables of a [`StreamNode`].
pub fn visit_stream_node_tables_inner<F>(
    stream_node: &mut StreamNode,
    internal_tables_only: bool,
    visit_child_recursively: bool,
    mut f: F,
) where
    F: FnMut(&mut Table, &str),
{
    macro_rules! always {
        ($table:expr, $name:expr) => {{
            let table = $table
                .as_mut()
                .unwrap_or_else(|| panic!("internal table {} should always exist", $name));
            f(table, $name);
        }};
    }

    macro_rules! optional {
        ($table:expr, $name:expr) => {
            if let Some(table) = &mut $table {
                f(table, $name);
            }
        };
    }

    macro_rules! repeated {
        ($tables:expr, $name:expr) => {
            for table in &mut $tables {
                f(table, $name);
            }
        };
    }

    let mut visit_body = |body: &mut NodeBody| {
        match body {
            // Join
            NodeBody::HashJoin(node) => {
                // TODO: make the degree table optional
                always!(node.left_table, "HashJoinLeft");
                always!(node.left_degree_table, "HashJoinDegreeLeft");
                always!(node.right_table, "HashJoinRight");
                always!(node.right_degree_table, "HashJoinDegreeRight");
            }
            NodeBody::TemporalJoin(node) => {
                optional!(node.memo_table, "TemporalJoinMemo");
            }
            NodeBody::DynamicFilter(node) => {
                always!(node.left_table, "DynamicFilterLeft");
                always!(node.right_table, "DynamicFilterRight");
            }

            // Aggregation
            NodeBody::HashAgg(node) => {
                assert_eq!(node.agg_call_states.len(), node.agg_calls.len());
                always!(node.intermediate_state_table, "HashAggState");
                for (call_idx, state) in node.agg_call_states.iter_mut().enumerate() {
                    match state.inner.as_mut().unwrap() {
                        agg_call_state::Inner::ValueState(_) => {}
                        agg_call_state::Inner::MaterializedInputState(s) => {
                            always!(s.table, &format!("HashAggCall{}", call_idx));
                        }
                    }
                }
                for (distinct_col, dedup_table) in node
                    .distinct_dedup_tables
                    .iter_mut()
                    .sorted_by_key(|(i, _)| *i)
                {
                    f(dedup_table, &format!("HashAggDedupForCol{}", distinct_col));
                }
            }
            NodeBody::SimpleAgg(node) => {
                assert_eq!(node.agg_call_states.len(), node.agg_calls.len());
                always!(node.intermediate_state_table, "SimpleAggState");
                for (call_idx, state) in node.agg_call_states.iter_mut().enumerate() {
                    match state.inner.as_mut().unwrap() {
                        agg_call_state::Inner::ValueState(_) => {}
                        agg_call_state::Inner::MaterializedInputState(s) => {
                            always!(s.table, &format!("SimpleAggCall{}", call_idx));
                        }
                    }
                }
                for (distinct_col, dedup_table) in node
                    .distinct_dedup_tables
                    .iter_mut()
                    .sorted_by_key(|(i, _)| *i)
                {
                    f(
                        dedup_table,
                        &format!("SimpleAggDedupForCol{}", distinct_col),
                    );
                }
            }

            // Top-N
            NodeBody::AppendOnlyTopN(node) => {
                always!(node.table, "AppendOnlyTopN");
            }
            NodeBody::TopN(node) => {
                always!(node.table, "TopN");
            }
            NodeBody::AppendOnlyGroupTopN(node) => {
                always!(node.table, "AppendOnlyGroupTopN");
            }
            NodeBody::GroupTopN(node) => {
                always!(node.table, "GroupTopN");
            }

            // Source
            NodeBody::Source(node) => {
                if let Some(source) = &mut node.source_inner {
                    always!(source.state_table, "Source");
                }
            }
            NodeBody::StreamFsFetch(node) => {
                if let Some(source) = &mut node.node_inner {
                    always!(source.state_table, "FsFetch");
                }
            }
            NodeBody::SourceBackfill(node) => {
                always!(node.state_table, "SourceBackfill")
            }

            // Sink
            NodeBody::Sink(node) => {
                // A sink with a kv log store should have a state table.
                optional!(node.table, "Sink")
            }

            // Now
            NodeBody::Now(node) => {
                always!(node.state_table, "Now");
            }

            // Watermark filter
            NodeBody::WatermarkFilter(node) => {
                assert!(!node.tables.is_empty());
                repeated!(node.tables, "WatermarkFilter");
            }

            // Shared arrangement
            NodeBody::Arrange(node) => {
                always!(node.table, "Arrange");
            }

            // Dedup
            NodeBody::AppendOnlyDedup(node) => {
                always!(node.state_table, "AppendOnlyDedup");
            }

            // EOWC over window
            NodeBody::EowcOverWindow(node) => {
                always!(node.state_table, "EowcOverWindow");
            }

            NodeBody::OverWindow(node) => {
                always!(node.state_table, "OverWindow");
            }

            // Sort
            NodeBody::Sort(node) => {
                always!(node.state_table, "Sort");
            }

            // Stream Scan
            NodeBody::StreamScan(node) => {
                optional!(node.state_table, "StreamScan")
            }

            // Stream Cdc Scan
            NodeBody::StreamCdcScan(node) => {
                always!(node.state_table, "StreamCdcScan")
            }

            // Note: add internal tables for new nodes here.
            NodeBody::Materialize(node) if !internal_tables_only => {
                always!(node.table, "Materialize")
            }

            NodeBody::GlobalApproxPercentile(node) => {
                always!(node.bucket_state_table, "GlobalApproxPercentileBucketState");
                always!(node.count_state_table, "GlobalApproxPercentileCountState");
            }

            // AsOf join
            NodeBody::AsOfJoin(node) => {
                always!(node.left_table, "AsOfJoinLeft");
                always!(node.right_table, "AsOfJoinRight");
            }
            _ => {}
        }
    };
    if visit_child_recursively {
        visit_stream_node(stream_node, visit_body)
    } else {
        visit_body(stream_node.node_body.as_mut().unwrap())
    }
}

pub fn visit_stream_node_internal_tables<F>(stream_node: &mut StreamNode, f: F)
where
    F: FnMut(&mut Table, &str),
{
    visit_stream_node_tables_inner(stream_node, true, true, f)
}

pub fn visit_stream_node_tables<F>(stream_node: &mut StreamNode, f: F)
where
    F: FnMut(&mut Table, &str),
{
    visit_stream_node_tables_inner(stream_node, false, true, f)
}

/// Visit the internal tables of a [`StreamFragment`].
pub fn visit_internal_tables<F>(fragment: &mut StreamFragment, f: F)
where
    F: FnMut(&mut Table, &str),
{
    visit_stream_node_internal_tables(fragment.node.as_mut().unwrap(), f)
}

/// Visit the tables of a [`StreamFragment`].
///
/// Compared to [`visit_internal_tables`], this function also visits the table of `Materialize` node.
pub fn visit_tables<F>(fragment: &mut StreamFragment, f: F)
where
    F: FnMut(&mut Table, &str),
{
    visit_stream_node_tables(fragment.node.as_mut().unwrap(), f)
}