risingwave_frontend/utils/
stream_graph_formatter.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
// 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::cmp::max;
use std::collections::{BTreeMap, HashMap};

use itertools::Itertools;
use pretty_xmlish::{Pretty, PrettyConfig};
use risingwave_common::util::stream_graph_visitor;
use risingwave_pb::catalog::Table;
use risingwave_pb::stream_plan::stream_fragment_graph::StreamFragmentEdge;
use risingwave_pb::stream_plan::{stream_node, DispatcherType, StreamFragmentGraph, StreamNode};

use crate::TableCatalog;

/// ice: in the future, we may allow configurable width, boundaries, etc.
pub fn explain_stream_graph(graph: &StreamFragmentGraph, is_verbose: bool) -> String {
    let mut output = String::with_capacity(2048);
    let mut config = PrettyConfig {
        need_boundaries: false,
        width: 80,
        ..Default::default()
    };
    StreamGraphFormatter::new(is_verbose).explain_graph(graph, &mut config, &mut output);
    output
}

/// A formatter to display the final stream plan graph, used for `explain (distsql) create
/// materialized view ...`
struct StreamGraphFormatter {
    /// exchange's `operator_id` -> edge
    edges: HashMap<u64, StreamFragmentEdge>,
    verbose: bool,
    tables: BTreeMap<u32, Table>,
}

impl StreamGraphFormatter {
    fn new(verbose: bool) -> Self {
        StreamGraphFormatter {
            edges: HashMap::default(),
            tables: BTreeMap::default(),
            verbose,
        }
    }

    /// collect the table catalog and return the table id
    fn add_table(&mut self, tb: &Table) -> u32 {
        self.tables.insert(tb.id, tb.clone());
        tb.id
    }

    fn explain_graph(
        &mut self,
        graph: &StreamFragmentGraph,
        config: &mut PrettyConfig,
        output: &mut String,
    ) {
        self.edges.clear();
        for edge in &graph.edges {
            self.edges.insert(edge.link_id, edge.clone());
        }
        let mut max_width = 80;
        for (_, fragment) in graph.fragments.iter().sorted_by_key(|(id, _)| **id) {
            output.push_str("Fragment ");
            output.push_str(&fragment.get_fragment_id().to_string());
            output.push('\n');
            let width = config.unicode(output, &self.explain_node(fragment.node.as_ref().unwrap()));
            max_width = max(width, max_width);
            config.width = max_width;
            output.push_str("\n\n");
        }
        for tb in self.tables.values() {
            let width = config.unicode(output, &self.explain_table(tb));
            max_width = max(width, max_width);
            config.width = max_width;
            output.push_str("\n\n");
        }
    }

    fn explain_table<'a>(&self, tb: &Table) -> Pretty<'a> {
        let tb = TableCatalog::from(tb.clone());
        let columns = tb
            .columns
            .iter()
            .map(|c| {
                let s = if self.verbose {
                    format!("{}: {}", c.name(), c.data_type())
                } else {
                    c.name().to_string()
                };
                Pretty::Text(s.into())
            })
            .collect();
        let columns = Pretty::Array(columns);
        let name = format!("Table {}", tb.id);
        let mut fields = Vec::with_capacity(5);
        fields.push(("columns", columns));
        fields.push((
            "primary key",
            Pretty::Array(tb.pk.iter().map(Pretty::debug).collect()),
        ));
        fields.push((
            "value indices",
            Pretty::Array(tb.value_indices.iter().map(Pretty::debug).collect()),
        ));
        fields.push((
            "distribution key",
            Pretty::Array(tb.distribution_key.iter().map(Pretty::debug).collect()),
        ));
        fields.push((
            "read pk prefix len hint",
            Pretty::debug(&tb.read_prefix_len_hint),
        ));
        if let Some(vnode_col_idx) = tb.vnode_col_index {
            fields.push(("vnode column idx", Pretty::debug(&vnode_col_idx)));
        }
        Pretty::childless_record(name, fields)
    }

    fn explain_node<'a>(&mut self, node: &StreamNode) -> Pretty<'a> {
        let one_line_explain = match node.get_node_body().unwrap() {
            stream_node::NodeBody::Exchange(_) => {
                let edge = self.edges.get(&node.operator_id).unwrap();
                let upstream_fragment_id = edge.upstream_id;
                let dist = edge.dispatch_strategy.as_ref().unwrap();
                format!(
                    "StreamExchange {} from {}",
                    match dist.r#type() {
                        DispatcherType::Unspecified => unreachable!(),
                        DispatcherType::Hash => format!("Hash({:?})", dist.dist_key_indices),
                        DispatcherType::Broadcast => "Broadcast".to_string(),
                        DispatcherType::Simple => "Single".to_string(),
                        DispatcherType::NoShuffle => "NoShuffle".to_string(),
                    },
                    upstream_fragment_id
                )
            }
            _ => node.identity.clone(),
        };

        let mut tables: Vec<(String, u32)> = Vec::with_capacity(7);
        let mut node_copy = node.clone();

        stream_graph_visitor::visit_stream_node_tables_inner(
            &mut node_copy,
            false,
            false,
            |table, table_name| {
                tables.push((table_name.to_string(), self.add_table(table)));
            },
        );

        let mut fields = Vec::with_capacity(3);
        if !tables.is_empty() {
            fields.push((
                "tables",
                Pretty::Array(
                    tables
                        .into_iter()
                        .map(|(name, id)| Pretty::Text(format!("{}: {}", name, id).into()))
                        .collect(),
                ),
            ));
        }
        if self.verbose {
            fields.push((
                "output",
                Pretty::Array(
                    node.fields
                        .iter()
                        .map(|f| Pretty::display(f.get_name()))
                        .collect(),
                ),
            ));
            fields.push((
                "stream key",
                Pretty::Array(
                    node.stream_key
                        .iter()
                        .map(|i| Pretty::display(node.fields[*i as usize].get_name()))
                        .collect(),
                ),
            ));
        }
        let children = node
            .input
            .iter()
            .map(|input| self.explain_node(input))
            .collect();
        Pretty::simple_record(one_line_explain, fields, children)
    }
}